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/01/02 10:18:22 UTC

[01/19] lucene-solr:jira/solr-9854: LUCENE-7595: Fix bug with RamUsageTester incorrectly handling Iterables outside Java Runtime

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-9854 2531fc8ae -> 4ca11b3ed


LUCENE-7595: Fix bug with RamUsageTester incorrectly handling Iterables outside Java Runtime


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

Branch: refs/heads/jira/solr-9854
Commit: db9190db9372ae88a7392a7186397441ce070a96
Parents: 3f24fd8
Author: Uwe Schindler <us...@apache.org>
Authored: Thu Dec 29 20:31:47 2016 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Thu Dec 29 20:31:47 2016 +0100

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/util/RamUsageTester.java            | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/db9190db/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
index daf81a9..6437d8e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
@@ -152,7 +152,7 @@ public final class RamUsageTester {
           }
           
           boolean needsReflection = true;
-          if (Constants.JRE_IS_MINIMUM_JAVA9) {
+          if (Constants.JRE_IS_MINIMUM_JAVA9 && obClazz.getName().startsWith("java.")) {
             // Java 9: Best guess for some known types, as we cannot precisely look into runtime classes:
             final ToLongFunction<Object> func = SIMPLE_TYPES.get(obClazz);
             if (func != null) { // some simple type like String where the size is easy to get from public properties


[07/19] lucene-solr:jira/solr-9854: SOLR-9900: fix false positives on range queries with ReversedWildcardFilterFactory

Posted by ab...@apache.org.
SOLR-9900: fix false positives on range queries with ReversedWildcardFilterFactory


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

Branch: refs/heads/jira/solr-9854
Commit: 5d042d3a49dfcf654b8bf8a96521d5404bfd3a7b
Parents: 6b00ee5
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Fri Dec 30 00:42:51 2016 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Fri Dec 30 00:43:29 2016 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                      |  2 ++
 .../org/apache/solr/parser/SolrQueryParserBase.java   | 14 ++++++++++++++
 .../analysis/TestReversedWildcardFilterFactory.java   | 12 ++++++++++++
 3 files changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5d042d3a/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b509e23..7f83de0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -298,6 +298,8 @@ Bug Fixes
 
 * SOLR-9901: Implement move in HdfsDirectoryFactory. (Mark Miller)
 
+* SOLR-9900: fix false positives on range queries with ReversedWildcardFilterFactory (Yonik Seeley via Mikhail Khludnev)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5d042d3a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
index 168bd49..f54e9e9 100644
--- a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
+++ b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.reverse.ReverseStringFilter;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.AutomatonQuery;
@@ -894,6 +895,19 @@ public abstract class SolrQueryParserBase extends QueryBuilder {
   protected Query getRangeQuery(String field, String part1, String part2, boolean startInclusive, boolean endInclusive) throws SyntaxError {
     checkNullField(field);
     SchemaField sf = schema.getField(field);
+
+    if (part1 == null) {
+      ReversedWildcardFilterFactory factory = getReversedWildcardFilterFactory(sf.getType());
+      if (factory != null) {
+        // There will be reversed tokens starting with u0001 that we want to exclude, so
+        // lets start at u0002 inclusive instead.
+        char[] buf = new char[1];
+        buf[0] = ReverseStringFilter.START_OF_HEADING_MARKER + 1;
+        part1 = new String(buf);
+        startInclusive = true;
+      }
+    }
+
     return sf.getType().getRangeQuery(parser, sf, part1, part2, startInclusive, endInclusive);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5d042d3a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java b/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
index 3ccc352..f7a49ac 100644
--- a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
+++ b/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
@@ -182,6 +182,18 @@ public class TestReversedWildcardFilterFactory extends SolrTestCaseJ4 {
     assertQ("false positive",
         req("+id:1 +one:*zemog*"),
         "//result[@numFound=0]");
+    
+    assertQ("no reverse, no false positive",
+        req("q", "+id:1 +three:[* TO a]", 
+            "debugQuery", "true"),
+        "//result[@numFound=0]");
+    {
+      String reverseField = random().nextBoolean() ? "one":"two";
+      assertQ("false positive",
+          req("q", "+id:1 +"+reverseField+":[* TO a]", 
+              "debugQuery", "true"),
+          "//result[@numFound=0]");
+    }
     assertQ("false positive",
         req("+id:1 +two:*zemog*"),
         "//result[@numFound=0]");


[08/19] lucene-solr:jira/solr-9854: LUCENE-7595: Disable another test not compatible with RamUsageTester

Posted by ab...@apache.org.
LUCENE-7595: Disable another test not compatible with RamUsageTester


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

Branch: refs/heads/jira/solr-9854
Commit: d65c02e8cc14f03389c2426ea3d3ddd75e12b1ec
Parents: db9190d
Author: Uwe Schindler <us...@apache.org>
Authored: Thu Dec 29 22:56:54 2016 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Thu Dec 29 22:56:54 2016 +0100

----------------------------------------------------------------------
 .../core/src/test/org/apache/lucene/search/TestLRUQueryCache.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d65c02e8/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index 9ebacf7..3acc3ea 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -381,6 +381,8 @@ public class TestLRUQueryCache extends LuceneTestCase {
   // by the cache itself, not cache entries, and we want to make sure that
   // memory usage is not grossly underestimated.
   public void testRamBytesUsedConstantEntryOverhead() throws IOException {
+    assumeFalse("LUCENE-7595: RamUsageTester does not work exact in Java 9 (estimations for maps and lists)", Constants.JRE_IS_MINIMUM_JAVA9);
+    
     final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true);
 
     final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {


[05/19] lucene-solr:jira/solr-9854: LUCENE-7564: Force single-threaded access to the AnalyzingInfixSuggester's SearcherManager when performing an acquire() or reassigning. This fixes failures in AnalyzingInfixSuggester.testRandomNRT().

Posted by ab...@apache.org.
LUCENE-7564: Force single-threaded access to the AnalyzingInfixSuggester's SearcherManager when performing an acquire() or reassigning.  This fixes failures in AnalyzingInfixSuggester.testRandomNRT().


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

Branch: refs/heads/jira/solr-9854
Commit: 93fdc20736d6e13736aceb091ab978bd8e03fcbb
Parents: a4335c0
Author: Steve Rowe <sa...@apache.org>
Authored: Thu Dec 29 15:51:37 2016 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Thu Dec 29 15:51:37 2016 -0500

----------------------------------------------------------------------
 .../analyzing/AnalyzingInfixSuggester.java      | 136 +++++++++++--------
 1 file changed, 81 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93fdc207/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
index b8c2dbd..2fbe4a8 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
@@ -136,6 +136,8 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
 
   /** {@link IndexSearcher} used for lookups. */
   protected SearcherManager searcherMgr;
+  
+  protected final Object searcherMgrLock = new Object();
 
   /** Default minimum number of leading characters before
    *  PrefixQuery is used (4). */
@@ -275,53 +277,55 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
   @Override
   public void build(InputIterator iter) throws IOException {
     
-    if (searcherMgr != null) {
-      searcherMgr.close();
-      searcherMgr = null;
-    }
+    synchronized (searcherMgrLock) {
+      if (searcherMgr != null) {
+        searcherMgr.close();
+        searcherMgr = null;
+      }
 
-    if (writer != null) {
-      writer.close();
-      writer = null;
-    }
+      if (writer != null) {
+        writer.close();
+        writer = null;
+      }
 
-    boolean success = false;
-    try {
-      // First pass: build a temporary normal Lucene index,
-      // just indexing the suggestions as they iterate:
-      writer = new IndexWriter(dir,
-                               getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
-      //long t0 = System.nanoTime();
-
-      // TODO: use threads?
-      BytesRef text;
-      while ((text = iter.next()) != null) {
-        BytesRef payload;
-        if (iter.hasPayloads()) {
-          payload = iter.payload();
-        } else {
-          payload = null;
-        }
+      boolean success = false;
+      try {
+        // First pass: build a temporary normal Lucene index,
+        // just indexing the suggestions as they iterate:
+        writer = new IndexWriter(dir,
+            getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
+        //long t0 = System.nanoTime();
+
+        // TODO: use threads?
+        BytesRef text;
+        while ((text = iter.next()) != null) {
+          BytesRef payload;
+          if (iter.hasPayloads()) {
+            payload = iter.payload();
+          } else {
+            payload = null;
+          }
 
-        add(text, iter.contexts(), iter.weight(), payload);
-      }
+          add(text, iter.contexts(), iter.weight(), payload);
+        }
 
-      //System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
-      if (commitOnBuild || closeIndexWriterOnBuild) {
-        commit();
-      }
-      searcherMgr = new SearcherManager(writer, null);
-      success = true;
-    } finally {
-      if (success) {
-        if (closeIndexWriterOnBuild) {
-          writer.close();
-          writer = null;
+        //System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
+        if (commitOnBuild || closeIndexWriterOnBuild) {
+          commit();
         }
-      } else {  // failure
-        if (writer != null) {
-          writer.rollback();
-          writer = null;
+        searcherMgr = new SearcherManager(writer, null);
+        success = true;
+      } finally {
+        if (success) {
+          if (closeIndexWriterOnBuild) {
+            writer.close();
+            writer = null;
+          }
+        } else {  // failure
+          if (writer != null) {
+            writer.rollback();
+            writer = null;
+          }
         }
       }
     }
@@ -369,10 +373,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
       } else {
         writer = new IndexWriter(dir, getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
       }
-      SearcherManager oldSearcherMgr = searcherMgr;
-      searcherMgr = new SearcherManager(writer, null);
-      if (oldSearcherMgr != null) {
-        oldSearcherMgr.close();
+      synchronized (searcherMgrLock) {
+        SearcherManager oldSearcherMgr = searcherMgr;
+        searcherMgr = new SearcherManager(writer, null);
+        if (oldSearcherMgr != null) {
+          oldSearcherMgr.close();
+        }
       }
     }
   }
@@ -642,7 +648,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     // only retrieve the first num hits now:
     Collector c2 = new EarlyTerminatingSortingCollector(c, SORT, num);
     List<LookupResult> results = null;
-    IndexSearcher searcher = searcherMgr.acquire();
+    SearcherManager mgr;
+    IndexSearcher searcher;
+    synchronized (searcherMgrLock) {
+      mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
+      searcher = mgr.acquire();
+    }
     try {
       //System.out.println("got searcher=" + searcher);
       searcher.search(finalQuery, c2);
@@ -653,7 +664,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
       // hits = searcher.search(query, null, num, SORT);
       results = createResults(searcher, hits, num, key, doHighlight, matchedTokens, prefixToken);
     } finally {
-      searcherMgr.release(searcher);
+      mgr.release(searcher);
     }
 
     //System.out.println((System.currentTimeMillis() - t0) + " msec for infix suggest");
@@ -853,7 +864,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     long mem = RamUsageEstimator.shallowSizeOf(this);
     try {
       if (searcherMgr != null) {
-        IndexSearcher searcher = searcherMgr.acquire();
+        SearcherManager mgr;
+        IndexSearcher searcher;
+        synchronized (searcherMgrLock) {
+          mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
+          searcher = mgr.acquire();
+        }
         try {
           for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
             LeafReader reader = FilterLeafReader.unwrap(context.reader());
@@ -862,7 +878,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
             }
           }
         } finally {
-          searcherMgr.release(searcher);
+          mgr.release(searcher);
         }
       }
       return mem;
@@ -876,7 +892,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     List<Accountable> resources = new ArrayList<>();
     try {
       if (searcherMgr != null) {
-        IndexSearcher searcher = searcherMgr.acquire();
+        SearcherManager mgr;
+        IndexSearcher searcher;
+        synchronized (searcherMgrLock) {
+          mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
+          searcher = mgr.acquire();
+        }
         try {
           for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
             LeafReader reader = FilterLeafReader.unwrap(context.reader());
@@ -885,7 +906,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
             }
           }
         } finally {
-          searcherMgr.release(searcher);
+          mgr.release(searcher);
         }
       }
       return Collections.unmodifiableList(resources);
@@ -899,11 +920,16 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     if (searcherMgr == null) {
       return 0;
     }
-    IndexSearcher searcher = searcherMgr.acquire();
+    SearcherManager mgr;
+    IndexSearcher searcher;
+    synchronized (searcherMgrLock) {
+      mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
+      searcher = mgr.acquire();
+    }
     try {
       return searcher.getIndexReader().numDocs();
     } finally {
-      searcherMgr.release(searcher);
+      mgr.release(searcher);
     }
   }
-};
+}


[03/19] lucene-solr:jira/solr-9854: SOLR-9905: Update CHANGES.txt

Posted by ab...@apache.org.
SOLR-9905: Update CHANGES.txt


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

Branch: refs/heads/jira/solr-9854
Commit: 00723827ff5ad5c129d3d8487d2c64469ea03239
Parents: 7dcb557
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Dec 29 14:42:31 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Dec 29 14:42:31 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00723827/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1383859..b509e23 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -202,6 +202,8 @@ New Features
 * SOLR-9897: Add hl.requireFieldMatch toggle support when using the UnifiedHighlighter.  Defaults to false like the
   other highlighters that support this. (David Smiley)
 
+* SOLR-9905: Add NullStream to isolate the performance of the ExportWriter (Joel Bernstein)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have


[11/19] lucene-solr:jira/solr-9854: SOLR-9843 Fix up DocValuesNotIndexedTest failures (cherry picked from commit f6a3557)

Posted by ab...@apache.org.
SOLR-9843 Fix up DocValuesNotIndexedTest failures
(cherry picked from commit f6a3557)


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

Branch: refs/heads/jira/solr-9854
Commit: 3ccd15a7658ad2821e8a2d2916781265db6f3afe
Parents: cb266d5
Author: Erick Erickson <er...@apache.org>
Authored: Thu Dec 29 18:10:34 2016 -0800
Committer: Erick Erickson <er...@apache.org>
Committed: Thu Dec 29 18:14:39 2016 -0800

----------------------------------------------------------------------
 .../org/apache/solr/cloud/DocValuesNotIndexedTest.java    | 10 ----------
 1 file changed, 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ccd15a7/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
index f5257f8..be9f9a3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
@@ -256,10 +256,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
       solrQuery.addSort("id", SolrQuery.ORDER.asc);
       final QueryResponse rsp = client.query(COLLECTION, solrQuery);
       SolrDocumentList res = rsp.getResults();
-      //TODO remove after SOLR-9843
-      if (order.length != res.getNumFound()) {
-        log.error("(3) About to fail, response is: " + rsp.toString());
-      }
       assertEquals("Should have exactly " + order.length + " documents returned", order.length, res.getNumFound());
       String expected;
       for (int idx = 0; idx < res.size(); ++idx) {
@@ -305,10 +301,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
       if (prop.getName().startsWith("bool")) expected = 3; //true, false and null
 
       List<Group> fieldCommandGroups = fieldCommand.getValues();
-      //TODO: remove me since this is excessive in the normal case, this is in for SOLR-9843
-      if (expected != fieldCommandGroups.size()) {
-        log.error("(1) About to fail assert, response is: " + rsp.toString());
-      }
       assertEquals("Did not find the expected number of groups for field " + prop.getName(), expected, fieldCommandGroups.size());
     }
   }
@@ -381,8 +373,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
               break;
             
             default:
-              //TODO remove me after SOLR-9843
-              log.error("(2) About to fail, response is: " + rsp.toString());
               fail("Unexpected number of elements in the group for " + prop.getName() + ": " + grp.getResult().size());
           }
         }


[14/19] lucene-solr:jira/solr-9854: SOLR-9684: Add schedule Streaming Expression

Posted by ab...@apache.org.
SOLR-9684: Add schedule Streaming Expression


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

Branch: refs/heads/jira/solr-9854
Commit: f3fe487970f1e21300bd556d226461a2d51b00f9
Parents: cc862d8
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 30 14:34:00 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 30 14:34:16 2016 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   1 +
 .../client/solrj/io/stream/SchedulerStream.java | 161 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   | 149 +++++++++++++++++
 3 files changed, 311 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3fe4879/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 13ce636..1610fea 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -140,6 +140,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("fetch", FetchStream.class)
       .withFunctionName("executor", ExecutorStream.class)
       .withFunctionName("null", NullStream.class)
+      .withFunctionName("schedule", SchedulerStream.class)
       // metrics
       .withFunctionName("min", MinMetric.class)
       .withFunctionName("max", MaxMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3fe4879/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java
new file mode 100644
index 0000000..f8506b9
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.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.client.solrj.io.stream;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.comp.StreamComparator;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The scheduler wraps two topics that represent high priority and low priority task queues.
+ * Each time the scheduler is called it will check to see if there are any high priority tasks in the queue. If there
+ * are high priority tasks, then the high priority queue will be read until it returns the EOF Tuple.
+ *
+ * If there are no tasks in the high priority queue, then the lower priority task queue will be opened and read until the EOF Tuple is
+ * returned.
+ *
+ * The scheduler is designed to be wrapped by the executor function and a daemon function can be used to call the executor iteratively.
+ **/
+
+public class SchedulerStream extends TupleStream implements Expressible {
+
+  private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private PushBackStream highPriorityTasks;
+  private PushBackStream tasks;
+  private TupleStream currentStream;
+
+  public SchedulerStream(StreamExpression expression, StreamFactory factory) throws IOException {
+    // grab all parameters out
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+
+
+    if(2 != streamExpressions.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
+    }
+
+    TupleStream stream1 = factory.constructStream(streamExpressions.get(0));
+    TupleStream stream2 = factory.constructStream(streamExpressions.get(1));
+
+    if(!(stream1 instanceof TopicStream) || !(stream2 instanceof TopicStream)) {
+      throw new IOException("The scheduler expects both stream parameters to be topics.");
+    }
+
+    init(new PushBackStream(stream1), new PushBackStream(stream2));
+  }
+
+  private void init(PushBackStream stream1, PushBackStream stream2) throws IOException{
+    this.highPriorityTasks = stream1;
+    this.tasks = stream2;
+  }
+
+  @Override
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    return toExpression(factory, true);
+  }
+
+  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
+
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    // stream
+    if(includeStreams) {
+      if (highPriorityTasks instanceof Expressible) {
+        expression.addParameter(((Expressible) highPriorityTasks).toExpression(factory));
+      } else {
+        throw new IOException("The SchedulerStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+
+      if (tasks instanceof Expressible) {
+        expression.addParameter(((Expressible) tasks).toExpression(factory));
+      } else {
+        throw new IOException("The SchedulerStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    return new StreamExplanation(getStreamNodeId().toString())
+        .withChildren(new Explanation[]{
+            highPriorityTasks.toExplanation(factory), tasks.toExplanation(factory)
+        })
+        .withFunctionName(factory.getFunctionName(this.getClass()))
+        .withImplementingClass(this.getClass().getName())
+        .withExpressionType(ExpressionType.STREAM_DECORATOR)
+        .withExpression(toExpression(factory, false).toString());
+  }
+
+  public void setStreamContext(StreamContext streamContext) {
+    this.highPriorityTasks.setStreamContext(streamContext);
+    tasks.setStreamContext(streamContext);
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList();
+    l.add(highPriorityTasks);
+    l.add(tasks);
+    return l;
+  }
+
+  public void open() throws IOException {
+    highPriorityTasks.open();
+    Tuple tuple = highPriorityTasks.read();
+    if(tuple.EOF) {
+      highPriorityTasks.close();
+      tasks.open();
+      currentStream = tasks;
+    } else {
+      highPriorityTasks.pushBack(tuple);
+      currentStream = highPriorityTasks;
+    }
+  }
+
+  public void close() throws IOException {
+      currentStream.close();
+  }
+
+  public Tuple read() throws IOException {
+    return currentStream.read();
+  }
+
+  public StreamComparator getStreamSort(){
+    return null;
+  }
+
+  public int getCost() {
+    return 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3fe4879/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 0c9d5b3..3bfe129 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
@@ -2816,6 +2816,155 @@ public class StreamExpressionTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testSchedulerStream() throws Exception {
+    Assume.assumeTrue(!useAlias);
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello1", "a_i", "0", "a_f", "1")
+        .add(id, "2", "a_s", "hello1", "a_i", "2", "a_f", "2")
+        .add(id, "3", "a_s", "hello1", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello1", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "5")
+        .add(id, "5", "a_s", "hello", "a_i", "10", "a_f", "6")
+        .add(id, "6", "a_s", "hello", "a_i", "11", "a_f", "7")
+        .add(id, "7", "a_s", "hello", "a_i", "12", "a_f", "8")
+        .add(id, "8", "a_s", "hello", "a_i", "13", "a_f", "9")
+        .add(id, "9", "a_s", "hello1", "a_i", "14", "a_f", "10")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
+        .withFunctionName("topic", TopicStream.class)
+        .withFunctionName("schedule", SchedulerStream.class);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    SolrClientCache cache = new SolrClientCache();
+
+    try {
+      FieldComparator comp = new FieldComparator("a_i", ComparatorOrder.ASCENDING);
+
+      expression = StreamExpressionParser.parse("schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0))");
+      stream = factory.constructStream(expression);
+      StreamContext context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+
+      Collections.sort(tuples, comp);
+      //The tuples from the first topic (high priority) should be returned.
+
+      assertEquals(tuples.size(), 4);
+      assertOrder(tuples, 5, 6, 7, 8);
+
+      expression = StreamExpressionParser.parse("schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0))");
+      stream = factory.constructStream(expression);
+      context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+      Collections.sort(tuples, comp);
+
+      //The Tuples from the second topic (Low priority) should be returned.
+      assertEquals(tuples.size(), 6);
+      assertOrder(tuples, 0, 1, 2, 3, 4, 9);
+
+      expression = StreamExpressionParser.parse("schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0))");
+      stream = factory.constructStream(expression);
+      context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+
+      //Both queus are empty.
+      assertEquals(tuples.size(), 0);
+
+    } finally {
+      cache.close();
+    }
+  }
+
+  @Test
+  public void testParallelSchedulerStream() throws Exception {
+    Assume.assumeTrue(!useAlias);
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello1", "a_i", "0", "a_f", "1")
+        .add(id, "2", "a_s", "hello1", "a_i", "2", "a_f", "2")
+        .add(id, "3", "a_s", "hello1", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello1", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "5")
+        .add(id, "5", "a_s", "hello", "a_i", "10", "a_f", "6")
+        .add(id, "6", "a_s", "hello", "a_i", "11", "a_f", "7")
+        .add(id, "7", "a_s", "hello", "a_i", "12", "a_f", "8")
+        .add(id, "8", "a_s", "hello", "a_i", "13", "a_f", "9")
+        .add(id, "9", "a_s", "hello1", "a_i", "14", "a_f", "10")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
+        .withFunctionName("topic", TopicStream.class)
+        .withFunctionName("parallel", ParallelStream.class)
+        .withFunctionName("schedule", SchedulerStream.class);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    SolrClientCache cache = new SolrClientCache();
+
+    try {
+      FieldComparator comp = new FieldComparator("a_i", ComparatorOrder.ASCENDING);
+
+      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0, partitionKeys=id)))");
+      stream = factory.constructStream(expression);
+      StreamContext context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+
+      Collections.sort(tuples, comp);
+      //The tuples from the first topic (high priority) should be returned.
+
+      assertEquals(tuples.size(), 4);
+      assertOrder(tuples, 5, 6, 7, 8);
+
+      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0, partitionKeys=id)))");
+      stream = factory.constructStream(expression);
+      context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+      Collections.sort(tuples, comp);
+
+      //The Tuples from the second topic (Low priority) should be returned.
+      assertEquals(tuples.size(), 6);
+      assertOrder(tuples, 0, 1, 2, 3, 4, 9);
+
+      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0, partitionKeys=id)))");
+      stream = factory.constructStream(expression);
+      context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+
+      //Both queus are empty.
+      assertEquals(tuples.size(), 0);
+
+    } finally {
+      cache.close();
+    }
+  }
+
+  @Test
   public void testParallelTopicStream() throws Exception {
 
     Assume.assumeTrue(!useAlias);


[13/19] lucene-solr:jira/solr-9854: SOLR-9668: introduce cursorMark='true' for SolrEntityProcessor

Posted by ab...@apache.org.
SOLR-9668: introduce cursorMark='true' for SolrEntityProcessor


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

Branch: refs/heads/jira/solr-9854
Commit: cc862d8e67f32d5447599d265f5d126541ed92c9
Parents: 26ee8e9
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Tue Dec 27 15:34:12 2016 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Fri Dec 30 20:50:53 2016 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../handler/dataimport/SolrEntityProcessor.java | 192 ++++++++++++-------
 .../dataimport/MockSolrEntityProcessor.java     |  18 +-
 .../TestSolrEntityProcessorEndToEnd.java        |  27 ++-
 .../dataimport/TestSolrEntityProcessorUnit.java |  70 +++++++
 5 files changed, 234 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 59dde90..874ac81 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -206,6 +206,8 @@ New Features
 
 * SOLR-9891: Add mkroot command to bin/solr and bin/solr.cmd (Erick Erickson)
 
+* SOLR-9668,SOLR-7197: introduce cursorMark='true' in SolrEntityProcessor (Yegor Kozlov, Raveendra Yerraguntl via Mikhail Khludnev)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java
index 5e62731..6d8726f 100644
--- a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java
+++ b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java
@@ -16,6 +16,18 @@
  */
 package org.apache.solr.handler.dataimport;
 
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
 import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
@@ -27,22 +39,12 @@ import org.apache.solr.client.solrj.impl.XMLResponseParser;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.CursorMarkParams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
-import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
-
 /**
  * <p>
  * An implementation of {@link EntityProcessor} which fetches values from a
@@ -139,81 +141,53 @@ public class SolrEntityProcessor extends EntityProcessorBase {
    * The following method changes the rowIterator mutable field. It requires
    * external synchronization. 
    */
-  private void buildIterator() {
+  protected void buildIterator() {
     if (rowIterator != null)  {
       SolrDocumentListIterator documentListIterator = (SolrDocumentListIterator) rowIterator;
       if (!documentListIterator.hasNext() && documentListIterator.hasMoreRows()) {
-        SolrDocumentList solrDocumentList = doQuery(documentListIterator
-            .getStart() + documentListIterator.getSize());
-        if (solrDocumentList != null) {
-          rowIterator = new SolrDocumentListIterator(solrDocumentList);
-        }
+        nextPage();
       }
-    } else  {
-      SolrDocumentList solrDocumentList = doQuery(0);
-      if (solrDocumentList != null) {
-        rowIterator = new SolrDocumentListIterator(solrDocumentList);
-      }
-      return;
+    } else {
+      Boolean cursor = new Boolean(context
+          .getResolvedEntityAttribute(CursorMarkParams.CURSOR_MARK_PARAM));
+      rowIterator = !cursor ? new SolrDocumentListIterator(new SolrDocumentList())
+          : new SolrDocumentListCursor(new SolrDocumentList(), CursorMarkParams.CURSOR_MARK_START);
+      nextPage();
     }
   }
   
-  protected SolrDocumentList doQuery(int start) {
-    this.queryString = context.getResolvedEntityAttribute(QUERY);
-    if (this.queryString == null) {
-      throw new DataImportHandlerException(
-          DataImportHandlerException.SEVERE,
-          "SolrEntityProcessor: parameter 'query' is required"
-      );
-    }
-
-    String rowsP = context.getResolvedEntityAttribute(CommonParams.ROWS);
-    if (rowsP != null) {
-      rows = Integer.parseInt(rowsP);
-    }
+  protected void nextPage() {
+    ((SolrDocumentListIterator)rowIterator).doQuery();
+  }
 
-    String fqAsString = context.getResolvedEntityAttribute(CommonParams.FQ);
-    if (fqAsString != null) {
-      this.filterQueries = fqAsString.split(",");
-    }
+  class SolrDocumentListCursor extends SolrDocumentListIterator {
+    
+    private final String cursorMark;
 
-    String fieldsAsString = context.getResolvedEntityAttribute(CommonParams.FL);
-    if (fieldsAsString != null) {
-      this.fields = fieldsAsString.split(",");
-    }
-    this.requestHandler = context.getResolvedEntityAttribute(CommonParams.QT);
-    String timeoutAsString = context.getResolvedEntityAttribute(TIMEOUT);
-    if (timeoutAsString != null) {
-      this.timeout = Integer.parseInt(timeoutAsString);
+    public SolrDocumentListCursor(SolrDocumentList solrDocumentList, String cursorMark) {
+      super(solrDocumentList);
+      this.cursorMark = cursorMark;
     }
 
-    SolrQuery solrQuery = new SolrQuery(queryString);
-    solrQuery.setRows(rows);
-    solrQuery.setStart(start);
-    if (fields != null) {
-      for (String field : fields) {
-        solrQuery.addField(field);
+    @Override
+    protected void passNextPage(SolrQuery solrQuery) {
+      String timeoutAsString = context.getResolvedEntityAttribute(TIMEOUT);
+      if (timeoutAsString != null) {
+        throw new DataImportHandlerException(SEVERE,"cursorMark can't be used with timeout");
       }
+      
+      solrQuery.set(CursorMarkParams.CURSOR_MARK_PARAM, cursorMark);
     }
-    solrQuery.setRequestHandler(requestHandler);
-    solrQuery.setFilterQueries(filterQueries);
-    solrQuery.setTimeAllowed(timeout * 1000);
     
-    QueryResponse response = null;
-    try {
-      response = solrClient.query(solrQuery);
-    } catch (SolrServerException | IOException e) {
-      if (ABORT.equals(onError)) {
-        wrapAndThrow(SEVERE, e);
-      } else if (SKIP.equals(onError)) {
-        wrapAndThrow(DataImportHandlerException.SKIP_ROW, e);
-      }
+    @Override
+    protected Iterator<Map<String,Object>> createNextPageIterator(QueryResponse response) {
+      return
+          new SolrDocumentListCursor(response.getResults(),
+              response.getNextCursorMark()) ;
     }
-    
-    return response == null ? null : response.getResults();
   }
   
-  private static class SolrDocumentListIterator implements Iterator<Map<String,Object>> {
+  class SolrDocumentListIterator implements Iterator<Map<String,Object>> {
     
     private final int start;
     private final int size;
@@ -230,6 +204,84 @@ public class SolrEntityProcessor extends EntityProcessorBase {
       this.size = solrDocumentList.size();
     }
 
+    protected QueryResponse doQuery() {
+      SolrEntityProcessor.this.queryString = context.getResolvedEntityAttribute(QUERY);
+      if (SolrEntityProcessor.this.queryString == null) {
+        throw new DataImportHandlerException(
+            DataImportHandlerException.SEVERE,
+            "SolrEntityProcessor: parameter 'query' is required"
+        );
+      }
+
+      String rowsP = context.getResolvedEntityAttribute(CommonParams.ROWS);
+      if (rowsP != null) {
+        rows = Integer.parseInt(rowsP);
+      }
+
+      String sortParam = context.getResolvedEntityAttribute(CommonParams.SORT);
+      
+      String fqAsString = context.getResolvedEntityAttribute(CommonParams.FQ);
+      if (fqAsString != null) {
+        SolrEntityProcessor.this.filterQueries = fqAsString.split(",");
+      }
+
+      String fieldsAsString = context.getResolvedEntityAttribute(CommonParams.FL);
+      if (fieldsAsString != null) {
+        SolrEntityProcessor.this.fields = fieldsAsString.split(",");
+      }
+      SolrEntityProcessor.this.requestHandler = context.getResolvedEntityAttribute(CommonParams.QT);
+     
+
+      SolrQuery solrQuery = new SolrQuery(queryString);
+      solrQuery.setRows(rows);
+      
+      if (sortParam!=null) {
+        solrQuery.setParam(CommonParams.SORT, sortParam);
+      }
+      
+      passNextPage(solrQuery);
+      
+      if (fields != null) {
+        for (String field : fields) {
+          solrQuery.addField(field);
+        }
+      }
+      solrQuery.setRequestHandler(requestHandler);
+      solrQuery.setFilterQueries(filterQueries);
+      
+      
+      QueryResponse response = null;
+      try {
+        response = solrClient.query(solrQuery);
+      } catch (SolrServerException | IOException | SolrException e) {
+        if (ABORT.equals(onError)) {
+          wrapAndThrow(SEVERE, e);
+        } else if (SKIP.equals(onError)) {
+          wrapAndThrow(DataImportHandlerException.SKIP_ROW, e);
+        }
+      }
+      
+      if (response != null) {
+        SolrEntityProcessor.this.rowIterator = createNextPageIterator(response);
+      }
+      return response;
+    }
+
+    protected Iterator<Map<String,Object>> createNextPageIterator(QueryResponse response) {
+      return new SolrDocumentListIterator(response.getResults());
+    }
+
+    protected void passNextPage(SolrQuery solrQuery) {
+      String timeoutAsString = context.getResolvedEntityAttribute(TIMEOUT);
+      if (timeoutAsString != null) {
+        SolrEntityProcessor.this.timeout = Integer.parseInt(timeoutAsString);
+      }
+      
+      solrQuery.setTimeAllowed(timeout * 1000);
+      
+      solrQuery.setStart(getStart() + getSize());
+    }
+    
     @Override
     public boolean hasNext() {
       return solrDocumentIterator.hasNext();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java
index 4ebca30..42e5f7d 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java
@@ -29,16 +29,28 @@ public class MockSolrEntityProcessor extends SolrEntityProcessor {
   private int queryCount = 0;
 
   private int rows;
+  
+  private int start = 0;
 
   public MockSolrEntityProcessor(List<SolrTestCaseJ4.Doc> docsData, int rows) {
     this.docsData = docsData;
     this.rows = rows;
   }
 
+  //@Override
+  //protected SolrDocumentList doQuery(int start) {
+  //  queryCount++;
+  //  return getDocs(start, rows);
+ // }
+  
   @Override
-  protected SolrDocumentList doQuery(int start) {
-    queryCount++;
-    return getDocs(start, rows);
+  protected void buildIterator() {
+    if (rowIterator==null || (!rowIterator.hasNext() && ((SolrDocumentListIterator)rowIterator).hasMoreRows())){
+      queryCount++;
+      SolrDocumentList docs = getDocs(start, rows);
+      rowIterator = new SolrDocumentListIterator(docs);
+      start += docs.size();
+    }
   }
 
   private SolrDocumentList getDocs(int start, int rows) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
index 8ef94c0..9e104ee 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
@@ -34,6 +34,8 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Files;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -179,7 +181,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
     
     try {
       addDocumentsToSolr(generateSolrDocuments(7));
-      runFullImport(generateDIHConfig("query='*:*' fl='id' rows='2'", false));
+      runFullImport(generateDIHConfig("query='*:*' fl='id' rows='2'"+(random().nextBoolean() ?" cursorMark='true' sort='id asc'":""), false));
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
       fail(e.getMessage());
@@ -252,7 +254,8 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
     assertQ(req("*:*"), "//result[@numFound='0']");
     
     try {
-      runFullImport(generateDIHConfig("query='bogus:3' rows='2' fl='id,desc' onError='abort'", false));
+      runFullImport(generateDIHConfig("query='bogus:3' rows='2' fl='id,desc' onError='"+
+            (random().nextBoolean() ? "abort" : "justtogetcoverage")+"'", false));
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
       fail(e.getMessage());
@@ -260,7 +263,27 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
     
     assertQ(req("*:*"), "//result[@numFound='0']");
   }
+  
+  public void testCursorMarkNoSort() throws SolrServerException, IOException {
+    assertQ(req("*:*"), "//result[@numFound='0']");
+    addDocumentsToSolr(generateSolrDocuments(7));
+    try {     
+      List<String> errors = Arrays.asList("sort='id'", //wrong sort spec
+          "", //no sort spec
+          "sort='id asc' timeout='12345'"); // sort is fine, but set timeout
+      Collections.shuffle(errors, random());
+      String attrs = "query='*:*' rows='2' fl='id,desc' cursorMark='true' "
+                                                            + errors.get(0);
+      runFullImport(generateDIHConfig(attrs,
+            false));
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      fail(e.getMessage());
+    }
     
+    assertQ(req("*:*"), "//result[@numFound='0']");
+  }
+  
   private static List<Map<String,Object>> generateSolrDocuments(int num) {
     List<Map<String,Object>> docList = new ArrayList<>();
     for (int i = 1; i <= num; i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java
index a8fcbb1..a2a9fff 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java
@@ -18,11 +18,23 @@ package org.apache.solr.handler.dataimport;
 
 import java.util.*;
 
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.CursorMarkParams;
+import org.apache.solr.handler.dataimport.SolrEntityProcessor.SolrDocumentListIterator;
+import org.junit.Test;
+
 /**
  * Unit test of SolrEntityProcessor. A very basic test outside of the DIH.
  */
 public class TestSolrEntityProcessorUnit extends AbstractDataImportHandlerTestCase {
 
+  private static final class NoNextMockProcessor extends SolrEntityProcessor {
+    @Override
+    protected void nextPage() {
+    }
+  }
+
   private static final String ID = "id";
 
   public void testQuery() {
@@ -85,6 +97,64 @@ public class TestSolrEntityProcessorUnit extends AbstractDataImportHandlerTestCa
       processor.destroy();
     }
   }
+  @Test (expected = DataImportHandlerException.class)
+  public void testNoQuery() {
+    SolrEntityProcessor processor = new SolrEntityProcessor();
+    
+    HashMap<String,String> entityAttrs = new HashMap<String,String>(){{put(SolrEntityProcessor.SOLR_SERVER,"http://route:66/no");}};
+    processor.init(getContext(null, null, null, null, Collections.emptyList(), 
+        entityAttrs));
+    try {
+    processor.buildIterator();
+    }finally {
+      processor.destroy();
+    }
+  }
+  
+  public void testPagingQuery() {
+    SolrEntityProcessor processor = new NoNextMockProcessor() ;
+    
+    HashMap<String,String> entityAttrs = new HashMap<String,String>(){{
+      put(SolrEntityProcessor.SOLR_SERVER,"http://route:66/no");
+      if (random().nextBoolean()) {
+        List<String> noCursor = Arrays.asList("","false",CursorMarkParams.CURSOR_MARK_START);//only 'true' not '*'
+        Collections.shuffle(noCursor, random());
+        put(CursorMarkParams.CURSOR_MARK_PARAM,  noCursor.get(0));
+      }}};
+    processor.init(getContext(null, null, null, null, Collections.emptyList(), 
+        entityAttrs));
+    try {
+    processor.buildIterator();
+    SolrQuery query = new SolrQuery();
+    ((SolrDocumentListIterator) processor.rowIterator).passNextPage(query);
+    assertEquals("0", query.get(CommonParams.START));
+    assertNull( query.get(CursorMarkParams.CURSOR_MARK_PARAM));
+    assertNotNull( query.get(CommonParams.TIME_ALLOWED));
+    }finally {
+      processor.destroy();
+    }
+  }
+  
+  public void testCursorQuery() {
+    SolrEntityProcessor processor = new NoNextMockProcessor() ;
+    
+    HashMap<String,String> entityAttrs = new HashMap<String,String>(){{
+      put(SolrEntityProcessor.SOLR_SERVER,"http://route:66/no");
+      put(CursorMarkParams.CURSOR_MARK_PARAM,"true");
+      }};
+    processor.init(getContext(null, null, null, null, Collections.emptyList(), 
+        entityAttrs));
+    try {
+    processor.buildIterator();
+    SolrQuery query = new SolrQuery();
+    ((SolrDocumentListIterator) processor.rowIterator).passNextPage(query);
+    assertNull(query.get(CommonParams.START));
+    assertEquals(CursorMarkParams.CURSOR_MARK_START, query.get(CursorMarkParams.CURSOR_MARK_PARAM));
+    assertNull( query.get(CommonParams.TIME_ALLOWED));
+    }finally {
+      processor.destroy();
+    }
+  }
 
   private List<Doc> generateUniqueDocs(int numDocs) {
     List<FldType> types = new ArrayList<>();


[04/19] lucene-solr:jira/solr-9854: LUCENE-5325: Add test for missing values in sorts

Posted by ab...@apache.org.
LUCENE-5325: Add test for missing values in sorts


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

Branch: refs/heads/jira/solr-9854
Commit: a4335c0e9f01275c7d6e807813d9818b6e59d76e
Parents: 0072382
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Dec 29 14:44:11 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 20:26:36 2016 +0000

----------------------------------------------------------------------
 .../org/apache/lucene/search/TestDoubleValuesSource.java    | 9 +++++++++
 .../test/org/apache/lucene/search/TestLongValuesSource.java | 9 +++++++++
 2 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a4335c0e/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
index 3860963..13a5168 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
@@ -53,6 +53,8 @@ public class TestDoubleValuesSource extends LuceneTestCase {
       document.add(new NumericDocValuesField("long", random().nextLong()));
       document.add(new FloatDocValuesField("float", random().nextFloat()));
       document.add(new DoubleDocValuesField("double", random().nextDouble()));
+      if (i == 545)
+        document.add(new DoubleDocValuesField("onefield", 45.72));
       iw.addDocument(document);
     }
     reader = iw.getReader();
@@ -67,6 +69,13 @@ public class TestDoubleValuesSource extends LuceneTestCase {
     super.tearDown();
   }
 
+  public void testSortMissing() throws Exception {
+    DoubleValuesSource onefield = DoubleValuesSource.fromDoubleField("onefield");
+    TopDocs results = searcher.search(new MatchAllDocsQuery(), 1, new Sort(onefield.getSortField(true)));
+    FieldDoc first = (FieldDoc) results.scoreDocs[0];
+    assertEquals(45.72, first.fields[0]);
+  }
+
   public void testSimpleFieldEquivalences() throws Exception {
     checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
     checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a4335c0e/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
index 9148ad5..8b20be5 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
@@ -49,6 +49,8 @@ public class TestLongValuesSource extends LuceneTestCase {
       document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
       document.add(new NumericDocValuesField("int", random().nextInt()));
       document.add(new NumericDocValuesField("long", random().nextLong()));
+      if (i == 545)
+        document.add(new NumericDocValuesField("onefield", 45));
       iw.addDocument(document);
     }
     reader = iw.getReader();
@@ -63,6 +65,13 @@ public class TestLongValuesSource extends LuceneTestCase {
     super.tearDown();
   }
 
+  public void testSortMissing() throws Exception {
+    LongValuesSource onefield = LongValuesSource.fromLongField("onefield");
+    TopDocs results = searcher.search(new MatchAllDocsQuery(), 1, new Sort(onefield.getSortField(true)));
+    FieldDoc first = (FieldDoc) results.scoreDocs[0];
+    assertEquals(45L, first.fields[0]);
+  }
+
   public void testSimpleFieldEquivalences() throws Exception {
     checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
     checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));


[10/19] lucene-solr:jira/solr-9854: SOLR-9891: Add mkroot command to bin/solr and bin/solr.cmd

Posted by ab...@apache.org.
SOLR-9891: Add mkroot command to bin/solr and bin/solr.cmd


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

Branch: refs/heads/jira/solr-9854
Commit: cb266d5fc775bd9d26ed7f0e68e9d0d12793f9b5
Parents: b584f9c
Author: Erick Erickson <er...@apache.org>
Authored: Thu Dec 29 17:39:48 2016 -0800
Committer: Erick Erickson <er...@apache.org>
Committed: Thu Dec 29 17:39:48 2016 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 solr/bin/solr                                   | 26 +++++++-
 solr/bin/solr.cmd                               | 21 ++++++-
 .../src/java/org/apache/solr/util/SolrCLI.java  | 65 +++++++++++++++++++-
 4 files changed, 108 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cb266d5f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7f83de0..59dde90 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -204,6 +204,8 @@ New Features
 
 * SOLR-9905: Add NullStream to isolate the performance of the ExportWriter (Joel Bernstein)
 
+* SOLR-9891: Add mkroot command to bin/solr and bin/solr.cmd (Erick Erickson)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cb266d5f/solr/bin/solr
----------------------------------------------------------------------
diff --git a/solr/bin/solr b/solr/bin/solr
index c1add26..fcf864b 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -493,6 +493,12 @@ function print_usage() {
     echo ""
     echo "             Only the node names are listed, not data"
     echo ""
+    echo "         mkroot makes a znode on Zookeeper with no data. Can be used to make a path of arbitrary"
+    echo "             depth but primarily intended to create a 'chroot'."
+    echo ""
+    echo "             <path>: The Zookeeper path to create. Leading slash is assumed if not present."
+    echo "                     Intermediate nodes are created as needed if not present."
+    echo ""
   fi
 } # end print_usage
 
@@ -507,6 +513,7 @@ function print_short_zk_usage() {
   echo "         solr zk rm [-r] <path> [-z zkHost]"
   echo "         solr zk mv <src> <dest> [-z zkHost]"
   echo "         solr zk ls [-r] <path> [-z zkHost]"
+  echo "         solr zk mkroot <path> [-z zkHost]"
   echo ""
 
   if [ "$1" == "" ]; then
@@ -1001,7 +1008,7 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
   if [ $# -gt 0 ]; then
     while true; do
       case "$1" in
-        -upconfig|upconfig|-downconfig|downconfig|cp|rm|mv|ls)
+        -upconfig|upconfig|-downconfig|downconfig|cp|rm|mv|ls|mkroot)
             if [ "${1:0:1}" == "-" ]; then
               ZK_OP=${1:1}
             else
@@ -1042,7 +1049,7 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
             shift
             break
         ;;
-        *)  # Pick up <src> <dst> or <path> params for rm, ls, cp, mv.
+        *)  # Pick up <src> <dst> or <path> params for rm, ls, cp, mv, mkroot.
             if [ "$1" == "" ]; then
               break # out-of-args, stop looping
             fi
@@ -1062,7 +1069,7 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
   fi
 
   if [ -z "$ZK_OP" ]; then
-    print_short_zk_usage "Zookeeper operation (one of 'upconfig', 'downconfig', 'rm', 'mv', 'cp', 'ls') is required!"
+    print_short_zk_usage "Zookeeper operation (one of 'upconfig', 'downconfig', 'rm', 'mv', 'cp', 'ls', 'mkroot') is required!"
   fi
 
   if [ -z "$ZK_HOST" ]; then
@@ -1088,6 +1095,13 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
     fi
   fi
 
+  if [[ "$ZK_OP" == "mkroot" ]]; then
+    if [[ -z "$ZK_SRC" ]]; then
+      print_short_zk_usage "<path> must be specified when using the 'mkroot' command."
+    fi
+  fi
+
+
   case "$ZK_OP" in
     upconfig)
       run_tool "$ZK_OP" -confname "$CONFIGSET_CONFNAME" -confdir "$CONFIGSET_CONFDIR" -zkHost "$ZK_HOST" -configsetsDir "$SOLR_TIP/server/solr/configsets"
@@ -1113,6 +1127,12 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
       fi
       run_tool "$ZK_OP" -path "$ZK_SRC" -recurse "$ZK_RECURSE" -zkHost "$ZK_HOST"
     ;;
+    mkroot)
+      if [ -z "$ZK_SRC" ]; then
+        print_short_zk_usage "Zookeeper path to list must be specified when using the 'mkroot' command"
+      fi
+      run_tool "$ZK_OP" -path "$ZK_SRC" -zkHost "$ZK_HOST"
+    ;;
     *)
       print_short_zk_usage "Unrecognized Zookeeper operation $ZK_OP"
     ;;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cb266d5f/solr/bin/solr.cmd
----------------------------------------------------------------------
diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd
index 4b6081f..04398bc 100644
--- a/solr/bin/solr.cmd
+++ b/solr/bin/solr.cmd
@@ -480,6 +480,13 @@ echo             ^<path^>: The Zookeeper path to use as the root.
 echo.
 echo             Only the node names are listed, not data
 echo.
+echo         mkroot makes a znode in Zookeeper with no data. Can be used to make a path of arbitrary
+echo                depth but primarily intended to create a 'chroot'."
+echo.
+echo             ^<path^>: The Zookeeper path to create. Leading slash is assumed if not present.
+echo                       Intermediate nodes are created as needed if not present.
+echo.
+
 goto done
 
 :zk_short_usage
@@ -492,6 +499,7 @@ echo         solr zk cp [-r] ^<src^> ^<dest^> [-z zkHost]
 echo         solr zk rm [-r] ^<path^> [-z zkHost]
 echo         solr zk mv ^<src^> ^<dest^> [-z zkHost]
 echo         solr zk ls [-r] ^<path^> [-z zkHost]
+echo         solr zk mkroot ^<path^> [-z zkHost]
 echo.
 IF "%ZK_FULL%"=="true" (
   goto zk_full_usage
@@ -1399,6 +1407,8 @@ IF "%1"=="-upconfig" (
   goto set_zk_op
 ) ELSE IF "%1"=="ls" (
   goto set_zk_op
+) ELSE IF "%1"=="mkroot" (
+  goto set_zk_op
 ) ELSE IF "%1"=="-n" (
   goto set_config_name
 ) ELSE IF "%1"=="-r" (
@@ -1561,13 +1571,22 @@ IF "!ZK_OP!"=="upconfig" (
   org.apache.solr.util.SolrCLI !ZK_OP! -zkHost !ZK_HOST! -path !ZK_SRC! -recurse !ZK_RECURSE!
 ) ELSE IF "!ZK_OP!"=="ls" (
   IF "%ZK_SRC"=="" (
-    set ERROR_MSG="Zookeeper path to remove must be specified when using the 'rm' command"
+    set ERROR_MSG="Zookeeper path to remove must be specified when using the 'ls' command"
     goto zk_short_usage
   )
   "%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^
   -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^
   -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^
   org.apache.solr.util.SolrCLI !ZK_OP! -zkHost !ZK_HOST! -path !ZK_SRC! -recurse !ZK_RECURSE!
+) ELSE IF "!ZK_OP!"=="mkroot" (
+  IF "%ZK_SRC"=="" (
+    set ERROR_MSG="Zookeeper path to create must be specified when using the 'mkroot' command"
+    goto zk_short_usage
+  )
+  "%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^
+  -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^
+  -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^
+  org.apache.solr.util.SolrCLI !ZK_OP! -zkHost !ZK_HOST! -path !ZK_SRC!
 ) ELSE (
   set ERROR_MSG="Unknown zk option !ZK_OP!"
   goto zk_short_usage

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cb266d5f/solr/core/src/java/org/apache/solr/util/SolrCLI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
index 4979848..bb2d554 100644
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -364,6 +364,8 @@ public class SolrCLI {
       return new ZkCpTool();
     else if ("ls".equals(toolType))
       return new ZkLsTool();
+    else if ("mkroot".equals(toolType))
+      return new ZkMkrootTool();
     else if ("assert".equals(toolType))
       return new AssertTool();
     else if ("utils".equals(toolType))
@@ -1986,7 +1988,7 @@ public class SolrCLI {
 
       if (zkHost == null) {
         throw new IllegalStateException("Solr at " + cli.getOptionValue("zkHost") +
-            " is running in standalone server mode, 'zk rm' can only be used when running in SolrCloud mode.\n");
+            " is running in standalone server mode, 'zk ls' can only be used when running in SolrCloud mode.\n");
       }
 
 
@@ -1999,12 +2001,71 @@ public class SolrCLI {
             " recurse: " + Boolean.toString(recurse));
         stdout.print(zkClient.listZnode(znode, recurse));
       } catch (Exception e) {
-        log.error("Could not complete rm operation for reason: " + e.getMessage());
+        log.error("Could not complete ls operation for reason: " + e.getMessage());
         throw (e);
       }
     }
   } // End zkLsTool class
 
+
+  public static class ZkMkrootTool extends ToolBase {
+
+    public ZkMkrootTool() {
+      this(System.out);
+    }
+
+    public ZkMkrootTool(PrintStream stdout) {
+      super(stdout);
+    }
+
+    @SuppressWarnings("static-access")
+    public Option[] getOptions() {
+      return new Option[]{
+          OptionBuilder
+              .withArgName("path")
+              .hasArg()
+              .isRequired(true)
+              .withDescription("Path to create")
+              .create("path"),
+          OptionBuilder
+              .withArgName("HOST")
+              .hasArg()
+              .isRequired(true)
+              .withDescription("Address of the Zookeeper ensemble; defaults to: " + ZK_HOST)
+              .create("zkHost")
+      };
+    }
+
+    public String getName() {
+      return "mkroot";
+    }
+
+    protected void runImpl(CommandLine cli) throws Exception {
+
+      String zkHost = getZkHost(cli);
+
+      if (zkHost == null) {
+        throw new IllegalStateException("Solr at " + cli.getOptionValue("zkHost") +
+            " is running in standalone server mode, 'zk mkroot' can only be used when running in SolrCloud mode.\n");
+      }
+
+
+      try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
+        echo("\nConnecting to ZooKeeper at " + zkHost + " ...");
+
+        String znode = cli.getOptionValue("path");
+        echo("Creating Zookeeper path " + znode + " on ZooKeeper at " + zkHost);
+        zkClient.makePath(znode, true);
+      } catch (Exception e) {
+        log.error("Could not complete mkroot operation for reason: " + e.getMessage());
+        throw (e);
+      }
+    }
+  } // End zkMkrootTool class
+
+
+
+
   public static class ZkCpTool extends ToolBase {
 
     public ZkCpTool() {


[02/19] lucene-solr:jira/solr-9854: SOLR-9905: Add NullStream to isolate the performance of the ExportWriter

Posted by ab...@apache.org.
SOLR-9905: Add NullStream to isolate the performance of the ExportWriter


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

Branch: refs/heads/jira/solr-9854
Commit: 7dcb557ab73da7fb7af0e8f698895e28dde4bbca
Parents: db9190d
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Dec 29 13:46:04 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Dec 29 14:39:16 2016 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   2 +-
 .../solr/client/solrj/io/stream/NullStream.java | 155 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   |  65 ++++++++
 3 files changed, 221 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dcb557a/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 c6f3c62..13ce636 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -139,7 +139,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("classify", ClassifyStream.class)
       .withFunctionName("fetch", FetchStream.class)
       .withFunctionName("executor", ExecutorStream.class)
-
+      .withFunctionName("null", NullStream.class)
       // metrics
       .withFunctionName("min", MinMetric.class)
       .withFunctionName("max", MaxMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dcb557a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java
new file mode 100644
index 0000000..bef3b1c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.solrj.io.stream;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Date;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.comp.StreamComparator;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+
+/**
+ *  The NullStream Iterates over a TupleStream and eats the tuples. It returns the tuple count in the EOF tuple.
+ *  Because the NullStreaam eats all the Tuples it see's it can be used as a simple tool for performance analysis of
+ *  underlying streams.
+ **/
+
+public class NullStream extends TupleStream implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  private TupleStream stream;
+  private long count;
+  private long start;
+  private Tuple eof;
+
+  public NullStream(TupleStream tupleStream) throws IOException {
+    init(tupleStream);
+  }
+
+  public NullStream(StreamExpression expression, StreamFactory factory) throws IOException {
+    // grab all parameters out
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+    TupleStream stream = factory.constructStream(streamExpressions.get(0));
+
+    init(stream);
+  }
+
+  private void init(TupleStream tupleStream) throws IOException{
+    this.stream = tupleStream;
+  }
+
+  @Override
+  public StreamExpression toExpression(StreamFactory factory) throws IOException{
+    return toExpression(factory, true);
+  }
+
+  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    if(includeStreams){
+      // stream
+      if(stream instanceof Expressible){
+        expression.addParameter(((Expressible)stream).toExpression(factory));
+      }
+      else{
+        throw new IOException("This RankStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+    }
+    else{
+      expression.addParameter("<stream>");
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    return new StreamExplanation(getStreamNodeId().toString())
+        .withChildren(new Explanation[]{
+            stream.toExplanation(factory)
+        })
+        .withFunctionName(factory.getFunctionName(this.getClass()))
+        .withImplementingClass(this.getClass().getName())
+        .withExpressionType(ExpressionType.STREAM_DECORATOR)
+        .withExpression(toExpression(factory, false).toString());
+  }
+
+  public void setStreamContext(StreamContext context) {
+    this.stream.setStreamContext(context);
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList<TupleStream>();
+    l.add(stream);
+    return l;
+  }
+
+  public void open() throws IOException {
+    start = new Date().getTime();
+    count = 0;
+    stream.open();
+  }
+
+  public void close() throws IOException {
+    stream.close();
+  }
+
+  public Tuple read() throws IOException {
+
+    if(eof != null) {
+      return eof;
+    }
+
+    while(true) {
+      Tuple tuple  = stream.read();
+      if(tuple.EOF) {
+        eof = tuple;
+        long end = new Date().getTime();
+        Tuple t = new Tuple(new HashMap());
+        t.put("nullCount", count);
+        t.put("timer", end-start);
+        return t;
+      } else {
+        ++count;
+      }
+    }
+  }
+
+  /** Return the stream sort - ie, the order in which records are returned */
+  public StreamComparator getStreamSort(){
+    return stream.getStreamSort();
+  }
+
+  public int getCost() {
+    return 0;
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dcb557a/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 7d48c0e..0c9d5b3 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
@@ -373,6 +373,71 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
   }
 
+
+  @Test
+  public void testNullStream() throws Exception {
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
+        .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
+        .add(id, "5", "a_s", "hello1", "a_i", "1", "a_f", "2")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("null", NullStream.class);
+
+    // Basic test
+    stream = factory.constructStream("null(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i asc\")");
+    tuples = getTuples(stream);
+    assertTrue(tuples.size() == 1);
+    assertTrue(tuples.get(0).getLong("nullCount") == 6);
+  }
+
+
+  @Test
+  public void testParallelNullStream() throws Exception {
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
+        .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
+        .add(id, "5", "a_s", "hello1", "a_i", "1", "a_f", "2")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("null", NullStream.class)
+        .withFunctionName("parallel", ParallelStream.class);
+
+    // Basic test
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"nullCount desc\", null(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), by=\"a_i asc\"))");
+    tuples = getTuples(stream);
+    assertTrue(tuples.size() == 2);
+    long nullCount = 0;
+    for(Tuple t : tuples) {
+      nullCount += t.getLong("nullCount");
+    }
+
+    assertEquals(nullCount, 6L);
+  }
+
+
   @Test
   public void testNulls() throws Exception {
 


[17/19] lucene-solr:jira/solr-9854: SOLR-9495: Update CHANGES.txt

Posted by ab...@apache.org.
SOLR-9495: Update CHANGES.txt


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

Branch: refs/heads/jira/solr-9854
Commit: 832d02bf494c8fea02398db31b55de4314f2be8a
Parents: 6167618
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 30 20:39:57 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 30 20:39:57 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/832d02bf/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6a99617..02167f3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -306,6 +306,8 @@ Bug Fixes
 
 * SOLR-9900: fix false positives on range queries with ReversedWildcardFilterFactory (Yonik Seeley via Mikhail Khludnev)
 
+* SOLR-9495: AIOBE with confusing message for incomplete sort spec in Streaming Expression (Gus Heck, Joel Bernstein)
+
 Other Changes
 ----------------------
 


[06/19] lucene-solr:jira/solr-9854: LUCENE-7564: add missing javadocs

Posted by ab...@apache.org.
LUCENE-7564: add missing javadocs


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

Branch: refs/heads/jira/solr-9854
Commit: 6b00ee5175d55d2f2a25ce6539dc12277022c898
Parents: 93fdc20
Author: Steve Rowe <sa...@apache.org>
Authored: Thu Dec 29 16:08:35 2016 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Thu Dec 29 16:08:53 2016 -0500

----------------------------------------------------------------------
 .../lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b00ee51/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
index 2fbe4a8..81880d4 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
@@ -137,6 +137,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
   /** {@link IndexSearcher} used for lookups. */
   protected SearcherManager searcherMgr;
   
+  /** Used to manage concurrent access to searcherMgr */
   protected final Object searcherMgrLock = new Object();
 
   /** Default minimum number of leading characters before


[15/19] lucene-solr:jira/solr-9854: SOLR-9684: Update CHANGES.txt

Posted by ab...@apache.org.
SOLR-9684: Update CHANGES.txt


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

Branch: refs/heads/jira/solr-9854
Commit: 36a691c50d680d1c6977e6185448e06cb21f653d
Parents: f3fe487
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 30 15:38:28 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 30 15:38:28 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/36a691c5/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 874ac81..6a99617 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -208,6 +208,8 @@ New Features
 
 * SOLR-9668,SOLR-7197: introduce cursorMark='true' in SolrEntityProcessor (Yegor Kozlov, Raveendra Yerraguntl via Mikhail Khludnev)
 
+* SOLR-9684: Add schedule Streaming Expression (Joel Bernstein)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have


[18/19] lucene-solr:jira/solr-9854: SOLR-9880 Fix concurrency bugs in tests.

Posted by ab...@apache.org.
SOLR-9880 Fix concurrency bugs in 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/fb2800b1
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/fb2800b1
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/fb2800b1

Branch: refs/heads/jira/solr-9854
Commit: fb2800b1497a67493c7f8944bda22f590bb9cc6b
Parents: 832d02b
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Sat Dec 31 20:46:32 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Sat Dec 31 20:46:32 2016 +0100

----------------------------------------------------------------------
 .../apache/solr/metrics/reporters/SolrGangliaReporterTest.java    | 3 ++-
 .../apache/solr/metrics/reporters/SolrGraphiteReporterTest.java   | 3 ++-
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fb2800b1/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
index 6773e0c..3d1c482 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
@@ -69,7 +69,8 @@ public class SolrGraphiteReporterTest extends SolrTestCaseJ4 {
       assertTrue(reporter instanceof SolrGraphiteReporter);
       Thread.sleep(5000);
       assertTrue(mock.lines.size() >= 3);
-      for (String line : mock.lines) {
+      String[] frozenLines = (String[])mock.lines.toArray(new String[mock.lines.size()]);
+      for (String line : frozenLines) {
         assertTrue(line, line.startsWith("test.solr.node.cores."));
       }
     } finally {


[12/19] lucene-solr:jira/solr-9854: LUCENE-7606: Normalization with CustomAnalyzer would only apply the last token filter.

Posted by ab...@apache.org.
LUCENE-7606: Normalization with CustomAnalyzer would only apply the last token filter.


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

Branch: refs/heads/jira/solr-9854
Commit: 26ee8e9bea70e857aa61764020337ce675066bd1
Parents: 3ccd15a
Author: Adrien Grand <jp...@gmail.com>
Authored: Fri Dec 30 10:30:40 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Fri Dec 30 10:30:40 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +++
 .../lucene/analysis/custom/CustomAnalyzer.java  |  2 +-
 .../analysis/custom/TestCustomAnalyzer.java     | 21 ++++++++++++++++++++
 .../apache/lucene/analysis/custom/mapping1.txt  |  1 +
 .../apache/lucene/analysis/custom/mapping2.txt  |  1 +
 5 files changed, 27 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 12b615d..4c49560 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -129,6 +129,9 @@ Bug Fixes
   using helpers for exclusive bounds that are consistent with Double.compare.
   (Adrien Grand, Dawid Weiss)
 
+* LUCENE-7606: Normalization with CustomAnalyzer would only apply the last
+  token filter. (Adrien Grand)
+
 Improvements
 
 * LUCENE-6824: TermAutomatonQuery now rewrites to TermQuery,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
index 466642c..1cfdfe3 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
@@ -145,7 +145,7 @@ public final class CustomAnalyzer extends Analyzer {
     for (TokenFilterFactory filter : tokenFilters) {
       if (filter instanceof MultiTermAwareComponent) {
         filter = (TokenFilterFactory) ((MultiTermAwareComponent) filter).getMultiTermComponent();
-        result = filter.create(in);
+        result = filter.create(result);
       }
     }
     return result;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
index aa69b70..d929bfd 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
@@ -30,6 +30,7 @@ import org.apache.lucene.analysis.CharFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.charfilter.HTMLStripCharFilterFactory;
+import org.apache.lucene.analysis.charfilter.MappingCharFilterFactory;
 import org.apache.lucene.analysis.core.KeywordTokenizerFactory;
 import org.apache.lucene.analysis.core.LowerCaseFilterFactory;
 import org.apache.lucene.analysis.core.LowerCaseTokenizer;
@@ -479,4 +480,24 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
     assertEquals(new BytesRef("2A"), analyzer2.normalize("dummy", "0�"));
   }
 
+  public void testNormalizationWithMultipleTokenFilters() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        // none of these components are multi-term aware so they should not be applied
+        .withTokenizer(WhitespaceTokenizerFactory.class, Collections.emptyMap())
+        .addTokenFilter(LowerCaseFilterFactory.class, Collections.emptyMap())
+        .addTokenFilter(ASCIIFoldingFilterFactory.class, Collections.emptyMap())
+        .build();
+    assertEquals(new BytesRef("a b e"), analyzer.normalize("dummy", "� B �"));
+  }
+
+  public void testNormalizationWithMultiplCharFilters() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        // none of these components are multi-term aware so they should not be applied
+        .withTokenizer(WhitespaceTokenizerFactory.class, Collections.emptyMap())
+        .addCharFilter(MappingCharFilterFactory.class, new HashMap<>(Collections.singletonMap("mapping", "org/apache/lucene/analysis/custom/mapping1.txt")))
+        .addCharFilter(MappingCharFilterFactory.class, new HashMap<>(Collections.singletonMap("mapping", "org/apache/lucene/analysis/custom/mapping2.txt")))
+        .build();
+    assertEquals(new BytesRef("e f c"), analyzer.normalize("dummy", "a b c"));
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping1.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping1.txt b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping1.txt
new file mode 100644
index 0000000..40aaf5a
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping1.txt
@@ -0,0 +1 @@
+"a" => "e"

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping2.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping2.txt b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping2.txt
new file mode 100644
index 0000000..cac0bea
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping2.txt
@@ -0,0 +1 @@
+"b" => "f"


[16/19] lucene-solr:jira/solr-9854: SOLR-9495: AIOBE with confusing message for incomplete sort spec in Streaming Expression

Posted by ab...@apache.org.
SOLR-9495: AIOBE with confusing message for incomplete sort spec in Streaming Expression


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

Branch: refs/heads/jira/solr-9854
Commit: 61676188d7f592f697933b6051806c0bc55b406a
Parents: 36a691c
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 30 19:54:00 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 30 19:55:31 2016 -0500

----------------------------------------------------------------------
 .../solr/client/solrj/io/stream/CloudSolrStream.java      |  4 ++++
 .../solr/client/solrj/io/stream/StreamExpressionTest.java | 10 ++++++++++
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61676188/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
index 0580122..f177585 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
@@ -332,6 +332,10 @@ public class CloudSolrStream extends TupleStream implements Expressible {
 
       String[] spec = s.trim().split("\\s+"); //This should take into account spaces in the sort spec.
       
+      if (spec.length != 2) {
+        throw new IOException("Invalid sort spec:" + s);
+      }
+
       String fieldName = spec[0].trim();
       String order = spec[1].trim();
       

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61676188/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 3bfe129..936d42f 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
@@ -162,6 +162,16 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       assertTrue(e.getMessage().contains("fl param expected for search function"));
     }
 
+
+    try {
+      expression = StreamExpressionParser.parse("search(" + COLLECTIONORALIAS + ", q=\"blah\", fl=\"id, a_f\", sort=\"a_f\")");
+      stream = new CloudSolrStream(expression, factory);
+      tuples = getTuples(stream);
+      throw new Exception("Should be an exception here");
+    } catch(Exception e) {
+      assertTrue(e.getMessage().contains("Invalid sort spec"));
+    }
+
   }
 
   @Test


[19/19] lucene-solr:jira/solr-9854: Merge branch 'master' into jira/solr-9854

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


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

Branch: refs/heads/jira/solr-9854
Commit: 4ca11b3ed20b0860bfafe99d5b1594184f9e3001
Parents: 2531fc8 fb2800b
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Jan 2 10:47:16 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Jan 2 10:47:16 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../lucene/analysis/custom/CustomAnalyzer.java  |   2 +-
 .../analysis/custom/TestCustomAnalyzer.java     |  21 ++
 .../apache/lucene/analysis/custom/mapping1.txt  |   1 +
 .../apache/lucene/analysis/custom/mapping2.txt  |   1 +
 .../lucene/search/TestDoubleValuesSource.java   |   9 +
 .../apache/lucene/search/TestLRUQueryCache.java |   2 +
 .../lucene/search/TestLongValuesSource.java     |   9 +
 .../analyzing/AnalyzingInfixSuggester.java      | 137 +++++++-----
 .../org/apache/lucene/util/RamUsageTester.java  |   2 +-
 solr/CHANGES.txt                                |  12 +
 solr/bin/solr                                   |  26 ++-
 solr/bin/solr.cmd                               |  21 +-
 .../handler/dataimport/SolrEntityProcessor.java | 192 ++++++++++------
 .../dataimport/MockSolrEntityProcessor.java     |  18 +-
 .../TestSolrEntityProcessorEndToEnd.java        |  27 ++-
 .../dataimport/TestSolrEntityProcessorUnit.java |  70 ++++++
 .../org/apache/solr/handler/StreamHandler.java  |   3 +-
 .../apache/solr/parser/SolrQueryParserBase.java |  14 ++
 .../src/java/org/apache/solr/util/SolrCLI.java  |  65 +++++-
 .../TestReversedWildcardFilterFactory.java      |  12 +
 .../solr/cloud/DocValuesNotIndexedTest.java     |  10 -
 .../reporters/SolrGangliaReporterTest.java      |   3 +-
 .../reporters/SolrGraphiteReporterTest.java     |   3 +-
 .../client/solrj/io/stream/CloudSolrStream.java |   4 +
 .../solr/client/solrj/io/stream/NullStream.java | 155 +++++++++++++
 .../client/solrj/io/stream/SchedulerStream.java | 161 +++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   | 224 +++++++++++++++++++
 28 files changed, 1056 insertions(+), 151 deletions(-)
----------------------------------------------------------------------



[09/19] lucene-solr:jira/solr-9854: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr

Posted by ab...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr


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

Branch: refs/heads/jira/solr-9854
Commit: b584f9c56c3ef22cb54c11fd5051b54f8ffed6c7
Parents: d65c02e 5d042d3
Author: Uwe Schindler <us...@apache.org>
Authored: Thu Dec 29 22:57:39 2016 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Thu Dec 29 22:57:39 2016 +0100

----------------------------------------------------------------------
 .../lucene/search/TestDoubleValuesSource.java   |   9 ++
 .../lucene/search/TestLongValuesSource.java     |   9 ++
 .../analyzing/AnalyzingInfixSuggester.java      | 137 +++++++++-------
 solr/CHANGES.txt                                |   4 +
 .../org/apache/solr/handler/StreamHandler.java  |   2 +-
 .../apache/solr/parser/SolrQueryParserBase.java |  14 ++
 .../TestReversedWildcardFilterFactory.java      |  12 ++
 .../solr/client/solrj/io/stream/NullStream.java | 155 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   |  65 ++++++++
 9 files changed, 351 insertions(+), 56 deletions(-)
----------------------------------------------------------------------