You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/04/20 10:20:39 UTC

[06/23] lucene-solr:feature/autoscaling: Squash-merge from master.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
index 17abf78..b9a2e78 100644
--- a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
+++ b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
@@ -21,14 +21,16 @@ import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Random;
 
+import com.codahale.metrics.Metric;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.join.ScoreMode;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.NamedList;
+import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.response.SolrQueryResponse;
@@ -198,22 +200,23 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
   public void testCacheHit() throws Exception {
     indexDataForScorring();
 
-    SolrCache cache = (SolrCache) h.getCore().getInfoRegistry()
-        .get("queryResultCache");
+    Map<String, Metric> metrics = h.getCoreContainer().getMetricManager().registry(h.getCore().getCoreMetricManager().getRegistryName()).getMetrics();
+
+    MetricsMap mm = (MetricsMap)metrics.get("CACHE.searcher.queryResultCache");
     {
-      final NamedList statPre = cache.getStatistics();
+      Map<String,Object> statPre = mm.getValue();
       h.query(req("q", "{!join from=movieId_s to=id score=Avg}title:first", "fl", "id", "omitHeader", "true"));
-      assertHitOrInsert(cache, statPre);
+      assertHitOrInsert(mm.getValue(), statPre);
     }
 
     {
-      final NamedList statPre = cache.getStatistics();
+      Map<String,Object> statPre = mm.getValue();
       h.query(req("q", "{!join from=movieId_s to=id score=Avg}title:first", "fl", "id", "omitHeader", "true"));
-      assertHit(cache, statPre);
+      assertHit(mm.getValue(), statPre);
     }
 
     {
-      NamedList statPre = cache.getStatistics();
+      Map<String,Object> statPre = mm.getValue();
 
       Random r = random();
       boolean changed = false;
@@ -234,14 +237,14 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
               //" b=" + boost + 
               "}" + q, "fl", "id", "omitHeader", "true")
       );
-      assertInsert(cache, statPre);
+      assertInsert(mm.getValue(), statPre);
 
-      statPre = cache.getStatistics();
+      statPre = mm.getValue();
       final String repeat = h.query(req("q", "{!join from=" + from + " to=" + to + " score=" + score.toLowerCase(Locale.ROOT) +
           //" b=" + boost
               "}" + q, "fl", "id", "omitHeader", "true")
       );
-      assertHit(cache, statPre);
+      assertHit(mm.getValue(), statPre);
 
       assertEquals("lowercase shouldn't change anything", resp, repeat);
 
@@ -254,6 +257,7 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
     // this queries are not overlap, with other in this test case. 
     // however it might be better to extract this method into the separate suite
     // for a while let's nuke a cache content, in case of repetitions
+    SolrCache cache = (SolrCache)h.getCore().getInfoRegistry().get("queryResultCache");
     cache.clear();
   }
 
@@ -264,32 +268,32 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
     return l.get(r.nextInt(l.size()));
   }
 
-  private void assertInsert(SolrCache cache, final NamedList statPre) {
+  private void assertInsert(Map<String,Object> current, final Map<String,Object> statPre) {
     assertEquals("it lookups", 1,
-        delta("lookups", cache.getStatistics(), statPre));
-    assertEquals("it doesn't hit", 0, delta("hits", cache.getStatistics(), statPre));
+        delta("lookups", current, statPre));
+    assertEquals("it doesn't hit", 0, delta("hits", current, statPre));
     assertEquals("it inserts", 1,
-        delta("inserts", cache.getStatistics(), statPre));
+        delta("inserts", current, statPre));
   }
 
-  private void assertHit(SolrCache cache, final NamedList statPre) {
+  private void assertHit(Map<String,Object> current, final Map<String,Object> statPre) {
     assertEquals("it lookups", 1,
-        delta("lookups", cache.getStatistics(), statPre));
-    assertEquals("it hits", 1, delta("hits", cache.getStatistics(), statPre));
+        delta("lookups", current, statPre));
+    assertEquals("it hits", 1, delta("hits", current, statPre));
     assertEquals("it doesn't insert", 0,
-        delta("inserts", cache.getStatistics(), statPre));
+        delta("inserts", current, statPre));
   }
 
-  private void assertHitOrInsert(SolrCache cache, final NamedList statPre) {
+  private void assertHitOrInsert(Map<String,Object> current, final Map<String,Object> statPre) {
     assertEquals("it lookups", 1,
-        delta("lookups", cache.getStatistics(), statPre));
-    final long mayHit = delta("hits", cache.getStatistics(), statPre);
+        delta("lookups", current, statPre));
+    final long mayHit = delta("hits", current, statPre);
     assertTrue("it may hit", 0 == mayHit || 1 == mayHit);
     assertEquals("or insert on cold", 1,
-        delta("inserts", cache.getStatistics(), statPre) + mayHit);
+        delta("inserts", current, statPre) + mayHit);
   }
 
-  private long delta(String key, NamedList a, NamedList b) {
+  private long delta(String key, Map<String,Object> a, Map<String,Object> b) {
     return (Long) a.get(key) - (Long) b.get(key);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorWithCollapseTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorWithCollapseTest.java b/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorWithCollapseTest.java
new file mode 100644
index 0000000..f985b41
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorWithCollapseTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.spelling;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.SpellingParams;
+import org.apache.solr.handler.component.SpellCheckComponent;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class SpellCheckCollatorWithCollapseTest  extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig-collapseqparser.xml", "schema11.xml");
+  }
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    clearIndex();
+    assertU(commit());
+  }
+  
+  @Test
+  public void test() throws Exception {
+    for(int i=0 ; i<200 ; i++) {
+      String[] doc = {"id","" + i, "group_i", "" + (i % 10), "a_s", ((i%2)==0 ? "love" : "peace")};
+      assertU(adoc(doc));
+      if(i%5==0) {
+        assertU(commit());
+      }
+    }
+    assertU(commit());
+    assertQ(
+        req(
+            SpellCheckComponent.COMPONENT_NAME, "true",
+            SpellCheckComponent.SPELLCHECK_DICT, "direct",
+            SpellingParams.SPELLCHECK_COUNT, "10",
+            SpellingParams.SPELLCHECK_COLLATE, "true",
+            SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "5",
+            SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
+            CommonParams.Q, "a_s:lpve",
+            CommonParams.QT, "spellCheckCompRH_Direct",
+            SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, "5",
+            CommonParams.FQ, "{!collapse field=group_i}",
+            "expand", "true"),
+        "//lst[@name='spellcheck']/lst[@name='collations']/str[@name='collation']='a_s:love'");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/test/org/apache/solr/spelling/SpellingQueryConverterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/spelling/SpellingQueryConverterTest.java b/solr/core/src/test/org/apache/solr/spelling/SpellingQueryConverterTest.java
index ed0749f..821fe73 100644
--- a/solr/core/src/test/org/apache/solr/spelling/SpellingQueryConverterTest.java
+++ b/solr/core/src/test/org/apache/solr/spelling/SpellingQueryConverterTest.java
@@ -94,6 +94,16 @@ public class SpellingQueryConverterTest extends LuceneTestCase {
     assertTrue("tokens is null and it shouldn't be", tokens != null);
     assertEquals("tokens Size: " + tokens.size() + " is not 1", 1, tokens.size());
     assertTrue("Token offsets do not match", isOffsetCorrect(original, tokens));
+    
+    String firstKeyword = "value1";
+    String secondKeyword = "value2";
+    original = "field-with-parenthesis:(" + firstKeyword + " " + secondKeyword + ")";
+    tokens = converter.convert(original);
+    assertTrue("tokens is null and it shouldn't be", tokens != null);
+    assertEquals("tokens Size: " + tokens.size() + " is not 2", 2, tokens.size());
+    assertTrue("Token offsets do not match", isOffsetCorrect(original, tokens));
+    assertTrue("first Token is not " + firstKeyword, new ArrayList<>(tokens).get(0).toString().equals(firstKeyword));
+    assertTrue("second Token is not " + secondKeyword, new ArrayList<>(tokens).get(1).toString().equals(secondKeyword));    
   }
 
   private boolean isOffsetCorrect(String s, Collection<Token> tokens) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/test/org/apache/solr/store/blockcache/BufferStoreTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/store/blockcache/BufferStoreTest.java b/solr/core/src/test/org/apache/solr/store/blockcache/BufferStoreTest.java
index e91d762..534793f 100644
--- a/solr/core/src/test/org/apache/solr/store/blockcache/BufferStoreTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blockcache/BufferStoreTest.java
@@ -17,9 +17,12 @@
 package org.apache.solr.store.blockcache;
 
 import java.math.BigDecimal;
+import java.util.Map;
 
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.common.util.NamedList;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -27,12 +30,18 @@ public class BufferStoreTest extends LuceneTestCase {
   private final static int blockSize = 1024;
 
   private Metrics metrics;
+  private MetricsMap metricsMap;
 
   private Store store;
 
   @Before
   public void setup() {
     metrics = new Metrics();
+    SolrMetricManager metricManager = new SolrMetricManager();
+    String registry = TestUtil.randomSimpleString(random(), 2, 10);
+    String scope = TestUtil.randomSimpleString(random(), 2, 10);
+    metrics.initializeMetrics(metricManager, registry, scope);
+    metricsMap = (MetricsMap) metricManager.registry(registry).getMetrics().get("CACHE." + scope + ".hdfsBlockCache");
     BufferStore.initNewBuffer(blockSize, blockSize, metrics);
     store = BufferStore.instance(blockSize);
   }
@@ -77,7 +86,7 @@ public class BufferStoreTest extends LuceneTestCase {
    *          whether buffers should have been lost since the last call
    */
   private void assertGaugeMetricsChanged(boolean allocated, boolean lost) {
-    NamedList<Number> stats = metrics.getStatistics();
+    Map<String,Object> stats = metricsMap.getValue();
 
     assertEquals("Buffer allocation metric not updating correctly.",
         allocated, isMetricPositive(stats, "buffercache.allocations"));
@@ -85,7 +94,7 @@ public class BufferStoreTest extends LuceneTestCase {
         lost, isMetricPositive(stats, "buffercache.lost"));
   }
 
-  private boolean isMetricPositive(NamedList<Number> stats, String metric) {
+  private boolean isMetricPositive(Map<String,Object> stats, String metric) {
     return new BigDecimal(stats.get(metric).toString()).compareTo(BigDecimal.ZERO) > 0;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/test/org/apache/solr/update/VersionInfoTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/VersionInfoTest.java b/solr/core/src/test/org/apache/solr/update/VersionInfoTest.java
index 6bccd95..e8a85bd 100644
--- a/solr/core/src/test/org/apache/solr/update/VersionInfoTest.java
+++ b/solr/core/src/test/org/apache/solr/update/VersionInfoTest.java
@@ -101,7 +101,7 @@ public class VersionInfoTest extends SolrTestCaseJ4 {
         versionFromTLog, versionFromIndex);
 
     // reload the core, which should reset the max
-    CoreContainer coreContainer = req.getCore().getCoreDescriptor().getCoreContainer();
+    CoreContainer coreContainer = req.getCore().getCoreContainer();
     coreContainer.reload(req.getCore().getName());
     maxVersionFromIndex = ulog.getMaxVersionFromIndex();
     assertEquals("max version from index should be equal to version of last doc added after reload",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java
index 432bb02..aa536a7 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java
@@ -99,7 +99,7 @@ public class ClassificationUpdateProcessorTest extends SolrTestCaseJ4 {
     updateProcessorToTest=new ClassificationUpdateProcessor(params,mockProcessor,reader,req().getSchema());
     updateProcessorToTest.processAdd(update);
 
-    assertThat(unseenDocument1.getFieldValue(PREDICTED_CLASS),is("class1"));
+    assertThat(unseenDocument1.getFieldValue(PREDICTED_CLASS),is("class2"));
   }
 
   @Test
@@ -119,7 +119,7 @@ public class ClassificationUpdateProcessorTest extends SolrTestCaseJ4 {
     updateProcessorToTest=new ClassificationUpdateProcessor(params,mockProcessor,reader,req().getSchema());
     updateProcessorToTest.processAdd(update);
 
-    assertThat(unseenDocument1.getFieldValue(TRAINING_CLASS),is("class1"));
+    assertThat(unseenDocument1.getFieldValue(TRAINING_CLASS),is("class2"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/test/org/apache/solr/util/MockCoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/MockCoreContainer.java b/solr/core/src/test/org/apache/solr/util/MockCoreContainer.java
index 054415a..5e8eab0 100644
--- a/solr/core/src/test/org/apache/solr/util/MockCoreContainer.java
+++ b/solr/core/src/test/org/apache/solr/util/MockCoreContainer.java
@@ -26,7 +26,7 @@ import org.apache.solr.core.CoreDescriptor;
 public class MockCoreContainer extends CoreContainer {
   public static class MockCoreDescriptor extends CoreDescriptor {
     public MockCoreDescriptor() {
-      super(new MockCoreContainer(), "mock", Paths.get("path"));
+      super("mock", Paths.get("path"), null, false);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java b/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
index aa02de5..4eb1169 100644
--- a/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
+++ b/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
@@ -18,6 +18,7 @@
 package org.apache.solr.util.stats;
 
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
@@ -45,7 +46,11 @@ public class MetricUtilsTest extends SolrTestCaseJ4 {
       timer.update(Math.abs(random().nextInt()) + 1, TimeUnit.NANOSECONDS);
     }
     // obtain timer metrics
-    NamedList lst = new NamedList(MetricUtils.convertTimer(timer, false));
+    Map<String,Object> map = new HashMap<>();
+    MetricUtils.convertTimer("", timer, MetricUtils.PropertyFilter.ALL, false, false, (k, v) -> {
+      map.putAll((Map<String,Object>)v);
+    });
+    NamedList lst = new NamedList(map);
     // check that expected metrics were obtained
     assertEquals(14, lst.size());
     final Snapshot snapshot = timer.getSnapshot();
@@ -84,7 +89,7 @@ public class MetricUtilsTest extends SolrTestCaseJ4 {
     Gauge<Long> error = () -> {throw new InternalError("Memory Pool not found error");};
     registry.register("memory.expected.error", error);
     MetricUtils.toMaps(registry, Collections.singletonList(MetricFilter.ALL), MetricFilter.ALL,
-        false, false, false, (k, o) -> {
+        MetricUtils.PropertyFilter.ALL, false, false, false, false, (k, o) -> {
       Map v = (Map)o;
       if (k.startsWith("counter")) {
         assertEquals(1L, v.get("count"));
@@ -114,7 +119,7 @@ public class MetricUtilsTest extends SolrTestCaseJ4 {
     });
     // test compact format
     MetricUtils.toMaps(registry, Collections.singletonList(MetricFilter.ALL), MetricFilter.ALL,
-        false, false, true, (k, o) -> {
+        MetricUtils.PropertyFilter.ALL, false, false, true, false, (k, o) -> {
           if (k.startsWith("counter")) {
             assertTrue(o instanceof Long);
             assertEquals(1L, o);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/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 ad85256..5c34cb6 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
@@ -16,7 +16,6 @@
  */
 package org.apache.solr.client.solrj.impl;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.net.ConnectException;
@@ -84,8 +83,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
 
-import static org.apache.solr.common.params.CommonParams.ID;
 import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS;
+import static org.apache.solr.common.params.CommonParams.ID;
 
 /**
  * SolrJ client class to communicate with SolrCloud.
@@ -393,7 +392,7 @@ public class CloudSolrClient extends SolrClient {
    */
   @Deprecated
   public CloudSolrClient(Collection<String> zkHosts, String chroot, HttpClient httpClient, LBHttpSolrClient lbSolrClient, boolean updatesToLeaders) {
-    this(zkHosts, chroot, httpClient, lbSolrClient, null, updatesToLeaders, false, null);
+    this(zkHosts, chroot, null, httpClient, lbSolrClient, null, updatesToLeaders, false, null);
   }
 
   /**
@@ -407,9 +406,14 @@ public class CloudSolrClient extends SolrClient {
    *          each host in the zookeeper ensemble. Note that with certain
    *          Collection types like HashSet, the order of hosts in the final
    *          connect string may not be in the same order you added them.
+   *          Provide only one of solrUrls or zkHosts.
    * @param chroot
    *          A chroot value for zookeeper, starting with a forward slash. If no
    *          chroot is required, use null.
+   * @param solrUrls
+   *          A list of Solr URLs to configure the underlying {@link HttpClusterStateProvider}, which will
+   *          use of the these URLs to fetch the list of live nodes for this Solr cluster. Provide only
+   *          one of solrUrls or zkHosts.
    * @param httpClient
    *          the {@link HttpClient} instance to be used for all requests. The provided httpClient should use a
    *          multi-threaded connection manager.  If null, a default HttpClient will be used.
@@ -424,6 +428,7 @@ public class CloudSolrClient extends SolrClient {
    */
   private CloudSolrClient(Collection<String> zkHosts,
                           String chroot,
+                          List<String> solrUrls,
                           HttpClient httpClient,
                           LBHttpSolrClient lbSolrClient,
                           LBHttpSolrClient.Builder lbHttpSolrClientBuilder,
@@ -433,7 +438,21 @@ public class CloudSolrClient extends SolrClient {
 
   ) {
     if (stateProvider == null) {
-      this.stateProvider = new ZkClientClusterStateProvider(zkHosts, chroot);
+      if (zkHosts != null && solrUrls != null) {
+        throw new IllegalArgumentException("Both zkHost(s) & solrUrl(s) have been specified. Only specify one.");
+      }
+      if (zkHosts != null) {
+        this.stateProvider = new ZkClientClusterStateProvider(zkHosts, chroot);
+      } else if (solrUrls != null && !solrUrls.isEmpty()) {
+        try {
+          this.stateProvider = new HttpClusterStateProvider(solrUrls, httpClient);
+        } catch (Exception e) {
+          throw new RuntimeException("Couldn't initialize a HttpClusterStateProvider (is/are the "
+              + "Solr server(s), "  + solrUrls + ", down?)", e);
+        }
+      } else {
+        throw new IllegalArgumentException("Both zkHosts and solrUrl cannot be null.");
+      }
     } else {
       this.stateProvider = stateProvider;
     }
@@ -1260,7 +1279,7 @@ public class CloudSolrClient extends SolrClient {
       Set<String> liveNodes = stateProvider.liveNodes();
       for (String liveNode : liveNodes) {
         theUrlList.add(ZkStateReader.getBaseUrlForNodeName(liveNode,
-            (String) stateProvider.getClusterProperties().getOrDefault(ZkStateReader.URL_SCHEME,"http")));
+            (String) stateProvider.getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
       }
     } else {
       
@@ -1366,7 +1385,7 @@ public class CloudSolrClient extends SolrClient {
     return rsp.getResponse();
   }
 
-  Set<String> getCollectionNames(String collection) {
+  private Set<String> getCollectionNames(String collection) {
     // Extract each comma separated collection name and store in a List.
     List<String> rawCollectionsList = StrUtils.splitSmart(collection, ",", true);
     Set<String> collectionNames = new HashSet<>();
@@ -1603,6 +1622,7 @@ public class CloudSolrClient extends SolrClient {
    */
   public static class Builder {
     private Collection<String> zkHosts;
+    private List<String> solrUrls;
     private HttpClient httpClient;
     private String zkChroot;
     private LBHttpSolrClient loadBalancedSolrClient;
@@ -1614,6 +1634,7 @@ public class CloudSolrClient extends SolrClient {
 
     public Builder() {
       this.zkHosts = new ArrayList();
+      this.solrUrls = new ArrayList();
       this.shardLeadersOnly = true;
     }
     
@@ -1630,8 +1651,29 @@ public class CloudSolrClient extends SolrClient {
       this.zkHosts.add(zkHost);
       return this;
     }
+
+    /**
+     * Provide a Solr URL to be used when configuring {@link CloudSolrClient} instances.
+     *
+     * Method may be called multiple times. One of the provided values will be used to fetch
+     * the list of live Solr nodes that the underlying {@link HttpClusterStateProvider} would be maintaining.
+     */
+    public Builder withSolrUrl(String solrUrl) {
+      this.solrUrls.add(solrUrl);
+      return this;
+    }
     
     /**
+     * Provide a list of Solr URL to be used when configuring {@link CloudSolrClient} instances.
+     * One of the provided values will be used to fetch the list of live Solr
+     * nodes that the underlying {@link HttpClusterStateProvider} would be maintaining.
+     */
+    public Builder withSolrUrl(Collection<String> solrUrls) {
+      this.solrUrls.addAll(solrUrls);
+      return this;
+    }
+
+    /**
      * Provides a {@link HttpClient} for the builder to use when creating clients.
      */
     public Builder withLBHttpSolrClientBuilder(LBHttpSolrClient.Builder lbHttpSolrClientBuilder) {
@@ -1723,25 +1765,22 @@ public class CloudSolrClient extends SolrClient {
      */
     public CloudSolrClient build() {
       if (stateProvider == null) {
-        stateProvider = new ZkClientClusterStateProvider(zkHosts, zkChroot);
+        if (!zkHosts.isEmpty()) {
+          stateProvider = new ZkClientClusterStateProvider(zkHosts, zkChroot);
+        }
+        else if (!this.solrUrls.isEmpty()) {
+          try {
+            stateProvider = new HttpClusterStateProvider(solrUrls, httpClient);
+          } catch (Exception e) {
+            throw new RuntimeException("Couldn't initialize a HttpClusterStateProvider (is/are the "
+                + "Solr server(s), "  + solrUrls + ", down?)", e);
+          }
+        } else {
+          throw new IllegalArgumentException("Both zkHosts and solrUrl cannot be null.");
+        }
       }
-      return new CloudSolrClient(zkHosts, zkChroot, httpClient, loadBalancedSolrClient, lbClientBuilder,
+      return new CloudSolrClient(zkHosts, zkChroot, solrUrls, httpClient, loadBalancedSolrClient, lbClientBuilder,
           shardLeadersOnly, directUpdatesToLeadersOnly, stateProvider);
     }
   }
-
-  public interface ClusterStateProvider extends Closeable {
-
-    ClusterState.CollectionRef getState(String collection);
-
-    Set<String> liveNodes();
-
-    String getAlias(String collection);
-
-    String getCollectionName(String name);
-
-    Map<String, Object> getClusterProperties();
-
-    void connect();
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/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
new file mode 100644
index 0000000..b913cd4
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java
@@ -0,0 +1,59 @@
+/*
+ * 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.impl;
+
+import java.io.Closeable;
+import java.util.Set;
+
+import org.apache.solr.common.cloud.ClusterState;
+
+public interface ClusterStateProvider extends Closeable {
+
+  /**
+   * Obtain the state of the collection (cluster status).
+   * @return the collection state, or null is collection doesn't exist
+   */
+  ClusterState.CollectionRef getState(String collection);
+
+  /**
+   * Obtain set of live_nodes for the cluster.
+   */
+  Set<String> liveNodes();
+
+  /**
+   * Given an alias, returns the collection name that this alias points to
+   */
+  String getAlias(String alias);
+
+  /**
+   * Given a name, returns the collection name if an alias by that name exists, or
+   * returns the name itself, if no alias exists.
+   */
+  String getCollectionName(String name);
+
+  /**
+   * Obtain a cluster property, or null if it doesn't exist.
+   */
+  Object getClusterProperty(String propertyName);
+
+  /**
+   * Obtain a cluster property, or the default value if it doesn't exist.
+   */
+  Object getClusterProperty(String propertyName, String def);
+
+  void connect();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/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
new file mode 100644
index 0000000..1fb9415
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
@@ -0,0 +1,261 @@
+/*
+ * 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.impl;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.http.client.HttpClient;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ClusterState.CollectionRef;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HttpClusterStateProvider implements ClusterStateProvider {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private String urlScheme;
+  volatile Set<String> liveNodes;
+  long liveNodesTimestamp = 0;
+  volatile Map<String, String> aliases;
+  long aliasesTimestamp = 0;
+
+  private int cacheTimeout = 5; // the liveNodes and aliases cache will be invalidated after 5 secs
+  final HttpClient httpClient;
+  final boolean clientIsInternal;
+
+  public HttpClusterStateProvider(List<String> solrUrls, HttpClient httpClient) throws Exception {
+    this.httpClient = httpClient == null? HttpClientUtil.createClient(null): httpClient;
+    this.clientIsInternal = httpClient == null;
+    for (String solrUrl: solrUrls) {
+      urlScheme = solrUrl.startsWith("https")? "https": "http";
+      try (SolrClient initialClient = new HttpSolrClient.Builder().withBaseSolrUrl(solrUrl).withHttpClient(httpClient).build()) {
+        Set<String> liveNodes = fetchLiveNodes(initialClient); // throws exception if unable to fetch
+        this.liveNodes = liveNodes;
+        liveNodesTimestamp = System.nanoTime();
+        break;
+      } catch (IOException e) {
+        log.warn("Attempt to fetch live_nodes from " + solrUrl + " failed.", e);
+      }
+    }
+
+    if (this.liveNodes == null || this.liveNodes.isEmpty()) {
+      throw new RuntimeException("Tried fetching live_nodes using Solr URLs provided, i.e. " + solrUrls + ". However, "
+          + "succeeded in obtaining the cluster state from none of them."
+          + "If you think your Solr cluster is up and is accessible,"
+          + " you could try re-creating a new CloudSolrClient using working"
+          + " solrUrl(s) or zkHost(s).");
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.clientIsInternal && this.httpClient != null) {
+      HttpClientUtil.close(httpClient);
+    }
+  }
+
+  @Override
+  public CollectionRef getState(String collection) {
+    for (String nodeName: liveNodes) {
+      try (HttpSolrClient client = new HttpSolrClient.Builder().
+          withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+          withHttpClient(httpClient).build()) {
+        ClusterState cs = fetchClusterState(client, collection);
+        return cs.getCollectionRef(collection);
+      } catch (SolrServerException | RemoteSolrException | IOException e) {
+        if (e.getMessage().contains(collection + " not found")) {
+          // Cluster state for the given collection was not found.
+          // Lets fetch/update our aliases:
+          getAliases(true);
+          return null;
+        }
+        log.warn("Attempt to fetch cluster state from " +
+            ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+      }
+    }
+    throw new RuntimeException("Tried fetching cluster state using the node names we knew of, i.e. " + liveNodes +". However, "
+        + "succeeded in obtaining the cluster state from none of them."
+        + "If you think your Solr cluster is up and is accessible,"
+        + " you could try re-creating a new CloudSolrClient using working"
+        + " solrUrl(s) or zkHost(s).");
+  }
+
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private ClusterState fetchClusterState(SolrClient client, String collection) throws SolrServerException, IOException {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("collection", collection);
+    params.set("action", "CLUSTERSTATUS");
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+    NamedList cluster = (SimpleOrderedMap) client.request(request).get("cluster");
+    Map<String, Object> collectionsMap = Collections.singletonMap(collection,
+        ((NamedList) cluster.get("collections")).get(collection));
+    int znodeVersion = (int)((Map<String, Object>)(collectionsMap).get(collection)).get("znodeVersion");
+    Set<String> liveNodes = new HashSet((List<String>)(cluster.get("live_nodes")));
+    this.liveNodes = liveNodes;
+    liveNodesTimestamp = System.nanoTime();
+    ClusterState cs = ClusterState.load(znodeVersion, collectionsMap, liveNodes, ZkStateReader.CLUSTER_STATE);
+    return cs;
+  }
+
+  @Override
+  public Set<String> liveNodes() {
+    if (liveNodes == null) {
+      throw new RuntimeException("We don't know of any live_nodes to fetch the"
+          + " latest live_nodes information from. "
+          + "If you think your Solr cluster is up and is accessible,"
+          + " you could try re-creating a new CloudSolrClient using working"
+          + " solrUrl(s) or zkHost(s).");
+    }
+    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)).
+            withHttpClient(httpClient).build()) {
+          Set<String> liveNodes = fetchLiveNodes(client);
+          this.liveNodes = (liveNodes);
+          liveNodesTimestamp = System.nanoTime();
+          return liveNodes;
+        } catch (Exception e) {
+          log.warn("Attempt to fetch live_nodes from " +
+              ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+        }
+      }
+      throw new RuntimeException("Tried fetching live_nodes using all the node names we knew of, i.e. " + liveNodes +". However, "
+          + "succeeded in obtaining the cluster state from none of them."
+          + "If you think your Solr cluster is up and is accessible,"
+          + " you could try re-creating a new CloudSolrClient using working"
+          + " solrUrl(s) or zkHost(s).");
+    } else {
+      return liveNodes; // cached copy is fresh enough
+    }
+  }
+
+  private static Set<String> fetchLiveNodes(SolrClient client) throws Exception {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", "CLUSTERSTATUS");
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+    NamedList cluster = (SimpleOrderedMap) client.request(request).get("cluster");
+    Set<String> liveNodes = new HashSet((List<String>)(cluster.get("live_nodes")));
+    return liveNodes;
+  }
+
+  @Override
+  public String getAlias(String alias) {
+    Map<String, String> aliases = getAliases(false);
+    return aliases.get(alias);
+  }
+
+  private Map<String, String> getAliases(boolean forceFetch) {
+    if (this.liveNodes == null) {
+      throw new RuntimeException("We don't know of any live_nodes to fetch the"
+          + " latest aliases information from. "
+          + "If you think your Solr cluster is up and is accessible,"
+          + " you could try re-creating a new CloudSolrClient using working"
+          + " solrUrl(s) or zkHost(s).");
+    }
+
+    if (forceFetch || this.aliases == null ||
+        TimeUnit.SECONDS.convert((System.nanoTime() - aliasesTimestamp), TimeUnit.NANOSECONDS) > getCacheTimeout()) {
+      for (String nodeName: liveNodes) {
+        try (HttpSolrClient client = new HttpSolrClient.Builder().
+            withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+            withHttpClient(httpClient).build()) {
+
+          Map<String, String> aliases = new CollectionAdminRequest.ListAliases().process(client).getAliases();
+          this.aliases = aliases;
+          this.aliasesTimestamp = System.nanoTime();
+          return Collections.unmodifiableMap(aliases);
+        } catch (SolrServerException | RemoteSolrException | IOException e) {
+          // Situation where we're hitting an older Solr which doesn't have LISTALIASES
+          if (e instanceof RemoteSolrException && ((RemoteSolrException)e).code()==400) {
+            log.warn("LISTALIASES not found, possibly using older Solr server. Aliases won't work"
+                + " unless you re-create the CloudSolrClient using zkHost(s) or upgrade Solr server", e);
+            this.aliases = Collections.emptyMap();
+            this.aliasesTimestamp = System.nanoTime();
+            return aliases;
+          }
+          log.warn("Attempt to fetch cluster state from " +
+              ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+        }
+      }
+
+      throw new RuntimeException("Tried fetching aliases using all the node names we knew of, i.e. " + liveNodes +". However, "
+          + "succeeded in obtaining the cluster state from none of them."
+          + "If you think your Solr cluster is up and is accessible,"
+          + " you could try re-creating a new CloudSolrClient using a working"
+          + " solrUrl or zkHost.");
+    } else {
+      return Collections.unmodifiableMap(this.aliases); // cached copy is fresh enough
+    }
+  }
+
+  @Override
+  public String getCollectionName(String name) {
+    Map<String, String> aliases = getAliases(false);
+    return aliases.containsKey(name) ? aliases.get(name): name;
+  }
+
+  @Override
+  public Object getClusterProperty(String propertyName) {
+    if (propertyName.equals(ZkStateReader.URL_SCHEME)) {
+      return this.urlScheme;
+    }
+    throw new UnsupportedOperationException("Fetching cluster properties not supported"
+        + " using the HttpClusterStateProvider. "
+        + "ZkClientClusterStateProvider can be used for this."); // TODO
+  }
+
+  @Override
+  public Object getClusterProperty(String propertyName, String def) {
+    if (propertyName.equals(ZkStateReader.URL_SCHEME)) {
+      return this.urlScheme;
+    }
+    throw new UnsupportedOperationException("Fetching cluster properties not supported"
+        + " using the HttpClusterStateProvider. "
+        + "ZkClientClusterStateProvider can be used for this."); // TODO
+  }
+
+  @Override
+  public void connect() {}
+
+  public int getCacheTimeout() {
+    return cacheTimeout;
+  }
+
+  public void setCacheTimeout(int cacheTimeout) {
+    this.cacheTimeout = cacheTimeout;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
index 3cb59cd..ed6ae7b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
@@ -360,10 +360,11 @@ public class LBHttpSolrClient extends SolrClient {
     boolean isNonRetryable = req.request instanceof IsUpdateRequest || ADMIN_PATHS.contains(req.request.getPath());
     List<ServerWrapper> skipped = null;
 
+    boolean timeAllowedExceeded = false;
     long timeAllowedNano = getTimeAllowedInNanos(req.getRequest());
     long timeOutTime = System.nanoTime() + timeAllowedNano;
     for (String serverStr : req.getServers()) {
-      if(isTimeExceeded(timeAllowedNano, timeOutTime)) {
+      if (timeAllowedExceeded = isTimeExceeded(timeAllowedNano, timeOutTime)) {
         break;
       }
       
@@ -400,22 +401,33 @@ public class LBHttpSolrClient extends SolrClient {
     // try the servers we previously skipped
     if (skipped != null) {
       for (ServerWrapper wrapper : skipped) {
-        if(isTimeExceeded(timeAllowedNano, timeOutTime)) {
+        if (timeAllowedExceeded = isTimeExceeded(timeAllowedNano, timeOutTime)) {
           break;
         }
 
-        ex = doRequest(wrapper.client, req, rsp, isNonRetryable, true, wrapper.getKey());
-        if (ex == null) {
-          return rsp; // SUCCESS
+        try {
+          MDC.put("LBHttpSolrClient.url", wrapper.client.getBaseURL());
+          ex = doRequest(wrapper.client, req, rsp, isNonRetryable, true, wrapper.getKey());
+          if (ex == null) {
+            return rsp; // SUCCESS
+          }
+        } finally {
+          MDC.remove("LBHttpSolrClient.url");
         }
       }
     }
 
 
+    final String solrServerExceptionMessage;
+    if (timeAllowedExceeded) {
+      solrServerExceptionMessage = "Time allowed to handle this request exceeded";
+    } else {
+      solrServerExceptionMessage = "No live SolrServers available to handle this request";
+    }
     if (ex == null) {
-      throw new SolrServerException("No live SolrServers available to handle this request");
+      throw new SolrServerException(solrServerExceptionMessage);
     } else {
-      throw new SolrServerException("No live SolrServers available to handle this request:" + zombieServers.keySet(), ex);
+      throw new SolrServerException(solrServerExceptionMessage+":" + zombieServers.keySet(), ex);
     }
 
   }
@@ -588,10 +600,11 @@ public class LBHttpSolrClient extends SolrClient {
     int maxTries = serverList.length;
     Map<String,ServerWrapper> justFailed = null;
 
+    boolean timeAllowedExceeded = false;
     long timeAllowedNano = getTimeAllowedInNanos(request);
     long timeOutTime = System.nanoTime() + timeAllowedNano;
     for (int attempts=0; attempts<maxTries; attempts++) {
-      if(isTimeExceeded(timeAllowedNano, timeOutTime)) {
+      if (timeAllowedExceeded = isTimeExceeded(timeAllowedNano, timeOutTime)) {
         break;
       }
       
@@ -619,7 +632,7 @@ public class LBHttpSolrClient extends SolrClient {
 
     // try other standard servers that we didn't try just now
     for (ServerWrapper wrapper : zombieServers.values()) {
-      if(isTimeExceeded(timeAllowedNano, timeOutTime)) {
+      if (timeAllowedExceeded = isTimeExceeded(timeAllowedNano, timeOutTime)) {
         break;
       }
       
@@ -646,10 +659,16 @@ public class LBHttpSolrClient extends SolrClient {
     }
 
 
+    final String solrServerExceptionMessage;
+    if (timeAllowedExceeded) {
+      solrServerExceptionMessage = "Time allowed to handle this request exceeded";
+    } else {
+      solrServerExceptionMessage = "No live SolrServers available to handle this request";
+    }
     if (ex == null) {
-      throw new SolrServerException("No live SolrServers available to handle this request");
+      throw new SolrServerException(solrServerExceptionMessage);
     } else {
-      throw new SolrServerException("No live SolrServers available to handle this request", ex);
+      throw new SolrServerException(solrServerExceptionMessage, ex);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
index 1e50127..1875c50 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
-public class ZkClientClusterStateProvider implements CloudSolrClient.ClusterStateProvider {
+public class ZkClientClusterStateProvider implements ClusterStateProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
 
@@ -72,14 +72,24 @@ public class ZkClientClusterStateProvider implements CloudSolrClient.ClusterStat
 
 
   @Override
-  public String getAlias(String collection) {
+  public String getAlias(String alias) {
     Aliases aliases = zkStateReader.getAliases();
-    return aliases.getCollectionAlias(collection);
+    return aliases.getCollectionAlias(alias);
   }
 
   @Override
-  public Map<String, Object> getClusterProperties() {
-    return zkStateReader.getClusterProperties();
+  public Object getClusterProperty(String propertyName) {
+    Map<String, Object> props = zkStateReader.getClusterProperties();
+    return props.get(propertyName);
+  }
+
+  @Override
+  public Object getClusterProperty(String propertyName, String def) {
+    Map<String, Object> props = zkStateReader.getClusterProperties();
+    if (props.containsKey(propertyName)) {
+      return props.get(propertyName);
+    }
+    return def;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConversionEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConversionEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConversionEvaluator.java
new file mode 100644
index 0000000..2849b49
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConversionEvaluator.java
@@ -0,0 +1,166 @@
+/*
+ * 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.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class ConversionEvaluator extends ComplexEvaluator {
+
+  enum LENGTH_CONSTANT {MILES, YARDS, FEET, INCHES, MILLIMETERS, CENTIMETERS, METERS, KILOMETERS};
+
+  private LENGTH_CONSTANT from;
+  private LENGTH_CONSTANT to;
+  private Convert convert;
+
+  public ConversionEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    if (3 != subEvaluators.size()) {
+      throw new EvaluatorException(new IOException(String.format(Locale.ROOT, "Invalid expression %s - expecting 3 value but found %d", expression, subEvaluators.size())));
+    }
+
+    try {
+      from = LENGTH_CONSTANT.valueOf(subEvaluators.get(0).toExpression(factory).toString().toUpperCase(Locale.ROOT));
+      to = LENGTH_CONSTANT.valueOf(subEvaluators.get(1).toExpression(factory).toString().toUpperCase(Locale.ROOT));
+      this.convert = getConvert(from, to);
+    } catch (IllegalArgumentException e) {
+      throw new EvaluatorException(e);
+    }
+  }
+
+  private String listParams() {
+    StringBuffer buf = new StringBuffer();
+    for(LENGTH_CONSTANT lc : LENGTH_CONSTANT.values()) {
+      if(buf.length() > 0) {
+        buf.append(", ");
+      }
+        buf.append(lc.toString());
+    }
+    return buf.toString();
+  }
+
+  @Override
+  public Object evaluate(Tuple tuple) throws IOException {
+
+    StreamEvaluator streamEvaluator = subEvaluators.get(2);
+    Object tupleValue = streamEvaluator.evaluate(tuple);
+
+    if (tupleValue == null) return null;
+
+    Number number = (Number)tupleValue;
+    double d = number.doubleValue();
+    return convert.convert(d);
+  }
+
+  private Convert getConvert(LENGTH_CONSTANT from, LENGTH_CONSTANT to) throws IOException {
+    switch(from) {
+      case INCHES:
+        switch(to) {
+          case MILLIMETERS:
+            return (double d) -> d*25.4;
+          case CENTIMETERS:
+            return (double d) -> d*2.54;
+          case METERS:
+            return (double d) -> d*0.0254;
+          default:
+            throw new EvaluatorException("No conversion available from "+from+" to "+to);
+        }
+      case FEET:
+        switch(to) {
+          case METERS:
+            return (double d) -> d * .30;
+        }
+      case YARDS:
+        switch(to) {
+          case METERS:
+            return (double d) -> d * .91;
+          case KILOMETERS:
+            return (double d) -> d * 0.00091;
+          default:
+            throw new EvaluatorException("No conversion available from "+from+" to "+to);
+        }
+      case MILES:
+        switch(to) {
+          case KILOMETERS:
+            return (double d) -> d * 1.61;
+          default:
+            throw new EvaluatorException("No conversion available from "+from+" to "+to);
+        }
+      case MILLIMETERS:
+        switch (to) {
+          case INCHES:
+            return (double d) -> d * 0.039;
+          default:
+            throw new EvaluatorException("No conversion available from "+from+" to "+to);
+        }
+      case CENTIMETERS:
+        switch(to) {
+          case INCHES:
+            return (double d) -> d * 0.39;
+          default:
+            throw new EvaluatorException("No conversion available from "+from+" to "+to);
+        }
+      case METERS:
+        switch(to) {
+          case FEET:
+            return (double d) -> d * 3.28;
+          default:
+            throw new EvaluatorException("No conversion available from "+from+" to "+to);
+        }
+      case KILOMETERS:
+        switch(to) {
+          case MILES:
+            return (double d) -> d * 0.62;
+          case FEET:
+            return (double d) -> d * 3280.8;
+        }
+      default:
+        throw new EvaluatorException("No conversion available from "+from);
+    }
+  }
+
+  private interface Convert {
+    public double convert(double d);
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    for (StreamEvaluator evaluator : subEvaluators) {
+      expression.addParameter(evaluator.toExpression(factory));
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(nodeId.toString())
+        .withExpressionType(Explanation.ExpressionType.EVALUATOR)
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EvaluatorException.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EvaluatorException.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EvaluatorException.java
new file mode 100644
index 0000000..d2098c2
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EvaluatorException.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+public class EvaluatorException extends IOException {
+  public EvaluatorException(Throwable t) {
+    super(t);
+  }
+
+  public EvaluatorException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
new file mode 100644
index 0000000..4af1f354
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
@@ -0,0 +1,136 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.TemporalAccessor;
+import java.time.temporal.UnsupportedTemporalTypeException;
+import java.util.Date;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * A generic date evaluator for use with a TemporalAccessor
+ */
+public abstract class TemporalEvaluator extends ComplexEvaluator {
+
+  private String field;
+
+  public TemporalEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    if (1 != subEvaluators.size()) {
+      throw new IOException(String.format(Locale.ROOT, "Invalid expression %s - expecting one value but found %d", expression, subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object evaluate(Tuple tuple) throws IOException {
+
+    Instant instant = null;
+    TemporalAccessor date = null;
+
+    //First evaluate the parameter
+    StreamEvaluator streamEvaluator = subEvaluators.get(0);
+    Object tupleValue = streamEvaluator.evaluate(tuple);
+
+    if (tupleValue == null) return null;
+
+    if(field == null) {
+      field = streamEvaluator.toExpression(constructingFactory).toString();
+    }
+
+    Map tupleContext = streamContext.getTupleContext();
+    date = (LocalDateTime)tupleContext.get(field); // Check to see if the date has already been created for this field
+
+    if(date == null) {
+      if (tupleValue instanceof String) {
+        instant = getInstant((String) tupleValue);
+      } else if (tupleValue instanceof Long) {
+        instant = Instant.ofEpochMilli((Long) tupleValue);
+      } else if (tupleValue instanceof Instant) {
+        instant = (Instant) tupleValue;
+      } else if (tupleValue instanceof Date) {
+        instant = ((Date) tupleValue).toInstant();
+      } else if (tupleValue instanceof TemporalAccessor) {
+        date = ((TemporalAccessor) tupleValue);
+        tupleContext.put(field, date); // Cache the date in the TupleContext
+      }
+    }
+
+    if (instant != null) {
+      if (TemporalEvaluatorEpoch.FUNCTION_NAME.equals(getFunction())) return instant.toEpochMilli();
+      date = LocalDateTime.ofInstant(instant, ZoneOffset.UTC);
+      tupleContext.put(field, date); // Cache the date in the TupleContext
+    }
+
+    if (date != null) {
+      try {
+        return evaluateDate(date);
+      } catch (UnsupportedTemporalTypeException utte) {
+        throw new IOException(String.format(Locale.ROOT, "It is not possible to call '%s' function on %s", getFunction(), date.getClass().getName()));
+      }
+    }
+
+    throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The parameter must be a string formatted ISO_INSTANT or of type Long,Instant,Date,LocalDateTime or TemporalAccessor.", String.valueOf(tupleValue)));
+  }
+
+  public abstract Object evaluateDate(TemporalAccessor aDate) throws IOException;
+  public abstract String getFunction();
+
+  protected Instant getInstant(String dateStr) throws IOException {
+
+    if (dateStr != null && !dateStr.isEmpty()) {
+      try {
+        return Instant.parse(dateStr);
+      } catch (DateTimeParseException e) {
+        throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The String must be formatted in the ISO_INSTANT date format.", dateStr));
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(getFunction());
+
+    for (StreamEvaluator evaluator : subEvaluators) {
+      expression.addParameter(evaluator.toExpression(factory));
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(nodeId.toString())
+        .withExpressionType(Explanation.ExpressionType.EVALUATOR)
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDay.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDay.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDay.java
new file mode 100644
index 0000000..88b5043
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDay.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a day stream evaluator
+ */
+public class TemporalEvaluatorDay extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "day";
+
+  public TemporalEvaluatorDay(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.DAY_OF_MONTH);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfQuarter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfQuarter.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfQuarter.java
new file mode 100644
index 0000000..f034377
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfQuarter.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a dayOfQuarter stream evaluator
+ */
+public class TemporalEvaluatorDayOfQuarter extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "dayOfQuarter";
+
+  public TemporalEvaluatorDayOfQuarter(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(IsoFields.DAY_OF_QUARTER);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfYear.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfYear.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfYear.java
new file mode 100644
index 0000000..339f938
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfYear.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a dayOfYear stream evaluator
+ */
+public class TemporalEvaluatorDayOfYear extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "dayOfYear";
+
+  public TemporalEvaluatorDayOfYear(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.DAY_OF_YEAR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
new file mode 100644
index 0000000..a8554b3
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
@@ -0,0 +1,53 @@
+/*
+ * 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.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.TemporalAccessor;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a epoch stream evaluator
+ */
+public class TemporalEvaluatorEpoch extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "epoch";
+
+  public TemporalEvaluatorEpoch(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) throws IOException {
+    if (aDate instanceof LocalDateTime) {
+      return ((LocalDateTime)aDate).atZone(ZoneOffset.UTC).toInstant().toEpochMilli();
+    }
+    throw new IOException(String.format(Locale.ROOT, "Unsupported function '%s' called on %s", FUNCTION_NAME, aDate.toString()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorHour.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorHour.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorHour.java
new file mode 100644
index 0000000..5480870
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorHour.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a hour stream evaluator
+ */
+public class TemporalEvaluatorHour extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "hour";
+
+  public TemporalEvaluatorHour(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.HOUR_OF_DAY);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMinute.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMinute.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMinute.java
new file mode 100644
index 0000000..9438e9d
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMinute.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a minute stream evaluator
+ */
+public class TemporalEvaluatorMinute extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "minute";
+
+  public TemporalEvaluatorMinute(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.MINUTE_OF_HOUR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMonth.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMonth.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMonth.java
new file mode 100644
index 0000000..b9a70fe
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMonth.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a month stream evaluator
+ */
+public class TemporalEvaluatorMonth extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "month";
+
+  public TemporalEvaluatorMonth(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.MONTH_OF_YEAR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorQuarter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorQuarter.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorQuarter.java
new file mode 100644
index 0000000..0144311
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorQuarter.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a quarter stream evaluator
+ */
+public class TemporalEvaluatorQuarter extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "quarter";
+
+  public TemporalEvaluatorQuarter(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(IsoFields.QUARTER_OF_YEAR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorSecond.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorSecond.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorSecond.java
new file mode 100644
index 0000000..f5b71fc
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorSecond.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a second stream evaluator
+ */
+public class TemporalEvaluatorSecond extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "second";
+
+  public TemporalEvaluatorSecond(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.SECOND_OF_MINUTE);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorWeek.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorWeek.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorWeek.java
new file mode 100644
index 0000000..1a2974a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorWeek.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a week stream evaluator
+ */
+public class TemporalEvaluatorWeek extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "week";
+
+  public TemporalEvaluatorWeek(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorYear.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorYear.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorYear.java
new file mode 100644
index 0000000..0b8d69c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorYear.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.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a year stream evaluator
+ */
+public class TemporalEvaluatorYear extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "year";
+
+  public TemporalEvaluatorYear(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.YEAR);
+  }
+
+}