You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dr...@apache.org on 2016/06/09 04:59:24 UTC

[1/8] lucene-solr:SOLR-9191: SOLR-7123: Syntax modified. Multiple paths can be added in the split parameter itself (pipe separated) [Forced Update!]

Repository: lucene-solr
Updated Branches:
  refs/heads/SOLR-9191 8465ed211 -> ed2621de8 (forced update)


SOLR-7123: Syntax modified. Multiple paths can be added in the split parameter itself (pipe separated)


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

Branch: refs/heads/SOLR-9191
Commit: 66cd0edc52a35fa43182a57c1f3dbfb420334f1e
Parents: 6ffd1ba
Author: Noble Paul <no...@apache.org>
Authored: Tue Jun 7 22:19:09 2016 +0530
Committer: anshum <an...@apache.org>
Committed: Tue Jun 7 13:31:40 2016 -0700

----------------------------------------------------------------------
 .../src/java/org/apache/solr/handler/loader/JsonLoader.java   | 7 +++----
 .../core/src/test/org/apache/solr/handler/JsonLoaderTest.java | 2 +-
 .../java/org/apache/solr/common/util/JsonRecordReader.java    | 6 +-----
 .../org/apache/solr/common/util/TestJsonRecordReader.java     | 6 ++----
 4 files changed, 7 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/66cd0edc/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
index ffbfe97..2c5b6ac 100644
--- a/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
+++ b/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
@@ -125,9 +125,8 @@ public class JsonLoader extends ContentStreamLoader {
       String path = (String) req.getContext().get(PATH);
       if (UpdateRequestHandler.DOC_PATH.equals(path) || "false".equals(req.getParams().get("json.command"))) {
         String split = req.getParams().get("split");
-        String childSplit = req.getParams().get("child.split");
         String[] f = req.getParams().getParams("f");
-        handleSplitMode(split, childSplit, f, reader);
+        handleSplitMode(split, f, reader);
         return;
       }
       parser = new JSONParser(reader);
@@ -194,7 +193,7 @@ public class JsonLoader extends ContentStreamLoader {
       }
     }
 
-    private void handleSplitMode(String split, String childSplit, String[] fields, final Reader reader) throws IOException {
+    private void handleSplitMode(String split, String[] fields, final Reader reader) throws IOException {
       if (split == null) split = "/";
       if (fields == null || fields.length == 0) fields = new String[]{"$FQN:/**"};
       final boolean echo = "true".equals(req.getParams().get("echo"));
@@ -209,7 +208,7 @@ public class JsonLoader extends ContentStreamLoader {
 
       }
 
-      JsonRecordReader jsonRecordReader = JsonRecordReader.getInst(split, childSplit, Arrays.asList(fields));
+      JsonRecordReader jsonRecordReader = JsonRecordReader.getInst(split, Arrays.asList(fields));
       jsonRecordReader.streamRecords(parser, new JsonRecordReader.Handler() {
         ArrayList docs = null;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/66cd0edc/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java b/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java
index a904d9e..7e4d28a 100644
--- a/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java
@@ -370,7 +370,7 @@ public class JsonLoaderTest extends SolrTestCaseJ4 {
         "b:[{c:c1, e:e1},{c:c2, e :e2, d:{p:q}}]," +
         "x:y" +
         "}}";
-    req = req("split", "/", "child.split" , "/a/b"   );
+    req = req("split", "/|/a/b"   );
     req.getContext().put("path","/update/json/docs");
     rsp = new SolrQueryResponse();
     p = new BufferingRequestProcessor(null);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/66cd0edc/solr/solrj/src/java/org/apache/solr/common/util/JsonRecordReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JsonRecordReader.java b/solr/solrj/src/java/org/apache/solr/common/util/JsonRecordReader.java
index 12c0d83..2025197 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/JsonRecordReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/JsonRecordReader.java
@@ -34,11 +34,10 @@ public class JsonRecordReader {
 
   private Node rootNode = new Node("/", (Node) null);
 
-  public static JsonRecordReader getInst(String split, String childSplit, List<String> fieldMappings) {
+  public static JsonRecordReader getInst(String split, List<String> fieldMappings) {
 
     JsonRecordReader jsonRecordReader = new JsonRecordReader();
     jsonRecordReader.addSplit(split);
-    if (childSplit != null) jsonRecordReader.addSplit(childSplit);
     for (String s : fieldMappings) {
       String path = s;
       int idx = s.indexOf(':');
@@ -52,9 +51,6 @@ public class JsonRecordReader {
     return jsonRecordReader;
   }
 
-  public static JsonRecordReader getInst(String split, List<String> fieldMappings) {
-    return getInst(split, null, fieldMappings);
-  }
 
   private JsonRecordReader() {
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/66cd0edc/solr/solrj/src/test/org/apache/solr/common/util/TestJsonRecordReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/common/util/TestJsonRecordReader.java b/solr/solrj/src/test/org/apache/solr/common/util/TestJsonRecordReader.java
index e11cc29..9da3bd5 100644
--- a/solr/solrj/src/test/org/apache/solr/common/util/TestJsonRecordReader.java
+++ b/solr/solrj/src/test/org/apache/solr/common/util/TestJsonRecordReader.java
@@ -42,8 +42,6 @@ public class TestJsonRecordReader extends SolrTestCaseJ4 {
         "     {\"c\":\"C2\",\"d\":\"D2\"}\n" +
         " ]\n" +
         "}";
-//    System.out.println(json);
-//    All parameters are mapped with field name
     JsonRecordReader streamer = JsonRecordReader.getInst("/b", Arrays.asList(
         "a_s:/a",
         "c_s:/b/c",
@@ -208,7 +206,7 @@ public class TestJsonRecordReader extends SolrTestCaseJ4 {
         "b:{c:d}," +
         "x:y" +
         "}}";
-    JsonRecordReader streamer = JsonRecordReader.getInst("/", "/a/b", Arrays.asList("/a/x", "/a/b/*"));
+    JsonRecordReader streamer = JsonRecordReader.getInst("/|/a/b", Arrays.asList("/a/x", "/a/b/*"));
     streamer.streamRecords(new StringReader(json), (record, path) -> {
       assertEquals(record.get("x"), "y");
       assertEquals(((Map) record.get(null)).get("c"), "d");
@@ -227,7 +225,7 @@ public class TestJsonRecordReader extends SolrTestCaseJ4 {
       assertEquals(m.get("c"), "c2");
       assertEquals(m.get("e"), "e2");
     });
-    streamer = JsonRecordReader.getInst("/", "/a/b", Arrays.asList("$FQN:/**"));
+    streamer = JsonRecordReader.getInst("/|/a/b", Arrays.asList("$FQN:/**"));
     streamer.streamRecords(new StringReader(json), (record, path) -> {
       assertEquals(record.get("a.x"), "y");
       List l = (List) record.get(null);


[2/8] lucene-solr:SOLR-9191: SOLR-8859: Fix AbstractSpatialFieldType to not cycle through all Spatial4j provided formats. And Fix RptWithGeometrySpatialField to be less brittle on init()

Posted by dr...@apache.org.
SOLR-8859: Fix AbstractSpatialFieldType to not cycle through all Spatial4j provided formats.
And Fix RptWithGeometrySpatialField to be less brittle on init()


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

Branch: refs/heads/SOLR-9191
Commit: fb37b3eb8c4130c8b5f53e1741e9585743b26e4d
Parents: 66cd0ed
Author: David Smiley <ds...@apache.org>
Authored: Tue Jun 7 16:48:34 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Tue Jun 7 16:48:34 2016 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +-
 .../solr/schema/AbstractSpatialFieldType.java   | 72 +++++++++-----------
 .../schema/RptWithGeometrySpatialField.java     | 11 +--
 .../solr/schema/SpatialRPTFieldTypeTest.java    | 32 ++++++---
 4 files changed, 65 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fb37b3eb/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 56c111b..bed02f2 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -81,8 +81,8 @@ New Features
   https://github.com/locationtech/spatial4j/blob/master/FORMATS.md
   To return the FeatureCollection as the root element, add '&omitHeader=true" (ryan)
 
-* SOLR-8859: AbstractSpatialFieldType will now convert Shapes to/from Strings
-  using the SpatialContext.  (ryan)
+* SOLR-8859: Spatial fields like RPT can now be configured to use Spatial4j registered shape formats
+  e.g. via format="GeoJSON".  (ryan, David Smiley)
 
 * SOLR-445: new ToleranteUpdateProcessorFactory to support skipping update commands that cause
   failures when sending multiple updates in a single request.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fb37b3eb/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
index 3130004..12fcea3 100644
--- a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
@@ -26,9 +26,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 
+import com.google.common.base.Throwables;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.IndexableField;
@@ -42,7 +44,6 @@ import org.apache.lucene.spatial.SpatialStrategy;
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.query.SpatialArgsParser;
 import org.apache.lucene.spatial.query.SpatialOperation;
-import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.response.TextResponseWriter;
@@ -64,10 +65,6 @@ import org.locationtech.spatial4j.shape.Shape;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Throwables;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-
 /**
  * Abstract base class for Solr FieldTypes based on a Lucene 4 {@link SpatialStrategy}.
  *
@@ -141,24 +138,21 @@ public abstract class AbstractSpatialFieldType<T extends SpatialStrategy> extend
     }
 
     final SupportedFormats fmts = ctx.getFormats();
-    final String format = args.remove(FORMAT);
-    if (format != null) {
-      shapeWriter = fmts.getWriter(format);
-      shapeReader = fmts.getReader(format);
-      if(shapeWriter==null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-            "Unknown Shape Format: "+ format);
-      }
-      if(shapeReader==null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-            "Unknown Shape Format: "+ format);
-      }
+    String format = args.remove(FORMAT);
+    if (format == null) {
+      format = "WKT";
+    }
+    shapeWriter = fmts.getWriter(format);
+    shapeReader = fmts.getReader(format);
+    if(shapeWriter==null) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Unknown Shape Format: "+ format);
     }
-    else {
-      // Otherwise, pick the first supported reader/writer
-      shapeWriter = fmts.getWriters().get(0);
-      shapeReader = fmts.getReaders().get(0);
+    if(shapeReader==null) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Unknown Shape Format: "+ format);
     }
+
     argsParser = newSpatialArgsParser();
   }
 
@@ -234,23 +228,29 @@ public abstract class AbstractSpatialFieldType<T extends SpatialStrategy> extend
 
   /** Create a {@link Shape} from the input string */
   public Shape parseShape(String str) {
+    str = str.trim();
     if (str.length() == 0)
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "empty string shape");
 
-    Shape shape = null;
-    if(shapeReader!=null) {
-      shape = shapeReader.readIfSupported(str);
-    }
-
-    if(shape==null) {
-      // Try all supported formats
-      shape = ctx.getFormats().read(str);
+    // If the first char is promising, try to parse with SpatialUtils.parsePoint
+    char firstChar = str.charAt(0);
+    if (firstChar == '+' || firstChar == '-' || (firstChar >= '0' && firstChar <= '9')) {
+      try {
+        return SpatialUtils.parsePoint(str, ctx);
+      } catch (Exception e) {//ignore
+      }
     }
 
-    if(shape==null) {
-       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to parse shape from: "+str);
+    try {
+      return shapeReader.read(str);
+    } catch (Exception e) {
+      String msg = "Unable to parse shape given formats" +
+          " \"lat,lon\", \"x y\" or as " + shapeReader.getFormatName() + " because " + e;
+      if (!msg.contains(str)) {
+        msg += " input: " + str;
+      }
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, msg, e);
     }
-    return shape;
   }
 
   /**
@@ -259,11 +259,7 @@ public abstract class AbstractSpatialFieldType<T extends SpatialStrategy> extend
    * The format can be selected using the initParam <code>format={WKT|GeoJSON}</code>
    */
   public String shapeToString(Shape shape) {
-    if(shapeWriter!=null) {
-      return shapeWriter.toString(shape);
-    }
-    // This will only happen if the context does not have any writers
-    throw new SolrException(ErrorCode.SERVER_ERROR, "ShapeWriter not configured");
+    return shapeWriter.toString(shape);
   }
 
   /** Called from {@link #getStrategy(String)} upon first use by fieldName. } */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fb37b3eb/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java b/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java
index b633174..ca2771c 100644
--- a/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java
+++ b/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java
@@ -18,11 +18,9 @@ package org.apache.solr.schema;
 
 import java.io.IOException;
 import java.lang.ref.WeakReference;
+import java.util.HashMap;
 import java.util.Map;
 
-import org.locationtech.spatial4j.context.SpatialContext;
-import org.locationtech.spatial4j.shape.Shape;
-import org.locationtech.spatial4j.shape.jts.JtsGeometry;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
@@ -36,6 +34,9 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.search.SolrCache;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.jts.JtsGeometry;
 
 /** A Solr Spatial FieldType based on {@link CompositeSpatialStrategy}.
  * @lucene.experimental */
@@ -48,7 +49,8 @@ public class RptWithGeometrySpatialField extends AbstractSpatialFieldType<Compos
 
   @Override
   protected void init(IndexSchema schema, Map<String, String> args) {
-    // Do NOT call super.init(); instead we delegate to an RPT field. Admittedly this is error prone.
+    Map<String, String> origArgs = new HashMap<>(args); // clone so we can feed it to an aggregated field type
+    super.init(schema, origArgs);
 
     //TODO Move this check to a call from AbstractSpatialFieldType.createFields() so the type can declare
     // if it supports multi-valued or not. It's insufficient here; we can't see if you set multiValued on the field.
@@ -65,6 +67,7 @@ public class RptWithGeometrySpatialField extends AbstractSpatialFieldType<Compos
     rptFieldType.setTypeName(getTypeName());
     rptFieldType.properties = properties;
     rptFieldType.init(schema, args);
+
     rptFieldType.argsParser = argsParser = newSpatialArgsParser();
     this.ctx = rptFieldType.ctx;
     this.distanceUnits = rptFieldType.distanceUnits;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fb37b3eb/solr/core/src/test/org/apache/solr/schema/SpatialRPTFieldTypeTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/SpatialRPTFieldTypeTest.java b/solr/core/src/test/org/apache/solr/schema/SpatialRPTFieldTypeTest.java
index f341832..a2afa2d 100644
--- a/solr/core/src/test/org/apache/solr/schema/SpatialRPTFieldTypeTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/SpatialRPTFieldTypeTest.java
@@ -21,6 +21,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.core.AbstractBadConfigTestBase;
 import org.junit.After;
 import org.junit.Before;
@@ -203,8 +204,8 @@ public class SpatialRPTFieldTypeTest extends AbstractBadConfigTestBase {
   }
 
   public void testShapeToFromStringWKT() throws Exception {
-    // Check WKT
-    setupRPTField("miles", "true", "WKT");
+    setupRPTField("miles", "true", "WKT", random().nextBoolean()
+        ? new SpatialRecursivePrefixTreeFieldType() : new RptWithGeometrySpatialField());
 
     AbstractSpatialFieldType ftype = (AbstractSpatialFieldType)
         h.getCore().getLatestSchema().getField("geo").getType();
@@ -214,11 +215,20 @@ public class SpatialRPTFieldTypeTest extends AbstractBadConfigTestBase {
     String out = ftype.shapeToString(shape);
 
     assertEquals(wkt, out);
+
+    //assert fails GeoJSON
+    try {
+      ftype.parseShape("{\"type\":\"Point\",\"coordinates\":[1,2]}");
+      fail("Should not parse GeoJSON if told format is WKT");
+    } catch (SolrException e) {// expected
+      System.out.println(e);
+    }
+
   }
 
   public void testShapeToFromStringGeoJSON() throws Exception {
-    // Check WKT
-    setupRPTField("miles", "true", "GeoJSON");
+    setupRPTField("miles", "true", "GeoJSON", random().nextBoolean()
+        ? new SpatialRecursivePrefixTreeFieldType() : new RptWithGeometrySpatialField());
 
     AbstractSpatialFieldType ftype = (AbstractSpatialFieldType)
         h.getCore().getLatestSchema().getField("geo").getType();
@@ -230,7 +240,7 @@ public class SpatialRPTFieldTypeTest extends AbstractBadConfigTestBase {
     assertEquals(json, out);
   }
 
-  private void setupRPTField(String distanceUnits, String geo, String format) throws Exception {
+  private void setupRPTField(String distanceUnits, String geo, String format, FieldType fieldType) throws Exception {
     deleteCore();
     File managedSchemaFile = new File(tmpConfDir, "managed-schema");
     Files.delete(managedSchemaFile.toPath()); // Delete managed-schema so it won't block parsing a new schema
@@ -243,7 +253,9 @@ public class SpatialRPTFieldTypeTest extends AbstractBadConfigTestBase {
     
     IndexSchema oldSchema = h.getCore().getLatestSchema();
 
-    SpatialRecursivePrefixTreeFieldType rptFieldType = new SpatialRecursivePrefixTreeFieldType();
+    if (fieldType == null) {
+      fieldType = new SpatialRecursivePrefixTreeFieldType();
+    }
     Map<String, String> rptMap = new HashMap<String,String>();
     if(distanceUnits!=null)
       rptMap.put("distanceUnits", distanceUnits);
@@ -252,9 +264,9 @@ public class SpatialRPTFieldTypeTest extends AbstractBadConfigTestBase {
     if(format!=null) {
       rptMap.put("format", format);
     }
-    rptFieldType.init(oldSchema, rptMap);
-    rptFieldType.setTypeName("location_rpt");
-    SchemaField newField = new SchemaField("geo", rptFieldType, SchemaField.STORED | SchemaField.INDEXED, null);
+    fieldType.init(oldSchema, rptMap);
+    fieldType.setTypeName("location_rpt");
+    SchemaField newField = new SchemaField("geo", fieldType, SchemaField.STORED | SchemaField.INDEXED, null);
     IndexSchema newSchema = oldSchema.addField(newField);
 
     h.getCore().setLatestSchema(newSchema);
@@ -263,6 +275,6 @@ public class SpatialRPTFieldTypeTest extends AbstractBadConfigTestBase {
   }
 
   private void setupRPTField(String distanceUnits, String geo) throws Exception {
-    setupRPTField(distanceUnits, geo, null);
+    setupRPTField(distanceUnits, geo, null, null);
   }
 }


[3/8] lucene-solr:SOLR-9191: LUCENE-7313: Remove DocValuesDocIdSet.

Posted by dr...@apache.org.
LUCENE-7313: Remove DocValuesDocIdSet.


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

Branch: refs/heads/SOLR-9191
Commit: 73045a4cd8ea5f0c8c4f418b6fbe99a94a21cf4a
Parents: fb37b3e
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Jun 8 14:20:40 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Jun 8 14:20:40 2016 +0200

----------------------------------------------------------------------
 .../apache/lucene/search/DocValuesDocIdSet.java | 157 -------------------
 1 file changed, 157 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/73045a4c/lucene/core/src/java/org/apache/lucene/search/DocValuesDocIdSet.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DocValuesDocIdSet.java b/lucene/core/src/java/org/apache/lucene/search/DocValuesDocIdSet.java
deleted file mode 100644
index 22c090c..0000000
--- a/lucene/core/src/java/org/apache/lucene/search/DocValuesDocIdSet.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search;
-
-import java.io.IOException;
-
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BitDocIdSet;
-import org.apache.lucene.util.FixedBitSet;
-
-/**
- * Base class for DocIdSet to be used with DocValues. The implementation
- * of its iterator is very stupid and slow if the implementation of the
- * {@link #matchDoc} method is not optimized, as iterators simply increment
- * the document id until {@code matchDoc(int)} returns true. Because of this
- * {@code matchDoc(int)} must be as fast as possible.
- * @lucene.internal
- */
-public abstract class DocValuesDocIdSet extends DocIdSet {
-
-  protected final int maxDoc;
-  protected final Bits acceptDocs;
-
-  public DocValuesDocIdSet(int maxDoc, Bits acceptDocs) {
-    this.maxDoc = maxDoc;
-    this.acceptDocs = acceptDocs;
-  }
-
-  /**
-   * this method checks, if a doc is a hit
-   */
-  protected abstract boolean matchDoc(int doc);
-
-  @Override
-  public long ramBytesUsed() {
-    return 0L;
-  }
-
-  @Override
-  public final Bits bits() {
-    return (acceptDocs == null) ? new Bits() {
-      @Override
-      public boolean get(int docid) {
-        return matchDoc(docid);
-      }
-
-      @Override
-      public int length() {
-        return maxDoc;
-      }
-    } : new Bits() {
-      @Override
-      public boolean get(int docid) {
-        return acceptDocs.get(docid) && matchDoc(docid);
-      }
-
-      @Override
-      public int length() {
-        return maxDoc;
-      }
-    };
-  }
-
-  @Override
-  public final DocIdSetIterator iterator() throws IOException {
-    if (acceptDocs == null) {
-      // Specialization optimization disregard acceptDocs
-      return new DocIdSetIterator() {
-        private int doc = -1;
-        
-        @Override
-        public int docID() {
-          return doc;
-        }
-      
-        @Override
-        public int nextDoc() {
-          do {
-            doc++;
-            if (doc >= maxDoc) {
-              return doc = NO_MORE_DOCS;
-            }
-          } while (!matchDoc(doc));
-          return doc;
-        }
-      
-        @Override
-        public int advance(int target) {
-          for(doc=target; doc<maxDoc; doc++) {
-            if (matchDoc(doc)) {
-              return doc;
-            }
-          }
-          return doc = NO_MORE_DOCS;
-        }
-
-        @Override
-        public long cost() {
-          return maxDoc;
-        }
-      };
-    } else if (acceptDocs instanceof FixedBitSet) {
-      // special case for FixedBitSet: use the iterator and filter it
-      // (used e.g. when Filters are chained by FilteredQuery)
-      return new FilteredDocIdSetIterator(new BitDocIdSet((FixedBitSet) acceptDocs).iterator()) {
-        @Override
-        protected boolean match(int doc) {
-          return DocValuesDocIdSet.this.matchDoc(doc);
-        }
-      };
-    } else {
-      // Stupid consultation of acceptDocs and matchDoc()
-      return new DocIdSetIterator() {
-        private int doc = -1;
-        
-        @Override
-        public int docID() {
-          return doc;
-        }
-      
-        @Override
-        public int nextDoc() {
-          return advance(doc + 1);
-        }
-      
-        @Override
-        public int advance(int target) {
-          for(doc=target; doc<maxDoc; doc++) {
-            if (acceptDocs.get(doc) && matchDoc(doc)) {
-              return doc;
-            }
-          }
-          return doc = NO_MORE_DOCS;
-        }
-
-        @Override
-        public long cost() {
-          return maxDoc;
-        }
-      };
-    }
-  }
-}


[6/8] lucene-solr:SOLR-9191: SOLR-9198: config APIs unable to add multiple values with same name

Posted by dr...@apache.org.
SOLR-9198: config APIs unable to add multiple values with same name


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

Branch: refs/heads/SOLR-9191
Commit: be011097445cb558d5ebf014e8f0e23085fbaec5
Parents: 86c053d
Author: Noble Paul <no...@apache.org>
Authored: Thu Jun 9 00:32:35 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Thu Jun 9 00:32:35 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 ++
 .../java/org/apache/solr/core/PluginInfo.java   | 16 +++++++++++--
 .../apache/solr/core/TestSolrConfigHandler.java | 24 ++++++++++++++++++++
 3 files changed, 40 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/be011097/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6cd982f..5d1d17c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -258,6 +258,8 @@ Bug Fixes
 
 * SOLR-8676: keep LOG4J_CONFIG in solr.cmd (Kristine Jetzke via Mikhail Khludnev)
 
+* SOLR-9198: config APIs unable to add multiple values with same name (noble)
+
 Optimizations
 ----------------------
 * SOLR-8722: Don't force a full ZkStateReader refresh on every Overseer operation.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/be011097/solr/core/src/java/org/apache/solr/core/PluginInfo.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/PluginInfo.java b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
index c6fe42c..832c8c2 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginInfo.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
@@ -67,8 +67,20 @@ public class PluginInfo implements MapSerializable{
     for (Map.Entry<String, Object> entry : map.entrySet()) {
       if (NAME.equals(entry.getKey()) || CLASS_NAME.equals(entry.getKey())) continue;
       Object value = entry.getValue();
-      if (value instanceof Map) value = new NamedList((Map) value);
-      initArgs.add(entry.getKey(), value);
+      if (value instanceof List) {
+        List list = (List) value;
+        if (!list.isEmpty() && list.get(0) instanceof Map) {//this is a subcomponent
+          for (Object o : list) {
+            if (o instanceof Map) o = new NamedList<>((Map) o);
+            initArgs.add(entry.getKey(), o);
+          }
+        } else {
+          initArgs.add(entry.getKey(), value);
+        }
+      } else {
+        if (value instanceof Map) value = new NamedList((Map) value);
+        initArgs.add(entry.getKey(), value);
+      }
     }
     this.type = type;
     this.name = (String) m.get(NAME);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/be011097/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
index 2d4a42a..250ccf0 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
@@ -426,6 +426,30 @@ public class TestSolrConfigHandler extends RestTestBase {
         Arrays.asList("config", "searchComponent","myspellcheck", "spellchecker", "class"),
         "solr.DirectSolrSpellChecker",
         10);
+
+    payload = "{\n" +
+        "    'add-requesthandler': {\n" +
+        "        name : '/dump100',\n" +
+        "        class : 'org.apache.solr.handler.DumpRequestHandler'," +
+        "        suggester: [{name: s1,lookupImpl: FuzzyLookupFactory, dictionaryImpl : DocumentDictionaryFactory}," +
+        "                    {name: s2,lookupImpl: FuzzyLookupFactory , dictionaryImpl : DocumentExpressionDictionaryFactory}]" +
+        "    }\n" +
+        "}";
+    runConfigCommand(writeHarness, "/config?wt=json", payload);
+    map = testForResponseElement(writeHarness,
+        testServerBaseUrl,
+        "/config?wt=json",
+        cloudSolrClient,
+        Arrays.asList("config", "requestHandler","/dump100", "class"),
+        "org.apache.solr.handler.DumpRequestHandler",
+        10);
+
+    map = getRespMap("/dump100?wt=json&json.nl=arrmap&initArgs=true", writeHarness);
+    List initArgs = (List) map.get("initArgs");
+    assertEquals(2, initArgs.size());
+    assertTrue(((Map)initArgs.get(0)).containsKey("suggester"));
+    assertTrue(((Map)initArgs.get(1)).containsKey("suggester"));
+    System.out.println(map);
   }
 
   public static Map testForResponseElement(RestTestHarness harness,


[4/8] lucene-solr:SOLR-9191: Add Version.LUCENE_6_2_0.

Posted by dr...@apache.org.
Add Version.LUCENE_6_2_0.


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

Branch: refs/heads/SOLR-9191
Commit: 18ac2978a91451356f46dc4e614f35f981ff36ee
Parents: 73045a4
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Jun 8 14:33:49 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Jun 8 14:33:49 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                                       | 3 +++
 lucene/core/src/java/org/apache/lucene/util/Version.java | 7 +++++++
 solr/CHANGES.txt                                         | 3 +++
 3 files changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/18ac2978/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index bd79c37..143067d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -6,6 +6,9 @@ http://s.apache.org/luceneversions
 ======================= Lucene 7.0.0 =======================
 (No Changes)
 
+======================= Lucene 6.2.0 =======================
+(No Changes)
+
 ======================= Lucene 6.1.0 =======================
 
 New Features

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/18ac2978/lucene/core/src/java/org/apache/lucene/util/Version.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index d5640d8..3bd817e 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -53,6 +53,13 @@ public final class Version {
   public static final Version LUCENE_6_1_0 = new Version(6, 1, 0);
 
   /**
+   * Match settings and bugs in Lucene's 6.2.0 release.
+   * @deprecated Use latest
+   */
+  @Deprecated
+  public static final Version LUCENE_6_2_0 = new Version(6, 2, 0);
+
+  /**
    * Match settings and bugs in Lucene's 7.0.0 release.
    *  <p>
    *  Use this to get the latest &amp; greatest settings, bug

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/18ac2978/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index bed02f2..6cd982f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -45,6 +45,9 @@ Optimizations
   check on every request and move connection lifecycle management towards the client.
   (Ryan Zezeski, Mark Miller, Shawn Heisey, Steve Davids)
 
+==================  6.2.0 ==================
+(No Changes)
+
 ==================  6.1.0 ==================
 
 Upgrading from Solr any prior release


[8/8] lucene-solr:SOLR-9191: SOLR-9191: OverseerTaskQueue.peekTopN() fatally flawed

Posted by dr...@apache.org.
SOLR-9191: OverseerTaskQueue.peekTopN() fatally flawed


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

Branch: refs/heads/SOLR-9191
Commit: ed2621de8402154440e3e1e1979eb620bd23b926
Parents: 6c0331b
Author: Scott Blum <dr...@gmail.com>
Authored: Tue Jun 7 01:52:16 2016 -0400
Committer: Scott Blum <dr...@apache.org>
Committed: Thu Jun 9 00:59:03 2016 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  6 +-
 .../org/apache/solr/cloud/DistributedQueue.java | 95 ++++++++++++--------
 .../solr/cloud/OverseerTaskProcessor.java       |  6 +-
 .../apache/solr/cloud/OverseerTaskQueue.java    | 48 ++++------
 .../apache/solr/cloud/DistributedQueueTest.java | 44 ++++++++-
 5 files changed, 127 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed2621de/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d73c61f..c8e7c1f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -46,7 +46,11 @@ Optimizations
   (Ryan Zezeski, Mark Miller, Shawn Heisey, Steve Davids)
 
 ==================  6.2.0 ==================
-(No Changes)
+
+Bug Fixes
+----------------------
+
+* SOLR-9191: OverseerTaskQueue.peekTopN() fatally flawed (Scott Blum, Noble Paul)
 
 ==================  6.1.0 ==================
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed2621de/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java b/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
index e424b7e..afed6f1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
@@ -17,14 +17,15 @@
 package org.apache.solr.cloud;
 
 import java.lang.invoke.MethodHandles;
-import java.util.Collections;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.NoSuchElementException;
-import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -32,6 +33,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCmdExecutor;
+import org.apache.solr.common.util.Pair;
 import org.apache.solr.util.stats.TimerContext;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -80,21 +82,15 @@ public class DistributedQueue {
   private TreeSet<String> knownChildren = new TreeSet<>();
 
   /**
-   * Used to wait on a non-empty queue; you must hold {@link #updateLock} and verify that
-   * {@link #knownChildren} is empty before waiting on this condition.
+   * Used to wait on ZK changes to the child list; you must hold {@link #updateLock} before waiting on this condition.
    */
-  private final Condition notEmpty = updateLock.newCondition();
+  private final Condition changed = updateLock.newCondition();
 
   /**
-   * If non-null, the last watcher to listen for child changes.
+   * If non-null, the last watcher to listen for child changes.  If null, the in-memory contents are dirty.
    */
   private ChildWatcher lastWatcher = null;
 
-  /**
-   * If true, ZK's child list probably doesn't match what's in memory.
-   */
-  private boolean isDirty = true;
-
   public DistributedQueue(SolrZkClient zookeeper, String dir) {
     this(zookeeper, dir, new Overseer.Stats());
   }
@@ -165,7 +161,7 @@ public class DistributedQueue {
         if (result != null) {
           return result;
         }
-        waitNanos = notEmpty.awaitNanos(waitNanos);
+        waitNanos = changed.awaitNanos(waitNanos);
       }
       return null;
     } finally {
@@ -222,7 +218,7 @@ public class DistributedQueue {
         if (result != null) {
           return result;
         }
-        notEmpty.await();
+        changed.await();
       }
     } finally {
       updateLock.unlock();
@@ -273,25 +269,19 @@ public class DistributedQueue {
   private String firstChild(boolean remove) throws KeeperException, InterruptedException {
     updateLock.lockInterruptibly();
     try {
-      // Try to fetch the first in-memory child.
-      if (!knownChildren.isEmpty()) {
+      // If we're not in a dirty state, and we have in-memory children, return from in-memory.
+      if (lastWatcher != null && !knownChildren.isEmpty()) {
         return remove ? knownChildren.pollFirst() : knownChildren.first();
       }
 
-      if (lastWatcher != null && !isDirty) {
-        // No children, no known updates, and a watcher is already set; nothing we can do.
-        return null;
-      }
-
       // Try to fetch an updated list of children from ZK.
       ChildWatcher newWatcher = new ChildWatcher();
       knownChildren = fetchZkChildren(newWatcher);
       lastWatcher = newWatcher; // only set after fetchZkChildren returns successfully
-      isDirty = false;
       if (knownChildren.isEmpty()) {
         return null;
       }
-      notEmpty.signalAll();
+      changed.signalAll();
       return remove ? knownChildren.pollFirst() : knownChildren.first();
     } finally {
       updateLock.unlock();
@@ -325,26 +315,63 @@ public class DistributedQueue {
   }
 
   /**
-   * Return the currently-known set of children from memory. If there are no children,
-   * waits up to {@code waitMillis} for at least one child to become available. May
-   * update the set of known children.
+   * Return the currently-known set of elements, using child names from memory. If no children are found, or no
+   * children pass {@code acceptFilter}, waits up to {@code waitMillis} for at least one child to become available.
+   * <p/>
+   * Package-private to support {@link OverseerTaskQueue} specifically.
    */
-  SortedSet<String> getChildren(long waitMillis) throws KeeperException, InterruptedException {
+  Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Function<String, Boolean> acceptFilter) throws KeeperException, InterruptedException {
+    List<String> foundChildren = new ArrayList<>();
     long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
-    while (waitNanos > 0) {
+    while (true) {
       // Trigger a fetch if needed.
-      firstElement();
+      firstChild(false);
+
       updateLock.lockInterruptibly();
       try {
-        if (!knownChildren.isEmpty()) {
-          return new TreeSet<>(knownChildren);
+        for (String child : knownChildren) {
+          if (acceptFilter.apply(child)) {
+            foundChildren.add(child);
+          }
         }
-        waitNanos = notEmpty.awaitNanos(waitNanos);
+        if (!foundChildren.isEmpty()) {
+          break;
+        }
+        if (waitNanos <= 0) {
+          break;
+        }
+        waitNanos = changed.awaitNanos(waitNanos);
       } finally {
         updateLock.unlock();
       }
+
+      if (!foundChildren.isEmpty()) {
+        break;
+      }
+    }
+
+    // Technically we could restart the method if we fail to actually obtain any valid children
+    // from ZK, but this is a super rare case, and the latency of the ZK fetches would require
+    // much more sophisticated waitNanos tracking.
+    List<Pair<String, byte[]>> result = new ArrayList<>();
+    for (String child : foundChildren) {
+      if (result.size() >= max) {
+        break;
+      }
+      try {
+        byte[] data = zookeeper.getData(dir + "/" + child, null, null, true);
+        result.add(new Pair<>(child, data));
+      } catch (KeeperException.NoNodeException e) {
+        // Another client deleted the node first, remove the in-memory and continue.
+        updateLock.lockInterruptibly();
+        try {
+          knownChildren.remove(child);
+        } finally {
+          updateLock.unlock();
+        }
+      }
     }
-    return Collections.emptySortedSet();
+    return result;
   }
 
   /**
@@ -418,10 +445,8 @@ public class DistributedQueue {
         if (lastWatcher == this) {
           lastWatcher = null;
         }
-        // Do no updates in this thread, just signal state back to client threads.
-        isDirty = true;
         // optimistically signal any waiters that the queue may not be empty now, so they can wake up and retry
-        notEmpty.signalAll();
+        changed.signalAll();
       } finally {
         updateLock.unlock();
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed2621de/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
index 93a7e6f..092ed97 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
@@ -190,9 +190,9 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
             cleanUpWorkQueue();
 
           List<QueueEvent> heads = workQueue.peekTopN(MAX_PARALLEL_TASKS, runningZKTasks, 2000L);
-
-          if (heads == null)
+          if (heads.isEmpty()) {
             continue;
+          }
 
           log.debug("Got {} tasks from work-queue : [{}]", heads.size(), heads.toString());
 
@@ -466,6 +466,8 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
           log.warn("Could not find and remove async call [" + asyncId + "] from the running map.");
         }
       }
+
+      workQueue.remove(head);
     }
 
     private void resetTaskWithException(OverseerMessageHandler messageHandler, String id, String asyncId, String taskKey, ZkNodeProps message) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed2621de/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
index 4cee814..aae7df2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
@@ -17,7 +17,6 @@
 package org.apache.solr.cloud;
 
 import java.lang.invoke.MethodHandles;
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
@@ -25,6 +24,7 @@ import java.util.TreeSet;
 
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.util.Pair;
 import org.apache.solr.util.stats.TimerContext;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -82,9 +82,8 @@ public class OverseerTaskQueue extends DistributedQueue {
 
   /**
    * Remove the event and save the response into the other path.
-   * 
    */
-  public byte[] remove(QueueEvent event) throws KeeperException,
+  public void remove(QueueEvent event) throws KeeperException,
       InterruptedException {
     TimerContext time = stats.time(dir + "_remove_event");
     try {
@@ -97,9 +96,10 @@ public class OverseerTaskQueue extends DistributedQueue {
         LOG.info("Response ZK path: " + responsePath + " doesn't exist."
             + "  Requestor may have disconnected from ZooKeeper");
       }
-      byte[] data = zookeeper.getData(path, null, null, true);
-      zookeeper.delete(path, -1, true);
-      return data;
+      try {
+        zookeeper.delete(path, -1, true);
+      } catch (KeeperException.NoNodeException ignored) {
+      }
     } finally {
       time.stop();
     }
@@ -227,44 +227,26 @@ public class OverseerTaskQueue extends DistributedQueue {
     ArrayList<QueueEvent> topN = new ArrayList<>();
 
     LOG.debug("Peeking for top {} elements. ExcludeSet: {}", n, excludeSet);
-    TimerContext time = null;
+    TimerContext time;
     if (waitMillis == Long.MAX_VALUE) time = stats.time(dir + "_peekTopN_wait_forever");
     else time = stats.time(dir + "_peekTopN_wait" + waitMillis);
 
     try {
-      for (String headNode : getChildren(waitMillis)) {
-        if (topN.size() < n) {
-          try {
-            String id = dir + "/" + headNode;
-            if (excludeSet.contains(id)) continue;
-            QueueEvent queueEvent = new QueueEvent(id,
-                zookeeper.getData(dir + "/" + headNode, null, null, true), null);
-            topN.add(queueEvent);
-          } catch (KeeperException.NoNodeException e) {
-            // Another client removed the node first, try next
-          }
-        } else {
-          if (topN.size() >= 1) {
-            printQueueEventsListElementIds(topN);
-            return topN;
-          }
-        }
-      }
-
-      if (topN.size() > 0 ) {
-        printQueueEventsListElementIds(topN);
-        return topN;
+      for (Pair<String, byte[]> element : peekElements(n, waitMillis, child -> !excludeSet.contains(dir + "/" + child))) {
+        topN.add(new QueueEvent(dir + "/" + element.first(),
+            element.second(), null));
       }
-      return null;
+      printQueueEventsListElementIds(topN);
+      return topN;
     } finally {
       time.stop();
     }
   }
 
   private static void printQueueEventsListElementIds(ArrayList<QueueEvent> topN) {
-    if(LOG.isDebugEnabled()) {
-      StringBuffer sb = new StringBuffer("[");
-      for(QueueEvent queueEvent: topN) {
+    if (LOG.isDebugEnabled() && !topN.isEmpty()) {
+      StringBuilder sb = new StringBuilder("[");
+      for (QueueEvent queueEvent : topN) {
         sb.append(queueEvent.getId()).append(", ");
       }
       sb.append("]");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed2621de/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java b/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
index 840e7e5..f42f101 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
@@ -19,7 +19,6 @@ package org.apache.solr.cloud;
 import java.nio.charset.Charset;
 import java.util.NoSuchElementException;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -137,6 +136,49 @@ public class DistributedQueueTest extends SolrTestCaseJ4 {
     assertNull(dq.poll());
   }
 
+  @Test
+  public void testPeekElements() throws Exception {
+    String dqZNode = "/distqueue/test";
+    byte[] data = "hello world".getBytes(UTF8);
+
+    DistributedQueue dq = makeDistributedQueue(dqZNode);
+
+    // Populate with data.
+    dq.offer(data);
+    dq.offer(data);
+    dq.offer(data);
+
+    // Should be able to get 0, 1, 2, or 3 instantly
+    for (int i = 0; i <= 3; ++i) {
+      assertEquals(i, dq.peekElements(i, 0, child -> true).size());
+    }
+
+    // Asking for more should return only 3.
+    assertEquals(3, dq.peekElements(4, 0, child -> true).size());
+
+    // If we filter everything out, we should block for the full time.
+    long start = System.nanoTime();
+    assertEquals(0, dq.peekElements(4, 1000, child -> false).size());
+    assertTrue(System.nanoTime() - start >= TimeUnit.MILLISECONDS.toNanos(500));
+
+    // If someone adds a new matching element while we're waiting, we should return immediately.
+    executor.submit(() -> {
+      try {
+        Thread.sleep(500);
+        dq.offer(data);
+      } catch (Exception e) {
+        // ignore
+      }
+    });
+    start = System.nanoTime();
+    assertEquals(1, dq.peekElements(4, 2000, child -> {
+      // The 4th element in the queue will end with a "3".
+      return child.endsWith("3");
+    }).size());
+    assertTrue(System.nanoTime() - start < TimeUnit.MILLISECONDS.toNanos(1000));
+    assertTrue(System.nanoTime() - start >= TimeUnit.MILLISECONDS.toNanos(250));
+  }
+
   private void forceSessionExpire() throws InterruptedException, TimeoutException {
     long sessionId = zkClient.getSolrZooKeeper().getSessionId();
     zkServer.expire(sessionId);


[5/8] lucene-solr:SOLR-9191: SOLR-9107: When creating a randomized SSL test config, 100% effective odds of using SSL and/or client auth should never be trumped by chance.

Posted by dr...@apache.org.
SOLR-9107: When creating a randomized SSL test config, 100% effective odds of using SSL and/or client auth should never be trumped by chance.


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

Branch: refs/heads/SOLR-9191
Commit: 86c053dd1055c5b2b4cfe3c8e6b573d3d1272b24
Parents: 18ac297
Author: Steve Rowe <sa...@apache.org>
Authored: Wed Jun 8 14:39:01 2016 -0400
Committer: Steve Rowe <sa...@apache.org>
Committed: Wed Jun 8 14:39:01 2016 -0400

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/86c053dd/solr/test-framework/src/java/org/apache/solr/util/RandomizeSSL.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/RandomizeSSL.java b/solr/test-framework/src/java/org/apache/solr/util/RandomizeSSL.java
index e7336d8..2450dad 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/RandomizeSSL.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/RandomizeSSL.java
@@ -105,9 +105,9 @@ public @interface RandomizeSSL {
       // even if we know SSL is disabled, always consume the same amount of randomness
       // that way all other test behavior should be consistent even if a user adds/removes @SuppressSSL
       
-      final boolean useSSL = TestUtil.nextInt(LuceneTestCase.random(), 0, 1000) < 
+      final boolean useSSL = TestUtil.nextInt(LuceneTestCase.random(), 0, 999) <
         (int)(1000 * getEffectiveOdds(ssl, LuceneTestCase.TEST_NIGHTLY, LuceneTestCase.RANDOM_MULTIPLIER));
-      final boolean useClientAuth = TestUtil.nextInt(LuceneTestCase.random(), 0, 1000) < 
+      final boolean useClientAuth = TestUtil.nextInt(LuceneTestCase.random(), 0, 999) <
         (int)(1000 * getEffectiveOdds(clientAuth, LuceneTestCase.TEST_NIGHTLY, LuceneTestCase.RANDOM_MULTIPLIER));
 
       return new SSLTestConfig(useSSL, useClientAuth);


[7/8] lucene-solr:SOLR-9191: CHANGES.txt-s: Move issues released with 6.0.1 into the 6.0.1 sections

Posted by dr...@apache.org.
CHANGES.txt-s: Move issues released with 6.0.1 into the 6.0.1 sections


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

Branch: refs/heads/SOLR-9191
Commit: 6c0331b8309603eaaf14b6677afba5ffe99f16a3
Parents: be01109
Author: Steve Rowe <sa...@apache.org>
Authored: Wed Jun 8 17:01:43 2016 -0400
Committer: Steve Rowe <sa...@apache.org>
Committed: Wed Jun 8 17:01:58 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt |  89 +++++++++++++------------
 solr/CHANGES.txt   | 173 ++++++++++++++++++++++++++----------------------
 2 files changed, 140 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c0331b8/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 143067d..26eff54 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -25,10 +25,6 @@ New Features
 * LUCENE-7234: Added InetAddressPoint.nextDown/nextUp to easily generate range
   queries with excluded bounds. (Adrien Grand)
 
-* LUCENE-7278: Spatial-extras DateRangePrefixTree's Calendar is now configurable, to
-  e.g. clear the Gregorian Change Date.  Also, toString(cal) is now identical to
-  DateTimeFormatter.ISO_INSTANT. (David Smiley)
-
 * LUCENE-7300: The misc module now has a directory wrapper that uses hard-links if
   applicable and supported when copying files from another FSDirectory in 
   Directory#copyFrom. (Simon Willnauer)
@@ -112,38 +108,6 @@ Bug Fixes
 * LUCENE-7168: Switch to stable encode for geo3d, remove quantization
   test leniency, remove dead code (Mike McCandless)
 
-* LUCENE-7187: Block join queries' Weight#extractTerms(...) implementations
-  should delegate to the wrapped weight. (Martijn van Groningen)
-
-* LUCENE-7188: remove incorrect sanity check in NRTCachingDirectory.listAll()
-  that led to IllegalStateException being thrown when nothing was wrong.
-  (David Smiley, yonik)  
-
-* LUCENE-7219: Make queryparser/xml (Point|LegacyNumeric)RangeQuery builders
-  match the underlying queries' (lower|upper)Term optionality logic.
-  (Kaneshanathan Srivisagan, Christine Poerschke)
-
-* LUCENE-7209: Fixed explanations of FunctionScoreQuery. (Adrien Grand)
-
-* LUCENE-7232: Fixed InetAddressPoint.newPrefixQuery, which was generating an
-  incorrect query when the prefix length was not a multiple of 8. (Adrien Grand)
-
-* LUCENE-7257: Fixed PointValues#size(IndexReader, String), docCount,
-  minPackedValue and maxPackedValue to skip leaves that do not have points
-  rather than raising an IllegalStateException. (Adrien Grand)
-
-* LUCENE-7279: JapaneseTokenizer throws ArrayIndexOutOfBoundsException
-  on some valid inputs (Mike McCandless)
-
-* LUCENE-7284: GapSpans needs to implement positionsCost(). (Daniel Bigham, Alan
-  Woodward)
-
-* LUCENE-7231: WeightedSpanTermExtractor didn't deal correctly with single-term
-  phrase queries. (Eva Popenda, Alan Woodward)
-
-* LUCENE-7293: Don't try to highlight GeoPoint queries (Britta Weber,
-  Nick Knize, Mike McCandless, Uwe Schindler)
-
 * LUCENE-7301: Multiple doc values updates to the same document within
   one update batch could be applied in the wrong order resulting in
   the wrong updated value (Ishan Chattopadhyaya, hossman, Mike McCandless)
@@ -156,12 +120,6 @@ Bug Fixes
   other ranges had more than one clause matching (Ahmet Arslan,
   hossman, Mike McCandless)
 
-Documentation
-
-* LUCENE-7223: Improve XXXPoint javadocs to make it clear that you
-  should separately add StoredField if you want to retrieve these
-  field values at search time (Greg Huber, Robert Muir, Mike McCandless)
-
 Other
 
 * LUCENE-7295: TermAutomatonQuery.hashCode calculates Automaton.toDot().hash,
@@ -200,7 +158,52 @@ Build
   (Uwe Schindler)
   
 ======================= Lucene 6.0.1 =======================
-(No Changes)
+
+New Features
+
+* LUCENE-7278: Spatial-extras DateRangePrefixTree's Calendar is now configurable, to
+  e.g. clear the Gregorian Change Date.  Also, toString(cal) is now identical to
+  DateTimeFormatter.ISO_INSTANT. (David Smiley)
+
+Bug Fixes
+
+* LUCENE-7187: Block join queries' Weight#extractTerms(...) implementations
+  should delegate to the wrapped weight. (Martijn van Groningen)
+
+* LUCENE-7209: Fixed explanations of FunctionScoreQuery. (Adrien Grand)
+
+* LUCENE-7232: Fixed InetAddressPoint.newPrefixQuery, which was generating an
+  incorrect query when the prefix length was not a multiple of 8. (Adrien Grand)
+
+* LUCENE-7279: JapaneseTokenizer throws ArrayIndexOutOfBoundsException
+  on some valid inputs (Mike McCandless)
+
+* LUCENE-7188: remove incorrect sanity check in NRTCachingDirectory.listAll()
+  that led to IllegalStateException being thrown when nothing was wrong.
+  (David Smiley, yonik)
+
+* LUCENE-7219: Make queryparser/xml (Point|LegacyNumeric)RangeQuery builders
+  match the underlying queries' (lower|upper)Term optionality logic.
+  (Kaneshanathan Srivisagan, Christine Poerschke)
+
+* LUCENE-7257: Fixed PointValues#size(IndexReader, String), docCount,
+  minPackedValue and maxPackedValue to skip leaves that do not have points
+  rather than raising an IllegalStateException. (Adrien Grand)
+
+* LUCENE-7284: GapSpans needs to implement positionsCost(). (Daniel Bigham, Alan
+  Woodward)
+
+* LUCENE-7231: WeightedSpanTermExtractor didn't deal correctly with single-term
+  phrase queries. (Eva Popenda, Alan Woodward)
+
+* LUCENE-7293: Don't try to highlight GeoPoint queries (Britta Weber,
+  Nick Knize, Mike McCandless, Uwe Schindler)
+
+Documentation
+
+* LUCENE-7223: Improve XXXPoint javadocs to make it clear that you
+  should separately add StoredField if you want to retrieve these
+  field values at search time (Greg Huber, Robert Muir, Mike McCandless)
 
 ======================= Lucene 6.0.0 =======================
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c0331b8/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5d1d17c..d73c61f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -158,92 +158,29 @@ New Features
 Bug Fixes
 ----------------------
 
-* SOLR-8857: HdfsUpdateLog does not use configured or new default number of version buckets and is
-  hard coded to 256. (Mark Miller, yonik, Gregory Chanan)
-
 * SOLR-8855: The HDFS BlockDirectory should not clean up it's cache on shutdown. (Mark Miller)
 
-* SOLR-8902: Make sure ReturnFields only returns the requested fields from (fl=) evn when 
-  DocumentTransformers ask for getExtraRequestFields()  (ryan)
-
-* SOLR-8875: SolrCloud Overseer clusterState could unexpectedly be null resulting in NPE.
-  (Scott Blum via David Smiley)
-
 * SOLR-8948: OverseerTaskQueue.containsTaskWithRequestId encounters json parse error if a
   SolrResponse node is in the overseer queue. (Jessica Cheng Mallet via shalin)
 
-* SOLR-8914: ZkStateReader's refreshLiveNodes(Watcher) is not thread safe. (Scott Blum, hoss,
-  sarowe, Erick Erickson, Mark Miller, shalin)
-
 * SOLR-7729: ConcurrentUpdateSolrClient ignores the collection parameter in some methods.
   (Nicolas Gavalda, Jorge Luis Betancourt Gonzalez via Mark Miller)
 
-* SOLR-8946: bin/post failed to detect stdin usage on Ubuntu; maybe other unixes. (David Smiley)
-
 * SOLR-8662: SchemaManager waits correctly for replicas to be notified of a new change.
   (sarowe, Noble Paul, Varun Thacker)
 
-* SOLR-9004: Fix "name" field type definition in films example. (Alexandre Rafalovitch via Varun Thacker)
-
 * SOLR-8983: Cleanup clusterstate and replicas for a failed create collection request
   (Varun Thacker, Anshum Gupta)
 
-* SOLR-8990: Fix top term links from schema browser page to use {!term} parser (hossman)
-
-* SOLR-8971: Preserve root cause when wrapping exceptions (hossman)
-
-* SOLR-8992: Restore Schema API GET method functionality removed in 6.0 (noble, Steve Rowe)
-
 * SOLR-9029: fix rare ZkStateReader visibility race during collection state format update (Scott Blum, hossman)
 
-* SOLR-9016: Fix SolrIdentifierValidator to not allow empty identifiers. (Shai Erera)
-
 * SOLR-9046: Fix solr.cmd that wrongly assumes Jetty will always listen on 0.0.0.0.
   (Bram Van Dam, Uwe Schindler)
 
-* SOLR-9034: Atomic updates failed to work when there were copyField targets that had docValues
-  enabled. (Karthik Ramachandran, Ishan Chattopadhyaya, yonik)
-
-* SOLR-9028: Fixed some test related bugs preventing SSL + ClientAuth from ever being tested (hossman)
-
-* SOLR-9059: NPE in SolrClientCache following collection reload (Joel Bernstein, Ryan Yacyshyn)
-
-* SOLR-8792: ZooKeeper ACL support fixed. (Esther Quansah, Ishan Chattopadhyaya, Steve Rowe)
-
 * SOLR-9064: Adds an explanation of the incoming stream to an UpdateStream's explanation (Dennis Gove)
 
-* SOLR-9030: The 'downnode' overseer command can trip asserts in ZkStateWriter.
-  (Scott Blum, Mark Miller, shalin)
-
-* SOLR-9036: Solr slave is doing full replication (entire index) of index after master restart.
-  (Lior Sapir, Mark Miller, shalin)
-
-* SOLR-9058: Makes HashJoinStream and OuterHashJoinStream support different field names in the 
-  incoming streams, eg. fieldA=fieldB. (Dennis Gove, Stephan Osthold)
-
-* SOLR-9093: Fix NullPointerException in TopGroupsShardResponseProcessor. (Christine Poerschke)
-
-* SOLR-8970: Change SSLTestConfig to use a keystore file that is included as a resource in the
-  test-framework jar so users subclassing SolrTestCaseJ4 don't need to preserve magic paths (hossman)
-
-* SOLR-9080, SOLR-9085: (6.0 bug) For years <= 1582, date math (round,add,sub) introduced error.  Range faceting
-  on such dates was also affected.  With this fixed, this is the first release range faceting works on BC years.
-  (David Smiley)
-
-* SOLR-9118: HashQParserPlugin should trim partition keys (Joel Bernstein)
-
-* SOLR-9117: The first SolrCore is leaked after reload. (Jessica Cheng Mallet via shalin)
-
-* SOLR-9116: Race condition causing occasional SolrIndexSearcher leak when SolrCore is reloaded.
-  (Jessica Cheng Mallet via shalin)
-
 * SOLR-9128: Fix error handling issues in Streaming classes (Joel Bernstein)
 
-* SOLR-8801: /bin/solr create script always returns exit code 0 when a collection/core already exists.
-  (Khalid Alharbi, Marius Grama via Steve Rowe)
-
-* SOLR-9134: Fix RestManager.addManagedResource return value. (Christine Poerschke)
-
 * SOLR-9151: Fix SolrCLI so that bin/solr -e cloud example can be run from any CWD (janhoy)
 
 * SOLR-9141: Fix ClassCastException when using the /sql handler count() function with 
@@ -290,9 +227,6 @@ Optimizations
 
 Other Changes
 ----------------------
-* SOLR-7516: Improve javadocs for JavaBinCodec, ObjectResolver and enforce the single-usage policy.
-  (Jason Gerlowski, Benoit Vanalderweireldt, shalin)
-
 * SOLR-8860: Remove back-compat handling of router format made in SOLR-4221 in 4.5.0. (shalin)
 
 * SOLR-8866: UpdateLog will now throw an exception if it doesn't know how to serialize a value.
@@ -313,11 +247,6 @@ Other Changes
 * SOLR-8097: Implement builder pattern design for constructing SolrJ clients and also deprecate direct construction
   of client objects. (Jason Gerlowski, Shawn Heisey, Anshum Gupta)
 
-* SOLR-8967: In SolrCloud mode, under the 'Core Selector' dropdown in the UI the Replication tab won't be displayed
-  anymore. The Replication tab is only beneficial to users running Solr in master-slave mode. (Varun Thacker)
-
-* SOLR-8985: Added back support for 'includeDynamic' flag to /schema/fields endpoint (noble)
-
 * SOLR-9015: Adds SelectStream as a default function in the StreamHandler (Dennis Gove)
 
 * SOLR-8929: Add an idea module for solr/server to enable launching start.jar (Scott Blum, Steve Rowe)
@@ -328,8 +257,6 @@ Other Changes
 
 * SOLR-9047: zkcli should allow alternative locations for log4j configuration (Gregory Chanan)
 
-* SOLR-9053: Upgrade commons-fileupload to 1.3.1, fixing a potential vulnerability (Jeff Field, Mike Drob via janhoy)
-
 * SOLR-9066: Make CountMetric return long instead of double (Kevin Risden)
 
 * SOLR-9065, SOLR-9072, SOLR-9132: Migrate some distributed tests to SolrCloudTestCase. (Alan Woodward)
@@ -346,11 +273,6 @@ Other Changes
 
 * SOLR-9105: Fix a bunch of typos across 103 files (Bartosz Krasi\u0144ski via janhoy)
 
-* SOLR-9115: Get rid of javax.xml.bind.DatatypeConverter in SimplePostTool
-  for Java 9 compatibility. (Uwe Schindler)
-
-* SOLR-9131: Fix "start solr" text in cluster.vm Velocity template (janhoy)
-
 * SOLR-9159: New cloud based concurrent atomic update test (hossman)
 
 * SOLR-9119: several static methods in ValueSourceParser have been made private (hossman)
@@ -368,7 +290,100 @@ Other Changes
 * SOLR-8445: fix line separator in log4j.properties files (Ahmet Arslan via Mikhail Khludnev)
 
 ==================  6.0.1 ==================
-(No Changes)
+
+Upgrade Notes
+----------------------
+
+* If you use historical dates, specifically on or before the year 1582, you should re-index.
+
+Bug Fixes
+----------------------
+
+* SOLR-8914: ZkStateReader's refreshLiveNodes(Watcher) is not thread safe. (Scott Blum, hoss,
+  sarowe, Erick Erickson, Mark Miller, shalin)
+
+* SOLR-9016: Fix SolrIdentifierValidator to not allow empty identifiers. (Shai Erera)
+
+* SOLR-8992: Restore Schema API GET method functionality removed in 6.0 (noble, Steve Rowe)
+
+* SOLR-9080, SOLR-9085: (6.0 bug) For years <= 1582, date math (round,add,sub) introduced error.  Range faceting
+  on such dates was also affected.  With this fixed, this is the first release range faceting works on BC years.
+  (David Smiley)
+
+* SOLR-8857: HdfsUpdateLog does not use configured or new default number of version buckets and is
+  hard coded to 256. (Mark Miller, yonik, Gregory Chanan)
+
+* SOLR-8902: Make sure ReturnFields only returns the requested fields from (fl=) evn when
+  DocumentTransformers ask for getExtraRequestFields()  (ryan)
+
+* SOLR-8875: SolrCloud Overseer clusterState could unexpectedly be null resulting in NPE.
+  (Scott Blum via David Smiley)
+
+* SOLR-8946: bin/post failed to detect stdin usage on Ubuntu; maybe other unixes. (David Smiley)
+
+* SOLR-9004: Fix "name" field type definition in films example. (Alexandre Rafalovitch via Varun Thacker)
+
+* SOLR-8990: Fix top term links from schema browser page to use {!term} parser (hossman)
+
+* SOLR-8971: Preserve root cause when wrapping exceptions (hossman)
+
+* SOLR-9034: Atomic updates failed to work when there were copyField targets that had docValues
+  enabled. (Karthik Ramachandran, Ishan Chattopadhyaya, yonik)
+
+* SOLR-9028: Fixed some test related bugs preventing SSL + ClientAuth from ever being tested (hossman)
+
+* SOLR-9059: NPE in SolrClientCache following collection reload (Joel Bernstein, Ryan Yacyshyn)
+
+* SOLR-8792: ZooKeeper ACL support fixed. (Esther Quansah, Ishan Chattopadhyaya, Steve Rowe)
+
+* SOLR-9030: The 'downnode' overseer command can trip asserts in ZkStateWriter.
+  (Scott Blum, Mark Miller, shalin)
+
+* SOLR-9036: Solr slave is doing full replication (entire index) of index after master restart.
+  (Lior Sapir, Mark Miller, shalin)
+
+* SOLR-9058: Makes HashJoinStream and OuterHashJoinStream support different field names in the
+  incoming streams, eg. fieldA=fieldB. (Dennis Gove, Stephan Osthold)
+
+* SOLR-9093: Fix NullPointerException in TopGroupsShardResponseProcessor. (Christine Poerschke)
+
+* SOLR-9118: HashQParserPlugin should trim partition keys (Joel Bernstein)
+
+* SOLR-9117: The first SolrCore is leaked after reload. (Jessica Cheng Mallet via shalin)
+
+* SOLR-9116: Race condition causing occasional SolrIndexSearcher leak when SolrCore is reloaded.
+  (Jessica Cheng Mallet via shalin)
+
+* SOLR-8801: /bin/solr create script always returns exit code 0 when a collection/core already exists.
+  (Khalid Alharbi, Marius Grama via Steve Rowe)
+
+* SOLR-9134: Fix RestManager.addManagedResource return value. (Christine Poerschke)
+
+Other Changes
+----------------------
+* SOLR-7516: Improve javadocs for JavaBinCodec, ObjectResolver and enforce the single-usage policy.
+  (Jason Gerlowski, Benoit Vanalderweireldt, shalin)
+
+* SOLR-8967: In SolrCloud mode, under the 'Core Selector' dropdown in the UI the Replication tab won't be displayed
+  anymore. The Replication tab is only beneficial to users running Solr in master-slave mode. (Varun Thacker)
+
+* SOLR-8985: Added back support for 'includeDynamic' flag to /schema/fields endpoint (noble)
+
+* SOLR-9131: Fix "start solr" text in cluster.vm Velocity template (janhoy)
+
+* SOLR-9053: Upgrade commons-fileupload to 1.3.1, fixing a potential vulnerability (Jeff Field, Mike Drob via janhoy)
+
+* SOLR-9115: Get rid of javax.xml.bind.DatatypeConverter in SimplePostTool
+  for Java 9 compatibility. (Uwe Schindler)
+
+* SOLR-5776,SOLR-9068,SOLR-8970:
+  - Refactor SSLConfig so that SSLTestConfig can provide SSLContexts using a NullSecureRandom
+    to prevent SSL tests from blocking on entropy starved machines.
+  - SSLTestConfig: Alternate (psuedo random) NullSecureRandom for Constants.SUN_OS.
+  - SSLTestConfig: Replace NullSecureRandom w/ NotSecurePsuedoRandom.
+  - Change SSLTestConfig to use a keystore file that is included as a resource in the
+    test-framework jar so users subclassing SolrTestCaseJ4 don't need to preserve magic paths.
+  (hossman)
 
 ==================  6.0.0 ==================