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

lucene-solr:jira/solr-6736: SOLR-6736: Initial branch with varunrajput's patch from Dec2015

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-6736 [created] ed99a219c


SOLR-6736: Initial branch with varunrajput's patch from Dec2015


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

Branch: refs/heads/jira/solr-6736
Commit: ed99a219c311db09e1993b4e5d3d879ab092cc78
Parents: 05c17c9
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Fri Feb 24 08:47:05 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Fri Feb 24 08:47:05 2017 +0530

----------------------------------------------------------------------
 .../solr/handler/admin/ConfigSetsHandler.java   |  83 +++++++++-
 .../solr/configsets/upload/newzkconf.zip        | Bin 0 -> 1994 bytes
 .../solr/configsets/upload/schema-minimal.xml   |  25 +++
 .../configsets/upload/solrconfig-minimal.xml    |  59 +++++++
 .../apache/solr/cloud/TestConfigSetsAPI.java    | 158 +++++++++++++++++++
 .../solr/common/params/ConfigSetParams.java     |   6 +
 6 files changed, 330 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed99a219/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 5d6f02c..cc15239 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
@@ -16,14 +16,20 @@
  */
 package org.apache.solr.handler.admin;
 
+import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
-
+import java.nio.charset.StandardCharsets;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.Path;
 import org.apache.solr.api.Api;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.cloud.OverseerSolrResponse;
@@ -36,6 +42,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.ConfigSetParams;
 import org.apache.solr.common.params.ConfigSetParams.ConfigSetAction;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.Utils;
@@ -43,6 +50,7 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET;
 import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.CONFIGSETS_ACTION_PREFIX;
@@ -63,6 +71,7 @@ public class ConfigSetsHandler extends RequestHandlerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
   public static long DEFAULT_ZK_TIMEOUT = 300*1000;
+  protected final boolean isUploadEnabled;
   private final ConfigSetsHandlerApi configSetsHandlerApi = new ConfigSetsHandlerApi(this);
 
   /**
@@ -72,6 +81,8 @@ public class ConfigSetsHandler extends RequestHandlerBase {
    */
   public ConfigSetsHandler(final CoreContainer coreContainer) {
     this.coreContainer = coreContainer;
+    isUploadEnabled = Boolean.parseBoolean(System.getProperty(
+        ConfigSetParams.ENABLE_CONFIGSET_UPLOAD, ConfigSetParams.ENABLE_CONFIGSET_UPLOAD_DEFAULT));
   }
 
 
@@ -95,6 +106,16 @@ public class ConfigSetsHandler extends RequestHandlerBase {
       ConfigSetAction action = ConfigSetAction.get(a);
       if (action == null)
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a);
+      if (action == ConfigSetAction.UPLOAD) {
+        if (isUploadEnabled) {
+          handleConfigUploadRequest(req, rsp);
+          return;
+        } else {
+          throw new SolrException(SolrException.ErrorCode.FORBIDDEN, 
+              "Uploads are not enabled. Please set the system property \"" 
+                  + ConfigSetParams.ENABLE_CONFIGSET_UPLOAD + "\" to true");
+        }
+      }
       invokeAction(req, rsp, action);
     } else {
       throw new SolrException(ErrorCode.BAD_REQUEST, "action is a required param");
@@ -121,6 +142,66 @@ public class ConfigSetsHandler extends RequestHandlerBase {
     }
   }
 
+  private void handleConfigUploadRequest(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+
+    String httpMethod = (String) req.getContext().get(ConfigSetParams.HTTP_METHOD);
+    if (!"POST".equals(httpMethod)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "The upload action supports POST requests only");
+    }
+
+    String configSetName = req.getParams().get(NAME);
+    if (StringUtils.isBlank(configSetName)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "The configuration name should be provided in the \"name\" parameter");
+    }
+
+    SolrZkClient zkClient = coreContainer.getZkController().getZkClient();
+    String configPathInZk = ZkConfigManager.CONFIGS_ZKNODE + Path.SEPARATOR + configSetName;
+
+    if (zkClient.exists(configPathInZk, true)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "The configuration " + configSetName + " already exists in zookeeper");
+    }
+
+    Iterator<ContentStream> contentStreamsIterator = req.getContentStreams().iterator();
+
+    if (!contentStreamsIterator.hasNext()) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "No stream found for the config data to be uploaded");
+    }
+
+    InputStream inputStream = contentStreamsIterator.next().getStream();
+
+    // Create a node for the configuration in zookeeper
+    zkClient.makePath(configPathInZk, true);
+
+    ZipInputStream zis = new ZipInputStream(inputStream, StandardCharsets.UTF_8);
+    ZipEntry zipEntry = null;
+    while ((zipEntry = zis.getNextEntry()) != null) {
+      String filePathInZk = configPathInZk + "/" + zipEntry.getName();
+      if (zipEntry.isDirectory()) {
+        zkClient.makePath(filePathInZk, true);
+      } else {
+        createZkNodeIfNotExistsAndSetData(zkClient, filePathInZk,
+            IOUtils.toByteArray(zis));
+      }
+    }
+
+    zis.close();
+
+    rsp.add("message", configSetName + " successfully uploaded!");
+  }
+
+      private void createZkNodeIfNotExistsAndSetData(SolrZkClient zkClient,
+          String filePathInZk, byte[] data) throws Exception {
+        if (!zkClient.exists(filePathInZk, true)) {
+          zkClient.create(filePathInZk, data, CreateMode.PERSISTENT, true);
+        } else {
+          zkClient.setData(filePathInZk, data, true);
+        }
+      }
+    
   private void handleResponse(String operation, ZkNodeProps m,
       SolrQueryResponse rsp, long timeout) throws KeeperException, InterruptedException {
     long time = System.nanoTime();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed99a219/solr/core/src/test-files/solr/configsets/upload/newzkconf.zip
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/newzkconf.zip b/solr/core/src/test-files/solr/configsets/upload/newzkconf.zip
new file mode 100644
index 0000000..e9db927
Binary files /dev/null and b/solr/core/src/test-files/solr/configsets/upload/newzkconf.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed99a219/solr/core/src/test-files/solr/configsets/upload/schema-minimal.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/schema-minimal.xml b/solr/core/src/test-files/solr/configsets/upload/schema-minimal.xml
new file mode 100644
index 0000000..9e2f947
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/schema-minimal.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<schema name="minimal" version="1.1">
+ <types>
+  <fieldType name="string" class="solr.StrField"/>
+ </types>
+ <fields>
+   <dynamicField name="*" type="string" indexed="true" stored="true" />
+ </fields>
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed99a219/solr/core/src/test-files/solr/configsets/upload/solrconfig-minimal.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/solrconfig-minimal.xml b/solr/core/src/test-files/solr/configsets/upload/solrconfig-minimal.xml
new file mode 100644
index 0000000..2f9609d
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/solrconfig-minimal.xml
@@ -0,0 +1,59 @@
+<?xml version="1.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.
+-->
+
+<!-- This is a "kitchen sink" config file that tests can use.
+     When writting a new test, feel free to add *new* items (plugins,
+     config options, etc...) as long as they don't break any existing
+     tests.  if you need to test something esoteric please add a new
+     "solrconfig-your-esoteric-purpose.xml" config file.
+
+     Note in particular that this test is used by MinimalSchemaTest so
+     Anything added to this file needs to work correctly even if there
+     is now uniqueKey or defaultSearch Field.
+  -->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+
+  </updateHandler>
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+
+  <requestHandler name="/admin/" class="org.apache.solr.handler.admin.AdminHandlers" />
+
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+</config>
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed99a219/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 2027a6d..aa78c1d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
@@ -18,8 +18,13 @@ package org.apache.solr.cloud;
 
 import java.io.ByteArrayInputStream;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
 import java.io.InputStreamReader;
+import java.io.StringReader;
+import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -29,8 +34,14 @@ import java.util.Set;
 
 import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
+import org.apache.http.HttpEntity;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.util.EntityUtils;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Create;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Delete;
@@ -45,11 +56,15 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.ConfigSetProperties;
+import org.apache.solr.core.TestDynamicLoading;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.noggit.JSONParser;
+import org.noggit.ObjectBuilder;
 
 import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET;
 import static org.apache.solr.common.params.CommonParams.NAME;
@@ -62,6 +77,10 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
 
   private MiniSolrCloudCluster solrCluster;
 
+  static {
+    System.setProperty(ConfigSetParams.ENABLE_CONFIGSET_UPLOAD, "true");
+  }
+
   @Override
   @Before
   public void setUp() throws Exception {
@@ -232,6 +251,145 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void testUploadErrors() throws Exception {
+    final SolrClient solrClient = new HttpSolrClient(
+        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+
+    ByteBuffer emptyData = ByteBuffer.allocate(0);
+
+    //Checking error when no configuration name is specified in request
+    Map map = postDataAndGetResponse(solrCluster.getSolrClient(),
+        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString()
+        + "/admin/configs?action=UPLOAD&wt=json", emptyData);
+    assertNotNull(map);
+    long statusCode = (long) getObjectByPath(map, false,
+        Arrays.asList("responseHeader", "status"));
+    assertEquals(400l, statusCode);
+
+    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+        AbstractZkTestCase.TIMEOUT, 45000, null);
+
+    //Create dummy config files in zookeeper
+    zkClient.makePath("/configs/myconf", true);
+    zkClient.create("/configs/myconf/firstDummyFile",
+        "first dummy content".getBytes(), CreateMode.PERSISTENT, true);
+    zkClient.create("/configs/myconf/anotherDummyFile",
+        "second dummy content".getBytes(), CreateMode.PERSISTENT, true);
+
+    //Checking error when configuration name specified already exists
+    map = postDataAndGetResponse(solrCluster.getSolrClient(),
+        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString()
+        + "/admin/configs?action=UPLOAD&wt=json&name=myconf", emptyData);
+    assertNotNull(map);
+    statusCode = (long) getObjectByPath(map, false,
+        Arrays.asList("responseHeader", "status"));
+    assertEquals(400l, statusCode);
+    assertTrue("Expected file doesnt exist in zk. It's possibly overwritten",
+        zkClient.exists("/configs/myconf/firstDummyFile", true));
+    assertTrue("Expected file doesnt exist in zk. It's possibly overwritten",
+        zkClient.exists("/configs/myconf/anotherDummyFile", true));
+
+    zkClient.close();
+    solrClient.close();
+  }
+
+  @Test
+  public void testUpload() throws Exception {
+    String configSetName = "newzkconfig";
+
+    // Read zipped sample config
+    ByteBuffer sampleZippedConfig = TestDynamicLoading
+        .getFileContent("solr/configsets/upload/newzkconf.zip");
+
+    final SolrClient solrClient = new HttpSolrClient(
+        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+
+    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+        AbstractZkTestCase.TIMEOUT, 45000, null);
+    try {
+      ZkConfigManager configManager = new ZkConfigManager(zkClient);
+      assertFalse(configManager.configExists(configSetName));
+
+      Map map = postDataAndGetResponse(solrCluster.getSolrClient(),
+          solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/admin/configs?action=UPLOAD&wt=json&name=newzkconf",
+          sampleZippedConfig);
+      assertNotNull(map);
+      long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
+      assertEquals(0l, statusCode);
+
+      assertTrue("schema-minimal.xml file should have been uploaded",
+          zkClient.exists("/configs/newzkconf/schema-minimal.xml", true));
+      assertTrue("schema-minimal.xml file contents on zookeeper are not exactly same as that of the file uploaded in config",
+          Arrays.equals(zkClient.getData("/configs/newzkconf/schema-minimal.xml", null, null, true),
+              readFile("solr/configsets/upload/schema-minimal.xml")));
+
+      assertTrue("solrconfig-minimal.xml file should have been uploaded",
+          zkClient.exists("/configs/newzkconf/solrconfig-minimal.xml", true));
+      assertTrue("solrconfig-minimal.xml file contents on zookeeper are not exactly same as that of the file uploaded in config",
+          Arrays.equals(zkClient.getData("/configs/newzkconf/solrconfig-minimal.xml", null, null, true),
+              readFile("solr/configsets/upload/solrconfig-minimal.xml")));
+    } finally {
+      zkClient.close();
+    }
+    solrClient.close();
+  }
+
+  public static Map postDataAndGetResponse(CloudSolrClient cloudClient,
+      String uri, ByteBuffer bytarr) throws IOException {
+    HttpPost httpPost = null;
+    HttpEntity entity;
+    String response = null;
+    Map m = null;
+    try {
+      httpPost = new HttpPost(uri);
+      httpPost.setHeader("Content-Type", "application/octet-stream");
+      httpPost.setEntity(new ByteArrayEntity(bytarr.array(), bytarr
+          .arrayOffset(), bytarr.limit()));
+      entity = cloudClient.getLbClient().getHttpClient().execute(httpPost)
+          .getEntity();
+      try {
+        response = EntityUtils.toString(entity, StandardCharsets.UTF_8);
+        m = (Map) ObjectBuilder.getVal(new JSONParser(
+            new StringReader(response)));
+      } catch (JSONParser.ParseException e) {
+        fail(e.getMessage());
+      }
+    } finally {
+      httpPost.releaseConnection();
+    }
+    return m;
+  }
+
+  private static Object getObjectByPath(Map root, boolean onlyPrimitive, java.util.List<String> hierarchy) {
+    Map obj = root;
+    for (int i = 0; i < hierarchy.size(); i++) {
+      String s = hierarchy.get(i);
+      if (i < hierarchy.size() - 1) {
+        if (!(obj.get(s) instanceof Map)) return null;
+        obj = (Map) obj.get(s);
+        if (obj == null) return null;
+      } else {
+        Object val = obj.get(s);
+        if (onlyPrimitive && val instanceof Map) {
+          return null;
+        }
+        return val;
+      }
+    }
+
+    return false;
+  }
+
+  private byte[] readFile(String fname) throws IOException {
+    byte[] buf = null;
+    try (FileInputStream fis = new FileInputStream(getFile(fname))) {
+      buf = new byte[fis.available()];
+      fis.read(buf);
+    }
+    return buf;
+  }
+  
+  @Test
   public void testDeleteErrors() throws Exception {
     final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
     final SolrClient solrClient = getHttpSolrClient(baseUrl);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ed99a219/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java b/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java
index 49c39ec..aad4c9e 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java
@@ -24,9 +24,15 @@ import java.util.Locale;
 public interface ConfigSetParams
 {
   public final static String ACTION = "action";
+  
+  String ENABLE_CONFIGSET_UPLOAD = "configs.upload";
+  String ENABLE_CONFIGSET_UPLOAD_DEFAULT = "false";
+  
+  String HTTP_METHOD = "httpMethod";
 
   public enum ConfigSetAction {
     CREATE,
+    UPLOAD,
     DELETE,
     LIST;