You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2021/10/22 00:58:53 UTC

[lucene-solr] branch branch_8x updated: SOLR-15708: SolrJ support for ConfigSet uploading

This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new ae0c25f  SOLR-15708: SolrJ support for ConfigSet uploading
ae0c25f is described below

commit ae0c25fac8b6d97510f39a7f47d9dd5310a9938f
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Thu Oct 21 16:02:05 2021 -0700

    SOLR-15708: SolrJ support for ConfigSet uploading
    
    (cherry picked from commit 58527ba5fd45ab8b2238f23850d46c5e58497446)
---
 solr/CHANGES.txt                                   |   1 +
 .../org/apache/solr/cloud/TestConfigSetsAPI.java   | 157 +++++++++++++--------
 .../solrj/request/ConfigSetAdminRequest.java       | 144 ++++++++++++++++++-
 .../solrj/request/TestConfigSetAdminRequest.java   |  33 +++++
 4 files changed, 274 insertions(+), 61 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a990591..91f67d0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -14,6 +14,7 @@ New Features
 * SOLR-7642: opt-in support to create ZK chroot on startup (Timothy Potter, Shawn Heisey, Mark Miller, Tomas Eduardo Fernandez Lobbe,
   Jan Høydahl, Steve Molloy, Isabelle Giguere, David Eric Pugh, Gus Heck, Christine Poerschke, Houston Putman)
 
+* SOLR-15708: SolrJ support for ConfigSet uploading (ab, hossman)
 
 Improvements
 ---------------------
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 60ae2c2..214c025 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
@@ -60,7 +60,7 @@ import org.apache.http.client.methods.HttpPut;
 import org.apache.http.entity.ByteArrayEntity;
 import org.apache.http.message.BasicHeader;
 import org.apache.http.util.EntityUtils;
-import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -69,6 +69,7 @@ import org.apache.solr.client.solrj.impl.CloudSolrClient;
 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;
+import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Upload;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
@@ -83,9 +84,10 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 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.common.util.ValidatingJsonMap;
 import org.apache.solr.core.ConfigSetProperties;
 import org.apache.solr.core.TestDynamicLoading;
-import org.apache.solr.security.AuthenticationPlugin;
+import org.apache.solr.security.BasicAuthPlugin;
 import org.apache.solr.security.AuthorizationContext;
 import org.apache.solr.security.AuthorizationPlugin;
 import org.apache.solr.security.AuthorizationResponse;
@@ -846,6 +848,7 @@ public class TestConfigSetsAPI extends SolrCloudTestCase {
   private static String getSecurityJson() throws KeeperException, InterruptedException {
     String securityJson = "{\n" +
             "  'authentication':{\n" +
+            "    'blockUnknown': false,\n" +
             "    'class':'" + MockAuthenticationPlugin.class.getName() + "'},\n" +
             "  'authorization':{\n" +
             "    'class':'" + MockAuthorizationPlugin.class.getName() + "'}}";
@@ -887,84 +890,110 @@ public class TestConfigSetsAPI extends SolrCloudTestCase {
   }
 
   private long uploadConfigSet(String configSetName, String suffix, String username,
-      boolean overwrite, boolean cleanup, boolean v2) throws IOException {
+                               boolean overwrite, boolean cleanup, boolean v2) throws IOException {
+    
     // Read zipped sample config
-    ByteBuffer sampleZippedConfig = TestDynamicLoading
-        .getFileContent(
-            createTempZipFile("solr/configsets/upload/"+configSetName), false);
-
-    return uploadGivenConfigSet(sampleZippedConfig, configSetName, suffix, username, overwrite, cleanup, v2);
+    return uploadGivenConfigSet(createTempZipFile("solr/configsets/upload/"+configSetName),
+                                configSetName, suffix, username, overwrite, cleanup, v2);
   }
 
   private long uploadBadConfigSet(String configSetName, String suffix, String username,
-                               boolean overwrite, boolean cleanup, boolean v2) throws IOException {
+                                  boolean overwrite, boolean cleanup, boolean v2) throws IOException {
+    
     // Read single file from sample configs. This should fail the unzipping
-    ByteBuffer sampleBadZippedFile = TestDynamicLoading.getFileContent(TestDynamicLoading.getFile("solr/configsets/upload/regular/solrconfig.xml").getAbsolutePath(), false);
-
-    return uploadGivenConfigSet(sampleBadZippedFile, configSetName, suffix, username, overwrite, cleanup, v2);
+    return uploadGivenConfigSet(SolrTestCaseJ4.getFile("solr/configsets/upload/regular/solrconfig.xml"),
+                                configSetName, suffix, username, overwrite, cleanup, v2);
   }
 
-  private long uploadGivenConfigSet(ByteBuffer file, String configSetName, String suffix, String username,
-                               boolean overwrite, boolean cleanup, boolean v2) throws IOException {
-    String uriEnding;
-    boolean usePut = false;
+  private long uploadGivenConfigSet(File file, String configSetName, String suffix, String username,
+                                    boolean overwrite, boolean cleanup, boolean v2) throws IOException {
+      
     if (v2) {
-      uriEnding = "/api/cluster/configs/" + configSetName+suffix + (!overwrite? "?overwrite=false" : "") + (cleanup? "?cleanup=true" : "");
-      usePut = true;
-    } else {
-      uriEnding = "/solr/admin/configs?action=UPLOAD&name="+configSetName+suffix + (overwrite? "&overwrite=true" : "") + (cleanup? "&cleanup=true" : "");
+      // TODO: switch to using V2Request
+    
+      final ByteBuffer fileBytes = TestDynamicLoading.getFileContent(file.getAbsolutePath(), false);
+      final String uriEnding = "/api/cluster/configs/" + configSetName+suffix + (!overwrite? "?overwrite=false" : "") + (cleanup? "?cleanup=true" : "");
+      final boolean usePut = true;
+      Map<?, ?> map = postDataAndGetResponse(cluster.getSolrClient(),
+                                             cluster.getJettySolrRunners().get(0).getBaseUrl().toString().replace("/solr", "") + uriEnding,
+                                             fileBytes, username, usePut);
+      assertNotNull(map);
+      long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
+      return statusCode;
+      
+    } // else "not" a V2 request...
+    
+    try {
+      return ((ConfigSetAdminResponse)((new Upload())
+                                       .setConfigSetName(configSetName + suffix)
+                                       .setUploadFile(file, "application/zip")
+                                       .setOverwrite(overwrite ? true : null) // expect server default to be 'false'
+                                       .setCleanup(cleanup ? true : null) // expect server default to be 'false'
+                                       .setBasicAuthCredentials(username, username) // for our MockAuthenticationPlugin
+                                       .process(cluster.getSolrClient()))
+              ).getStatus();
+    } catch (SolrServerException e1) {
+      throw new AssertionError("Server error uploading configset: " + e1.toString(), e1);
+    } catch (SolrException e2) {
+      return e2.code();
     }
-
-    @SuppressWarnings({"rawtypes"})
-    Map map = postDataAndGetResponse(cluster.getSolrClient(),
-            cluster.getJettySolrRunners().get(0).getBaseUrl().toString().replace("/solr", "") + uriEnding,
-            file, username, usePut);
-    assertNotNull(map);
-    long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
-    return statusCode;
   }
 
   private long uploadSingleConfigSetFile(String configSetName, String suffix, String username,
-                                         String filePath, String uploadPath, boolean overwrite, boolean cleanup, boolean v2) throws IOException {
+                                         String localFilePath, String uploadPath, boolean overwrite, boolean cleanup, boolean v2) throws IOException {
     // Read single file from sample configs
-    ByteBuffer sampleConfigFile = TestDynamicLoading.getFileContent(TestDynamicLoading.getFile(filePath).getAbsolutePath(), false);
+    final File file = SolrTestCaseJ4.getFile(localFilePath);
 
-    String uriEnding;
-    boolean usePut = false;
     if (v2) {
-      uriEnding = "/api/cluster/configs/" + configSetName+suffix + "/" + uploadPath + (!overwrite? "?overwrite=false" : "") + (cleanup? "?cleanup=true" : "");
-      usePut = true;
-    } else {
-      uriEnding = "/solr/admin/configs?action=UPLOAD&name="+configSetName+suffix+"&filePath="+uploadPath + (overwrite? "&overwrite=true" : "") + (cleanup? "&cleanup=true" : "");
+      // TODO: switch to use V2Request
+      
+      final ByteBuffer sampleConfigFile = TestDynamicLoading.getFileContent(file.getAbsolutePath(), false);
+      final String uriEnding = "/api/cluster/configs/" + configSetName+suffix + "/" + uploadPath + (!overwrite? "?overwrite=false" : "") + (cleanup? "?cleanup=true" : "");
+      final boolean usePut = true;
+
+      Map<?, ?> map = postDataAndGetResponse(cluster.getSolrClient(),
+                                             cluster.getJettySolrRunners().get(0).getBaseUrl().toString().replace("/solr", "") + uriEnding,
+                                             sampleConfigFile, username, usePut);
+      assertNotNull(map);
+      long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
+      return statusCode;
+      
+    } // else "not" a V2 request...
+    
+    try {
+      return ((ConfigSetAdminResponse)((new Upload())
+                                       .setConfigSetName(configSetName + suffix)
+                                       .setFilePath(uploadPath)
+                                       .setUploadFile(file, "application/octet-stream") // NOTE: server doesn't actually care, and test plumbing doesn't tell us
+                                       .setOverwrite(overwrite ? true : null) // expect server default to be 'false'
+                                       .setCleanup(cleanup ? true : null) // expect server default to be 'false'
+                                       .setBasicAuthCredentials(username, username) // for our MockAuthenticationPlugin
+                                       .process(cluster.getSolrClient()))
+              ).getStatus();
+    } catch (SolrServerException e1) {
+      throw new AssertionError("Server error uploading file to configset: " + e1.toString(), e1);
+    } catch (SolrException e2) {
+      return e2.code();
     }
-
-    @SuppressWarnings({"rawtypes"})
-    Map map = postDataAndGetResponse(cluster.getSolrClient(),
-            cluster.getJettySolrRunners().get(0).getBaseUrl().toString().replace("/solr", "") + uriEnding,
-            sampleConfigFile, username, usePut);
-    assertNotNull(map);
-    long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
-    return statusCode;
   }
 
+  
   /**
    * Create a zip file (in the temp directory) containing all the files within the specified directory
-   * and return the path for the zip file.
+   * and return the zip file.
    */
-  private String createTempZipFile(String directoryPath) {
-    File zipFile = new File(cluster.getBaseDir().toFile().getAbsolutePath() +
-        File.separator + TestUtil.randomSimpleString(random(), 6, 8) + ".zip");
-
-    File directory = TestDynamicLoading.getFile(directoryPath);
-    if (log.isInfoEnabled()) {
-      log.info("Directory: {}", directory.getAbsolutePath());
-    }
+  private File createTempZipFile(String directoryPath) {
     try {
+      final File zipFile = createTempFile("configset","zip").toFile();
+      final File directory = SolrTestCaseJ4.getFile(directoryPath);
+      if (log.isInfoEnabled()) {
+        log.info("Directory: {}", directory.getAbsolutePath());
+      }
       zip (directory, zipFile);
       if (log.isInfoEnabled()) {
         log.info("Zipfile: {}", zipFile.getAbsolutePath());
       }
-      return zipFile.getAbsolutePath();
+      return zipFile;
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
@@ -1263,11 +1292,20 @@ public class TestConfigSetsAPI extends SolrCloudTestCase {
     }
   }
 
-  public static class MockAuthenticationPlugin extends AuthenticationPlugin {
+  public static class MockAuthenticationPlugin extends BasicAuthPlugin {
 
     @Override
-    public void init(Map<String, Object> pluginConfig) {
-
+    public AuthenticationProvider getAuthenticationProvider(Map<String, Object> pluginConfig) {
+      return new AuthenticationProvider() {
+        @Override public void init(Map<String,Object> ignored) { }
+        @Override public ValidatingJsonMap getSpec() { return Utils.getSpec("cluster.security.BasicAuth.Commands").getSpec(); }
+        @Override public boolean authenticate(String user, String pwd) {
+          return user.equals(pwd);
+        }
+        @Override public Map<String, String> getPromptHeaders() {
+          return Collections.emptyMap();
+        }
+      };
     }
 
     @Override
@@ -1275,10 +1313,9 @@ public class TestConfigSetsAPI extends SolrCloudTestCase {
       if (((HttpServletRequest)request).getHeader("user") != null) {
         final Principal p = new BasicUserPrincipal("solr");
         filterChain.doFilter(wrap((HttpServletRequest)request, p, "solr"), response);
-      } else {
-        filterChain.doFilter(request, response);
+        return true;
       }
-      return true;
+      return super.doAuthenticate(request, response, filterChain);
     }
 
     HttpServletRequest wrap(HttpServletRequest request, Principal principal, String username) {
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 ab06a9f..e0d81c4 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
@@ -16,8 +16,15 @@
  */
 package org.apache.solr.client.solrj.request;
 
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
 import java.util.Map;
 import java.util.Properties;
+import java.util.Collection;
+import java.util.Collections;
 
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
@@ -27,8 +34,13 @@ import org.apache.solr.common.params.ConfigSetParams.ConfigSetAction;
 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.ContentStreamBase.FileStream;
+
 import static org.apache.solr.common.params.CommonParams.NAME;
 
+import org.apache.commons.io.IOUtils;
+
 /**
  * This class is experimental and subject to change.
  *
@@ -100,7 +112,137 @@ public abstract class ConfigSetAdminRequest
     }
   }
 
-  // CREATE request
+  /**
+   * Uploads files to create a new configset, or modify an existing config set.
+   *
+   * When creating a new configset, the file to be uploaded must be a ZIP file containing the entire configset being uploaded.
+   * When modifing an existing configset, the file to be uploaded should either be a ZIP file containing the entire configset 
+   * being uploaded, or an individual file to upload if {@link #setFilePath} is being used.
+   */
+  public static class Upload extends ConfigSetSpecificAdminRequest<Upload> {
+    private static final String NO_STREAM_ERROR = "There must be a ContentStream or File to Upload";
+    
+    protected ContentStream stream;
+    protected String filePath;
+    
+    protected Boolean overwrite;
+    protected Boolean cleanup;
+    
+    public Upload() {
+      action = ConfigSetAction.UPLOAD;
+      setMethod(SolrRequest.METHOD.POST);
+    }
+
+    @Override
+    protected Upload getThis() {
+      return this;
+    }
+
+    /** Optional {@link ConfigSetParams#FILE_PATH} to indicate a single file is being uploaded into an existing configset */
+    public final Upload setFilePath(final String filePath) {
+      this.filePath = filePath;
+      return getThis();
+    }
+
+    /** @see #setFilePath */
+    public final String getFilePath() {
+      return filePath;
+    }
+
+    /** 
+     * A convinience method for specifying an existing File to use as the upload data.
+     *
+     * This should either be a ZIP file containing the entire configset being uploaded, or
+     * an individual file to upload into an existing configset if {@link #setFilePath} is being used.
+     *
+     * @see #setUploadStream
+     */
+    public final Upload setUploadFile(final File file, final String contentType) {
+      final FileStream fileStream = new FileStream(file);
+      fileStream.setContentType(contentType);
+      return setUploadStream(fileStream);
+    }
+
+    /** @see ConfigSetParams#OVERWRITE */
+    public final Upload setOverwrite(final Boolean overwrite) {
+      this.overwrite = overwrite;
+      return getThis();
+    }
+
+    /** @see #setOverwrite */
+    public final Boolean getOverwrite() {
+      return overwrite;
+    }
+    
+    /** @see ConfigSetParams#CLEANUP */
+    public final Upload setCleanup(final Boolean cleanup) {
+      this.cleanup = cleanup;
+      return getThis();
+    }
+    
+    /** @see #setCleanup */
+    public final Boolean getCleanup() {
+      return cleanup;
+    }
+    
+    /** 
+     * Specify the ContentStream to upload.
+     *
+     * This should either be a ZIP file containing the entire configset being uploaded, or
+     * an individual file to upload into an existing configset if {@link #setFilePath} is being used.
+     *
+     * @see #setUploadStream
+     */
+    public final Upload setUploadStream(final ContentStream stream) {
+      this.stream = stream;
+      return getThis();
+    }
+    
+    @Override
+    public Collection<ContentStream> getContentStreams() throws IOException {
+      return Collections.singletonList(stream);
+    }
+    
+    @Override
+    public RequestWriter.ContentWriter getContentWriter(String expectedType) {
+      if (null == stream) {
+        throw new NullPointerException(NO_STREAM_ERROR);
+      }
+      return new RequestWriter.ContentWriter() {
+        @Override
+        public void write(OutputStream os) throws IOException {
+          try(InputStream inStream = stream.getStream()) {
+            IOUtils.copy(inStream, os);
+          }
+        }
+        
+        @Override
+        public String getContentType() {
+          return stream.getContentType();
+        }
+      };
+    }
+    
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+
+      if (null == stream) {
+        throw new NullPointerException(NO_STREAM_ERROR);
+      }
+      
+      params.setNonNull(ConfigSetParams.FILE_PATH, filePath);
+      params.setNonNull(ConfigSetParams.CLEANUP, cleanup);
+      params.setNonNull(ConfigSetParams.OVERWRITE, overwrite);
+          
+      return params;
+    }
+  }
+
+  /**
+   * Creates a new config set by cloning an existing "base" configset.
+   * To create a new configset from scratch using a ZIP file you wish to upload, use the {@link Upload} command instead
+   */
   public static class Create extends ConfigSetSpecificAdminRequest<Create> {
     protected static String PROPERTY_PREFIX = "configSetProp";
     protected String baseConfigSetName;
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 5c78159..0b6e3ad 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
@@ -16,9 +16,13 @@
  */
 package org.apache.solr.client.solrj.request;
 
+import java.io.File;
+
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
+import org.apache.solr.common.params.ConfigSetParams;
+
 import org.junit.Test;
 
 /**
@@ -34,6 +38,35 @@ public class TestConfigSetAdminRequest extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void testUpload() throws Exception {
+    final File tmpFile = createTempFile().toFile();
+    ConfigSetAdminRequest.Upload upload = new ConfigSetAdminRequest.Upload();
+    verifyException(upload, "ConfigSet");
+    
+    upload.setConfigSetName("name");
+    verifyException(upload, "There must be a ContentStream");
+    
+    upload.setUploadFile(tmpFile, "application/zip");
+    
+    assertEquals(1, upload.getContentStreams().size());
+    assertEquals("application/zip", upload.getContentStreams().stream().findFirst().get().getContentType());
+    
+    assertNull(upload.getParams().get(ConfigSetParams.FILE_PATH));
+    assertNull(upload.getParams().get(ConfigSetParams.OVERWRITE));
+    assertNull(upload.getParams().get(ConfigSetParams.CLEANUP));
+    
+    upload.setUploadFile(tmpFile, "application/xml")
+      .setFilePath("solrconfig.xml")
+      .setOverwrite(true);
+    
+    assertEquals(1, upload.getContentStreams().size());
+    assertEquals("application/xml", upload.getContentStreams().stream().findFirst().get().getContentType());
+    
+    assertEquals("solrconfig.xml", upload.getParams().get(ConfigSetParams.FILE_PATH));
+    assertEquals("true", upload.getParams().get(ConfigSetParams.OVERWRITE));
+  }
+  
+  @Test
   public void testCreate() {
     ConfigSetAdminRequest.Create create = new ConfigSetAdminRequest.Create();
     verifyException(create, "ConfigSet");