You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2017/06/21 07:11:00 UTC

[34/50] [abbrv] lucene-solr:feature/autoscaling: SOLR-6671: Possible to set solr.data.home property as root dir for all data

SOLR-6671: Possible to set solr.data.home property as root dir for all data


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

Branch: refs/heads/feature/autoscaling
Commit: 39dfb7808ac11c369985549dff06441f0cf5b93c
Parents: 0ec9d64
Author: Jan Høydahl <ja...@apache.org>
Authored: Tue Jun 20 13:20:44 2017 +0200
Committer: Jan Høydahl <ja...@apache.org>
Committed: Tue Jun 20 13:21:14 2017 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  1 +
 solr/bin/solr                                   | 18 +++++-
 solr/bin/solr.cmd                               | 32 +++++++++-
 solr/bin/solr.in.cmd                            |  4 ++
 solr/bin/solr.in.sh                             |  4 ++
 .../solr/core/CachingDirectoryFactory.java      | 10 ++++
 .../org/apache/solr/core/DirectoryFactory.java  | 24 +++++++-
 .../apache/solr/core/DirectoryFactoryTest.java  | 61 ++++++++++++++++++++
 ...adir-and-directoryfactory-in-solrconfig.adoc |  2 +
 9 files changed, 150 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39dfb780/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5c865a1..9267039 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -909,6 +909,7 @@ New Features
   parsers' behavior remains the same: queries will be split on whitespace before sending individual terms to analysis.
   (Steve Rowe)
 
+* SOLR-6671: Possible to set solr.data.home property as root dir for all data (janhoy, Shawn Heisey, Mark Miller)
 
 Bug Fixes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39dfb780/solr/bin/solr
----------------------------------------------------------------------
diff --git a/solr/bin/solr b/solr/bin/solr
index d21167e1..a22ef93 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -299,7 +299,7 @@ function print_usage() {
     echo ""
   elif [[ "$CMD" == "start" || "$CMD" == "restart" ]]; then
     echo ""
-    echo "Usage: solr $CMD [-f] [-c] [-h hostname] [-p port] [-d directory] [-z zkHost] [-m memory] [-e example] [-s solr.solr.home] [-a \"additional-options\"] [-V]"
+    echo "Usage: solr $CMD [-f] [-c] [-h hostname] [-p port] [-d directory] [-z zkHost] [-m memory] [-e example] [-s solr.solr.home] [-t solr.data.home] [-a \"additional-options\"] [-V]"
     echo ""
     echo "  -f            Start Solr in foreground; default starts Solr in the background"
     echo "                  and sends stdout / stderr to solr-PORT-console.log"
@@ -329,6 +329,9 @@ function print_usage() {
     echo "                  This parameter is ignored when running examples (-e), as the solr.solr.home depends"
     echo "                  on which example is run. The default value is server/solr."
     echo ""
+    echo "  -t <dir>      Sets the solr.data.home system property, where Solr will store data (index)."
+    echo "                  If not set, Solr uses solr.solr.home for config and data."
+    echo ""
     echo "  -e <example>  Name of the example to run; available examples:"
     echo "      cloud:         SolrCloud example"
     echo "      techproducts:  Comprehensive example illustrating many of Solr's core capabilities"
@@ -1387,6 +1390,10 @@ if [ $# -gt 0 ]; then
             SOLR_HOME="$2"
             shift 2
         ;;
+        -t|-data.home)
+            SOLR_DATA_HOME="$2"
+            shift 2
+        ;;
         -e|-example)
             if [[ -z "$2" || "${2:0:1}" == "-" ]]; then
               print_usage "$SCRIPT_CMD" "Example name is required when using the $1 option!"
@@ -1682,6 +1689,10 @@ if [ ! -e "$SOLR_HOME" ]; then
   echo -e "\nSolr home directory $SOLR_HOME not found!\n"
   exit 1
 fi
+if [[ $SOLR_DATA_HOME ]] && [ ! -e "$SOLR_DATA_HOME" ]; then
+  echo -e "\nSolr data home directory $SOLR_DATA_HOME not found!\n"
+  exit 1
+fi
 if $verbose ; then
   q=""
 else
@@ -1882,6 +1893,9 @@ function launch_solr() {
       echo -e "    SOLR_LOG_LEVEL  = $SOLR_LOG_LEVEL"
     fi
 
+    if [ "$SOLR_DATA_HOME" != "" ]; then
+      echo -e "    SOLR_DATA_HOME  = $SOLR_DATA_HOME"
+    fi
     echo -e "\n"
   fi
     
@@ -1897,7 +1911,7 @@ function launch_solr() {
     "${REMOTE_JMX_OPTS[@]}" "${CLOUD_MODE_OPTS[@]}" $SOLR_LOG_LEVEL_OPT -Dsolr.log.dir="$SOLR_LOGS_DIR" \
     "-Djetty.port=$SOLR_PORT" "-DSTOP.PORT=$stop_port" "-DSTOP.KEY=$STOP_KEY" \
     "${SOLR_HOST_ARG[@]}" "-Duser.timezone=$SOLR_TIMEZONE" \
-    "-Djetty.home=$SOLR_SERVER_DIR" "-Dsolr.solr.home=$SOLR_HOME" "-Dsolr.install.dir=$SOLR_TIP" \
+    "-Djetty.home=$SOLR_SERVER_DIR" "-Dsolr.solr.home=$SOLR_HOME" "-Dsolr.data.home=$SOLR_DATA_HOME" "-Dsolr.install.dir=$SOLR_TIP" \
     "${LOG4J_CONFIG[@]}" "${SOLR_OPTS[@]}")
 
   if [ "$SOLR_MODE" == "solrcloud" ]; then

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39dfb780/solr/bin/solr.cmd
----------------------------------------------------------------------
diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd
index 0afea02..8772200 100644
--- a/solr/bin/solr.cmd
+++ b/solr/bin/solr.cmd
@@ -285,7 +285,7 @@ goto done
 
 :start_usage
 @echo.
-@echo Usage: solr %SCRIPT_CMD% [-f] [-c] [-h hostname] [-p port] [-d directory] [-z zkHost] [-m memory] [-e example] [-s solr.solr.home] [-a "additional-options"] [-V]
+@echo Usage: solr %SCRIPT_CMD% [-f] [-c] [-h hostname] [-p port] [-d directory] [-z zkHost] [-m memory] [-e example] [-s solr.solr.home] [-t solr.data.home] [-a "additional-options"] [-V]
 @echo.
 @echo   -f            Start Solr in foreground; default starts Solr in the background
 @echo                   and sends stdout / stderr to solr-PORT-console.log
@@ -312,6 +312,9 @@ goto done
 @echo                   This parameter is ignored when running examples (-e), as the solr.solr.home depends
 @echo                   on which example is run. The default value is server/solr.
 @echo.
+@echo   -t dir        Sets the solr.data.home system property, used as root for ^<instance_dir^>/data directories.
+@echo                   If not set, Solr uses solr.solr.home for both config and data.
+@echo.
 @echo   -e example    Name of the example to run; available examples:
 @echo       cloud:          SolrCloud example
 @echo       techproducts:   Comprehensive example illustrating many of Solr's core capabilities
@@ -602,6 +605,7 @@ IF "%1"=="-cloud" goto set_cloud_mode
 IF "%1"=="-d" goto set_server_dir
 IF "%1"=="-dir" goto set_server_dir
 IF "%1"=="-s" goto set_solr_home_dir
+IF "%1"=="-t" goto set_solr_data_dir
 IF "%1"=="-solr.home" goto set_solr_home_dir
 IF "%1"=="-e" goto set_example
 IF "%1"=="-example" goto set_example
@@ -697,6 +701,24 @@ SHIFT
 SHIFT
 goto parse_args
 
+:set_solr_data_dir
+
+set "arg=%~2"
+IF "%arg%"=="" (
+  set SCRIPT_ERROR=Directory name is required!
+  goto invalid_cmd_line
+)
+
+set firstChar=%arg:~0,1%
+IF "%firstChar%"=="-" (
+  set SCRIPT_ERROR=Expected directory but found %2 instead!
+  goto invalid_cmd_line
+)
+set "SOLR_DATA_HOME=%~2"
+SHIFT
+SHIFT
+goto parse_args
+
 :set_example
 
 set "arg=%~2"
@@ -897,6 +919,7 @@ IF [%SOLR_LOGS_DIR%] == [] (
   set SOLR_LOGS_DIR=%SOLR_LOGS_DIR:"=%
 )
 set SOLR_LOGS_DIR_QUOTED="%SOLR_LOGS_DIR%"
+set SOLR_DATA_HOME_QUOTED="%SOLR_DATA_HOME%"
 
 set "EXAMPLE_DIR=%SOLR_TIP%\example"
 set TMP=!SOLR_HOME:%EXAMPLE_DIR%=!
@@ -1139,6 +1162,10 @@ IF "%verbose%"=="1" (
     @echo     SOLR_LOG_LEVEL  = !SOLR_LOG_LEVEL!
   )
 
+  IF NOT "%SOLR_DATA_HOME%"=="" (
+    @echo     SOLR_DATA_HOME  = !SOLR_DATA_HOME!
+  )
+
   @echo.
 )
 
@@ -1155,6 +1182,7 @@ IF "%SOLR_SSL_ENABLED%"=="true" (
 )
 IF NOT "%SOLR_LOG_LEVEL%"=="" set "START_OPTS=%START_OPTS% -Dsolr.log.level=%SOLR_LOG_LEVEL%"
 set "START_OPTS=%START_OPTS% -Dsolr.log.dir=%SOLR_LOGS_DIR_QUOTED%"
+IF NOT "%SOLR_DATA_HOME%"=="" set "START_OPTS=%START_OPTS% -Dsolr.data.home=%SOLR_DATA_HOME_QUOTED%"
 IF NOT DEFINED LOG4J_CONFIG set "LOG4J_CONFIG=file:%SOLR_SERVER_DIR%\resources\log4j.properties"
 
 cd /d "%SOLR_SERVER_DIR%"
@@ -1833,4 +1861,4 @@ REM Safe echo which does not mess with () in strings
 set "eout=%1"
 set eout=%eout:"=%
 echo !eout!
-GOTO :eof
+GOTO :eof
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39dfb780/solr/bin/solr.in.cmd
----------------------------------------------------------------------
diff --git a/solr/bin/solr.in.cmd b/solr/bin/solr.in.cmd
index 077b68d..6071263 100644
--- a/solr/bin/solr.in.cmd
+++ b/solr/bin/solr.in.cmd
@@ -68,6 +68,10 @@ REM Path to a directory for Solr to store cores and their data. By default, Solr
 REM If solr.xml is not stored in ZooKeeper, this directory needs to contain solr.xml
 REM set SOLR_HOME=
 
+REM Path to a directory that Solr will use as root for data folders for each core.
+REM If not set, defaults to <instance_dir>/data. Overridable per core through 'dataDir' core property
+REM set SOLR_DATA_HOME=
+
 REM Changes the logging level. Valid values: ALL, TRACE, DEBUG, INFO, WARN, ERROR, FATAL, OFF. Default is INFO
 REM This is an alternative to changing the rootLogger in log4j.properties
 REM set SOLR_LOG_LEVEL=INFO

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39dfb780/solr/bin/solr.in.sh
----------------------------------------------------------------------
diff --git a/solr/bin/solr.in.sh b/solr/bin/solr.in.sh
index 6c6c229..060ece9 100644
--- a/solr/bin/solr.in.sh
+++ b/solr/bin/solr.in.sh
@@ -86,6 +86,10 @@
 # If solr.xml is not stored in ZooKeeper, this directory needs to contain solr.xml
 #SOLR_HOME=
 
+# Path to a directory that Solr will use as root for data folders for each core.
+# If not set, defaults to <instance_dir>/data. Overridable per core through 'dataDir' core property
+#SOLR_DATA_HOME=
+
 # Solr provides a default Log4J configuration properties file in server/resources
 # however, you may want to customize the log settings and file appender location
 # so you can point the script to use a different log4j.properties file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39dfb780/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
index e710063..5301370 100644
--- a/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
@@ -19,6 +19,7 @@ package org.apache.solr.core;
 import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -395,6 +396,15 @@ public abstract class CachingDirectoryFactory extends DirectoryFactory {
     maxWriteMBPerSecMerge = (Double) args.get("maxWriteMBPerSecMerge");
     maxWriteMBPerSecRead = (Double) args.get("maxWriteMBPerSecRead");
     maxWriteMBPerSecDefault = (Double) args.get("maxWriteMBPerSecDefault");
+
+    dataHomePath = args.get(DATA_HOME) == null ? null : Paths.get((String) args.get(DATA_HOME));
+    if (dataHomePath == null && System.getProperty(DATA_HOME) != null && System.getProperty(DATA_HOME).length() > 0) {
+      // If solr.data.home is not specified explicitly in solrconfig.xml, fall back to global System Property with same name
+      dataHomePath = Paths.get(System.getProperty(DATA_HOME));
+    }
+    if (dataHomePath != null) {
+      log.info(DATA_HOME + "=" + dataHomePath);
+    }
   }
   
   /*

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39dfb780/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index 20824ab..f60958d 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -24,6 +24,8 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.NoSuchFileException;
 import java.util.Arrays;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.Collections;
 import java.util.List;
 
@@ -53,6 +55,11 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
 
   protected static final String INDEX_W_TIMESTAMP_REGEX = "index\\.[0-9]{17}"; // see SnapShooter.DATE_FMT
 
+  public static final String DATA_HOME = "solr.data.home";
+
+  // May be set by sub classes as data root, in which case getDataHome will use it as base
+  protected Path dataHomePath;
+
   // hint about what the directory contains - default is index directory
   public enum DirContext {DEFAULT, META_DATA}
 
@@ -315,9 +322,22 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
     return false;
   }
 
+  /**
+   * Get the data home folder. If solr.data.home is set, that is used, else base on instanceDir
+   * @param cd core descriptor instance
+   * @return a String with absolute path to data direcotry
+   */
   public String getDataHome(CoreDescriptor cd) throws IOException {
-    // by default, we go off the instance directory
-    return cd.getInstanceDir().resolve(cd.getDataDir()).toAbsolutePath().toString();
+    String dataDir;
+    if (dataHomePath != null) {
+      String instanceDirLastPath = cd.getInstanceDir().getName(cd.getInstanceDir().getNameCount()-1).toString();
+      dataDir = Paths.get(coreContainer.getSolrHome()).resolve(dataHomePath)
+          .resolve(instanceDirLastPath).resolve(cd.getDataDir()).toAbsolutePath().toString();
+    } else {
+      // by default, we go off the instance directory
+      dataDir = cd.getInstanceDir().resolve(cd.getDataDir()).toAbsolutePath().toString();
+    }
+    return dataDir;
   }
 
   public void cleanupOldIndexDirectories(final String dataDirPath, final String currentIndexDirPath, boolean afterCoreReload) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39dfb780/solr/core/src/test/org/apache/solr/core/DirectoryFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/DirectoryFactoryTest.java b/solr/core/src/test/org/apache/solr/core/DirectoryFactoryTest.java
index 59868b5..35198b7 100755
--- a/solr/core/src/test/org/apache/solr/core/DirectoryFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/core/DirectoryFactoryTest.java
@@ -16,7 +16,17 @@
  */
 package org.apache.solr.core;
 
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.Properties;
+
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.handler.admin.CoreAdminHandler;
+import org.apache.solr.handler.component.HttpShardHandlerFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 public class DirectoryFactoryTest extends LuceneTestCase {
 
@@ -28,4 +38,55 @@ public class DirectoryFactoryTest extends LuceneTestCase {
     assertEquals("hdfs", DirectoryFactory.LOCK_TYPE_HDFS);
   }
 
+  @After
+  @Before
+  public void clean() {
+    System.clearProperty("solr.data.home");
+    System.clearProperty("solr.solr.home");
+  }
+
+  @Test
+  public void testGetDataHome() throws Exception {
+    MockCoreContainer cc = new MockCoreContainer("/solr/home");
+    Properties cp = cc.getContainerProperties();
+    boolean zkAware = cc.isZooKeeperAware();
+    RAMDirectoryFactory rdf = new RAMDirectoryFactory();
+    rdf.initCoreContainer(cc);
+    rdf.init(new NamedList());
+
+    // No solr.data.home property set. Absolute instanceDir
+    assertEquals("/tmp/inst1/data", rdf.getDataHome(new CoreDescriptor("core_name", Paths.get("/tmp/inst1"), cp, zkAware)));
+
+    // Simulate solr.data.home set in solrconfig.xml <directoryFactory> tag
+    NamedList args = new NamedList();
+    args.add("solr.data.home", "/solrdata/");
+    rdf.init(args);
+    assertEquals("/solrdata/inst_dir/data", rdf.getDataHome(new CoreDescriptor("core_name", Paths.get("inst_dir"), cp, zkAware)));
+
+    // solr.data.home set with System property, and relative path
+    System.setProperty("solr.data.home", "solrdata");
+    rdf.init(new NamedList());
+    assertEquals("/solr/home/solrdata/inst_dir/data", rdf.getDataHome(new CoreDescriptor("core_name", Paths.get("inst_dir"), cp, zkAware)));
+    // Test parsing last component of instanceDir, and using custom dataDir
+    assertEquals("/solr/home/solrdata/myinst/mydata", rdf.getDataHome(new CoreDescriptor("core_name", Paths.get("/path/to/myinst"), cp, zkAware, "dataDir", "mydata")));
+  }
+
+
+  private static class MockCoreContainer extends CoreContainer {
+
+    private final String mockSolrHome;
+
+    public MockCoreContainer(String solrHome) throws IOException {
+      super(new Object());
+      mockSolrHome = solrHome;
+      this.shardHandlerFactory = new HttpShardHandlerFactory();
+      this.coreAdminHandler = new CoreAdminHandler();
+    }
+
+    @Override
+    public String getSolrHome() {
+      return mockSolrHome;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39dfb780/solr/solr-ref-guide/src/datadir-and-directoryfactory-in-solrconfig.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/datadir-and-directoryfactory-in-solrconfig.adoc b/solr/solr-ref-guide/src/datadir-and-directoryfactory-in-solrconfig.adoc
index 8ee43be..c68a3ad 100644
--- a/solr/solr-ref-guide/src/datadir-and-directoryfactory-in-solrconfig.adoc
+++ b/solr/solr-ref-guide/src/datadir-and-directoryfactory-in-solrconfig.adoc
@@ -33,6 +33,8 @@ The `${solr.core.name}` substitution will cause the name of the current core to
 
 If you are using replication to replicate the Solr index (as described in <<legacy-scaling-and-distribution.adoc#legacy-scaling-and-distribution,Legacy Scaling and Distribution>>), then the `<dataDir>` directory should correspond to the index directory used in the replication configuration.
 
+NOTE: If the environment variable `SOLR_DATA_HOME` if defined, or if `solr.data.home` is configured for your DirectoryFactory, the location of data directory will be `<SOLR_DATA_HOME>/<instance_name>/data`.
+
 [[DataDirandDirectoryFactoryinSolrConfig-SpecifyingtheDirectoryFactoryForYourIndex]]
 == Specifying the DirectoryFactory For Your Index