mirror of https://github.com/apache/lucene.git
SOLR-9194: Enhance the bin/solr script to perform file operations to/from Zookeeper
This commit is contained in:
parent
1fa0d0419a
commit
a851d5f557
|
@ -78,6 +78,7 @@ New Features
|
|||
|
||||
* SOLR-9251: Support for a new tag 'role' in replica placement rules (noble)
|
||||
|
||||
* SOLR-9194: Enhance the bin/solr script to perform file operations to/from Zookeeper (Erick Erickson, janhoy)
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
|
|
254
solr/bin/solr
254
solr/bin/solr
|
@ -354,28 +354,86 @@ function print_usage() {
|
|||
echo " Solr instance and will use the port of the first server it finds."
|
||||
echo ""
|
||||
elif [ "$CMD" == "zk" ]; then
|
||||
echo "Usage: solr zk [-upconfig|-downconfig] [-d confdir] [-n configName] [-z zkHost]"
|
||||
print_short_zk_usage ""
|
||||
echo " Be sure to check the Solr logs in case of errors."
|
||||
echo ""
|
||||
echo " -upconfig to move a configset from the local machine to Zookeeper."
|
||||
echo " -z zkHost Optional Zookeeper connection string for all commands. If specified it"
|
||||
echo " overrides the 'ZK_HOST=...'' defined in solr.in.sh."
|
||||
echo ""
|
||||
echo " -downconfig to move a configset from Zookeeper to the local machine."
|
||||
echo " upconfig uploads a configset from the local machine to Zookeeper. (Backcompat: -upconfig)"
|
||||
echo ""
|
||||
echo " -n configName Name of the configset in Zookeeper that will be the destination of"
|
||||
echo " 'upconfig' and the source for 'downconfig'."
|
||||
echo " downconfig downloads a configset from Zookeeper to the local machine. (Backcompat: -downconfig)"
|
||||
echo ""
|
||||
echo " -d confdir The local directory the configuration will be uploaded from for"
|
||||
echo " 'upconfig' or downloaded to for 'downconfig'. For 'upconfig', this"
|
||||
echo " can be one of the example configsets, basic_configs, data_driven_schema_configs or"
|
||||
echo " sample_techproducts_configs or an arbitrary directory."
|
||||
echo " -n configName Name of the configset in Zookeeper that will be the destination of"
|
||||
echo " 'upconfig' and the source for 'downconfig'."
|
||||
echo ""
|
||||
echo " -z zkHost Zookeeper connection string."
|
||||
echo " -d confdir The local directory the configuration will be uploaded from for"
|
||||
echo " 'upconfig' or downloaded to for 'downconfig'. If 'confdir' is a child of"
|
||||
echo " ...solr/server/solr/configsets' then the configs will be copied from/to"
|
||||
echo " that directory. Otherwise it is interpreted as a simple local path."
|
||||
echo ""
|
||||
echo " NOTE: Solr must have been started least once (or have it running) before using this command."
|
||||
echo " This initialized Zookeeper for Solr"
|
||||
echo " cp copies files or folders to/from Zookeeper or Zokeeper -> Zookeeper"
|
||||
echo " -r Recursively copy <src> to <dst>. Command will fail if <src> has children and "
|
||||
echo " -r is not specified. Optional"
|
||||
echo ""
|
||||
echo " <src>, <dest> : [file:][/]path/to/local/file or zk:/path/to/zk/node"
|
||||
echo " NOTE: <src> and <dest> may both be Zookeeper resources prefixed by 'zk:'"
|
||||
echo " When <src> is a zk resource, <dest> may be '.'"
|
||||
echo " If <dest> ends with '/', then <dest> will be a local folder or parent znode and the last"
|
||||
echo " element of the <src> path will be appended."
|
||||
echo ""
|
||||
echo " The 'file:' prefix is stripped, thus 'file:/' specifies an absolute local path and"
|
||||
echo " 'file:somewhere' specifies a relative local path. All paths on Zookeeper are absolute"
|
||||
echo " so the slash is required."
|
||||
echo ""
|
||||
echo " Zookeeper nodes CAN have data, so moving a single file to a parent znode"
|
||||
echo " will overlay the data on the parent Znode so specifying the trailing slash"
|
||||
echo " is important."
|
||||
echo ""
|
||||
echo " Wildcards are not supported"
|
||||
echo ""
|
||||
echo " rm deletes files or folders on Zookeeper"
|
||||
echo " -r Recursively delete if <path> is a directory. Command will fail if <path>"
|
||||
echo " has children and -r is not specified. Optional"
|
||||
echo " <path> : [zk:]/path/to/zk/node. <path> may not be the root ('/')"
|
||||
echo ""
|
||||
echo " mv moves (renames) znodes on Zookeeper"
|
||||
echo " <src>, <dest> : Zookeeper nodes, the 'zk:' prefix is optional."
|
||||
echo " If <dest> ends with '/', then <dest> will be a parent znode"
|
||||
echo " and the last element of the <src> path will be appended."
|
||||
echo " Zookeeper nodes CAN have data, so moving a single file to a parent znode"
|
||||
echo " will overlay the data on the parent Znode so specifying the trailing slash"
|
||||
echo " is important."
|
||||
echo ""
|
||||
echo " ls lists the znodes on Zookeeper"
|
||||
echo " -r recursively descends the path listing all znodes. Optional"
|
||||
echo " <path>: The Zookeeper path to use as the root."
|
||||
echo ""
|
||||
echo " Only the node names are listed, not data"
|
||||
echo ""
|
||||
fi
|
||||
} # end print_usage
|
||||
|
||||
function print_short_zk_usage() {
|
||||
|
||||
if [ "$1" != "" ]; then
|
||||
echo -e "\nERROR: $1\n"
|
||||
fi
|
||||
|
||||
echo " Usage: solr zk upconfig|downconfig -d <confdir> -n <configName> [-z zkHost]"
|
||||
echo " solr zk cp [-r] <src> <dest> [-z zkHost]"
|
||||
echo " solr zk rm [-r] <path> [-z zkHost]"
|
||||
echo " solr zk mv <src> <dest> [-z zkHost]"
|
||||
echo " solr zk ls [-r] <path> [-z zkHost]"
|
||||
echo ""
|
||||
|
||||
if [ "$1" == "" ]; then
|
||||
echo "Type bin/solr zk -help for full usage help"
|
||||
else
|
||||
exit 1
|
||||
fi
|
||||
}
|
||||
|
||||
# used to show the script is still alive when waiting on work to complete
|
||||
function spinner() {
|
||||
local pid=$1
|
||||
|
@ -571,7 +629,7 @@ if [ "$SCRIPT_CMD" == "healthcheck" ]; then
|
|||
;;
|
||||
-z|-zkhost)
|
||||
if [[ -z "$2" || "${2:0:1}" == "-" ]]; then
|
||||
print_usage "$SCRIPT_CMD" "ZooKeepeer connection string is required when using the $1 option!"
|
||||
print_usage "$SCRIPT_CMD" "ZooKeeper connection string is required when using the $1 option!"
|
||||
exit 1
|
||||
fi
|
||||
ZK_HOST="$2"
|
||||
|
@ -821,93 +879,131 @@ if [[ "$SCRIPT_CMD" == "delete" ]]; then
|
|||
exit $?
|
||||
fi
|
||||
|
||||
# Upload or download a configset to Zookeeper
|
||||
ZK_RECURSE=false
|
||||
# Zookeeper file maintenance (upconfig, downconfig, files up/down etc.)
|
||||
# It's a little clumsy to have the parsing go round and round for upconfig and downconfig, but that's
|
||||
# necessary for back-compat
|
||||
if [[ "$SCRIPT_CMD" == "zk" ]]; then
|
||||
|
||||
if [ $# -gt 0 ]; then
|
||||
while true; do
|
||||
case "$1" in
|
||||
-z|-zkhost)
|
||||
if [[ -z "$2" || "${2:0:1}" == "-" ]]; then
|
||||
print_usage "$SCRIPT_CMD" "ZooKeepeer connection string is required when using the $1 option!"
|
||||
exit 1
|
||||
fi
|
||||
ZK_HOST="$2"
|
||||
shift 2
|
||||
;;
|
||||
-n|-confname)
|
||||
if [[ -z "$2" || "${2:0:1}" == "-" ]]; then
|
||||
print_usage "$SCRIPT_CMD" "Configuration name is required when using the $1 option!"
|
||||
exit 1
|
||||
fi
|
||||
CONFIGSET_CONFNAME="$2"
|
||||
shift 2
|
||||
;;
|
||||
-d|-confdir)
|
||||
if [[ -z "$2" || "${2:0:1}" == "-" ]]; then
|
||||
print_usage "$SCRIPT_CMD" "Configuration directory is required when using the $1 option!"
|
||||
exit 1
|
||||
fi
|
||||
CONFIGSET_CONFDIR="$2"
|
||||
shift 2
|
||||
;;
|
||||
-upconfig)
|
||||
ZK_OP="upconfig"
|
||||
shift 1
|
||||
;;
|
||||
-downconfig)
|
||||
ZK_OP="downconfig"
|
||||
shift 1
|
||||
;;
|
||||
-help|-usage|-h)
|
||||
print_usage "$SCRIPT_CMD"
|
||||
exit 0
|
||||
;;
|
||||
--)
|
||||
shift
|
||||
break
|
||||
;;
|
||||
*)
|
||||
if [ "$1" != "" ]; then
|
||||
print_usage "$SCRIPT_CMD" "Unrecognized or misplaced argument: $1!"
|
||||
exit 1
|
||||
-upconfig|upconfig|-downconfig|downconfig|cp|rm|mv|ls)
|
||||
if [ "${1:0:1}" == "-" ]; then
|
||||
ZK_OP=${1:1}
|
||||
else
|
||||
ZK_OP=$1
|
||||
fi
|
||||
shift 1
|
||||
;;
|
||||
-z|-zkhost)
|
||||
if [[ -z "$2" || "${2:0:1}" == "-" ]]; then
|
||||
print_short_zk_usage "$SCRIPT_CMD" "ZooKeeper connection string is required when using the $1 option!"
|
||||
fi
|
||||
ZK_HOST="$2"
|
||||
shift 2
|
||||
;;
|
||||
-n|-confname)
|
||||
if [[ -z "$2" || "${2:0:1}" == "-" ]]; then
|
||||
print_short_zk_usage "$SCRIPT_CMD" "Configuration name is required when using the $1 option!"
|
||||
fi
|
||||
CONFIGSET_CONFNAME="$2"
|
||||
shift 2
|
||||
;;
|
||||
-d|-confdir)
|
||||
if [[ -z "$2" || "${2:0:1}" == "-" ]]; then
|
||||
print_short_zk_usage "$SCRIPT_CMD" "Configuration directory is required when using the $1 option!"
|
||||
fi
|
||||
CONFIGSET_CONFDIR="$2"
|
||||
shift 2
|
||||
;;
|
||||
-r)
|
||||
ZK_RECURSE="true"
|
||||
shift
|
||||
;;
|
||||
-help|-usage|-h)
|
||||
print_usage "$SCRIPT_CMD"
|
||||
exit 0
|
||||
;;
|
||||
--)
|
||||
shift
|
||||
break
|
||||
;;
|
||||
*) # Pick up <src> <dst> or <path> params for rm, ls, cp, mv.
|
||||
if [ "$1" == "" ]; then
|
||||
break # out-of-args, stop looping
|
||||
fi
|
||||
if [ -z "$ZK_SRC" ]; then
|
||||
ZK_SRC=$1
|
||||
else
|
||||
if [ -z "$ZK_DST" ]; then
|
||||
ZK_DST=$1
|
||||
else
|
||||
break # out-of-args, stop looping
|
||||
print_short_zk_usage "Unrecognized or misplaced command $1"
|
||||
fi
|
||||
;;
|
||||
fi
|
||||
shift
|
||||
;;
|
||||
esac
|
||||
done
|
||||
fi
|
||||
|
||||
if [ -z "$ZK_OP" ]; then
|
||||
echo "Zookeeper operation (one of '-upconfig' or '-downconfig') is required!"
|
||||
print_usage "$SCRIPT_CMD"
|
||||
exit 1
|
||||
print_short_zk_usage "Zookeeper operation (one of 'upconfig', 'downconfig', 'rm', 'mv', 'cp', 'ls') is required!"
|
||||
fi
|
||||
|
||||
if [ -z "$ZK_HOST" ]; then
|
||||
echo "Zookeeper address (-z) argument is required!"
|
||||
print_usage "$SCRIPT_CMD"
|
||||
exit 1
|
||||
print_short_zk_usage "Zookeeper address (-z) argument is required or ZK_HOST must be specified in the solr.in.sh file."
|
||||
fi
|
||||
|
||||
if [ -z "$CONFIGSET_CONFDIR" ]; then
|
||||
echo "Local directory of the configset (-d) argument is required!"
|
||||
print_usage "$SCRIPT_CMD"
|
||||
exit 1
|
||||
if [[ "$ZK_OP" == "upconfig" || "$ZK_OP" == "downconfig" ]]; then
|
||||
if [ -z "$CONFIGSET_CONFDIR" ]; then
|
||||
print_short_zk_usage "Local directory of the configset (-d) argument is required!"
|
||||
fi
|
||||
|
||||
if [ -z "$CONFIGSET_CONFNAME" ]; then
|
||||
print_short_zk_usage "Configset name on Zookeeper (-n) argument is required!"
|
||||
fi
|
||||
fi
|
||||
|
||||
if [ -z "$CONFIGSET_CONFNAME" ]; then
|
||||
echo "Configset name on Zookeeper (-n) argument is required!"
|
||||
print_usage "$SCRIPT_CMD"
|
||||
exit 1
|
||||
if [[ "$ZK_OP" == "cp" || "$ZK_OP" == "mv" ]]; then
|
||||
if [[ -z "$ZK_SRC" || -z "$ZK_DST" ]]; then
|
||||
print_short_zk_usage "<source> and <destination> must be specified when using either the 'mv' or 'cp' commands."
|
||||
fi
|
||||
if [[ "$ZK_OP" == "cp" && "${ZK_SRC:0:3}" != "zk:" && "${ZK_DST:0:3}" != "zk:" ]]; then
|
||||
print_short_zk_usage "One of the source or desintation paths must be prefixed by 'zk:' for the 'cp' command."
|
||||
fi
|
||||
fi
|
||||
|
||||
if [ "$ZK_OP" == "upconfig" ]; then
|
||||
run_tool "$ZK_OP" -confname "$CONFIGSET_CONFNAME" -confdir "$CONFIGSET_CONFDIR" -zkHost "$ZK_HOST" -configsetsDir "$SOLR_TIP/server/solr/configsets"
|
||||
else
|
||||
run_tool "$ZK_OP" -confname "$CONFIGSET_CONFNAME" -confdir "$CONFIGSET_CONFDIR" -zkHost "$ZK_HOST"
|
||||
fi
|
||||
case "$ZK_OP" in
|
||||
upconfig)
|
||||
run_tool "$ZK_OP" -confname "$CONFIGSET_CONFNAME" -confdir "$CONFIGSET_CONFDIR" -zkHost "$ZK_HOST" -configsetsDir "$SOLR_TIP/server/solr/configsets"
|
||||
;;
|
||||
downconfig)
|
||||
run_tool "$ZK_OP" -confname "$CONFIGSET_CONFNAME" -confdir "$CONFIGSET_CONFDIR" -zkHost "$ZK_HOST"
|
||||
;;
|
||||
rm)
|
||||
if [ -z "$ZK_SRC" ]; then
|
||||
print_short_zk_usage "Zookeeper path to remove must be specified when using the 'rm' command"
|
||||
fi
|
||||
run_tool "$ZK_OP" -path "$ZK_SRC" -zkHost "$ZK_HOST" -recurse "$ZK_RECURSE"
|
||||
;;
|
||||
mv)
|
||||
run_tool "$ZK_OP" -src "$ZK_SRC" -dst "$ZK_DST" -zkHost "$ZK_HOST"
|
||||
;;
|
||||
cp)
|
||||
run_tool "$ZK_OP" -src "$ZK_SRC" -dst "$ZK_DST" -zkHost "$ZK_HOST" -recurse "$ZK_RECURSE"
|
||||
;;
|
||||
ls)
|
||||
if [ -z "$ZK_SRC" ]; then
|
||||
print_short_zk_usage "Zookeeper path to list must be specified when using the 'ls' command"
|
||||
fi
|
||||
run_tool "$ZK_OP" -path "$ZK_SRC" -recurse "$ZK_RECURSE" -zkHost "$ZK_HOST"
|
||||
;;
|
||||
*)
|
||||
print_short_zk_usage "Unrecognized Zookeeper operation $ZK_OP"
|
||||
;;
|
||||
esac
|
||||
|
||||
exit $?
|
||||
fi
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
@REM
|
||||
@REM
|
||||
@REM Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
@REM contributor license agreements. See the NOTICE file distributed with
|
||||
@REM this work for additional information regarding copyright ownership.
|
||||
|
@ -137,6 +137,7 @@ IF "%1"=="delete" (
|
|||
IF "%1"=="zk" (
|
||||
set SCRIPT_CMD=zk
|
||||
SHIFT
|
||||
set ZK_RECURSE=false
|
||||
goto parse_zk_args
|
||||
)
|
||||
|
||||
|
@ -164,7 +165,7 @@ goto done
|
|||
:script_usage
|
||||
@echo.
|
||||
@echo Usage: solr COMMAND OPTIONS
|
||||
@echo where COMMAND is one of: start, stop, restart, healthcheck, create, create_core, create_collection, delete, version, upconfig, downconfig
|
||||
@echo where COMMAND is one of: start, stop, restart, healthcheck, create, create_core, create_collection, delete, version, zk
|
||||
@echo.
|
||||
@echo Standalone server example (start Solr running in the background on port 8984):
|
||||
@echo.
|
||||
|
@ -344,26 +345,83 @@ echo.
|
|||
goto done
|
||||
|
||||
:zk_usage
|
||||
set ZK_FULL=true
|
||||
goto zk_short_usage
|
||||
:zk_full_usage
|
||||
echo Be sure to check the Solr logs in case of errors.
|
||||
echo.
|
||||
echo Usage: solr zk [-downconfig or -upconfig] [-d confdir] [-n configName] [-z zkHost]
|
||||
echo -z zkHost Optional Zookeeper connection string for all commands. If specified it
|
||||
echo overrides the 'ZK_HOST=...'' defined in solr.in.sh.
|
||||
echo.
|
||||
echo -upconfig to move a configset from the local machine to Zookeeper
|
||||
echo upconfig uploads a configset from the local machine to Zookeeper. (Backcompat: -upconfig)
|
||||
echo.
|
||||
echo -downconfig to move a configset from Zookeeper to the local machine
|
||||
echo downconfig downloads a configset from Zookeeper to the local machine. (Backcompat: -downconfig)
|
||||
echo.
|
||||
echo -n configName Name of the configset in Zookeeper that will be the destination of
|
||||
echo 'upconfig' and the source for 'downconfig'.
|
||||
echo -n configName Name of the configset in Zookeeper that will be the destination of
|
||||
echo 'upconfig' and the source for 'downconfig'.
|
||||
echo.
|
||||
echo -d confdir The local directory the configuration will be uploaded from for
|
||||
echo 'upconfig' or downloaded to for 'downconfig'. For 'upconfig', this
|
||||
echo can be one of the example configsets, basic_configs, data_driven_schema_configs or
|
||||
echo sample_techproducts_configs or an arbitrary directory.
|
||||
echo -d confdir The local directory the configuration will be uploaded from for
|
||||
echo 'upconfig' or downloaded to for 'downconfig'. If 'confdir' is a child of
|
||||
echo ...solr/server/solr/configsets' then the configs will be copied from/to
|
||||
echo that directory. Otherwise it is interpreted as a simple local path.
|
||||
echo.
|
||||
echo -z zkHost Zookeeper connection string.
|
||||
echo cp copies files or folders to/from Zookeeper or Zokeeper -^> Zookeeper
|
||||
echo -r Recursively copy ^<src^> to ^<dst^>. Command will fail if ^<src^> has children and
|
||||
echo -r is not specified. Optional
|
||||
echo.
|
||||
echo NOTE: Solr must have been started least once (or have it running) before using this command.
|
||||
echo This initialized Zookeeper for Solr
|
||||
echo. ^<src^>, ^<dest^> : [file:][/]path/to/local/file or zk:/path/to/zk/node
|
||||
echo NOTE: ^<src^> and ^<dest^> may both be Zookeeper resources prefixed by 'zk:'
|
||||
echo When ^<src^> is a zk resource, ^<dest^> may be '.'
|
||||
echo If ^<dest^> ends with '/', then ^<dest^> will be a local folder or parent znode and the last
|
||||
echo element of the ^<src^> path will be appended.
|
||||
echo.
|
||||
echo The 'file:' prefix is stripped, thus 'file:/' specifies an absolute local path and
|
||||
echo 'file:somewhere' specifies a relative local path. All paths on Zookeeper are absolute
|
||||
echo so the slash is required.
|
||||
echo.
|
||||
echo Zookeeper nodes CAN have data, so moving a single file to a parent znode
|
||||
echo will overlay the data on the parent Znode so specifying the trailing slash
|
||||
echo is important.
|
||||
echo.
|
||||
echo Wildcards are not supported
|
||||
echo.
|
||||
echo rm deletes files or folders on Zookeeper
|
||||
echo -r Recursively delete if ^<path^> is a directory. Command will fail if ^<path^>
|
||||
echo has children and -r is not specified. Optional
|
||||
echo ^<path^> : [zk:]/path/to/zk/node. ^<path^> may not be the root ('/')"
|
||||
echo.
|
||||
echo mv moves (renames) znodes on Zookeeper
|
||||
echo ^<src^>, ^<dest^> : Zookeeper nodes, the 'zk:' prefix is optional.
|
||||
echo If ^<dest^> ends with '/', then ^<dest^> will be a parent znode
|
||||
echo and the last element of the ^<src^> path will be appended.
|
||||
echo Zookeeper nodes CAN have data, so moving a single file to a parent znode
|
||||
echo will overlay the data on the parent Znode so specifying the trailing slash
|
||||
echo is important.
|
||||
echo.
|
||||
echo ls lists the znodes on Zookeeper
|
||||
echo -r recursively descends the path listing all znodes. Optional
|
||||
echo ^<path^>: The Zookeeper path to use as the root.
|
||||
echo.
|
||||
echo Only the node names are listed, not data
|
||||
echo.
|
||||
goto done
|
||||
|
||||
:zk_short_usage
|
||||
IF NOT "!ERROR_MSG!"=="" (
|
||||
echo ERROR: !ERROR_MSG!
|
||||
echo.
|
||||
)
|
||||
echo Usage: solr zk upconfig^|downconfig -d ^<confdir^> -n ^<configName^> [-z zkHost]
|
||||
echo solr zk cp [-r] ^<src^> ^<dest^> [-z zkHost]
|
||||
echo solr zk rm [-r] ^<path^> [-z zkHost]
|
||||
echo solr zk mv ^<src^> ^<dest^> [-z zkHost]
|
||||
echo solr zk ls [-r] ^<path^> [-z zkHost]
|
||||
echo.
|
||||
IF "%ZK_FULL%"=="true" (
|
||||
goto zk_full_usage
|
||||
) ELSE (
|
||||
echo Type bin/solr zk -help for full usage help
|
||||
)
|
||||
goto done
|
||||
|
||||
|
||||
|
@ -405,6 +463,7 @@ IF "%1"=="-key" goto set_stop_key
|
|||
IF "%1"=="-all" goto set_stop_all
|
||||
IF "%firstTwo%"=="-D" goto set_passthru
|
||||
IF NOT "%1"=="" goto invalid_cmd_line
|
||||
goto invalid_cmd_line
|
||||
|
||||
:set_script_cmd
|
||||
set SCRIPT_CMD=%1
|
||||
|
@ -1159,27 +1218,63 @@ org.apache.solr.util.SolrCLI delete -name !DELETE_NAME! -deleteConfig !DELETE_CO
|
|||
|
||||
goto done
|
||||
|
||||
REM Clumsy to do the state machine thing for -d and -n, but that's required for back-compat
|
||||
:parse_zk_args
|
||||
IF [%1]==[] goto run_zk
|
||||
IF "%1"=="-upconfig" goto set_zk_op_up
|
||||
IF "%1"=="-downconfig" goto set_zk_op_down
|
||||
IF "%1"=="-n" goto set_config_name
|
||||
IF "%1"=="-configname" goto set_config_name
|
||||
IF "%1"=="-d" goto set_configdir
|
||||
IF "%1"=="-confdir" goto set_configdir
|
||||
IF "%1"=="-z" goto set_config_zk
|
||||
IF "%1"=="/?" goto usage
|
||||
IF "%1"=="-h" goto zk_usage
|
||||
IF "%1"=="-help" goto zk_usage
|
||||
IF "%1"=="-upconfig" (
|
||||
goto set_zk_op
|
||||
) ELSE IF "%1"=="upconfig" (
|
||||
goto set_zk_op
|
||||
) ELSE IF "%1"=="-downconfig" (
|
||||
goto set_zk_op
|
||||
) ELSE IF "%1"=="downconfig" (
|
||||
goto set_zk_op
|
||||
) ELSE IF "%1"=="cp" (
|
||||
goto set_zk_op
|
||||
) ELSE IF "%1"=="mv" (
|
||||
goto set_zk_op
|
||||
) ELSE IF "%1"=="rm" (
|
||||
goto set_zk_op
|
||||
) ELSE IF "%1"=="ls" (
|
||||
goto set_zk_op
|
||||
) ELSE IF "%1"=="-n" (
|
||||
goto set_config_name
|
||||
) ELSE IF "%1"=="-r" (
|
||||
goto set_zk_recurse
|
||||
) ELSE IF "%1"=="-configname" (
|
||||
goto set_config_name
|
||||
) ELSE IF "%1"=="-d" (
|
||||
goto set_configdir
|
||||
) ELSE IF "%1"=="-confdir" (
|
||||
goto set_configdir
|
||||
) ELSE IF "%1"=="-z" (
|
||||
goto set_config_zk
|
||||
) ELSE IF "%1"=="/?" (
|
||||
goto zk_usage
|
||||
) ELSE IF "%1"=="-h" (
|
||||
goto zk_usage
|
||||
) ELSE IF "%1"=="-help" (
|
||||
goto zk_usage
|
||||
) ELSE IF "!ZK_SRC!"=="" (
|
||||
if not "%~1"=="" (
|
||||
goto set_zk_src
|
||||
)
|
||||
goto zk_usage
|
||||
) ELSE IF "!ZK_DST!"=="" (
|
||||
IF "%ZK_OP%"=="cp" (
|
||||
goto set_zk_dst
|
||||
)
|
||||
IF "%ZK_OP%"=="mv" (
|
||||
goto set_zk_dst
|
||||
)
|
||||
set ZK_DST="_"
|
||||
) ELSE IF NOT "%1"=="" (
|
||||
set ERROR_MSG="Unrecognized or misplaced zk argument %1%"
|
||||
goto zk_short_usage
|
||||
)
|
||||
goto run_zk
|
||||
|
||||
:set_zk_op_up
|
||||
set ZK_OP=upconfig
|
||||
SHIFT
|
||||
goto parse_zk_args
|
||||
|
||||
:set_zk_op_down
|
||||
set ZK_OP=downconfig
|
||||
:set_zk_op
|
||||
set ZK_OP=%~1
|
||||
SHIFT
|
||||
goto parse_zk_args
|
||||
|
||||
|
@ -1196,43 +1291,118 @@ SHIFT
|
|||
goto parse_zk_args
|
||||
|
||||
:set_config_zk
|
||||
set CONFIGSET_ZK=%~2
|
||||
set ZK_HOST=%~2
|
||||
SHIFT
|
||||
SHIFT
|
||||
goto parse_zk_args
|
||||
|
||||
:set_zk_src
|
||||
set ZK_SRC=%~1
|
||||
SHIFT
|
||||
goto parse_zk_args
|
||||
|
||||
:set_zk_dst
|
||||
set ZK_DST=%~1
|
||||
SHIFT
|
||||
goto parse_zk_args
|
||||
|
||||
:set_zk_recurse
|
||||
set ZK_RECURSE="true"
|
||||
SHIFT
|
||||
goto parse_zk_args
|
||||
|
||||
:run_zk
|
||||
IF "!ZK_OP!"=="" (
|
||||
set "SCRIPT_ERROR=One of '-upconfig' or '-downconfig' is required for %SCRIPT_CMD%"
|
||||
goto invalid_cmd_line
|
||||
set "ERROR_MSG=Invalid command specified for zk sub-command"
|
||||
goto zk_short_usage
|
||||
)
|
||||
|
||||
IF "!CONFIGSET_NAME!"=="" (
|
||||
set "SCRIPT_ERROR=Name (-n) is a required parameter for %SCRIPT_CMD%"
|
||||
goto invalid_cmd_line
|
||||
IF "!ZK_HOST!"=="" (
|
||||
set "ERROR_MSG=Must specify -z zkHost"
|
||||
goto zk_short_usage
|
||||
)
|
||||
|
||||
if "!CONFIGSET_DIR!"=="" (
|
||||
set "SCRIPT_ERROR=Name (-d) is a required parameter for %SCRIPT_CMD%"
|
||||
goto err
|
||||
IF "!ZK_OP!"=="-upconfig" (
|
||||
set ZK_OP="upconfig"
|
||||
)
|
||||
|
||||
if "!CONFIGSET_ZK!"=="" (
|
||||
set "SCRIPT_ERROR=Name (-z) is a required parameter for %SCRIPT_CMD%"
|
||||
goto err
|
||||
IF "!ZK_OP!"=="-downconfig" (
|
||||
set ZK_OP="downconfig"
|
||||
)
|
||||
|
||||
IF "!ZK_OP!"=="upconfig" (
|
||||
"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -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 !ZK_OP! -confname !CONFIGSET_NAME! -confdir !CONFIGSET_DIR! -zkHost !CONFIGSET_ZK! -configsetsDir "%SOLR_TIP%/server/solr/configsets"
|
||||
IF "!CONFIGSET_NAME!"=="" (
|
||||
set ERROR_MSG="-n option must be set for upconfig"
|
||||
goto zk_short_usage
|
||||
)
|
||||
IF "!CONFIGSET_DIR!"=="" (
|
||||
set ERROR_MSG="The -d option must be set for upconfig."
|
||||
goto zk_short_usage
|
||||
)
|
||||
"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -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 !ZK_OP! -confname !CONFIGSET_NAME! -confdir !CONFIGSET_DIR! -zkHost !ZK_HOST! -configsetsDir "%SOLR_TIP%/server/solr/configsets"
|
||||
) ELSE IF "!ZK_OP!"=="downconfig" (
|
||||
IF "!CONFIGSET_NAME!"=="" (
|
||||
set ERROR_MSG="-n option must be set for downconfig"
|
||||
goto zk_short_usage
|
||||
)
|
||||
IF "!CONFIGSET_DIR!"=="" (
|
||||
set ERROR_MSG="The -d option must be set for downconfig."
|
||||
goto zk_short_usage
|
||||
)
|
||||
"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -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 !ZK_OP! -confname !CONFIGSET_NAME! -confdir !CONFIGSET_DIR! -zkHost !ZK_HOST!
|
||||
) ELSE IF "!ZK_OP!"=="cp" (
|
||||
IF "%ZK_SRC%"=="" (
|
||||
set ERROR_MSG="<src> must be specified for 'cp' command"
|
||||
goto zk_short_usage
|
||||
)
|
||||
IF "%ZK_DST%"=="" (
|
||||
set ERROR_MSG=<dest> must be specified for 'cp' command"
|
||||
goto zk_short_usage
|
||||
)
|
||||
IF NOT "!ZK_SRC:~0,3!"=="zk:" (
|
||||
IF NOT "!%ZK_DST:~0,3!"=="zk:" (
|
||||
set ERROR_MSG="At least one of src or dst must be prefixed by 'zk:'"
|
||||
goto zk_short_usage
|
||||
)
|
||||
)
|
||||
"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -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 !ZK_OP! -zkHost !ZK_HOST! -src !ZK_SRC! -dst !ZK_DST! -recurse !ZK_RECURSE!
|
||||
) ELSE IF "!ZK_OP!"=="mv" (
|
||||
IF "%ZK_SRC%"=="" (
|
||||
set ERROR_MSG="<src> must be specified for 'mv' command"
|
||||
goto zk_short_usage
|
||||
)
|
||||
IF "%ZK_DST%"=="" (
|
||||
set ERROR_MSG="<dest> must be specified for 'mv' command"
|
||||
goto zk_short_usage
|
||||
)
|
||||
"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -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 !ZK_OP! -zkHost !ZK_HOST! -src !ZK_SRC! -dst !ZK_DST!
|
||||
) ELSE IF "!ZK_OP!"=="rm" (
|
||||
IF "%ZK_SRC"=="" (
|
||||
set ERROR_MSG="Zookeeper path to remove must be specified when using the 'rm' command"
|
||||
goto zk_short_usage
|
||||
)
|
||||
"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -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 !ZK_OP! -zkHost !ZK_HOST! -path !ZK_SRC! -recurse !ZK_RECURSE!
|
||||
) ELSE IF "!ZK_OP!"=="ls" (
|
||||
IF "%ZK_SRC"=="" (
|
||||
set ERROR_MSG="Zookeeper path to remove must be specified when using the 'rm' command"
|
||||
goto zk_short_usage
|
||||
)
|
||||
"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -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 !ZK_OP! -zkHost !ZK_HOST! -path !ZK_SRC! -recurse !ZK_RECURSE!
|
||||
) ELSE (
|
||||
"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -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 !ZK_OP! -confname !CONFIGSET_NAME! -confdir !CONFIGSET_DIR! -zkHost !CONFIGSET_ZK!
|
||||
set ERROR_MSG="Unknown zk option !ZK_OP!"
|
||||
goto zk_short_usage
|
||||
)
|
||||
|
||||
goto done
|
||||
|
||||
:invalid_cmd_line
|
||||
|
@ -1260,7 +1430,7 @@ IF "%FIRST_ARG%"=="start" (
|
|||
) ELSE IF "%FIRST_ARG%"=="create_collection" (
|
||||
goto create_collection_usage
|
||||
) ELSE IF "%FIRST_ARG%"=="zk" (
|
||||
goto zk_usage
|
||||
goto zk_short_usage
|
||||
) ELSE (
|
||||
goto script_usage
|
||||
)
|
||||
|
@ -1300,12 +1470,12 @@ for /f "tokens=3" %%a in ("!JAVAVEROUT!") do (
|
|||
set JAVA_VERSION_INFO=%%a
|
||||
REM Remove surrounding quotes
|
||||
set JAVA_VERSION_INFO=!JAVA_VERSION_INFO:"=!
|
||||
|
||||
|
||||
REM Extract the major Java version, e.g. 7, 8, 9, 10 ...
|
||||
for /f "tokens=2 delims=." %%a in ("!JAVA_VERSION_INFO!") do (
|
||||
set JAVA_MAJOR_VERSION=%%a
|
||||
)
|
||||
|
||||
|
||||
REM Don't look for "_{build}" if we're on IBM J9.
|
||||
if NOT "%JAVA_VENDOR%" == "IBM J9" (
|
||||
for /f "delims=_ tokens=2" %%a in ("!JAVA_VERSION_INFO!") do (
|
||||
|
@ -1333,4 +1503,4 @@ REM Safe echo which does not mess with () in strings
|
|||
set "eout=%1"
|
||||
set eout=%eout:"=%
|
||||
echo !eout!
|
||||
GOTO :eof
|
||||
GOTO :eof
|
|
@ -89,6 +89,8 @@ import org.apache.solr.common.SolrException;
|
|||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.Slice;
|
||||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.common.cloud.ZkConfigManager;
|
||||
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
|
@ -146,7 +148,7 @@ public class SolrCLI {
|
|||
// since this is a CLI, spare the user the stacktrace
|
||||
String excMsg = exc.getMessage();
|
||||
if (excMsg != null) {
|
||||
System.err.println("\nERROR: "+excMsg+"\n");
|
||||
System.err.println("\nERROR: " + excMsg + "\n");
|
||||
toolExitStatus = 1;
|
||||
} else {
|
||||
throw exc;
|
||||
|
@ -165,53 +167,13 @@ public class SolrCLI {
|
|||
HttpClientUtil.addRequestInterceptor((httpRequest, httpContext) -> {
|
||||
String pair = ss.get(0) + ":" + ss.get(1);
|
||||
byte[] encodedBytes = Base64.encodeBase64(pair.getBytes(UTF_8));
|
||||
httpRequest.addHeader(new BasicHeader("Authorization", "Basic "+ new String(encodedBytes, UTF_8)));
|
||||
httpRequest.addHeader(new BasicHeader("Authorization", "Basic " + new String(encodedBytes, UTF_8)));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract void runImpl(CommandLine cli) throws Exception;
|
||||
|
||||
// It's a little awkward putting this in ToolBase, but to re-use it in upconfig and create, _and_ have access
|
||||
// to the (possibly) redirected "stdout", it needs to go here unless we reorganize things a bit.
|
||||
protected void upconfig(CloudSolrClient cloudSolrClient, CommandLine cli, String confname, String configSet) throws IOException {
|
||||
|
||||
File configSetDir = null;
|
||||
// we try to be flexible and allow the user to specify a configuration directory instead of a configset name
|
||||
File possibleConfigDir = new File(configSet);
|
||||
if (possibleConfigDir.isDirectory()) {
|
||||
configSetDir = possibleConfigDir;
|
||||
} else {
|
||||
File configsetsDir = new File(cli.getOptionValue("configsetsDir"));
|
||||
if (!configsetsDir.isDirectory())
|
||||
throw new FileNotFoundException(configsetsDir.getAbsolutePath() + " not found!");
|
||||
|
||||
// upload the configset if it exists
|
||||
configSetDir = new File(configsetsDir, configSet);
|
||||
if (!configSetDir.isDirectory()) {
|
||||
throw new FileNotFoundException("Specified config " + configSet +
|
||||
" not found in " + configsetsDir.getAbsolutePath());
|
||||
}
|
||||
}
|
||||
|
||||
File confDir = new File(configSetDir, "conf");
|
||||
if (!confDir.isDirectory()) {
|
||||
// config dir should contain a conf sub-directory but if not and there's a solrconfig.xml, then use it
|
||||
if ((new File(configSetDir, "solrconfig.xml")).isFile()) {
|
||||
confDir = configSetDir;
|
||||
} else {
|
||||
throw new IllegalArgumentException("Specified configuration directory " + configSetDir.getAbsolutePath() +
|
||||
" is invalid;\nit should contain either conf sub-directory or solrconfig.xml");
|
||||
}
|
||||
}
|
||||
|
||||
//
|
||||
echo("Uploading " + confDir.getAbsolutePath() +
|
||||
" for config " + confname + " to ZooKeeper at " + cloudSolrClient.getZkHost());
|
||||
cloudSolrClient.uploadConfig(confDir.toPath(), confname);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helps build SolrCloud aware tools by initializing a CloudSolrClient
|
||||
* instance before running the tool.
|
||||
|
@ -236,6 +198,9 @@ public class SolrCLI {
|
|||
|
||||
cloudSolrClient.connect();
|
||||
runCloudTool(cloudSolrClient, cli);
|
||||
} catch (Exception e) {
|
||||
log.error("Could not complete mv operation for reason: " + e.getMessage());
|
||||
throw (e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -301,6 +266,7 @@ public class SolrCLI {
|
|||
HttpClientUtil.setHttpClientBuilder(builder);
|
||||
log.info("Set HttpClientConfigurer from: "+builderClassName);
|
||||
} catch (Exception ex) {
|
||||
log.error(ex.getMessage());
|
||||
throw new RuntimeException("Error during loading of configurer '"+builderClassName+"'.", ex);
|
||||
}
|
||||
}
|
||||
|
@ -391,6 +357,14 @@ public class SolrCLI {
|
|||
return new ConfigSetUploadTool();
|
||||
else if ("downconfig".equals(toolType))
|
||||
return new ConfigSetDownloadTool();
|
||||
else if ("rm".equals(toolType))
|
||||
return new ZkRmTool();
|
||||
else if ("mv".equals(toolType))
|
||||
return new ZkMvTool();
|
||||
else if ("cp".equals(toolType))
|
||||
return new ZkCpTool();
|
||||
else if ("ls".equals(toolType))
|
||||
return new ZkLsTool();
|
||||
|
||||
// If you add a built-in tool to this class, add it here to avoid
|
||||
// classpath scanning
|
||||
|
@ -417,6 +391,10 @@ public class SolrCLI {
|
|||
formatter.printHelp("run_example", getToolOptions(new RunExampleTool()));
|
||||
formatter.printHelp("upconfig", getToolOptions(new ConfigSetUploadTool()));
|
||||
formatter.printHelp("downconfig", getToolOptions(new ConfigSetDownloadTool()));
|
||||
formatter.printHelp("rm", getToolOptions(new ZkRmTool()));
|
||||
formatter.printHelp("cp", getToolOptions(new ZkCpTool()));
|
||||
formatter.printHelp("mv", getToolOptions(new ZkMvTool()));
|
||||
formatter.printHelp("ls", getToolOptions(new ZkLsTool()));
|
||||
|
||||
List<Class<Tool>> toolClasses = findToolClassesInPackage("org.apache.solr.util");
|
||||
for (Class<Tool> next : toolClasses) {
|
||||
|
@ -1461,7 +1439,12 @@ public class SolrCLI {
|
|||
} else if (configExistsInZk) {
|
||||
echo("Re-using existing configuration directory "+confname);
|
||||
} else {
|
||||
upconfig(cloudSolrClient, cli, confname, cli.getOptionValue("confdir", DEFAULT_CONFIG_SET));
|
||||
Path confPath = ZkConfigManager.getConfigsetPath(confname, cli.getOptionValue("confdir", DEFAULT_CONFIG_SET),
|
||||
cli.getOptionValue("configsetsDir"));
|
||||
|
||||
echo("Uploading " + confPath.toAbsolutePath().toString() +
|
||||
" for config " + confname + " to ZooKeeper at " + cloudSolrClient.getZkHost());
|
||||
cloudSolrClient.uploadConfig(confPath, confname);
|
||||
}
|
||||
|
||||
// since creating a collection is a heavy-weight operation, check for existence first
|
||||
|
@ -1726,11 +1709,20 @@ public class SolrCLI {
|
|||
" is running in standalone server mode, upconfig can only be used when running in SolrCloud mode.\n");
|
||||
}
|
||||
|
||||
try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder().withZkHost(zkHost).build()) {
|
||||
String confName = cli.getOptionValue("confname");
|
||||
try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
|
||||
echo("\nConnecting to ZooKeeper at " + zkHost + " ...");
|
||||
cloudSolrClient.connect();
|
||||
upconfig(cloudSolrClient, cli, cli.getOptionValue("confname"), cli.getOptionValue("confdir"));
|
||||
Path confPath = ZkConfigManager.getConfigsetPath(confName, cli.getOptionValue("confdir"), cli.getOptionValue("configsetsDir"));
|
||||
|
||||
echo("Uploading " + confPath.toAbsolutePath().toString() +
|
||||
" for config " + cli.getOptionValue("confname") + " to ZooKeeper at " + zkHost);
|
||||
|
||||
zkClient.upConfig(confPath, confName);
|
||||
} catch (Exception e) {
|
||||
log.error("Could not complete upconfig operation for reason: " + e.getMessage());
|
||||
throw (e);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1781,32 +1773,330 @@ public class SolrCLI {
|
|||
}
|
||||
|
||||
|
||||
try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder().withZkHost(zkHost).build()) {
|
||||
try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
|
||||
echo("\nConnecting to ZooKeeper at " + zkHost + " ...");
|
||||
cloudSolrClient.connect();
|
||||
downconfig(cloudSolrClient, cli.getOptionValue("confname"), cli.getOptionValue("confdir"));
|
||||
}
|
||||
}
|
||||
String confName = cli.getOptionValue("confname");
|
||||
String confDir = cli.getOptionValue("confdir");
|
||||
Path configSetPath = Paths.get(confDir);
|
||||
// we try to be nice about having the "conf" in the directory, and we create it if it's not there.
|
||||
if (configSetPath.endsWith("/conf") == false) {
|
||||
configSetPath = Paths.get(configSetPath.toString(), "conf");
|
||||
}
|
||||
if (Files.exists(configSetPath) == false) {
|
||||
Files.createDirectories(configSetPath);
|
||||
}
|
||||
echo("Downloading configset " + confName + " from ZooKeeper at " + zkHost +
|
||||
" to directory " + configSetPath.toAbsolutePath());
|
||||
|
||||
protected void downconfig(CloudSolrClient cloudSolrClient, String confname, String confdir) throws IOException {
|
||||
|
||||
Path configSetPath = Paths.get(confdir);
|
||||
// we try to be nice about having the "conf" in the directory, and we create it if it's not there.
|
||||
if (configSetPath.endsWith("/conf") == false) {
|
||||
configSetPath = Paths.get(configSetPath.toString(), "conf");
|
||||
}
|
||||
if (Files.exists(configSetPath) == false) {
|
||||
Files.createDirectories(configSetPath);
|
||||
zkClient.downConfig(confName, configSetPath);
|
||||
} catch (Exception e) {
|
||||
log.error("Could not complete downconfig operation for reason: " + e.getMessage());
|
||||
throw (e);
|
||||
}
|
||||
|
||||
// Try to download the configset
|
||||
echo("Downloading configset " + confname + " from ZooKeeper at " + cloudSolrClient.getZkHost() +
|
||||
" to directory " + configSetPath.toAbsolutePath());
|
||||
cloudSolrClient.downloadConfig(confname, configSetPath);
|
||||
}
|
||||
|
||||
} // End ConfigSetDownloadTool class
|
||||
|
||||
public static class ZkRmTool extends ToolBase {
|
||||
|
||||
public ZkRmTool() {
|
||||
this(System.out);
|
||||
}
|
||||
|
||||
public ZkRmTool(PrintStream stdout) {
|
||||
super(stdout);
|
||||
}
|
||||
|
||||
@SuppressWarnings("static-access")
|
||||
public Option[] getOptions() {
|
||||
return new Option[]{
|
||||
OptionBuilder
|
||||
.withArgName("path")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Path to remove")
|
||||
.create("path"),
|
||||
OptionBuilder
|
||||
.withArgName("recurse")
|
||||
.hasArg()
|
||||
.isRequired(false)
|
||||
.withDescription("Recurse (true|false, default is false)")
|
||||
.create("recurse"),
|
||||
OptionBuilder
|
||||
.withArgName("HOST")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Address of the Zookeeper ensemble; defaults to: " + ZK_HOST)
|
||||
.create("zkHost")
|
||||
};
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return "rm";
|
||||
}
|
||||
|
||||
protected void runImpl(CommandLine cli) throws Exception {
|
||||
|
||||
String zkHost = getZkHost(cli);
|
||||
|
||||
if (zkHost == null) {
|
||||
throw new IllegalStateException("Solr at " + cli.getOptionValue("zkHost") +
|
||||
" is running in standalone server mode, 'zk rm' can only be used when running in SolrCloud mode.\n");
|
||||
}
|
||||
String target = cli.getOptionValue("path");
|
||||
Boolean recurse = Boolean.parseBoolean(cli.getOptionValue("recurse"));
|
||||
|
||||
String znode = target;
|
||||
if (target.toLowerCase(Locale.ROOT).startsWith("zk:")) {
|
||||
znode = target.substring(3);
|
||||
}
|
||||
if (znode.equals("/")) {
|
||||
throw new SolrServerException("You may not remove the root ZK node ('/')!");
|
||||
}
|
||||
echo("\nConnecting to ZooKeeper at " + zkHost + " ...");
|
||||
try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
|
||||
if (recurse == false && zkClient.getChildren(znode, null, true).size() != 0) {
|
||||
throw new SolrServerException("Zookeeper node " + znode + " has children and recurse has NOT been specified");
|
||||
}
|
||||
echo("Removing Zookeeper node " + znode + " from ZooKeeper at " + zkHost +
|
||||
" recurse: " + Boolean.toString(recurse));
|
||||
zkClient.clean(znode);
|
||||
} catch (Exception e) {
|
||||
log.error("Could not complete rm operation for reason: " + e.getMessage());
|
||||
throw (e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
} // End RmTool class
|
||||
|
||||
public static class ZkLsTool extends ToolBase {
|
||||
|
||||
public ZkLsTool() {
|
||||
this(System.out);
|
||||
}
|
||||
|
||||
public ZkLsTool(PrintStream stdout) {
|
||||
super(stdout);
|
||||
}
|
||||
|
||||
@SuppressWarnings("static-access")
|
||||
public Option[] getOptions() {
|
||||
return new Option[]{
|
||||
OptionBuilder
|
||||
.withArgName("path")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Path to list")
|
||||
.create("path"),
|
||||
OptionBuilder
|
||||
.withArgName("recurse")
|
||||
.hasArg()
|
||||
.isRequired(false)
|
||||
.withDescription("Recurse (true|false, default is false)")
|
||||
.create("recurse"),
|
||||
OptionBuilder
|
||||
.withArgName("HOST")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Address of the Zookeeper ensemble; defaults to: " + ZK_HOST)
|
||||
.create("zkHost")
|
||||
};
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return "ls";
|
||||
}
|
||||
|
||||
protected void runImpl(CommandLine cli) throws Exception {
|
||||
|
||||
String zkHost = getZkHost(cli);
|
||||
|
||||
if (zkHost == null) {
|
||||
throw new IllegalStateException("Solr at " + cli.getOptionValue("zkHost") +
|
||||
" is running in standalone server mode, 'zk rm' can only be used when running in SolrCloud mode.\n");
|
||||
}
|
||||
|
||||
|
||||
try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
|
||||
echo("\nConnecting to ZooKeeper at " + zkHost + " ...");
|
||||
|
||||
String znode = cli.getOptionValue("path");
|
||||
Boolean recurse = Boolean.parseBoolean(cli.getOptionValue("recurse"));
|
||||
echo("Getting listing for Zookeeper node " + znode + " from ZooKeeper at " + zkHost +
|
||||
" recurse: " + Boolean.toString(recurse));
|
||||
stdout.print(zkClient.listZnode(znode, recurse));
|
||||
} catch (Exception e) {
|
||||
log.error("Could not complete rm operation for reason: " + e.getMessage());
|
||||
throw (e);
|
||||
}
|
||||
}
|
||||
} // End zkLsTool class
|
||||
|
||||
public static class ZkCpTool extends ToolBase {
|
||||
|
||||
public ZkCpTool() {
|
||||
this(System.out);
|
||||
}
|
||||
|
||||
public ZkCpTool(PrintStream stdout) {
|
||||
super(stdout);
|
||||
}
|
||||
|
||||
@SuppressWarnings("static-access")
|
||||
public Option[] getOptions() {
|
||||
return new Option[]{
|
||||
OptionBuilder
|
||||
.withArgName("src")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Source file or directory, may be local or a Znode")
|
||||
.create("src"),
|
||||
OptionBuilder
|
||||
.withArgName("dst")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Destination of copy, may be local or a Znode.")
|
||||
.create("dst"),
|
||||
OptionBuilder
|
||||
.withArgName("recurse")
|
||||
.hasArg()
|
||||
.isRequired(false)
|
||||
.withDescription("Recurse (true|false, default is false)")
|
||||
.create("recurse"),
|
||||
OptionBuilder
|
||||
.withArgName("HOST")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Address of the Zookeeper ensemble; defaults to: " + ZK_HOST)
|
||||
.create("zkHost")
|
||||
};
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return "cp";
|
||||
}
|
||||
|
||||
protected void runImpl(CommandLine cli) throws Exception {
|
||||
|
||||
String zkHost = getZkHost(cli);
|
||||
if (zkHost == null) {
|
||||
throw new IllegalStateException("Solr at " + cli.getOptionValue("solrUrl") +
|
||||
" is running in standalone server mode, cp can only be used when running in SolrCloud mode.\n");
|
||||
}
|
||||
|
||||
try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
|
||||
echo("\nConnecting to ZooKeeper at " + zkHost + " ...");
|
||||
String src = cli.getOptionValue("src");
|
||||
String dst = cli.getOptionValue("dst");
|
||||
Boolean recurse = Boolean.parseBoolean(cli.getOptionValue("recurse"));
|
||||
echo("Copying from '" + src + "' to '" + dst + "'. ZooKeeper at " + zkHost);
|
||||
|
||||
boolean srcIsZk = src.toLowerCase(Locale.ROOT).startsWith("zk:");
|
||||
boolean dstIsZk = dst.toLowerCase(Locale.ROOT).startsWith("zk:");
|
||||
|
||||
String srcName = src;
|
||||
if (srcIsZk) {
|
||||
srcName = src.substring(3);
|
||||
} else if (srcName.toLowerCase(Locale.ROOT).startsWith("file:")) {
|
||||
srcName = srcName.substring(5);
|
||||
}
|
||||
|
||||
String dstName = dst;
|
||||
if (dstIsZk) {
|
||||
dstName = dst.substring(3);
|
||||
} else {
|
||||
if (dstName.toLowerCase(Locale.ROOT).startsWith("file:")) {
|
||||
dstName = dstName.substring(5);
|
||||
}
|
||||
}
|
||||
zkClient.zkTransfer(srcName, srcIsZk, dstName, dstIsZk, recurse);
|
||||
} catch (Exception e) {
|
||||
log.error("Could not complete the zk operation for reason: " + e.getMessage());
|
||||
throw (e);
|
||||
}
|
||||
}
|
||||
} // End CpTool class
|
||||
|
||||
|
||||
public static class ZkMvTool extends ToolBase {
|
||||
|
||||
public ZkMvTool() {
|
||||
this(System.out);
|
||||
}
|
||||
|
||||
public ZkMvTool(PrintStream stdout) {
|
||||
super(stdout);
|
||||
}
|
||||
|
||||
@SuppressWarnings("static-access")
|
||||
public Option[] getOptions() {
|
||||
return new Option[]{
|
||||
OptionBuilder
|
||||
.withArgName("src")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Source Znode to movej from.")
|
||||
.create("src"),
|
||||
OptionBuilder
|
||||
.withArgName("dst")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Destination Znode to move to.")
|
||||
.create("dst"),
|
||||
OptionBuilder
|
||||
.withArgName("HOST")
|
||||
.hasArg()
|
||||
.isRequired(true)
|
||||
.withDescription("Address of the Zookeeper ensemble; defaults to: " + ZK_HOST)
|
||||
.create("zkHost")
|
||||
};
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return "mv";
|
||||
}
|
||||
|
||||
protected void runImpl(CommandLine cli) throws Exception {
|
||||
|
||||
String zkHost = getZkHost(cli);
|
||||
if (zkHost == null) {
|
||||
throw new IllegalStateException("Solr at " + cli.getOptionValue("solrUrl") +
|
||||
" is running in standalone server mode, downconfig can only be used when running in SolrCloud mode.\n");
|
||||
}
|
||||
|
||||
|
||||
try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
|
||||
echo("\nConnecting to ZooKeeper at " + zkHost + " ...");
|
||||
String src = cli.getOptionValue("src");
|
||||
String dst = cli.getOptionValue("dst");
|
||||
|
||||
if (src.toLowerCase(Locale.ROOT).startsWith("file:") || dst.toLowerCase(Locale.ROOT).startsWith("file:")) {
|
||||
throw new SolrServerException("mv command operates on znodes and 'file:' has been specified.");
|
||||
}
|
||||
String source = src;
|
||||
if (src.toLowerCase(Locale.ROOT).startsWith("zk")) {
|
||||
source = src.substring(3);
|
||||
}
|
||||
|
||||
String dest = dst;
|
||||
if (dst.toLowerCase(Locale.ROOT).startsWith("zk")) {
|
||||
dest = dst.substring(3);
|
||||
}
|
||||
|
||||
echo("Moving Znode " + source + " to " + dest + " on ZooKeeper at " + zkHost);
|
||||
zkClient.moveZnode(source, dest);
|
||||
} catch (Exception e) {
|
||||
log.error("Could not complete mv operation for reason: " + e.getMessage());
|
||||
throw (e);
|
||||
}
|
||||
|
||||
}
|
||||
} // End MvTool class
|
||||
|
||||
|
||||
|
||||
public static class DeleteTool extends ToolBase {
|
||||
|
||||
public DeleteTool() { this(System.out); }
|
||||
|
|
|
@ -0,0 +1,28 @@
|
|||
<?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">
|
||||
<fieldType name="string" class="solr.StrField"/>
|
||||
<fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
|
||||
<fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
|
||||
<dynamicField name="*" type="string" indexed="true" stored="true"/>
|
||||
<!-- for versioning -->
|
||||
<field name="_version_" type="long" indexed="true" stored="true"/>
|
||||
<field name="_root_" type="int" indexed="true" stored="true" multiValued="false" required="false"/>
|
||||
<field name="id" type="string" indexed="true" stored="true"/>
|
||||
<uniqueKey>id</uniqueKey>
|
||||
</schema>
|
|
@ -0,0 +1,48 @@
|
|||
<?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.
|
||||
-->
|
||||
|
||||
<!-- Minimal solrconfig.xml with /select, /admin and /update only -->
|
||||
|
||||
<config>
|
||||
|
||||
<dataDir>${solr.data.dir:}</dataDir>
|
||||
|
||||
<directoryFactory name="DirectoryFactory"
|
||||
class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
|
||||
<schemaFactory class="ClassicIndexSchemaFactory"/>
|
||||
|
||||
<luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
|
||||
|
||||
<updateHandler class="solr.DirectUpdateHandler2">
|
||||
<commitWithin>
|
||||
<softCommit>${solr.commitwithin.softcommit:true}</softCommit>
|
||||
</commitWithin>
|
||||
<updateLog></updateLog>
|
||||
</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>
|
||||
</config>
|
||||
|
|
@ -0,0 +1,62 @@
|
|||
# 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.
|
||||
|
||||
#-----------------------------------------------------------------------
|
||||
# a couple of test stopwords to test that the words are really being
|
||||
# configured from this file:
|
||||
# So far, this is here only to be able to test the configset upload
|
||||
# and download and local->ZK recursive upload and download process
|
||||
#
|
||||
|
||||
stopworda
|
||||
stopwordb
|
||||
|
||||
#Standard english stop words taken from Lucene's StopAnalyzer
|
||||
a
|
||||
an
|
||||
and
|
||||
are
|
||||
as
|
||||
at
|
||||
be
|
||||
but
|
||||
by
|
||||
for
|
||||
if
|
||||
in
|
||||
into
|
||||
is
|
||||
it
|
||||
no
|
||||
not
|
||||
of
|
||||
on
|
||||
or
|
||||
s
|
||||
such
|
||||
t
|
||||
that
|
||||
the
|
||||
their
|
||||
then
|
||||
there
|
||||
these
|
||||
they
|
||||
this
|
||||
to
|
||||
was
|
||||
will
|
||||
with
|
||||
|
|
@ -0,0 +1,645 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.FileVisitResult;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.nio.file.SimpleFileVisitor;
|
||||
import java.nio.file.attribute.BasicFileAttributes;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.util.SolrCLI;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
|
||||
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
configureCluster(1)
|
||||
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
|
||||
.configure();
|
||||
zkAddr = cluster.getZkServer().getZkAddress();
|
||||
zkClient = new SolrZkClient(zkAddr, 30000);
|
||||
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void closeConn() {
|
||||
zkClient.close();
|
||||
}
|
||||
|
||||
private static String zkAddr;
|
||||
private static SolrZkClient zkClient;
|
||||
|
||||
@Test
|
||||
public void testUpconfig() throws Exception {
|
||||
// Use a full, explicit path for configset.
|
||||
Path src = TEST_PATH().resolve("configsets").resolve("cloud-subdirs").resolve("conf");
|
||||
Path configSet = TEST_PATH().resolve("configsets").resolve("cloud-subdirs");
|
||||
copyConfigUp(src, configSet, "upconfig1");
|
||||
// Now do we have that config up on ZK?
|
||||
verifyZkLocalPathsMatch(src, "/configs/upconfig1");
|
||||
|
||||
// Now just use a name in the configsets directory, do we find it?
|
||||
configSet = TEST_PATH().resolve("configsets");
|
||||
|
||||
String[] args = new String[]{
|
||||
"-confname", "upconfig2",
|
||||
"-confdir", "cloud-subdirs",
|
||||
"-zkHost", zkAddr,
|
||||
"-configsetsDir", configSet.toAbsolutePath().toString(),
|
||||
};
|
||||
|
||||
SolrCLI.ConfigSetUploadTool tool = new SolrCLI.ConfigSetUploadTool();
|
||||
|
||||
int res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
assertEquals("tool should have returned 0 for success ", 0, res);
|
||||
// Now do we have that config up on ZK?
|
||||
verifyZkLocalPathsMatch(src, "/configs/upconfig2");
|
||||
|
||||
// do we barf on a bogus path?
|
||||
args = new String[]{
|
||||
"-confname", "upconfig3",
|
||||
"-confdir", "nothinghere",
|
||||
"-zkHost", zkAddr,
|
||||
"-configsetsDir", configSet.toAbsolutePath().toString(),
|
||||
};
|
||||
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
assertTrue("tool should have returned non-zero for failure ", 0 != res);
|
||||
|
||||
String content = new String(zkClient.getData("/configs/upconfig2/schema.xml", null, null, true), StandardCharsets.UTF_8);
|
||||
assertTrue("There should be content in the node! ", content.contains("Apache Software Foundation"));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDownconfig() throws Exception {
|
||||
Path tmp = createTempDir("downConfigNewPlace");
|
||||
|
||||
// First we need a configset on ZK to bring down.
|
||||
Path src = TEST_PATH().resolve("configsets").resolve("cloud-subdirs").resolve("conf");
|
||||
Path configSet = TEST_PATH().resolve("configsets").resolve("cloud-subdirs");
|
||||
copyConfigUp(src, configSet, "downconfig1");
|
||||
// Now do we have that config up on ZK?
|
||||
verifyZkLocalPathsMatch(src, "/configs/downconfig1");
|
||||
|
||||
String[] args = new String[]{
|
||||
"-confname", "downconfig1",
|
||||
"-confdir", tmp.toAbsolutePath().toString(),
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
SolrCLI.ConfigSetDownloadTool downTool = new SolrCLI.ConfigSetDownloadTool();
|
||||
int res = downTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(downTool.getOptions()), args));
|
||||
assertEquals("Download should have succeeded.", 0, res);
|
||||
verifyZkLocalPathsMatch(Paths.get(tmp.toAbsolutePath().toString(), "conf"), "/configs/downconfig1");
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCp() throws Exception {
|
||||
// First get something up on ZK
|
||||
Path src = TEST_PATH().resolve("configsets").resolve("cloud-subdirs").resolve("conf");
|
||||
Path configSet = TEST_PATH().resolve("configsets").resolve("cloud-subdirs");
|
||||
|
||||
copyConfigUp(src, configSet, "cp1");
|
||||
|
||||
// Now copy it somewhere else on ZK.
|
||||
String[] args = new String[]{
|
||||
"-src", "zk:/configs/cp1",
|
||||
"-dst", "zk:/cp2",
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
SolrCLI.ZkCpTool cpTool = new SolrCLI.ZkCpTool();
|
||||
|
||||
int res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy from zk -> zk should have succeeded.", 0, res);
|
||||
verifyZnodesMatch("/configs/cp1", "/cp2");
|
||||
|
||||
|
||||
// try with zk->local
|
||||
Path tmp = createTempDir("tmpNewPlace2");
|
||||
args = new String[]{
|
||||
"-src", "zk:/configs/cp1",
|
||||
"-dst", "file:/" + tmp.toAbsolutePath().toString(),
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy should have succeeded.", 0, res);
|
||||
verifyZkLocalPathsMatch(tmp, "/configs/cp1");
|
||||
|
||||
|
||||
// try with zk->local no file: prefix
|
||||
tmp = createTempDir("tmpNewPlace3");
|
||||
args = new String[]{
|
||||
"-src", "zk:/configs/cp1",
|
||||
"-dst", tmp.toAbsolutePath().toString(),
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy should have succeeded.", 0, res);
|
||||
verifyZkLocalPathsMatch(tmp, "/configs/cp1");
|
||||
|
||||
|
||||
// try with local->zk
|
||||
args = new String[]{
|
||||
"-src", src.toAbsolutePath().toString(),
|
||||
"-dst", "zk:/cp3",
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy should have succeeded.", 0, res);
|
||||
verifyZkLocalPathsMatch(src, "/cp3");
|
||||
|
||||
// try with local->zk, file: specified
|
||||
args = new String[]{
|
||||
"-src", "file:" + src.toAbsolutePath().toString(),
|
||||
"-dst", "zk:/cp4",
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy should have succeeded.", 0, res);
|
||||
verifyZkLocalPathsMatch(src, "/cp4");
|
||||
|
||||
|
||||
// try with recurse not specified
|
||||
args = new String[]{
|
||||
"-src", "file:" + src.toAbsolutePath().toString(),
|
||||
"-dst", "zk:/cp5Fail",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertTrue("Copy should NOT have succeeded, recurse not specified.", 0 != res);
|
||||
|
||||
// try with recurse = false
|
||||
args = new String[]{
|
||||
"-src", "file:" + src.toAbsolutePath().toString(),
|
||||
"-dst", "zk:/cp6Fail",
|
||||
"-recurse", "false",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertTrue("Copy should NOT have succeeded, recurse set to false.", 0 != res);
|
||||
|
||||
|
||||
// NOTE: really can't test copying to '.' because the test framework doesn't allow altering the source tree
|
||||
// and at least IntelliJ's CWD is in the source tree.
|
||||
|
||||
// copy to local ending in '/'
|
||||
//src and cp3 and cp4 are valid
|
||||
String localSlash = tmp.normalize() + "/cpToLocal/";
|
||||
args = new String[]{
|
||||
"-src", "zk:/cp3/schema.xml",
|
||||
"-dst", localSlash,
|
||||
"-recurse", "false",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy should nave created intermediate directory locally.", 0, res);
|
||||
assertTrue("File should have been copied to a directory successfully", Files.exists(Paths.get(localSlash, "schema.xml")));
|
||||
|
||||
// copy to ZK ending in '/'.
|
||||
//src and cp3 are valid
|
||||
args = new String[]{
|
||||
"-src", "file:" + src.normalize().toAbsolutePath().toString() + "/solrconfig.xml",
|
||||
"-dst", "zk:/powerup/",
|
||||
"-recurse", "false",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy up to intermediate file should have succeeded.", 0, res);
|
||||
assertTrue("Should have created an intermediate node on ZK", zkClient.exists("/powerup/solrconfig.xml", true));
|
||||
|
||||
// copy individual file up
|
||||
//src and cp3 are valid
|
||||
args = new String[]{
|
||||
"-src", "file:" + src.normalize().toAbsolutePath().toString() + "/solrconfig.xml",
|
||||
"-dst", "zk:/copyUpFile.xml",
|
||||
"-recurse", "false",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy up to named file should have succeeded.", 0, res);
|
||||
assertTrue("Should NOT have created an intermediate node on ZK", zkClient.exists("/copyUpFile.xml", true));
|
||||
|
||||
// copy individual file down
|
||||
//src and cp3 are valid
|
||||
|
||||
String localNamed = tmp.normalize().toString() + "/localnamed/renamed.txt";
|
||||
args = new String[]{
|
||||
"-src", "zk:/cp4/solrconfig.xml",
|
||||
"-dst", "file:" + localNamed,
|
||||
"-recurse", "false",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy to local named file should have succeeded.", 0, res);
|
||||
Path locPath = Paths.get(localNamed);
|
||||
assertTrue("Should have found file: " + localNamed, Files.exists(locPath));
|
||||
assertTrue("Should be an individual file", Files.isRegularFile(locPath));
|
||||
assertTrue("File should have some data", Files.size(locPath) > 100);
|
||||
boolean foundApache = false;
|
||||
for (String line : Files.readAllLines(locPath, Charset.forName("UTF-8"))) {
|
||||
if (line.contains("Apache Software Foundation")) {
|
||||
foundApache = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue("Should have found Apache Software Foundation in the file! ", foundApache);
|
||||
|
||||
|
||||
// Test copy from somwehere in ZK to the root of ZK.
|
||||
args = new String[]{
|
||||
"-src", "zk:/cp4/solrconfig.xml",
|
||||
"-dst", "zk:/",
|
||||
"-recurse", "false",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args));
|
||||
assertEquals("Copy from somewhere in ZK to ZK root should have succeeded.", 0, res);
|
||||
assertTrue("Should have found znode /solrconfig.xml: ", zkClient.exists("/solrconfig.xml", true));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMv() throws Exception {
|
||||
|
||||
// First get something up on ZK
|
||||
Path src = TEST_PATH().resolve("configsets").resolve("cloud-subdirs").resolve("conf");
|
||||
Path configSet = TEST_PATH().resolve("configsets").resolve("cloud-subdirs");
|
||||
|
||||
copyConfigUp(src, configSet, "mv1");
|
||||
|
||||
// Now move it somewhere else.
|
||||
String[] args = new String[]{
|
||||
"-src", "zk:/configs/mv1",
|
||||
"-dst", "zk:/mv2",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
SolrCLI.ZkMvTool mvTool = new SolrCLI.ZkMvTool();
|
||||
|
||||
int res = mvTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(mvTool.getOptions()), args));
|
||||
assertEquals("Move should have succeeded.", 0, res);
|
||||
|
||||
// Now does the moved directory match the original on disk?
|
||||
verifyZkLocalPathsMatch(src, "/mv2");
|
||||
// And are we sure the old path is gone?
|
||||
assertFalse("/configs/mv1 Znode should not be there: ", zkClient.exists("/configs/mv1", true));
|
||||
|
||||
// Files are in mv2
|
||||
// Now fail if we specify "file:". Everything should still be in /mv2
|
||||
args = new String[]{
|
||||
"-src", "file:/mv2",
|
||||
"-dst", "/mv3",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
// Still in mv2
|
||||
res = mvTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(mvTool.getOptions()), args));
|
||||
assertTrue("Move should NOT have succeeded with file: specified.", 0 != res);
|
||||
|
||||
// Let's move it to yet another place with no zk: prefix.
|
||||
args = new String[]{
|
||||
"-src", "/mv2",
|
||||
"-dst", "/mv4",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = mvTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(mvTool.getOptions()), args));
|
||||
assertEquals("Move should have succeeded.", 0, res);
|
||||
|
||||
assertFalse("Znode /mv3 really should be gone", zkClient.exists("/mv3", true));
|
||||
|
||||
// Now does the moved directory match the original on disk?
|
||||
verifyZkLocalPathsMatch(src, "/mv4");
|
||||
|
||||
args = new String[]{
|
||||
"-src", "/mv4/solrconfig.xml",
|
||||
"-dst", "/testmvsingle/solrconfig.xml",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = mvTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(mvTool.getOptions()), args));
|
||||
assertEquals("Move should have succeeded.", 0, res);
|
||||
assertTrue("Should be able to move a single file", zkClient.exists("/testmvsingle/solrconfig.xml", true));
|
||||
|
||||
zkClient.makePath("/parentNode", true);
|
||||
|
||||
// what happens if the destination ends with a slash?
|
||||
args = new String[]{
|
||||
"-src", "/mv4/schema.xml",
|
||||
"-dst", "/parentnode/",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = mvTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(mvTool.getOptions()), args));
|
||||
assertEquals("Move should have succeeded.", 0, res);
|
||||
assertTrue("Should be able to move a single file to a parent znode", zkClient.exists("/parentnode/schema.xml", true));
|
||||
String content = new String(zkClient.getData("/parentnode/schema.xml", null, null, true), StandardCharsets.UTF_8);
|
||||
assertTrue("There should be content in the node! ", content.contains("Apache Software Foundation"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLs() throws Exception {
|
||||
|
||||
Path src = TEST_PATH().resolve("configsets").resolve("cloud-subdirs").resolve("conf");
|
||||
Path configSet = TEST_PATH().resolve("configsets").resolve("cloud-subdirs");
|
||||
|
||||
copyConfigUp(src, configSet, "lister");
|
||||
|
||||
// Should only find a single level.
|
||||
String[] args = new String[]{
|
||||
"-path", "/configs",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
PrintStream ps = new PrintStream(baos, false, StandardCharsets.UTF_8.name());
|
||||
SolrCLI.ZkLsTool tool = new SolrCLI.ZkLsTool(ps);
|
||||
|
||||
|
||||
int res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
String content = new String(baos.toByteArray(), StandardCharsets.UTF_8);
|
||||
|
||||
assertEquals("List should have succeeded", res, 0);
|
||||
assertTrue("Return should contain the conf directory", content.contains("lister"));
|
||||
assertFalse("Return should NOT contain a child node", content.contains("solrconfig.xml"));
|
||||
|
||||
|
||||
// simple ls recurse=false
|
||||
args = new String[]{
|
||||
"-path", "/configs",
|
||||
"-recurse", "false",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
content = new String(baos.toByteArray(), StandardCharsets.UTF_8);
|
||||
|
||||
assertEquals("List should have succeeded", res, 0);
|
||||
assertTrue("Return should contain the conf directory", content.contains("lister"));
|
||||
assertFalse("Return should NOT contain a child node", content.contains("solrconfig.xml"));
|
||||
|
||||
// recurse=true
|
||||
args = new String[]{
|
||||
"-path", "/configs",
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
content = new String(baos.toByteArray(), StandardCharsets.UTF_8);
|
||||
|
||||
assertEquals("List should have succeeded", res, 0);
|
||||
assertTrue("Return should contain the conf directory", content.contains("lister"));
|
||||
assertTrue("Return should contain a child node", content.contains("solrconfig.xml"));
|
||||
|
||||
// Saw a case where going from root foo'd, so test it.
|
||||
args = new String[]{
|
||||
"-path", "/",
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
content = new String(baos.toByteArray(), StandardCharsets.UTF_8);
|
||||
|
||||
assertEquals("List should have succeeded", res, 0);
|
||||
assertTrue("Return should contain the conf directory", content.contains("lister"));
|
||||
assertTrue("Return should contain a child node", content.contains("solrconfig.xml"));
|
||||
|
||||
args = new String[]{
|
||||
"-path", "/",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
content = new String(baos.toByteArray(), StandardCharsets.UTF_8);
|
||||
assertEquals("List should have succeeded", res, 0);
|
||||
assertFalse("Return should not contain /zookeeper", content.contains("/zookeeper"));
|
||||
|
||||
// Saw a case where ending in slash foo'd, so test it.
|
||||
args = new String[]{
|
||||
"-path", "/configs/",
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
content = new String(baos.toByteArray(), StandardCharsets.UTF_8);
|
||||
|
||||
assertEquals("List should have succeeded", res, 0);
|
||||
assertTrue("Return should contain the conf directory", content.contains("lister"));
|
||||
assertTrue("Return should contain a child node", content.contains("solrconfig.xml"));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRm() throws Exception {
|
||||
Path src = TEST_PATH().resolve("configsets").resolve("cloud-subdirs").resolve("conf");
|
||||
Path configSet = TEST_PATH().resolve("configsets").resolve("cloud-subdirs");
|
||||
|
||||
copyConfigUp(src, configSet, "rm1");
|
||||
copyConfigUp(src, configSet, "rm2");
|
||||
|
||||
// Should fail if recurse not set.
|
||||
String[] args = new String[]{
|
||||
"-path", "/configs/rm1",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
SolrCLI.ZkRmTool tool = new SolrCLI.ZkRmTool();
|
||||
|
||||
int res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
|
||||
assertTrue("Should have failed to remove node with children unless -recurse is set to true", res != 0);
|
||||
|
||||
// Are we sure all the znodes are still there?
|
||||
verifyZkLocalPathsMatch(src, "/configs/rm1");
|
||||
|
||||
args = new String[]{
|
||||
"-path", "zk:/configs/rm1",
|
||||
"-recurse", "false",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
|
||||
assertTrue("Should have failed to remove node with children if -recurse is set to false", res != 0);
|
||||
|
||||
args = new String[]{
|
||||
"-path", "/configs/rm1",
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
assertEquals("Should have removed node /configs/rm1", res, 0);
|
||||
assertFalse("Znode /configs/toremove really should be gone", zkClient.exists("/configs/rm1", true));
|
||||
|
||||
// Check that zk prefix also works.
|
||||
args = new String[]{
|
||||
"-path", "zk:/configs/rm2",
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
assertEquals("Should have removed node /configs/rm2", res, 0);
|
||||
assertFalse("Znode /configs/toremove2 really should be gone", zkClient.exists("/configs/rm2", true));
|
||||
|
||||
// This should silently just refuse to do anything to the / or /zookeeper
|
||||
args = new String[]{
|
||||
"-path", "zk:/",
|
||||
"-recurse", "true",
|
||||
"-zkHost", zkAddr,
|
||||
};
|
||||
|
||||
copyConfigUp(src, configSet, "rm3");
|
||||
res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
assertFalse("Should fail when trying to remove /.", res == 0);
|
||||
}
|
||||
|
||||
private void getAllKids(String zkRoot, Set<String> paths) throws KeeperException, InterruptedException {
|
||||
for (String node : zkClient.getChildren(zkRoot, null, true)) {
|
||||
paths.add(node);
|
||||
getAllKids(zkRoot + "/" + node, paths);
|
||||
}
|
||||
}
|
||||
|
||||
// We can use this for testing since the goal is to move "some stuff" up to ZK.
|
||||
// The fact that they're in configsets is irrelevant.
|
||||
private void copyConfigUp(Path src, Path configSet, String confName) throws Exception {
|
||||
String[] args = new String[]{
|
||||
"-confname", confName,
|
||||
"-confdir", src.toAbsolutePath().toString(),
|
||||
"-zkHost", zkAddr,
|
||||
"-configsetsDir", configSet.toAbsolutePath().toString(),
|
||||
};
|
||||
|
||||
SolrCLI.ConfigSetUploadTool tool = new SolrCLI.ConfigSetUploadTool();
|
||||
|
||||
int res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
|
||||
assertEquals("Tool should have returned 0 for success, returned: " + res, res, 0);
|
||||
|
||||
}
|
||||
|
||||
// Check that all children of fileRoot are children of zkRoot and vice-versa
|
||||
private void verifyZkLocalPathsMatch(Path fileRoot, String zkRoot) throws IOException, KeeperException, InterruptedException {
|
||||
verifyAllFilesAreZNodes(fileRoot, zkRoot);
|
||||
verifyAllZNodesAreFiles(fileRoot, zkRoot);
|
||||
}
|
||||
|
||||
void verifyAllZNodesAreFiles(Path fileRoot, String zkRoot) throws KeeperException, InterruptedException {
|
||||
|
||||
for (String node : zkClient.getChildren(zkRoot, null, true)) {
|
||||
Path thisPath = Paths.get(fileRoot.toAbsolutePath().toString(), node);
|
||||
assertTrue("Znode " + node + " should have been found on disk at " + fileRoot.toAbsolutePath().toString(),
|
||||
Files.exists(thisPath));
|
||||
verifyAllZNodesAreFiles(thisPath, zkRoot + "/" + node);
|
||||
}
|
||||
}
|
||||
|
||||
void verifyAllFilesAreZNodes(Path fileRoot, String zkRoot) throws IOException {
|
||||
Files.walkFileTree(fileRoot, new SimpleFileVisitor<Path>() {
|
||||
void checkPathOnZk(Path path) {
|
||||
String znode = zkRoot + path.toAbsolutePath().toString().substring(fileRoot.toAbsolutePath().toString().length());
|
||||
try {
|
||||
assertTrue("Should have found " + znode + " on Zookeeper", zkClient.exists(znode, true));
|
||||
} catch (Exception e) {
|
||||
fail("Caught unexpected exception " + e.getMessage() + " Znode we were checking " + znode);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
|
||||
assertTrue("Path should start at proper place!", file.startsWith(fileRoot));
|
||||
checkPathOnZk(file);
|
||||
return FileVisitResult.CONTINUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException {
|
||||
|
||||
checkPathOnZk(dir);
|
||||
return FileVisitResult.CONTINUE;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
// Insure that all znodes in first are in second and vice-versa
|
||||
private void verifyZnodesMatch(String first, String second) throws KeeperException, InterruptedException {
|
||||
verifyFirstNodesInSecond(first, second);
|
||||
verifyFirstNodesInSecond(second, first);
|
||||
}
|
||||
|
||||
private void verifyFirstNodesInSecond(String first, String second) throws KeeperException, InterruptedException {
|
||||
for (String node : zkClient.getChildren(first, null, true)) {
|
||||
String fNode = first + "/" + node;
|
||||
String sNode = second + "/" + node;
|
||||
assertTrue("Node " + sNode + " not found. Exists on " + fNode, zkClient.exists(sNode, true));
|
||||
verifyFirstNodesInSecond(fNode, sNode);
|
||||
}
|
||||
}
|
||||
|
||||
public static String createZkNodeName(String zkRoot, Path root, Path file) {
|
||||
String relativePath = root.relativize(file).toString();
|
||||
// Windows shenanigans
|
||||
String separator = root.getFileSystem().getSeparator();
|
||||
if ("\\".equals(separator))
|
||||
relativePath = relativePath.replaceAll("\\\\", "/");
|
||||
return zkRoot + "/" + relativePath;
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -585,7 +585,7 @@ public class CloudSolrClient extends SolrClient {
|
|||
* @param unit the units of the wait parameter
|
||||
* @param predicate a {@link CollectionStatePredicate} to check the collection state
|
||||
* @throws InterruptedException on interrupt
|
||||
* @throws TimeoutException on timeout
|
||||
* @throws TimeoutException on timeout
|
||||
*/
|
||||
public void waitForState(String collection, long wait, TimeUnit unit, CollectionStatePredicate predicate)
|
||||
throws InterruptedException, TimeoutException {
|
||||
|
@ -1453,7 +1453,7 @@ public class CloudSolrClient extends SolrClient {
|
|||
|
||||
/* Log the constructed connection string and then initialize. */
|
||||
final String zkHostString = zkBuilder.toString();
|
||||
log.info("Final constructed zkHost string: " + zkHostString);
|
||||
log.debug("Final constructed zkHost string: " + zkHostString);
|
||||
return zkHostString;
|
||||
}
|
||||
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.solr.common.cloud;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.StringUtils;
|
||||
import org.apache.solr.common.cloud.ZkClientConnectionStrategy.ZkUpdate;
|
||||
|
@ -27,7 +28,6 @@ import org.apache.zookeeper.CreateMode;
|
|||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
import org.apache.zookeeper.KeeperException.NodeExistsException;
|
||||
import org.apache.zookeeper.KeeperException.NotEmptyException;
|
||||
import org.apache.zookeeper.Op;
|
||||
import org.apache.zookeeper.OpResult;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
|
@ -52,8 +52,10 @@ import java.io.StringReader;
|
|||
import java.io.StringWriter;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Path;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -123,7 +125,7 @@ public class SolrZkClient implements Closeable {
|
|||
ZkClientConnectionStrategy strat, final OnReconnect onReconnect, BeforeReconnect beforeReconnect, ZkACLProvider zkACLProvider) {
|
||||
this.zkClientConnectionStrategy = strat;
|
||||
this.zkServerAddress = zkServerAddress;
|
||||
|
||||
|
||||
if (strat == null) {
|
||||
strat = new DefaultConnectionStrategy();
|
||||
}
|
||||
|
@ -701,22 +703,6 @@ public class SolrZkClient implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
// yeah, it's recursive :(
|
||||
public void clean(String path) throws InterruptedException, KeeperException {
|
||||
traverseZkTree(path, znode -> {
|
||||
try {
|
||||
if (!znode.equals("/")) {
|
||||
try {
|
||||
delete(znode, -1, true);
|
||||
} catch (NotEmptyException e) {
|
||||
clean(znode);
|
||||
}
|
||||
}
|
||||
} catch (NoNodeException r) {
|
||||
return;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates if zkHost contains a chroot. See http://zookeeper.apache.org/doc/r3.2.2/zookeeperProgrammers.html#ch_zkSessions
|
||||
|
@ -772,7 +758,7 @@ public class SolrZkClient implements Closeable {
|
|||
* @param root the root node to recursively update
|
||||
*/
|
||||
public void updateACLs(final String root) throws KeeperException, InterruptedException {
|
||||
traverseZkTree(root, path -> {
|
||||
ZkMaintenanceUtils.traverseZkTree(this, root, ZkMaintenanceUtils.VISIT_ORDER.VISIT_POST, path -> {
|
||||
try {
|
||||
setACL(path, getZkACLProvider().getACLsToAdd(path), true);
|
||||
log.info("Updated ACL on " + path);
|
||||
|
@ -783,38 +769,38 @@ public class SolrZkClient implements Closeable {
|
|||
});
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
private interface ZkVisitor {
|
||||
/**
|
||||
* Visit the target path
|
||||
* @param path the path to visit
|
||||
*/
|
||||
void visit(String path) throws InterruptedException, KeeperException;
|
||||
// Some pass-throughs to allow less code disruption to other classes that use SolrZkClient.
|
||||
public void clean(String path) throws InterruptedException, KeeperException {
|
||||
ZkMaintenanceUtils.clean(this, path);
|
||||
}
|
||||
|
||||
/**
|
||||
* Recursively visit a zk tree rooted at path and apply the given visitor to each path. Exists as a separate method
|
||||
* because some of the logic can get nuanced.
|
||||
*
|
||||
* @param path the path to start from
|
||||
* @param visitor the operation to perform on each path
|
||||
*/
|
||||
private void traverseZkTree(final String path, final ZkVisitor visitor) throws InterruptedException, KeeperException {
|
||||
List<String> children;
|
||||
try {
|
||||
children = getChildren(path, null, true);
|
||||
} catch (NoNodeException r) {
|
||||
return;
|
||||
}
|
||||
for (String string : children) {
|
||||
// we can't do anything to the built-in zookeeper node
|
||||
if (path.equals("/") && string.equals("zookeeper")) continue;
|
||||
if (path.equals("/")) {
|
||||
traverseZkTree(path + string, visitor);
|
||||
} else {
|
||||
traverseZkTree(path + "/" + string, visitor);
|
||||
}
|
||||
}
|
||||
visitor.visit(path);
|
||||
public void upConfig(Path confPath, String confName) throws IOException {
|
||||
ZkMaintenanceUtils.upConfig(this, confPath, confName);
|
||||
}
|
||||
|
||||
public String listZnode(String path, Boolean recurse) throws KeeperException, InterruptedException, SolrServerException {
|
||||
return ZkMaintenanceUtils.listZnode(this, path, recurse);
|
||||
}
|
||||
|
||||
public void downConfig(String confName, Path confPath) throws IOException {
|
||||
ZkMaintenanceUtils.downConfig(this, confName, confPath);
|
||||
}
|
||||
|
||||
public void zkTransfer(String src, Boolean srcIsZk,
|
||||
String dst, Boolean dstIsZk,
|
||||
Boolean recurse) throws SolrServerException, KeeperException, InterruptedException, IOException {
|
||||
ZkMaintenanceUtils.zkTransfer(this, src, srcIsZk, dst, dstIsZk, recurse);
|
||||
}
|
||||
|
||||
public void moveZnode(String src, String dst) throws SolrServerException, KeeperException, InterruptedException {
|
||||
ZkMaintenanceUtils.moveZnode(this, src, dst);
|
||||
}
|
||||
|
||||
public void uploadToZK(final Path rootPath, final String zkPath,
|
||||
final Pattern filenameExclusions) throws IOException {
|
||||
ZkMaintenanceUtils.uploadToZK(this, rootPath, zkPath, filenameExclusions);
|
||||
}
|
||||
public void downloadFromZK(String zkPath, Path dir) throws IOException {
|
||||
ZkMaintenanceUtils.downloadFromZK(this, zkPath, dir);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,13 +22,12 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.nio.file.FileVisitResult;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.SimpleFileVisitor;
|
||||
import java.nio.file.attribute.BasicFileAttributes;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
@ -46,76 +45,14 @@ public class ZkConfigManager {
|
|||
public static final Pattern UPLOAD_FILENAME_EXCLUDE_PATTERN = Pattern.compile(UPLOAD_FILENAME_EXCLUDE_REGEX);
|
||||
|
||||
private final SolrZkClient zkClient;
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new ZkConfigManager
|
||||
* @param zkClient the {@link SolrZkClient} to use
|
||||
*/
|
||||
public ZkConfigManager(SolrZkClient zkClient) {
|
||||
this.zkClient = zkClient;
|
||||
}
|
||||
public ZkConfigManager(SolrZkClient zkClient) { this.zkClient = zkClient; }
|
||||
|
||||
private void uploadToZK(final Path rootPath, final String zkPath,
|
||||
final Pattern filenameExclusions) throws IOException {
|
||||
|
||||
if (!Files.exists(rootPath))
|
||||
throw new IOException("Path " + rootPath + " does not exist");
|
||||
|
||||
Files.walkFileTree(rootPath, new SimpleFileVisitor<Path>(){
|
||||
@Override
|
||||
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
|
||||
String filename = file.getFileName().toString();
|
||||
if (filenameExclusions != null && filenameExclusions.matcher(filename).matches()) {
|
||||
logger.info("uploadToZK skipping '{}' due to filenameExclusions '{}'", filename, filenameExclusions);
|
||||
return FileVisitResult.CONTINUE;
|
||||
}
|
||||
String zkNode = createZkNodeName(zkPath, rootPath, file);
|
||||
try {
|
||||
zkClient.makePath(zkNode, file.toFile(), false, true);
|
||||
} catch (KeeperException | InterruptedException e) {
|
||||
throw new IOException("Error uploading file " + file.toString() + " to zookeeper path " + zkNode,
|
||||
SolrZkClient.checkInterrupted(e));
|
||||
}
|
||||
return FileVisitResult.CONTINUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException {
|
||||
return (dir.getFileName().toString().startsWith(".")) ? FileVisitResult.SKIP_SUBTREE : FileVisitResult.CONTINUE;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private static String createZkNodeName(String zkRoot, Path root, Path file) {
|
||||
String relativePath = root.relativize(file).toString();
|
||||
// Windows shenanigans
|
||||
String separator = root.getFileSystem().getSeparator();
|
||||
if ("\\".equals(separator))
|
||||
relativePath = relativePath.replaceAll("\\\\", "/");
|
||||
return zkRoot + "/" + relativePath;
|
||||
}
|
||||
|
||||
private void downloadFromZK(String zkPath, Path dir) throws IOException {
|
||||
try {
|
||||
List<String> files = zkClient.getChildren(zkPath, null, true);
|
||||
Files.createDirectories(dir);
|
||||
for (String file : files) {
|
||||
List<String> children = zkClient.getChildren(zkPath + "/" + file, null, true);
|
||||
if (children.size() == 0) {
|
||||
byte[] data = zkClient.getData(zkPath + "/" + file, null, null, true);
|
||||
Path filename = dir.resolve(file);
|
||||
logger.info("Writing file {}", filename);
|
||||
Files.write(filename, data);
|
||||
} else {
|
||||
downloadFromZK(zkPath + "/" + file, dir.resolve(file));
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (KeeperException | InterruptedException e) {
|
||||
throw new IOException("Error downloading files from zookeeper path " + zkPath + " to " + dir.toString(),
|
||||
SolrZkClient.checkInterrupted(e));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Upload files from a given path to a config in Zookeeper
|
||||
|
@ -125,7 +62,7 @@ public class ZkConfigManager {
|
|||
* if an I/O error occurs or the path does not exist
|
||||
*/
|
||||
public void uploadConfigDir(Path dir, String configName) throws IOException {
|
||||
uploadToZK(dir, CONFIGS_ZKNODE + "/" + configName, UPLOAD_FILENAME_EXCLUDE_PATTERN);
|
||||
zkClient.uploadToZK(dir, CONFIGS_ZKNODE + "/" + configName, UPLOAD_FILENAME_EXCLUDE_PATTERN);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -138,7 +75,7 @@ public class ZkConfigManager {
|
|||
*/
|
||||
public void uploadConfigDir(Path dir, String configName,
|
||||
Pattern filenameExclusions) throws IOException {
|
||||
uploadToZK(dir, CONFIGS_ZKNODE + "/" + configName, filenameExclusions);
|
||||
zkClient.uploadToZK(dir, CONFIGS_ZKNODE + "/" + configName, filenameExclusions);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -149,7 +86,7 @@ public class ZkConfigManager {
|
|||
* if an I/O error occurs or the config does not exist
|
||||
*/
|
||||
public void downloadConfigDir(String configName, Path dir) throws IOException {
|
||||
downloadFromZK(CONFIGS_ZKNODE + "/" + configName, dir);
|
||||
zkClient.downloadFromZK(CONFIGS_ZKNODE + "/" + configName, dir);
|
||||
}
|
||||
|
||||
public List<String> listConfigs() throws IOException {
|
||||
|
@ -240,4 +177,37 @@ public class ZkConfigManager {
|
|||
public void copyConfigDir(String fromConfig, String toConfig, Set<String> copiedToZkPaths) throws IOException {
|
||||
copyConfigDirFromZk(CONFIGS_ZKNODE + "/" + fromConfig, CONFIGS_ZKNODE + "/" + toConfig, copiedToZkPaths);
|
||||
}
|
||||
|
||||
// This method is used by configSetUploadTool and CreateTool to resolve the configset directory.
|
||||
// Check several possibilities:
|
||||
// 1> configSetDirParam + confname is defined in configsets, thus has a conf/solrconfig.xml
|
||||
// 2> configSet is a path that contains conf/solrconfig.xml
|
||||
// 3> configSet is t
|
||||
public static Path getConfigsetPath(String confname, String configSet, String configSetDirParam) throws IOException {
|
||||
|
||||
// one of the canned configsets.
|
||||
Path ret = Paths.get(configSetDirParam, configSet, "conf", "solrconfig.xml").normalize();
|
||||
if (Files.exists(ret)) {
|
||||
return Paths.get(configSetDirParam, configSet, "conf").normalize();
|
||||
}
|
||||
|
||||
// a local path to the parent of a "conf" directory
|
||||
ret = Paths.get(configSet, "conf", "solrconfig.xml").normalize();
|
||||
if (Files.exists(ret)) {
|
||||
return Paths.get(configSet, "conf").normalize();
|
||||
}
|
||||
|
||||
// A local path to the source, probably already includes "conf".
|
||||
ret = Paths.get(configSet, "solrconfig.xml").normalize();
|
||||
if (Files.exists(ret)) {
|
||||
return Paths.get(configSet).normalize();
|
||||
}
|
||||
|
||||
throw new IllegalArgumentException(String.format(Locale.ROOT,
|
||||
"Could not find solrconfig.xml at %s, %s or %s",
|
||||
Paths.get(configSetDirParam, configSet, "conf", "solrconfig.xml").normalize().toAbsolutePath().toString(),
|
||||
Paths.get(configSet, "conf", "solrconfig.xml").normalize().toAbsolutePath().toString(),
|
||||
Paths.get(configSet, "solrconfig.xml").normalize().toAbsolutePath().toString()
|
||||
));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,367 @@
|
|||
package org.apache.solr.common.cloud;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.nio.file.FileVisitResult;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.nio.file.SimpleFileVisitor;
|
||||
import java.nio.file.attribute.BasicFileAttributes;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Class to hold ZK upload/download/move common code. With the advent of the upconfig/downconfig/cp/ls/mv commands
|
||||
* in bin/solr it made sense to keep the individual transfer methods in a central place, so here it is.
|
||||
*/
|
||||
class ZkMaintenanceUtils {
|
||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
|
||||
private ZkMaintenanceUtils() {} // don't let it be instantiated, all methods are static.
|
||||
/**
|
||||
* Lists a ZNode child and (optionally) the znodes of all the children. No data is dumped.
|
||||
*
|
||||
* @param path The node to remove on Zookeeper
|
||||
* @param recurse Whether to remove children.
|
||||
* @throws KeeperException Could not perform the Zookeeper operation.
|
||||
* @throws InterruptedException Thread interrupted
|
||||
* @throws SolrServerException zookeeper node has children and recurse not specified.
|
||||
* @returns an indented list of the znodes suitable for display
|
||||
*/
|
||||
public static String listZnode(SolrZkClient zkClient, String path, Boolean recurse) throws KeeperException, InterruptedException, SolrServerException {
|
||||
String root = path;
|
||||
|
||||
if (path.toLowerCase(Locale.ROOT).startsWith("zk:")) {
|
||||
root = path.substring(3);
|
||||
}
|
||||
if (root.equals("/") == false && root.endsWith("/")) {
|
||||
root = root.substring(0, root.length() - 1);
|
||||
}
|
||||
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
||||
if (recurse == false) {
|
||||
for (String node : zkClient.getChildren(root, null, true)) {
|
||||
if (node.equals("zookeeper") == false) {
|
||||
sb.append(node).append(System.lineSeparator());
|
||||
}
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
traverseZkTree(zkClient, root, VISIT_ORDER.VISIT_PRE, znode -> {
|
||||
if (znode.startsWith("/zookeeper")) return; // can't do anything with this node!
|
||||
int iPos = znode.lastIndexOf("/");
|
||||
if (iPos > 0) {
|
||||
for (int idx = 0; idx < iPos; ++idx) sb.append(" ");
|
||||
sb.append(znode.substring(iPos + 1)).append(System.lineSeparator());
|
||||
} else {
|
||||
sb.append(znode).append(System.lineSeparator());
|
||||
}
|
||||
});
|
||||
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Copy between local file system and Zookeeper, or from one Zookeeper node to another,
|
||||
* optionally copying recursively.
|
||||
*
|
||||
* @param src Source to copy from. Both src and dst may be Znodes. However, both may NOT be local
|
||||
* @param dst The place to copy the files too. Both src and dst may be Znodes. However both may NOT be local
|
||||
* @param recurse if the source is a directory, reccursively copy the contents iff this is true.
|
||||
* @throws SolrServerException Explanatory exception due to bad params, failed operation, etc.
|
||||
* @throws KeeperException Could not perform the Zookeeper operation.
|
||||
* @throws InterruptedException Thread interrupted
|
||||
*/
|
||||
public static void zkTransfer(SolrZkClient zkClient, String src, Boolean srcIsZk,
|
||||
String dst, Boolean dstIsZk,
|
||||
Boolean recurse) throws SolrServerException, KeeperException, InterruptedException, IOException {
|
||||
|
||||
if (srcIsZk == false && dstIsZk == false) {
|
||||
throw new SolrServerException("One or both of source or destination must specify ZK nodes.");
|
||||
}
|
||||
|
||||
// Make sure -recurse is specified if the source has children.
|
||||
if (recurse == false) {
|
||||
if (srcIsZk) {
|
||||
if (zkClient.getChildren(src, null, true).size() != 0) {
|
||||
throw new SolrServerException("Zookeeper node " + src + " has children and recurse is false");
|
||||
}
|
||||
} else if (Files.isDirectory(Paths.get(src))) {
|
||||
throw new SolrServerException("Local path " + Paths.get(src).toAbsolutePath() + " is a directory and recurse is false");
|
||||
}
|
||||
}
|
||||
if (srcIsZk == false && dstIsZk == false) {
|
||||
throw new SolrServerException("At least one of the source and dest parameters must be prefixed with 'zk:' ");
|
||||
}
|
||||
dst = normalizeDest(src, dst);
|
||||
|
||||
if (srcIsZk && dstIsZk) {
|
||||
traverseZkTree(zkClient, src, VISIT_ORDER.VISIT_PRE, new ZkCopier(zkClient, src, dst));
|
||||
return;
|
||||
}
|
||||
if (dstIsZk) {
|
||||
uploadToZK(zkClient, Paths.get(src), dst, null);
|
||||
return;
|
||||
}
|
||||
|
||||
// Copying individual files from ZK requires special handling since downloadFromZK assumes it's a directory.
|
||||
// This is kind of a weak test for the notion of "directory" on Zookeeper.
|
||||
if (zkClient.getChildren(src, null, true).size() > 0) {
|
||||
downloadFromZK(zkClient, src, Paths.get(dst));
|
||||
return;
|
||||
}
|
||||
|
||||
if (Files.isDirectory(Paths.get(dst))) {
|
||||
if (dst.endsWith("/") == false) dst += "/";
|
||||
dst = normalizeDest(src, dst);
|
||||
}
|
||||
byte[] data = zkClient.getData(src, null, null, true);
|
||||
Path filename = Paths.get(dst);
|
||||
Files.createDirectories(filename.getParent());
|
||||
log.info("Writing file {}", filename);
|
||||
Files.write(filename, data);
|
||||
}
|
||||
|
||||
private static String normalizeDest(String srcName, String dstName) {
|
||||
// Pull the last element of the src path and add it to the dst.
|
||||
if (dstName.endsWith("/")) {
|
||||
int pos = srcName.lastIndexOf("/");
|
||||
if (pos < 0) {
|
||||
dstName += srcName;
|
||||
} else {
|
||||
dstName += srcName.substring(pos + 1);
|
||||
}
|
||||
} else if (dstName.equals(".")) {
|
||||
dstName = Paths.get(".").normalize().toAbsolutePath().toString();
|
||||
}
|
||||
return dstName;
|
||||
}
|
||||
|
||||
public static void moveZnode(SolrZkClient zkClient, String src, String dst) throws SolrServerException, KeeperException, InterruptedException {
|
||||
String destName = normalizeDest(src, dst);
|
||||
|
||||
// Special handling if the source has no children, i.e. copying just a single file.
|
||||
if (zkClient.getChildren(src, null, true).size() == 0) {
|
||||
zkClient.makePath(destName, false, true);
|
||||
zkClient.setData(destName, zkClient.getData(src, null, null, true), true);
|
||||
} else {
|
||||
traverseZkTree(zkClient, src, VISIT_ORDER.VISIT_PRE, new ZkCopier(zkClient, src, destName));
|
||||
}
|
||||
|
||||
// Insure all source znodes are present in dest before deleting the source.
|
||||
// throws error if not all there so the source is left intact. Throws error if source and dest don't match.
|
||||
checkAllZnodesThere(zkClient, src, destName);
|
||||
|
||||
clean(zkClient, src);
|
||||
}
|
||||
|
||||
|
||||
// Insure that all the nodes in one path match the nodes in the other as a safety check before removing
|
||||
// the source in a 'mv' command.
|
||||
private static void checkAllZnodesThere(SolrZkClient zkClient, String src, String dst) throws KeeperException, InterruptedException, SolrServerException {
|
||||
|
||||
for (String node : zkClient.getChildren(src, null, true)) {
|
||||
if (zkClient.exists(dst + "/" + node, true) == false) {
|
||||
throw new SolrServerException("mv command did not move node " + dst + "/" + node + " source left intact");
|
||||
}
|
||||
checkAllZnodesThere(zkClient, src + "/" + node, dst + "/" + node);
|
||||
}
|
||||
}
|
||||
|
||||
// This not just a copy operation since the config manager takes care of construction the znode path to configsets
|
||||
public static void downConfig(SolrZkClient zkClient, String confName, Path confPath) throws IOException {
|
||||
ZkConfigManager manager = new ZkConfigManager(zkClient);
|
||||
|
||||
// Try to download the configset
|
||||
manager.downloadConfigDir(confName, confPath);
|
||||
}
|
||||
|
||||
// This not just a copy operation since the config manager takes care of construction the znode path to configsets
|
||||
public static void upConfig(SolrZkClient zkClient, Path confPath, String confName) throws IOException {
|
||||
ZkConfigManager manager = new ZkConfigManager(zkClient);
|
||||
|
||||
// Try to download the configset
|
||||
manager.uploadConfigDir(confPath, confName);
|
||||
}
|
||||
|
||||
// yeah, it's recursive :(
|
||||
public static void clean(SolrZkClient zkClient, String path) throws InterruptedException, KeeperException {
|
||||
traverseZkTree(zkClient, path, VISIT_ORDER.VISIT_POST, znode -> {
|
||||
try {
|
||||
if (!znode.equals("/")) {
|
||||
try {
|
||||
zkClient.delete(znode, -1, true);
|
||||
} catch (KeeperException.NotEmptyException e) {
|
||||
clean(zkClient, znode);
|
||||
}
|
||||
}
|
||||
} catch (KeeperException.NoNodeException r) {
|
||||
return;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static void uploadToZK(SolrZkClient zkClient, final Path rootPath, final String zkPath,
|
||||
final Pattern filenameExclusions) throws IOException {
|
||||
|
||||
if (!Files.exists(rootPath))
|
||||
throw new IOException("Path " + rootPath + " does not exist");
|
||||
|
||||
Files.walkFileTree(rootPath, new SimpleFileVisitor<Path>() {
|
||||
@Override
|
||||
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
|
||||
String filename = file.getFileName().toString();
|
||||
if (filenameExclusions != null && filenameExclusions.matcher(filename).matches()) {
|
||||
log.info("uploadToZK skipping '{}' due to filenameExclusions '{}'", filename, filenameExclusions);
|
||||
return FileVisitResult.CONTINUE;
|
||||
}
|
||||
String zkNode = createZkNodeName(zkPath, rootPath, file);
|
||||
try {
|
||||
zkClient.makePath(zkNode, file.toFile(), false, true);
|
||||
} catch (KeeperException | InterruptedException e) {
|
||||
throw new IOException("Error uploading file " + file.toString() + " to zookeeper path " + zkNode,
|
||||
SolrZkClient.checkInterrupted(e));
|
||||
}
|
||||
return FileVisitResult.CONTINUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException {
|
||||
return (dir.getFileName().toString().startsWith(".")) ? FileVisitResult.SKIP_SUBTREE : FileVisitResult.CONTINUE;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static void downloadFromZK(SolrZkClient zkClient, String zkPath, Path dir) throws IOException {
|
||||
try {
|
||||
List<String> files = zkClient.getChildren(zkPath, null, true);
|
||||
Files.createDirectories(dir);
|
||||
for (String file : files) {
|
||||
List<String> children = zkClient.getChildren(zkPath + "/" + file, null, true);
|
||||
if (children.size() == 0) {
|
||||
byte[] data = zkClient.getData(zkPath + "/" + file, null, null, true);
|
||||
Path filename = dir.resolve(file);
|
||||
log.info("Writing file {}", filename);
|
||||
Files.write(filename, data);
|
||||
} else {
|
||||
downloadFromZK(zkClient, zkPath + "/" + file, dir.resolve(file));
|
||||
}
|
||||
}
|
||||
} catch (KeeperException | InterruptedException e) {
|
||||
throw new IOException("Error downloading files from zookeeper path " + zkPath + " to " + dir.toString(),
|
||||
SolrZkClient.checkInterrupted(e));
|
||||
}
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
public interface ZkVisitor {
|
||||
/**
|
||||
* Visit the target path
|
||||
*
|
||||
* @param path the path to visit
|
||||
*/
|
||||
void visit(String path) throws InterruptedException, KeeperException;
|
||||
}
|
||||
|
||||
public enum VISIT_ORDER {
|
||||
VISIT_PRE,
|
||||
VISIT_POST
|
||||
}
|
||||
|
||||
/**
|
||||
* Recursively visit a zk tree rooted at path and apply the given visitor to each path. Exists as a separate method
|
||||
* because some of the logic can get nuanced.
|
||||
*
|
||||
* @param path the path to start from
|
||||
* @param visitOrder whether to call the visitor at the at the ending or beginning of the run.
|
||||
* @param visitor the operation to perform on each path
|
||||
*/
|
||||
public static void traverseZkTree(SolrZkClient zkClient, final String path, final VISIT_ORDER visitOrder, final ZkVisitor visitor)
|
||||
throws InterruptedException, KeeperException {
|
||||
if (visitOrder == VISIT_ORDER.VISIT_PRE) {
|
||||
visitor.visit(path);
|
||||
}
|
||||
List<String> children;
|
||||
try {
|
||||
children = zkClient.getChildren(path, null, true);
|
||||
} catch (KeeperException.NoNodeException r) {
|
||||
return;
|
||||
}
|
||||
for (String string : children) {
|
||||
// we can't do anything to the built-in zookeeper node
|
||||
if (path.equals("/") && string.equals("zookeeper")) continue;
|
||||
if (path.startsWith("/zookeeper")) continue;
|
||||
if (path.equals("/")) {
|
||||
traverseZkTree(zkClient, path + string, visitOrder, visitor);
|
||||
} else {
|
||||
traverseZkTree(zkClient, path + "/" + string, visitOrder, visitor);
|
||||
}
|
||||
}
|
||||
if (visitOrder == VISIT_ORDER.VISIT_POST) {
|
||||
visitor.visit(path);
|
||||
}
|
||||
}
|
||||
|
||||
// Take into account Windows file separaters when making a Znode's name.
|
||||
public static String createZkNodeName(String zkRoot, Path root, Path file) {
|
||||
String relativePath = root.relativize(file).toString();
|
||||
// Windows shenanigans
|
||||
String separator = root.getFileSystem().getSeparator();
|
||||
if ("\\".equals(separator))
|
||||
relativePath = relativePath.replaceAll("\\\\", "/");
|
||||
return zkRoot + "/" + relativePath;
|
||||
}
|
||||
}
|
||||
|
||||
class ZkCopier implements ZkMaintenanceUtils.ZkVisitor {
|
||||
|
||||
String source;
|
||||
String dest;
|
||||
SolrZkClient zkClient;
|
||||
|
||||
ZkCopier(SolrZkClient zkClient, String source, String dest) {
|
||||
this.source = source;
|
||||
this.dest = dest;
|
||||
if (dest.endsWith("/")) {
|
||||
this.dest = dest.substring(0, dest.length() - 1);
|
||||
}
|
||||
this.zkClient = zkClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(String path) throws InterruptedException, KeeperException {
|
||||
String finalDestination = dest;
|
||||
if (path.equals(source) == false) finalDestination += "/" + path.substring(source.length() + 1);
|
||||
if (finalDestination.endsWith("/") || path.endsWith("/")) {
|
||||
int eoe = 99;
|
||||
}
|
||||
zkClient.makePath(finalDestination, false, true);
|
||||
zkClient.setData(finalDestination, zkClient.getData(path, null, null, true), true);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue