You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/09/07 07:49:22 UTC

[13/50] [abbrv] lucene-solr:jira/http2: SOLR-11861 baseConfigSet default

SOLR-11861 baseConfigSet default


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

Branch: refs/heads/jira/http2
Commit: b1b0963947503dac20e84950ca30511e9aace9e1
Parents: e0eb7ba
Author: David Smiley <ds...@apache.org>
Authored: Tue Sep 4 14:02:46 2018 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Tue Sep 4 14:02:46 2018 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 ++
 .../cloud/OverseerConfigSetMessageHandler.java  | 30 +++++++++++---------
 .../solr/handler/admin/ConfigSetsHandler.java   | 20 +++++++------
 .../apache/solr/cloud/TestConfigSetsAPI.java    | 14 ++++-----
 solr/solr-ref-guide/src/configsets-api.adoc     |  2 +-
 .../solrj/request/ConfigSetAdminRequest.java    |  5 ++--
 .../apispec/cluster.configs.Commands.json       |  2 +-
 .../request/TestConfigSetAdminRequest.java      |  2 --
 8 files changed, 41 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b09639/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c6e89b1..56f5668 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -198,6 +198,9 @@ New Features
 * SOLR-12715: NodeAddedTrigger should support adding replicas to new nodes by setting preferredOperation=addreplica.
   (shalin)
 
+* SOLR-11861: When creating a configSet via the API, the "baseConfigSet" parameter now defaults to "_default".
+  (Amrit Sarkar, David Smiley)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b09639/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
index 2f2859f..6812971 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
@@ -47,6 +47,7 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.params.ConfigSetParams.ConfigSetAction.CREATE;
+import static org.apache.solr.handler.admin.ConfigSetsHandlerApi.DEFAULT_CONFIGSET_NAME;
 
 /**
  * A {@link OverseerMessageHandler} that handles ConfigSets API related
@@ -95,10 +96,10 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
       if (!operation.startsWith(CONFIGSETS_ACTION_PREFIX)) {
         throw new SolrException(ErrorCode.BAD_REQUEST,
             "Operation does not contain proper prefix: " + operation
-            + " expected: " + CONFIGSETS_ACTION_PREFIX);
+                + " expected: " + CONFIGSETS_ACTION_PREFIX);
       }
       operation = operation.substring(CONFIGSETS_ACTION_PREFIX.length());
-      log.info("OverseerConfigSetMessageHandler.processMessage : "+ operation + " , "+ message.toString());
+      log.info("OverseerConfigSetMessageHandler.processMessage : " + operation + " , " + message.toString());
 
       ConfigSetParams.ConfigSetAction action = ConfigSetParams.ConfigSetAction.get(operation);
       if (action == null) {
@@ -120,7 +121,7 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
 
       if (configSetName == null) {
         SolrException.log(log, "Operation " + operation + " failed", e);
-      } else  {
+      } else {
         SolrException.log(log, "ConfigSet: " + configSetName + " operation: " + operation
             + " failed", e);
       }
@@ -128,7 +129,7 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
       results.add("Operation " + operation + " caused exception:", e);
       SimpleOrderedMap nl = new SimpleOrderedMap();
       nl.add("msg", e.getMessage());
-      nl.add("rspCode", e instanceof SolrException ? ((SolrException)e).code() : -1);
+      nl.add("rspCode", e instanceof SolrException ? ((SolrException) e).code() : -1);
       results.add("exception", nl);
     }
     return new OverseerSolrResponse(results);
@@ -210,16 +211,20 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
       operation = operation.substring(CONFIGSETS_ACTION_PREFIX.length());
       ConfigSetParams.ConfigSetAction action = ConfigSetParams.ConfigSetAction.get(operation);
       if (action == CREATE) {
-        return message.getStr(BASE_CONFIGSET);
+        String baseConfigSetName = message.getStr(BASE_CONFIGSET);
+        if (baseConfigSetName == null || baseConfigSetName.length() == 0) {
+          baseConfigSetName = DEFAULT_CONFIGSET_NAME;
+        }
+        return baseConfigSetName;
       }
     }
     return null;
   }
 
   private NamedList getConfigSetProperties(String path) throws IOException {
-    byte [] oldPropsData = null;
+    byte[] oldPropsData = null;
     try {
-     oldPropsData = zkStateReader.getZkClient().getData(path, null, null, true);
+      oldPropsData = zkStateReader.getZkClient().getData(path, null, null, true);
     } catch (KeeperException.NoNodeException e) {
       log.info("no existing ConfigSet properties found");
     } catch (KeeperException | InterruptedException e) {
@@ -283,10 +288,7 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
       throw new SolrException(ErrorCode.BAD_REQUEST, "ConfigSet name not specified");
     }
 
-    String baseConfigSetName = message.getStr(BASE_CONFIGSET);
-    if (baseConfigSetName == null || baseConfigSetName.length() == 0) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, "Base ConfigSet name not specified");
-    }
+    String baseConfigSetName = message.getStr(BASE_CONFIGSET, DEFAULT_CONFIGSET_NAME);
 
     ZkConfigManager configManager = new ZkConfigManager(zkStateReader.getZkClient());
     if (configManager.configExists(configSetName)) {
@@ -303,7 +305,7 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
     Map<String, Object> props = getNewProperties(message);
     if (props != null) {
       // read the old config properties and do a merge, if necessary
-      NamedList oldProps = getConfigSetProperties(getPropertyPath(baseConfigSetName,propertyPath));
+      NamedList oldProps = getConfigSetProperties(getPropertyPath(baseConfigSetName, propertyPath));
       if (oldProps != null) {
         mergeOldProperties(props, oldProps);
       }
@@ -317,7 +319,7 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
         try {
           zkStateReader.getZkClient().makePath(
               getPropertyPath(configSetName, propertyPath),
-                  propertyData, CreateMode.PERSISTENT, null, false, true);
+              propertyData, CreateMode.PERSISTENT, null, false, true);
         } catch (KeeperException | InterruptedException e) {
           throw new IOException("Error writing new properties",
               SolrZkClient.checkInterrupted(e));
@@ -365,7 +367,7 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
     NamedList properties = getConfigSetProperties(getPropertyPath(configSetName, propertyPath));
     if (properties != null) {
       Object immutable = properties.get(ConfigSetProperties.IMMUTABLE_CONFIGSET_ARG);
-      boolean isImmutableConfigSet = immutable  != null ? Boolean.parseBoolean(immutable.toString()) : false;
+      boolean isImmutableConfigSet = immutable != null ? Boolean.parseBoolean(immutable.toString()) : false;
       if (!force && isImmutableConfigSet) {
         throw new SolrException(ErrorCode.BAD_REQUEST, "Requested delete of immutable ConfigSet: " + configSetName);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b09639/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
index f8fc533..fbd0fb6 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
@@ -66,6 +66,7 @@ import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.params.ConfigSetParams.ConfigSetAction.CREATE;
 import static org.apache.solr.common.params.ConfigSetParams.ConfigSetAction.DELETE;
 import static org.apache.solr.common.params.ConfigSetParams.ConfigSetAction.LIST;
+import static org.apache.solr.handler.admin.ConfigSetsHandlerApi.DEFAULT_CONFIGSET_NAME;
 
 /**
  * A {@link org.apache.solr.request.SolrRequestHandler} for ConfigSets API requests.
@@ -73,7 +74,7 @@ import static org.apache.solr.common.params.ConfigSetParams.ConfigSetAction.LIST
 public class ConfigSetsHandler extends RequestHandlerBase implements PermissionNameProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
-  public static long DEFAULT_ZK_TIMEOUT = 300*1000;
+  public static long DEFAULT_ZK_TIMEOUT = 300 * 1000;
   private final ConfigSetsHandlerApi configSetsHandlerApi = new ConfigSetsHandlerApi(this);
 
   /**
@@ -90,11 +91,11 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
     if (coreContainer == null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Core container instance missing");
+          "Core container instance missing");
     }
 
     // Make sure that the core is ZKAware
-    if(!coreContainer.isZooKeeperAware()) {
+    if (!coreContainer.isZooKeeperAware()) {
       throw new SolrException(ErrorCode.BAD_REQUEST,
           "Solr instance is not running in SolrCloud mode.");
     }
@@ -195,7 +196,7 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
   }
 
   private void createZkNodeIfNotExistsAndSetData(SolrZkClient zkClient,
-      String filePathInZk, byte[] data) throws Exception {
+                                                 String filePathInZk, byte[] data) throws Exception {
     if (!zkClient.exists(filePathInZk, true)) {
       zkClient.create(filePathInZk, data, CreateMode.PERSISTENT, true);
     } else {
@@ -204,7 +205,7 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
   }
 
   private void handleResponse(String operation, ZkNodeProps m,
-      SolrQueryResponse rsp, long timeout) throws KeeperException, InterruptedException {
+                              SolrQueryResponse rsp, long timeout) throws KeeperException, InterruptedException {
     long time = System.nanoTime();
 
     QueueEvent event = coreContainer.getZkController()
@@ -216,7 +217,7 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
       SimpleOrderedMap exp = (SimpleOrderedMap) response.getResponse().get("exception");
       if (exp != null) {
         Integer code = (Integer) exp.get("rspCode");
-        rsp.setException(new SolrException(code != null && code != -1 ? ErrorCode.getErrorCode(code) : ErrorCode.SERVER_ERROR, (String)exp.get("msg")));
+        rsp.setException(new SolrException(code != null && code != -1 ? ErrorCode.getErrorCode(code) : ErrorCode.SERVER_ERROR, (String) exp.get("msg")));
       }
     } else {
       if (System.nanoTime() - time >= TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS)) {
@@ -236,7 +237,7 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
   }
 
   private static Map<String, Object> copyPropertiesWithPrefix(SolrParams params, Map<String, Object> props, String prefix) {
-    Iterator<String> iter =  params.getParameterNamesIterator();
+    Iterator<String> iter = params.getParameterNamesIterator();
     while (iter.hasNext()) {
       String param = iter.next();
       if (param.startsWith(prefix)) {
@@ -254,6 +255,7 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
   public String getDescription() {
     return "Manage SolrCloud ConfigSets";
   }
+
   @Override
   public Category getCategory() {
     return Category.ADMIN;
@@ -263,7 +265,9 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
     CREATE_OP(CREATE) {
       @Override
       Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetsHandler h) throws Exception {
-        Map<String, Object> props = CollectionsHandler.copy(req.getParams().required(), null, NAME, BASE_CONFIGSET);
+        String baseConfigSetName = req.getParams().get(BASE_CONFIGSET, DEFAULT_CONFIGSET_NAME);
+        Map<String, Object> props = CollectionsHandler.copy(req.getParams().required(), null, NAME);
+        props.put(BASE_CONFIGSET, baseConfigSetName);
         return copyPropertiesWithPrefix(req.getParams(), props, PROPERTY_PREFIX + ".");
       }
     },

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b09639/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
index 71a0c2f..f513645 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
@@ -16,11 +16,6 @@
  */
 package org.apache.solr.cloud;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET;
-import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.core.ConfigSetProperties.DEFAULT_FILENAME;
-
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileInputStream;
@@ -99,6 +94,10 @@ import org.noggit.ObjectBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.core.ConfigSetProperties.DEFAULT_FILENAME;
+
 /**
  * Simple ConfigSets API tests on user errors and simple success cases.
  */
@@ -136,9 +135,8 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
     CreateNoErrorChecking create = new CreateNoErrorChecking();
     verifyException(solrClient, create, NAME);
 
-    // no base ConfigSet name
+    // set ConfigSet
     create.setConfigSetName("configSetName");
-    verifyException(solrClient, create, BASE_CONFIGSET);
 
     // ConfigSet already exists
     Create alreadyExists = new Create();
@@ -156,7 +154,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
   @Test
   public void testCreate() throws Exception {
     // no old, no new
-    verifyCreate("baseConfigSet1", "configSet1", null, null);
+    verifyCreate(null, "configSet1", null, null);
 
     // no old, new
     verifyCreate("baseConfigSet2", "configSet2",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b09639/solr/solr-ref-guide/src/configsets-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/configsets-api.adoc b/solr/solr-ref-guide/src/configsets-api.adoc
index 8874f21..1fbab30 100644
--- a/solr/solr-ref-guide/src/configsets-api.adoc
+++ b/solr/solr-ref-guide/src/configsets-api.adoc
@@ -130,7 +130,7 @@ name::
 The configset to be created. This parameter is required.
 
 baseConfigSet::
-The name of the configset to copy as a base. This parameter is required.
+The name of the configset to copy as a base. This defaults to `_default`
 
 configSetProp._property_=_value_::
 A configset property from the base configset to override in the copied configset.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b09639/solr/solrj/src/java/org/apache/solr/client/solrj/request/ConfigSetAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/ConfigSetAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/ConfigSetAdminRequest.java
index 6916877..3db8589 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/ConfigSetAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/ConfigSetAdminRequest.java
@@ -135,10 +135,9 @@ public abstract class ConfigSetAdminRequest
     @Override
     public SolrParams getParams() {
       ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
-      if (baseConfigSetName == null) {
-        throw new RuntimeException( "no Base ConfigSet specified!" );
+      if (baseConfigSetName != null) {
+        params.set("baseConfigSet", baseConfigSetName);
       }
-      params.set("baseConfigSet", baseConfigSetName);
       if (properties != null) {
         for (Map.Entry entry : properties.entrySet()) {
           params.set(PROPERTY_PREFIX + "." + entry.getKey().toString(),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b09639/solr/solrj/src/resources/apispec/cluster.configs.Commands.json
----------------------------------------------------------------------
diff --git a/solr/solrj/src/resources/apispec/cluster.configs.Commands.json b/solr/solrj/src/resources/apispec/cluster.configs.Commands.json
index 3792686..f8b3b3a 100644
--- a/solr/solrj/src/resources/apispec/cluster.configs.Commands.json
+++ b/solr/solrj/src/resources/apispec/cluster.configs.Commands.json
@@ -28,7 +28,7 @@
           "additionalProperties" : true
         }
       },
-      "required" : ["name", "baseConfigSet"]
+      "required" : ["name"]
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b09639/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestConfigSetAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestConfigSetAdminRequest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestConfigSetAdminRequest.java
index a3409bd..50e4f5d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestConfigSetAdminRequest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestConfigSetAdminRequest.java
@@ -38,8 +38,6 @@ public class TestConfigSetAdminRequest extends SolrTestCaseJ4 {
     ConfigSetAdminRequest.Create create = new ConfigSetAdminRequest.Create();
     verifyException(create, "ConfigSet");
     create.setConfigSetName("name");
-    verifyException(create, "Base ConfigSet");
-    create.setBaseConfigSetName("baseConfigSet");
     create.getParams();
   }