You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2017/05/12 23:39:05 UTC

[38/58] [abbrv] lucene-solr:jira/solr-10233: SOLR-10431: Make it possible to invoke v2 api calls using SolrJ

SOLR-10431: Make it possible to invoke v2 api calls using SolrJ


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

Branch: refs/heads/jira/solr-10233
Commit: cc8b5bab0bdad4d57ac3764e9c3b313cb07fcddc
Parents: 95968c6
Author: Cao Manh Dat <da...@apache.org>
Authored: Thu May 11 09:06:03 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Thu May 11 09:06:03 2017 +0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../solr/security/BasicAuthIntegrationTest.java |  15 ++-
 .../solr/client/solrj/impl/CloudSolrClient.java |  23 +++-
 .../solr/client/solrj/impl/HttpSolrClient.java  |  14 ++
 .../solr/client/solrj/request/V2Request.java    | 127 +++++++++++++++++++
 .../apache/solr/common/params/CommonParams.java |   4 +-
 .../client/solrj/request/TestV2Request.java     |  89 +++++++++++++
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   2 +
 8 files changed, 265 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc8b5bab/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b7aaa8b..530a065 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -90,6 +90,8 @@ New Features
 
 * SOLR-10262: Add support for configurable metrics implementations. (ab)
 
+* SOLR-10431: Make it possible to invoke v2 api calls using SolrJ (Cao Manh Dat, Noble Paul, shalin)
+
 Bug Fixes
 ----------------------
 * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc8b5bab/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index 5231dd8..157f61a 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -45,6 +45,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.DocCollection;
@@ -82,10 +83,10 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
 
   @Test
   public void testBasicAuth() throws Exception {
-
+    boolean isUseV2Api = random().nextBoolean();
     String authcPrefix = "/admin/authentication";
     String authzPrefix = "/admin/authorization";
-    if(random().nextBoolean()){
+    if(isUseV2Api){
       authcPrefix = "/____v2/cluster/security/authentication";
       authzPrefix = "/____v2/cluster/security/authorization";
     }
@@ -110,8 +111,14 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
           "'set-user': {'harry':'HarryIsCool'}\n" +
           "}";
 
-      GenericSolrRequest genericReq = new GenericSolrRequest(SolrRequest.METHOD.POST, authcPrefix, new ModifiableSolrParams());
-      genericReq.setContentStreams(Collections.singletonList(new ContentStreamBase.ByteArrayStream(command.getBytes(UTF_8), "")));
+      final SolrRequest genericReq;
+      if (isUseV2Api) {
+        genericReq = new V2Request.Builder("/cluster/security/authentication").withMethod(SolrRequest.METHOD.POST).build();
+      } else {
+        genericReq = new GenericSolrRequest(SolrRequest.METHOD.POST, authcPrefix, new ModifiableSolrParams());
+        ((GenericSolrRequest)genericReq).setContentStreams(Collections.singletonList(new ContentStreamBase.ByteArrayStream(command.getBytes(UTF_8), "")));
+      }
+
 
       HttpSolrClient.RemoteSolrException exp = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
         cluster.getSolrClient().request(genericReq);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc8b5bab/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index ac388d2..3596333 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -53,6 +53,7 @@ import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.IsUpdateRequest;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -1089,8 +1090,12 @@ public class CloudSolrClient extends SolrClient {
     // collections is stale and needs to be refreshed ... this code has no impact on internal collections
     String stateVerParam = null;
     List<DocCollection> requestedCollections = null;
+    boolean isCollectionRequestOfV2 = false;
+    if (request instanceof V2Request) {
+      isCollectionRequestOfV2 = ((V2Request) request).isPerCollectionRequest();
+    }
     boolean isAdmin = ADMIN_PATHS.contains(request.getPath());
-    if (collection != null &&  !isAdmin) { // don't do _stateVer_ checking for admin requests
+    if (collection != null &&  !isAdmin && !isCollectionRequestOfV2) { // don't do _stateVer_ checking for admin, v2 api requests
       Set<String> requestedCollectionNames = getCollectionNames(collection);
 
       StringBuilder stateVerParamBuilder = null;
@@ -1147,8 +1152,10 @@ public class CloudSolrClient extends SolrClient {
     } catch (Exception exc) {
 
       Throwable rootCause = SolrException.getRootCause(exc);
-      // don't do retry support for admin requests or if the request doesn't have a collection specified
-      if (collection == null || isAdmin) {
+      // don't do retry support for admin requests
+      // or if the request doesn't have a collection specified
+      // or request is v2 api and its method is not GET
+      if (collection == null || isAdmin || (request instanceof V2Request && request.getMethod() != SolrRequest.METHOD.GET)) {
         if (exc instanceof SolrServerException) {
           throw (SolrServerException)exc;
         } else if (exc instanceof IOException) {
@@ -1274,7 +1281,15 @@ public class CloudSolrClient extends SolrClient {
       reqParams = new ModifiableSolrParams();
     }
     List<String> theUrlList = new ArrayList<>();
-    if (ADMIN_PATHS.contains(request.getPath())) {
+    if (request instanceof V2Request) {
+      Set<String> liveNodes = stateProvider.liveNodes();
+      if (!liveNodes.isEmpty()) {
+        List<String> liveNodesList = new ArrayList<>(liveNodes);
+        Collections.shuffle(liveNodesList);
+        theUrlList.add(ZkStateReader.getBaseUrlForNodeName(liveNodesList.get(0),
+            (String) stateProvider.getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
+      }
+    } else if (ADMIN_PATHS.contains(request.getPath())) {
       Set<String> liveNodes = stateProvider.liveNodes();
       for (String liveNode : liveNodes) {
         theUrlList.add(ZkStateReader.getBaseUrlForNodeName(liveNode,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc8b5bab/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
index 5b272fc..1692aa9 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
@@ -42,6 +42,7 @@ import org.apache.http.HttpStatus;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.entity.UrlEncodedFormEntity;
+import org.apache.http.client.methods.HttpDelete;
 import org.apache.http.client.methods.HttpEntityEnclosingRequestBase;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
@@ -65,6 +66,7 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -373,6 +375,14 @@ public class HttpSolrClient extends SolrClient {
     if (collection != null)
       basePath += "/" + collection;
 
+    if (request instanceof V2Request) {
+      if (System.getProperty("solr.v2RealPath") == null) {
+        basePath = baseUrl.replace("/solr", "/v2");
+      } else {
+        basePath = baseUrl + "/____v2";
+      }
+    }
+
     if (SolrRequest.METHOD.GET == request.getMethod()) {
       if (streams != null) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "GET can't send streams!");
@@ -381,6 +391,10 @@ public class HttpSolrClient extends SolrClient {
       return new HttpGet(basePath + path + wparams.toQueryString());
     }
 
+    if (SolrRequest.METHOD.DELETE == request.getMethod()) {
+      return new HttpDelete(basePath + path + wparams.toQueryString());
+    }
+
     if (SolrRequest.METHOD.POST == request.getMethod() || SolrRequest.METHOD.PUT == request.getMethod()) {
 
       String url = basePath + path;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc8b5bab/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
new file mode 100644
index 0000000..2b33593
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.request;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.regex.Pattern;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.ContentStreamBase;
+
+public class V2Request extends SolrRequest {
+  static final Pattern COLL_REQ_PATTERN = Pattern.compile("/(c|collections)/[^/]+/(?!shards)");
+  private InputStream payload;
+  private SolrParams solrParams;
+
+  private V2Request(METHOD m, String resource, InputStream payload) {
+    super(m, resource);
+    this.payload = payload;
+  }
+
+  @Override
+  public SolrParams getParams() {
+    return solrParams;
+  }
+
+  @Override
+  public Collection<ContentStream> getContentStreams() throws IOException {
+    if (payload != null) {
+      return Collections.singleton(new ContentStreamBase() {
+        @Override
+        public InputStream getStream() throws IOException {
+          return payload;
+        }
+
+        @Override
+        public String getContentType() {
+          return "application/json";
+        }
+      });
+    }
+    return null;
+  }
+
+  public boolean isPerCollectionRequest() {
+    return COLL_REQ_PATTERN.matcher(getPath()).find();
+  }
+
+  @Override
+  protected SolrResponse createResponse(SolrClient client) {
+    return null;
+  }
+
+  public static class Builder {
+    private String resource;
+    private METHOD method = METHOD.GET;
+    private InputStream payload;
+    private SolrParams params;
+
+    /**
+     * Create a Builder object based on the provided resource.
+     * The default method is GET.
+     *
+     * @param resource resource of the request for example "/collections" or "/cores/core-name"
+     */
+    public Builder(String resource) {
+      if (!resource.startsWith("/")) resource = "/" + resource;
+      this.resource = resource;
+    }
+
+    public Builder withMethod(METHOD m) {
+      this.method = m;
+      return this;
+    }
+
+    /**
+     * Set payload for request.
+     * @param payload as UTF-8 String
+     * @return builder object
+     */
+    public Builder withPayload(String payload) {
+      this.payload = new ByteArrayInputStream(payload.getBytes(StandardCharsets.UTF_8));
+      return this;
+    }
+
+    public Builder withPayLoad(InputStream payload) {
+      this.payload = payload;
+      return this;
+    }
+
+    public Builder withParams(SolrParams params) {
+      this.params = params;
+      return this;
+    }
+
+    public V2Request build() {
+      V2Request v2Request = new V2Request(method, resource, payload);
+      v2Request.solrParams = params;
+      return v2Request;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc8b5bab/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
index c3a011c..589ef7e 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
@@ -188,9 +188,7 @@ public interface CommonParams {
       CONFIGSETS_HANDLER_PATH,
       AUTHC_PATH,
       AUTHZ_PATH,
-      METRICS_PATH,
-      "/____v2/cluster/security/authentication",
-      "/____v2/cluster/security/authorization"));
+      METRICS_PATH));
 
   /** valid values for: <code>echoParams</code> */
   enum EchoParamStyle {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc8b5bab/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV2Request.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV2Request.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV2Request.java
new file mode 100644
index 0000000..61f771e
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV2Request.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.request;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.util.NamedList;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestV2Request extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(4)
+        .addConfig("config", getFile("solrj/solr/collection1/conf").toPath())
+        .configure();
+  }
+
+  public void assertSuccess(SolrClient client, V2Request request) throws IOException, SolrServerException {
+    NamedList<Object> res = client.request(request);
+    assertTrue("The request failed", res.get("responseHeader").toString().contains("status=0"));
+  }
+
+  @Test
+  public void testIsCollectionRequest() {
+    assertFalse(new V2Request.Builder("/collections").build().isPerCollectionRequest());
+    assertFalse(new V2Request.Builder("/collections/a/shards").build().isPerCollectionRequest());
+    assertFalse(new V2Request.Builder("/collections/a/shards/").build().isPerCollectionRequest());
+    assertTrue(new V2Request.Builder("/collections/a/update").build().isPerCollectionRequest());
+    assertTrue(new V2Request.Builder("/c/a/update").build().isPerCollectionRequest());
+    assertTrue(new V2Request.Builder("/c/a/schema").build().isPerCollectionRequest());
+    assertFalse(new V2Request.Builder("/c/a").build().isPerCollectionRequest());
+  }
+
+  @Test
+  public void testHttpSolrClient() throws Exception {
+    HttpSolrClient solrClient = new HttpSolrClient.Builder(
+        cluster.getJettySolrRunner(0).getBaseUrl().toString()).build();
+    doTest(solrClient);
+    solrClient.close();
+  }
+
+  @Test
+  public void testCloudSolrClient() throws Exception {
+    doTest(cluster.getSolrClient());
+  }
+
+  private void doTest(SolrClient client) throws IOException, SolrServerException {
+    assertSuccess(client, new V2Request.Builder("/collections")
+        .withMethod(SolrRequest.METHOD.POST)
+        .withPayload("{" +
+            "  'create' : {" +
+            "    'name' : 'test'," +
+            "    'numShards' : 2," +
+            "    'replicationFactor' : 2," +
+            "    'config' : 'config'" +
+            "  }" +
+            "}").build());
+    assertSuccess(client, new V2Request.Builder("/c").build());
+    assertSuccess(client, new V2Request.Builder("/c/_introspect").build());
+    assertSuccess(client, new V2Request.Builder("/c/test").withMethod(SolrRequest.METHOD.DELETE).build());
+    NamedList<Object> res = client.request(new V2Request.Builder("/c").build());
+    List collections = (List) res.get("collections");
+    assertEquals(0, collections.size());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc8b5bab/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 54ab06d..cd49357 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -269,6 +269,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     initCoreDataDir = createTempDir("init-core-data").toFile();
     System.err.println("Creating dataDir: " + initCoreDataDir.getAbsolutePath());
 
+    System.setProperty("solr.v2RealPath", "true");
     System.setProperty("zookeeper.forceSync", "no");
     System.setProperty("jetty.testMode", "true");
     System.setProperty("enable.update.log", usually() ? "true" : "false");
@@ -310,6 +311,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
       ObjectReleaseTracker.clear();
       TestInjection.reset();
       initCoreDataDir = null;
+      System.clearProperty("solr.v2RealPath");
       System.clearProperty("zookeeper.forceSync");
       System.clearProperty("jetty.testMode");
       System.clearProperty("tests.shardhandler.randomSeed");