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

lucene-solr:master: SOLR-7955: Auto create .system collection on first request if it does not exist

Repository: lucene-solr
Updated Branches:
  refs/heads/master c1d9b87d7 -> e200b8a2a


SOLR-7955: Auto create .system collection on first request if it does not exist


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

Branch: refs/heads/master
Commit: e200b8a2a418cdb145acb51d1181b1b60362a926
Parents: c1d9b87
Author: Noble Paul <no...@apache.org>
Authored: Thu Feb 2 15:26:26 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Thu Feb 2 15:26:26 2017 +1030

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 ++
 .../org/apache/solr/servlet/HttpSolrCall.java   | 42 ++++++++++++++++++-
 .../apache/solr/handler/TestBlobHandler.java    | 44 +++++++++++---------
 .../solr/handler/TestSystemCollAutoCreate.java  | 29 +++++++++++++
 4 files changed, 97 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e200b8a2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 44c5316..b4f9016 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -124,6 +124,9 @@ New Features
 * SOLR-9933: SolrCoreParser now supports configuration of custom SpanQueryBuilder classes.
   (Daniel Collins, Christine Poerschke)
 
+* SOLR-7955: Auto create .system collection on first request if it does not exist (noble)
+
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e200b8a2/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index b244015..521c7bb 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -18,7 +18,6 @@ package org.apache.solr.servlet;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
-
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -37,6 +36,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.input.CloseShieldInputStream;
@@ -73,16 +73,17 @@ import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
-import org.apache.solr.common.util.ValidatingJsonMap;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.common.util.ValidatingJsonMap;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.ContentStreamHandlerBase;
 import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.request.SolrRequestHandler;
@@ -103,6 +104,7 @@ import org.apache.solr.update.processor.DistributingUpdateProcessorFactory;
 import org.apache.solr.util.CommandOperation;
 import org.apache.solr.util.JsonSchemaValidator;
 import org.apache.solr.util.RTimerTree;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -111,9 +113,13 @@ import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.RELOAD;
+import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.common.params.CoreAdminParams.ACTION;
+import static org.apache.solr.handler.admin.CollectionsHandler.SYSTEM_COLL;
 import static org.apache.solr.servlet.SolrDispatchFilter.Action.ADMIN;
 import static org.apache.solr.servlet.SolrDispatchFilter.Action.FORWARD;
 import static org.apache.solr.servlet.SolrDispatchFilter.Action.PASSTHROUGH;
@@ -296,6 +302,9 @@ public class HttpSolrCall {
       // if we couldn't find it locally, look on other nodes
       extractRemotePath(corename, origCorename, idx);
       if (action != null) return;
+      //core is not available locally or remotely
+      autoCreateSystemColl();
+      if(action != null) return;
     }
 
     // With a valid core...
@@ -331,6 +340,35 @@ public class HttpSolrCall {
     action = PASSTHROUGH;
   }
 
+  protected void autoCreateSystemColl() throws Exception {
+    if (SYSTEM_COLL.equals(corename) && "POST".equals(req.getMethod()) && !cores.getZkController().getClusterState().hasCollection(SYSTEM_COLL)) {
+      log.info("Going to auto-create .system collection");
+      SolrQueryResponse rsp = new SolrQueryResponse();
+      String repFactor = String.valueOf(Math.min(3, cores.getZkController().getClusterState().getLiveNodes().size()));
+      cores.getCollectionsHandler().handleRequestBody(new LocalSolrQueryRequest(null,
+          new ModifiableSolrParams()
+              .add(ACTION, CREATE.toString())
+              .add( NAME, SYSTEM_COLL)
+              .add(REPLICATION_FACTOR, repFactor)), rsp);
+      if (rsp.getValues().get("success") == null) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Could not auto-create .system collection: "+ Utils.toJSONString(rsp.getValues()));
+      }
+      TimeOut timeOut = new TimeOut(3, TimeUnit.SECONDS);
+      for (; ; ) {
+        if (cores.getZkController().getClusterState().getCollectionOrNull(SYSTEM_COLL) != null) {
+          break;
+        } else {
+          if (timeOut.hasTimedOut()) {
+            throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find .system collection even after 3 seconds");
+          }
+          Thread.sleep(50);
+        }
+      }
+
+      action = RETRY;
+    }
+  }
+
   protected String lookupAliases(String collName) {
     ZkStateReader reader = cores.getZkController().getZkStateReader();
     aliases = reader.getAliases();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e200b8a2/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java b/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java
index 4fda926..c395d20 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java
@@ -90,27 +90,33 @@ public class TestBlobHandler extends AbstractFullDistribZkTestBase {
           "field",
           "type")));
 
-      byte[] bytarr = new byte[1024];
-      for (int i = 0; i < bytarr.length; i++) bytarr[i] = (byte) (i % 127);
-      byte[] bytarr2 = new byte[2048];
-      for (int i = 0; i < bytarr2.length; i++) bytarr2[i] = (byte) (i % 127);
-      String blobName = "test";
-      postAndCheck(cloudClient, baseUrl, blobName, ByteBuffer.wrap(bytarr), 1);
-      postAndCheck(cloudClient, baseUrl, blobName, ByteBuffer.wrap(bytarr2), 2);
-
-      url = baseUrl + "/.system/blob/test/1";
-      map = TestSolrConfigHandlerConcurrent.getAsMap(url, cloudClient);
-      List l = (List) Utils.getObjectByPath(map, false, Arrays.asList("response", "docs"));
-      assertNotNull("" + map, l);
-      assertTrue("" + map, l.size() > 0);
-      map = (Map) l.get(0);
-      assertEquals("" + bytarr.length, String.valueOf(map.get("size")));
-
-      compareInputAndOutput(baseUrl + "/.system/blob/test?wt=filestream", bytarr2);
-      compareInputAndOutput(baseUrl + "/.system/blob/test/1?wt=filestream", bytarr);
+      checkBlobPost(baseUrl, cloudClient);
     }
   }
 
+  static void checkBlobPost(String baseUrl, CloudSolrClient cloudClient) throws Exception {
+    String url;
+    Map map;
+    byte[] bytarr = new byte[1024];
+    for (int i = 0; i < bytarr.length; i++) bytarr[i] = (byte) (i % 127);
+    byte[] bytarr2 = new byte[2048];
+    for (int i = 0; i < bytarr2.length; i++) bytarr2[i] = (byte) (i % 127);
+    String blobName = "test";
+    postAndCheck(cloudClient, baseUrl, blobName, ByteBuffer.wrap(bytarr), 1);
+    postAndCheck(cloudClient, baseUrl, blobName, ByteBuffer.wrap(bytarr2), 2);
+
+    url = baseUrl + "/.system/blob/test/1";
+    map = TestSolrConfigHandlerConcurrent.getAsMap(url, cloudClient);
+    List l = (List) Utils.getObjectByPath(map, false, Arrays.asList("response", "docs"));
+    assertNotNull("" + map, l);
+    assertTrue("" + map, l.size() > 0);
+    map = (Map) l.get(0);
+    assertEquals("" + bytarr.length, String.valueOf(map.get("size")));
+
+    compareInputAndOutput(baseUrl + "/.system/blob/test?wt=filestream", bytarr2, cloudClient);
+    compareInputAndOutput(baseUrl + "/.system/blob/test/1?wt=filestream", bytarr, cloudClient);
+  }
+
   public static void createSystemCollection(SolrClient client) throws SolrServerException, IOException {
     CollectionAdminResponse response1;
     CollectionAdminRequest.Create createCollectionRequest = new CollectionAdminRequest.Create()
@@ -152,7 +158,7 @@ public class TestBlobHandler extends AbstractFullDistribZkTestBase {
     return new String(Utils.toJSON(map), StandardCharsets.UTF_8);
   }
 
-  private void compareInputAndOutput(String url, byte[] bytarr) throws IOException {
+  static void compareInputAndOutput(String url, byte[] bytarr, CloudSolrClient cloudClient) throws IOException {
 
     HttpClient httpClient = cloudClient.getLbClient().getHttpClient();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e200b8a2/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java b/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java
new file mode 100644
index 0000000..cadda58
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler;
+
+
+import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.common.cloud.DocCollection;
+
+public class TestSystemCollAutoCreate extends AbstractFullDistribZkTestBase {
+  public void testAutoCreate() throws Exception {
+    TestBlobHandler.checkBlobPost(cloudJettys.get(0).jetty.getBaseUrl().toExternalForm(), cloudClient);
+    DocCollection sysColl = cloudClient.getZkStateReader().getClusterState().getCollection(".system");
+  }
+}