You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by th...@apache.org on 2015/05/22 20:06:25 UTC

svn commit: r1681177 - in /lucene/dev/trunk/solr: ./ bin/ core/src/java/org/apache/solr/util/ core/src/test/org/apache/solr/cloud/ server/solr/configsets/data_driven_schema_configs/conf/

Author: thelabdude
Date: Fri May 22 18:06:25 2015
New Revision: 1681177

URL: http://svn.apache.org/r1681177
Log:
SOLR-7582: Allow auto-commit to be set with system properties in data_driven_schema_configs and enable auto soft-commits for the bin/solr -e cloud example using the Config API.

Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/bin/solr
    lucene/dev/trunk/solr/bin/solr.cmd
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SolrCloudExampleTest.java
    lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1681177&r1=1681176&r2=1681177&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Fri May 22 18:06:25 2015
@@ -443,6 +443,10 @@ Other Changes
 * SOLR-7463: Stop forcing MergePolicy's "NoCFSRatio" based on the IWC "useCompoundFile" configuration
   (Tomás Fernández Löbbe)
 
+* SOLR-7582: Allow auto-commit to be set with system properties in data_driven_schema_configs and
+  enable auto soft-commits for the bin/solr -e cloud example using the Config API.
+  (Timothy Potter)
+
 ==================  5.1.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

Modified: lucene/dev/trunk/solr/bin/solr
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/bin/solr?rev=1681177&r1=1681176&r2=1681177&view=diff
==============================================================================
--- lucene/dev/trunk/solr/bin/solr (original)
+++ lucene/dev/trunk/solr/bin/solr Fri May 22 18:06:25 2015
@@ -1573,6 +1573,11 @@ else
     -confname "$CLOUD_COLLECTION" -confdir "$CLOUD_CONFIG" \
     -configsetsDir "$SOLR_TIP/server/solr/configsets" -solrUrl "$SOLR_URL_SCHEME://$SOLR_TOOL_HOST:$SOLR_PORT/solr"
 
+  # enable soft-autocommits for the gettingstarted collection
+  echo -e "\nEnabling auto soft-commits with maxTime 3 secs using the Config API"
+  run_tool config -collection "$CLOUD_COLLECTION" -solrUrl "$SOLR_URL_SCHEME://$SOLR_TOOL_HOST:$SOLR_PORT/solr" \
+    -property updateHandler.autoSoftCommit.maxTime -value 3000
+
   echo -e "\n\nSolrCloud example running, please visit $SOLR_URL_SCHEME://$SOLR_TOOL_HOST:$SOLR_PORT/solr \n\n"
 fi
 

Modified: lucene/dev/trunk/solr/bin/solr.cmd
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/bin/solr.cmd?rev=1681177&r1=1681176&r2=1681177&view=diff
==============================================================================
--- lucene/dev/trunk/solr/bin/solr.cmd (original)
+++ lucene/dev/trunk/solr/bin/solr.cmd Fri May 22 18:06:25 2015
@@ -1099,6 +1099,12 @@ goto create_collection
   org.apache.solr.util.SolrCLI create_collection -name !CLOUD_COLLECTION! -shards !CLOUD_NUM_SHARDS! -replicationFactor !CLOUD_REPFACT! ^
   -confdir !CLOUD_CONFIG! -configsetsDir "%SOLR_SERVER_DIR%\solr\configsets" -zkHost %zk_host%
 
+@echo.
+echo Enabling auto soft-commits with maxTime 3 secs using the Config API
+"%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^
+  -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^
+  org.apache.solr.util.SolrCLI config -collection !CLOUD_COLLECTION! -property updateHandler.autoSoftCommit.maxTime -value 3000 -zkHost %zk_host%
+
 echo.
 echo SolrCloud example is running, please visit !SOLR_URL_SCHEME!://%SOLR_TOOL_HOST%:%NODE1_PORT%/solr"
 echo.

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java?rev=1681177&r1=1681176&r2=1681177&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java Fri May 22 18:06:25 2015
@@ -27,6 +27,7 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Enumeration;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
@@ -60,12 +61,14 @@ import org.apache.http.util.EntityUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -75,6 +78,8 @@ import org.apache.solr.common.cloud.ZkCo
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.NamedList;
 import org.noggit.CharArr;
 import org.noggit.JSONParser;
 import org.noggit.JSONWriter;
@@ -253,6 +258,8 @@ public class SolrCLI {
       return new CreateTool();
     else if ("delete".equals(toolType))
       return new DeleteTool();
+    else if ("config".equals(toolType))
+      return new ConfigTool();
 
     // If you add a built-in tool to this class, add it here to avoid
     // classpath scanning
@@ -275,6 +282,7 @@ public class SolrCLI {
     formatter.printHelp("create_core", getToolOptions(new CreateCoreTool()));
     formatter.printHelp("create", getToolOptions(new CreateTool()));
     formatter.printHelp("delete", getToolOptions(new DeleteTool()));
+    formatter.printHelp("config", getToolOptions(new ConfigTool()));
 
     List<Class<Tool>> toolClasses = findToolClassesInPackage("org.apache.solr.util");
     for (Class<Tool> next : toolClasses) {
@@ -1098,6 +1106,35 @@ public class SolrCLI {
             .create("configsetsDir")
   };
 
+  /**
+   * Get the base URL of a live Solr instance from either the solrUrl command-line option from ZooKeeper.
+   */
+  public static String resolveSolrUrl(CommandLine cli) throws Exception {
+    String solrUrl = cli.getOptionValue("solrUrl");
+    if (solrUrl == null) {
+      String zkHost = cli.getOptionValue("zkHost");
+      if (zkHost == null)
+        throw new IllegalStateException("Must provide either the '-solrUrl' or '-zkHost' parameters!");
+
+      LogManager.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
+      LogManager.getLogger("org.apache.solr.common.cloud").setLevel(Level.WARN);
+      try (CloudSolrClient cloudSolrClient = new CloudSolrClient(zkHost)) {
+        cloudSolrClient.connect();
+        Set<String> liveNodes = cloudSolrClient.getZkStateReader().getClusterState().getLiveNodes();
+        if (liveNodes.isEmpty())
+          throw new IllegalStateException("No live nodes found! Cannot determine 'solrUrl' from ZooKeeper: "+zkHost);
+
+        String firstLiveNode = liveNodes.iterator().next();
+        solrUrl = cloudSolrClient.getZkStateReader().getBaseUrlForNodeName(firstLiveNode);
+      }
+    }
+    return solrUrl;
+  }
+
+  /**
+   * Get the ZooKeeper connection string from either the zkHost command-line option or by looking it
+   * up from a running Solr instance based on the solrUrl option.
+   */
   public static String getZkHost(CommandLine cli) throws Exception {
     String zkHost = cli.getOptionValue("zkHost");
     if (zkHost != null)
@@ -1136,6 +1173,18 @@ public class SolrCLI {
     return zkHost;
   }
 
+  public static boolean safeCheckCollectionExists(String url, String collection) {
+    boolean exists = false;
+    try {
+      Map<String,Object> existsCheckResult = getJson(url);
+      List<String> collections = (List<String>) existsCheckResult.get("collections");
+      exists = collections != null && collections.contains(collection);
+    } catch (Exception exc) {
+      // just ignore it since we're only interested in a positive result here
+    }
+    return exists;
+  }
+
   /**
    * Supports create_collection command in the bin/solr script.
    */
@@ -1301,18 +1350,6 @@ public class SolrCLI {
       return 0;
     }
 
-    protected boolean safeCheckCollectionExists(String url, String collection) {
-      boolean exists = false;
-      try {
-        Map<String,Object> existsCheckResult = getJson(url);
-        List<String> collections = (List<String>) existsCheckResult.get("collections");
-        exists = collections != null && collections.contains(collection);
-      } catch (Exception exc) {
-        // just ignore it since we're only interested in a positive result here
-      }
-      return exists;
-    }
-
     protected int optionAsInt(CommandLine cli, String option, int defaultVal) {
       return Integer.parseInt(cli.getOptionValue(option, String.valueOf(defaultVal)));
     }
@@ -1733,4 +1770,115 @@ public class SolrCLI {
     }
 
   } // end DeleteTool class
+
+  /**
+   * Sends a POST to the Config API to perform a specified action.
+   */
+  public static class ConfigTool implements Tool {
+
+    @Override
+    public String getName() {
+      return "config";
+    }
+
+    @SuppressWarnings("static-access")
+    @Override
+    public Option[] getOptions() {
+      return new Option[] {
+          OptionBuilder
+              .withArgName("ACTION")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Config API action, one of: set-property, unset-property; default is set-property")
+              .create("action"),
+          OptionBuilder
+              .withArgName("PROP")
+              .hasArg()
+              .isRequired(true)
+              .withDescription("Name of the Config API property to apply the action to, such as: updateHandler.autoSoftCommit.maxTime")
+              .create("property"),
+          OptionBuilder
+              .withArgName("VALUE")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Set the property to this value; accepts JSON objects and strings")
+              .create("value"),
+          OptionBuilder
+              .withArgName("COLL")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Collection; defaults to gettingstarted")
+              .create("collection"),
+          OptionBuilder
+              .withArgName("HOST")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Address of the Zookeeper ensemble")
+              .create("zkHost"),
+          OptionBuilder
+              .withArgName("HOST")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Base Solr URL, which can be used to determine the zkHost if that's not known")
+              .create("solrUrl")
+      };
+    }
+
+    @Override
+    public int runTool(CommandLine cli) throws Exception {
+      String solrUrl = resolveSolrUrl(cli);
+      String action = cli.getOptionValue("action", "set-property");
+      String collection = cli.getOptionValue("collection", "gettingstarted");
+      String property = cli.getOptionValue("property");
+      String value = cli.getOptionValue("value");
+
+      Map<String,Object> jsonObj = new HashMap<>();
+      if (value != null) {
+        Map<String,String> setMap = new HashMap<>();
+        setMap.put(property, value);
+        jsonObj.put(action, setMap);
+      } else {
+        jsonObj.put(action, property);
+      }
+
+      CharArr arr = new CharArr();
+      (new JSONWriter(arr, 0)).write(jsonObj);
+      String jsonBody = arr.toString();
+
+      String updatePath = "/"+collection+"/config";
+
+      System.out.println("\nPOSTing request to Config API: "+solrUrl+updatePath);
+      System.out.println(jsonBody);
+      System.out.println();
+
+      int exitStatus = 0;
+      try (SolrClient solrClient = new HttpSolrClient(solrUrl)) {
+        NamedList<Object> result = postJsonToSolr(solrClient, updatePath, jsonBody);
+        Integer statusCode = (Integer)((NamedList)result.get("responseHeader")).get("status");
+        if (statusCode == 0) {
+          if (value != null) {
+            System.out.println("Successfully "+action+" "+property+" to "+value);
+          } else {
+            System.out.println("Successfully "+action+" "+property);
+          }
+        } else {
+          String errMsg = "Failed to "+action+" property due to:\n"+result;
+          System.err.println("\nERROR: "+errMsg+"\n");
+          exitStatus = 1;
+        }
+      }
+      return exitStatus;
+    }
+
+  } // end ConfigTool class
+
+  public static final String JSON_CONTENT_TYPE = "application/json";
+
+  public static NamedList<Object> postJsonToSolr(SolrClient solrClient, String updatePath, String jsonBody) throws Exception {
+    ContentStreamBase.StringStream contentStream = new ContentStreamBase.StringStream(jsonBody);
+    contentStream.setContentType(JSON_CONTENT_TYPE);
+    ContentStreamUpdateRequest req = new ContentStreamUpdateRequest(updatePath);
+    req.addContentStream(contentStream);
+    return solrClient.request(req);
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SolrCloudExampleTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SolrCloudExampleTest.java?rev=1681177&r1=1681176&r2=1681177&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SolrCloudExampleTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SolrCloudExampleTest.java Fri May 22 18:06:25 2015
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
@@ -136,8 +137,75 @@ public class SolrCloudExampleTest extend
     QueryResponse qr = cloudClient.query(new SolrQuery("*:*"));
     int numFound = (int)qr.getResults().getNumFound();
     assertEquals("*:* found unexpected number of documents", expectedXmlDocCount, numFound);
-                 
+
+    log.info("Updating Config for " + testCollectionName);
+    doTestConfigUpdate(testCollectionName, solrUrl);
+
+    log.info("Running healthcheck for " + testCollectionName);
+    doTestHealthcheck(testCollectionName, cloudClient.getZkHost());
+
+    // verify the delete action works too
+    log.info("Running delete for "+testCollectionName);
+    doTestDeleteAction(testCollectionName, solrUrl);
 
     log.info("testLoadDocsIntoGettingStartedCollection succeeded ... shutting down now!");
   }
+
+  protected void doTestHealthcheck(String testCollectionName, String zkHost) throws Exception {
+    String[] args = new String[]{
+        "healthcheck",
+        "-collection", testCollectionName,
+        "-zkHost", zkHost
+    };
+    SolrCLI.HealthcheckTool tool = new SolrCLI.HealthcheckTool();
+    CommandLine cli =
+        SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args);
+    assertTrue("Healthcheck action failed!", tool.runTool(cli) == 0);
+  }
+
+  protected void doTestDeleteAction(String testCollectionName, String solrUrl) throws Exception {
+    String[] args = new String[] {
+        "delete",
+        "-name", testCollectionName,
+        "-solrUrl", solrUrl
+    };
+    SolrCLI.DeleteTool tool = new SolrCLI.DeleteTool();
+    CommandLine cli =
+        SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args);
+    assertTrue("Delete action failed!", tool.runTool(cli) == 0);
+    assertTrue(!SolrCLI.safeCheckCollectionExists(solrUrl, testCollectionName)); // it should not exist anymore
+  }
+
+  /**
+   * Uses the SolrCLI config action to activate soft auto-commits for the getting started collection.
+   */
+  protected void doTestConfigUpdate(String testCollectionName, String solrUrl) throws Exception {
+    if (!solrUrl.endsWith("/"))
+      solrUrl += "/";
+    String configUrl = solrUrl + testCollectionName + "/config";
+
+    Map<String, Object> configJson = SolrCLI.getJson(configUrl);
+    Object maxTimeFromConfig = SolrCLI.atPath("/config/updateHandler/autoSoftCommit/maxTime", configJson);
+    assertNotNull(maxTimeFromConfig);
+    assertEquals(new Long(-1L), maxTimeFromConfig);
+
+    String prop = "updateHandler.autoSoftCommit.maxTime";
+    Long maxTime = new Long(3000L);
+    String[] args = new String[]{
+        "config",
+        "-collection", testCollectionName,
+        "-property", prop,
+        "-value", maxTime.toString(),
+        "-solrUrl", solrUrl
+    };
+    SolrCLI.ConfigTool tool = new SolrCLI.ConfigTool();
+    CommandLine cli = SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args);
+    log.info("Sending set-property '" + prop + "'=" + maxTime + " to SolrCLI.ConfigTool.");
+    assertTrue("Set config property failed!", tool.runTool(cli) == 0);
+
+    configJson = SolrCLI.getJson(configUrl);
+    maxTimeFromConfig = SolrCLI.atPath("/config/updateHandler/autoSoftCommit/maxTime", configJson);
+    assertNotNull(maxTimeFromConfig);
+    assertEquals(maxTime, maxTimeFromConfig);
+  }
 }

Modified: lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml?rev=1681177&r1=1681176&r2=1681177&view=diff
==============================================================================
--- lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml (original)
+++ lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml Fri May 22 18:06:25 2015
@@ -360,7 +360,7 @@
          have some sort of hard autoCommit to limit the log size.
       -->
     <autoCommit>
-      <maxTime>15000</maxTime>
+      <maxTime>${solr.autoCommit.maxTime:15000}</maxTime>
       <openSearcher>false</openSearcher>
     </autoCommit>
 
@@ -369,11 +369,10 @@
          but does not ensure that data is synced to disk.  This is
          faster and more near-realtime friendly than a hard commit.
       -->
-    <!--
-      <autoSoftCommit> 
-        <maxTime>1000</maxTime> 
-      </autoSoftCommit>
-     -->
+
+    <autoSoftCommit>
+      <maxTime>${solr.autoSoftCommit.maxTime:-1}</maxTime>
+    </autoSoftCommit>
 
     <!-- Update Related Event Listeners