Merge branch 'apache/master' (4/16/15) into hbase-11339
API conflicts and test fixes Update LoadTestTool.COLUMN_FAMILY -> DEFAULT_COLUMN_FAMILY due HBASE-11842 Use new 1.0+ api in some tests Use updated Scanners internal api Fix to take into account HBASE-13203 - procedure v2 table delete Conflicts: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
This commit is contained in:
commit
0e20bbf6a3
|
@ -14,3 +14,4 @@ hbase-*/test
|
|||
*.iws
|
||||
*.iml
|
||||
*.ipr
|
||||
patchprocess/
|
||||
|
|
22
bin/hbase
22
bin/hbase
|
@ -34,8 +34,9 @@
|
|||
# HBASE_CLASSPATH_PREFIX Extra Java CLASSPATH entries that should be
|
||||
# prefixed to the system classpath.
|
||||
#
|
||||
# HBASE_HEAPSIZE The maximum amount of heap to use, in MB.
|
||||
# Default is 1000.
|
||||
# HBASE_HEAPSIZE The maximum amount of heap to use.
|
||||
# Default is unset and uses the JVMs default setting
|
||||
# (usually 1/4th of the available memory).
|
||||
#
|
||||
# HBASE_LIBRARY_PATH HBase additions to JAVA_LIBRARY_PATH for adding
|
||||
# native libraries.
|
||||
|
@ -214,14 +215,17 @@ if [ "$HBASE_LIBRARY_PATH" != "" ]; then
|
|||
fi
|
||||
|
||||
#If avail, add Hadoop to the CLASSPATH and to the JAVA_LIBRARY_PATH
|
||||
HADOOP_IN_PATH=$(PATH="${HADOOP_HOME:-${HADOOP_PREFIX}}/bin:$PATH" which hadoop 2>/dev/null)
|
||||
if [ -f ${HADOOP_IN_PATH} ]; then
|
||||
HADOOP_JAVA_LIBRARY_PATH=$(HADOOP_CLASSPATH="$CLASSPATH" ${HADOOP_IN_PATH} \
|
||||
org.apache.hadoop.hbase.util.GetJavaProperty java.library.path 2>/dev/null)
|
||||
if [ -n "$HADOOP_JAVA_LIBRARY_PATH" ]; then
|
||||
JAVA_LIBRARY_PATH=$(append_path "${JAVA_LIBRARY_PATH}" "$HADOOP_JAVA_LIBRARY_PATH")
|
||||
# Allow this functionality to be disabled
|
||||
if [ "$HBASE_DISABLE_HADOOP_CLASSPATH_LOOKUP" != "true" ] ; then
|
||||
HADOOP_IN_PATH=$(PATH="${HADOOP_HOME:-${HADOOP_PREFIX}}/bin:$PATH" which hadoop 2>/dev/null)
|
||||
if [ -f ${HADOOP_IN_PATH} ]; then
|
||||
HADOOP_JAVA_LIBRARY_PATH=$(HADOOP_CLASSPATH="$CLASSPATH" ${HADOOP_IN_PATH} \
|
||||
org.apache.hadoop.hbase.util.GetJavaProperty java.library.path 2>/dev/null)
|
||||
if [ -n "$HADOOP_JAVA_LIBRARY_PATH" ]; then
|
||||
JAVA_LIBRARY_PATH=$(append_path "${JAVA_LIBRARY_PATH}" "$HADOOP_JAVA_LIBRARY_PATH")
|
||||
fi
|
||||
CLASSPATH=$(append_path "${CLASSPATH}" `${HADOOP_IN_PATH} classpath 2>/dev/null`)
|
||||
fi
|
||||
CLASSPATH=$(append_path "${CLASSPATH}" `${HADOOP_IN_PATH} classpath 2>/dev/null`)
|
||||
fi
|
||||
|
||||
# Add user-specified CLASSPATH last
|
||||
|
|
|
@ -31,7 +31,7 @@
|
|||
# HBASE_SSH_OPTS Options passed to ssh when running remote commands.
|
||||
#
|
||||
|
||||
usage="Usage: hbase-cleanup.sh (--cleanZk|--cleanHdfs|--cleanAll)"
|
||||
usage="Usage: hbase-cleanup.sh (--cleanZk|--cleanHdfs|--cleanAll|--cleanAcls)"
|
||||
|
||||
bin=`dirname "$0"`
|
||||
bin=`cd "$bin">/dev/null; pwd`
|
||||
|
@ -40,7 +40,7 @@ bin=`cd "$bin">/dev/null; pwd`
|
|||
. "$bin"/hbase-config.sh
|
||||
|
||||
case $1 in
|
||||
--cleanZk|--cleanHdfs|--cleanAll)
|
||||
--cleanZk|--cleanHdfs|--cleanAll|--cleanAcls)
|
||||
matches="yes" ;;
|
||||
*) ;;
|
||||
esac
|
||||
|
@ -90,6 +90,11 @@ execute_hdfs_command() {
|
|||
"$bin"/hbase org.apache.hadoop.fs.FsShell $command 2>&1
|
||||
}
|
||||
|
||||
execute_clean_acls() {
|
||||
command=$1;
|
||||
"$bin"/hbase org.apache.hadoop.hbase.zookeeper.ZkAclReset $command 2>&1
|
||||
}
|
||||
|
||||
clean_up() {
|
||||
case $1 in
|
||||
--cleanZk)
|
||||
|
@ -102,6 +107,9 @@ clean_up() {
|
|||
execute_zk_command "rmr ${zparent}";
|
||||
execute_hdfs_command "-rmr ${hrootdir}"
|
||||
;;
|
||||
--cleanAcls)
|
||||
execute_clean_acls;
|
||||
;;
|
||||
*)
|
||||
;;
|
||||
esac
|
||||
|
|
|
@ -126,14 +126,12 @@ export MALLOC_ARENA_MAX=${MALLOC_ARENA_MAX:-4}
|
|||
if [ -z "$JAVA_HOME" ]; then
|
||||
cat 1>&2 <<EOF
|
||||
+======================================================================+
|
||||
| Error: JAVA_HOME is not set and Java could not be found |
|
||||
| Error: JAVA_HOME is not set |
|
||||
+----------------------------------------------------------------------+
|
||||
| Please download the latest Sun JDK from the Sun Java web site |
|
||||
| > http://java.sun.com/javase/downloads/ < |
|
||||
| > http://www.oracle.com/technetwork/java/javase/downloads |
|
||||
| |
|
||||
| HBase requires Java 1.7 or later. |
|
||||
| NOTE: This script will find Sun Java whether you install using the |
|
||||
| binary or the RPM based installer. |
|
||||
+======================================================================+
|
||||
EOF
|
||||
exit 1
|
||||
|
|
|
@ -28,8 +28,9 @@
|
|||
@rem
|
||||
@rem HBASE_CLASSPATH Extra Java CLASSPATH entries.
|
||||
@rem
|
||||
@rem HBASE_HEAPSIZE The maximum amount of heap to use, in MB.
|
||||
@rem Default is 1000.
|
||||
@rem HBASE_HEAPSIZE The maximum amount of heap to use.
|
||||
@rem Default is unset and uses the JVMs default setting
|
||||
@rem (usually 1/4th of the available memory).
|
||||
@rem
|
||||
@rem HBASE_OPTS Extra Java runtime options.
|
||||
@rem
|
||||
|
@ -87,7 +88,7 @@ if "%hbase-command%"=="" (
|
|||
goto :eof
|
||||
)
|
||||
|
||||
set JAVA_HEAP_MAX=-Xmx1000m
|
||||
set JAVA_HEAP_MAX=""
|
||||
set JAVA_OFFHEAP_MAX=""
|
||||
|
||||
rem check envvars which might override default args
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
#!/bin/sh
|
||||
#!/usr/bin/env bash
|
||||
#/**
|
||||
# * Copyright 2007 The Apache Software Foundation
|
||||
# *
|
||||
|
@ -54,5 +54,9 @@ shift;
|
|||
|
||||
for i in $*
|
||||
do
|
||||
run_master $cmd $i
|
||||
if [[ "$i" =~ ^[0-9]+$ ]]; then
|
||||
run_master $cmd $i
|
||||
else
|
||||
echo "Invalid argument"
|
||||
fi
|
||||
done
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
#!/bin/sh
|
||||
#!/usr/bin/env bash
|
||||
#/**
|
||||
# * Copyright 2007 The Apache Software Foundation
|
||||
# *
|
||||
|
@ -52,5 +52,9 @@ shift;
|
|||
|
||||
for i in $*
|
||||
do
|
||||
run_regionserver $cmd $i
|
||||
if [[ "$i" =~ ^[0-9]+$ ]]; then
|
||||
run_regionserver $cmd $i
|
||||
else
|
||||
echo "Invalid argument"
|
||||
fi
|
||||
done
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin
|
|||
import org.apache.hadoop.hbase.client.Get
|
||||
import org.apache.hadoop.hbase.client.Scan
|
||||
import org.apache.hadoop.hbase.client.HTable
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory
|
||||
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
|
||||
import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
|
||||
import org.apache.hadoop.hbase.filter.FilterList;
|
||||
|
@ -100,7 +100,7 @@ def isSuccessfulScan(admin, r)
|
|||
scan = Scan.new(r.getStartKey(), r.getStartKey())
|
||||
scan.setBatch(1)
|
||||
scan.setCaching(1)
|
||||
scan.setFilter(FilterList.new(FirstKeyOnlyFilter.new(),InclusiveStopFilter().new(r.getStartKey())))
|
||||
scan.setFilter(FilterList.new(FirstKeyOnlyFilter.new(),InclusiveStopFilter.new(r.getStartKey())))
|
||||
begin
|
||||
table = HTable.new(admin.getConfiguration(), r.getTableName())
|
||||
scanner = table.getScanner(scan)
|
||||
|
@ -243,7 +243,7 @@ end
|
|||
|
||||
# Now get list of regions on targetServer
|
||||
def getRegions(config, servername)
|
||||
connection = HConnectionManager::getConnection(config);
|
||||
connection = ConnectionFactory::createConnection(config);
|
||||
return ProtobufUtil::getOnlineRegions(connection.getAdmin(ServerName.valueOf(servername)));
|
||||
end
|
||||
|
||||
|
|
|
@ -54,9 +54,9 @@ import org.apache.hadoop.hbase.client.Scan
|
|||
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter
|
||||
import org.apache.hadoop.hbase.util.Bytes
|
||||
import org.apache.hadoop.hbase.HRegionInfo
|
||||
import org.apache.hadoop.hbase.client.MetaScanner
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor
|
||||
import org.apache.hadoop.hbase.HTableDescriptor
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory
|
||||
|
||||
# disable debug logging on this script for clarity
|
||||
log_level = org.apache.log4j.Level::ERROR
|
||||
|
@ -138,8 +138,8 @@ while true
|
|||
if $tablename.nil?
|
||||
server_count = admin.getClusterStatus().getRegionsCount()
|
||||
else
|
||||
connection = HConnectionManager::getConnection(config);
|
||||
server_count = MetaScanner::allTableRegions(config, connection, $TableName ,false).size()
|
||||
connection = ConnectionFactory::createConnection(config);
|
||||
server_count = MetaTableAccessor::allTableRegions(connection, $TableName).size()
|
||||
end
|
||||
print "Region Status: #{server_count} / #{meta_count}\n"
|
||||
if SHOULD_WAIT and server_count < meta_count
|
||||
|
|
|
@ -24,10 +24,11 @@
|
|||
@rem Extra Java CLASSPATH elements. Optional.
|
||||
@rem set HBASE_CLASSPATH=
|
||||
|
||||
@rem The maximum amount of heap to use, in MB. Default is 1000.
|
||||
@rem The maximum amount of heap to use. Default is left to JVM default.
|
||||
@rem set HBASE_HEAPSIZE=1000
|
||||
|
||||
@rem Uncomment below if you intend to use off heap cache.
|
||||
@rem Uncomment below if you intend to use off heap cache. For example, to allocate 8G of
|
||||
@rem offheap, set the value to "8G".
|
||||
@rem set HBASE_OFFHEAPSIZE=1000
|
||||
|
||||
@rem For example, to allocate 8G of offheap, to 8G:
|
||||
|
|
|
@ -31,14 +31,12 @@
|
|||
# Extra Java CLASSPATH elements. Optional.
|
||||
# export HBASE_CLASSPATH=
|
||||
|
||||
# The maximum amount of heap to use, in MB. Default is 1000.
|
||||
# export HBASE_HEAPSIZE=1000
|
||||
# The maximum amount of heap to use. Default is left to JVM default.
|
||||
# export HBASE_HEAPSIZE=1G
|
||||
|
||||
# Uncomment below if you intend to use off heap cache.
|
||||
# export HBASE_OFFHEAPSIZE=1000
|
||||
|
||||
# For example, to allocate 8G of offheap, to 8G:
|
||||
# export HBASE_OFFHEAPSIZE=8G
|
||||
# Uncomment below if you intend to use off heap cache. For example, to allocate 8G of
|
||||
# offheap, set the value to "8G".
|
||||
# export HBASE_OFFHEAPSIZE=1G
|
||||
|
||||
# Extra Java runtime options.
|
||||
# Below are what we set by default. May only work with SUN JVM.
|
||||
|
|
|
@ -76,13 +76,14 @@ log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%
|
|||
log4j.logger.org.apache.zookeeper=INFO
|
||||
#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=INFO
|
||||
log4j.logger.org.apache.hadoop.hbase.META=INFO
|
||||
# Make these two classes INFO-level. Make them DEBUG to see more zk debug.
|
||||
log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
|
||||
log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
|
||||
#log4j.logger.org.apache.hadoop.dfs=DEBUG
|
||||
# Set this class to log INFO only otherwise its OTT
|
||||
# Enable this to get detailed connection error/retry logging.
|
||||
# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
|
||||
# log4j.logger.org.apache.hadoop.hbase.client.ConnectionImplementation=TRACE
|
||||
|
||||
|
||||
# Uncomment this line to enable tracing on _every_ RPC call (this can be a lot of output)
|
||||
|
@ -90,5 +91,4 @@ log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
|
|||
|
||||
# Uncomment the below if you want to remove logging of client region caching'
|
||||
# and scan of hbase:meta messages
|
||||
# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=INFO
|
||||
# log4j.logger.org.apache.hadoop.hbase.client.MetaScanner=INFO
|
||||
# log4j.logger.org.apache.hadoop.hbase.client.ConnectionImplementation=INFO
|
||||
|
|
|
@ -65,6 +65,7 @@ a branch (e.g. 0.98), or a particular commit hash. If ref2 is omitted, master
|
|||
will be used.
|
||||
|
||||
Options:
|
||||
-a, --all Do not filter by interface annotations.
|
||||
-b, --binary-only Only run the check for binary compatibility.
|
||||
-f, --force-download Download dependencies (i.e. Java ACC), even if they are
|
||||
already present.
|
||||
|
@ -88,8 +89,8 @@ __EOF
|
|||
GETOPT=${GETOPT:-/usr/bin/env getopt}
|
||||
|
||||
# Parse command line arguments and check for proper syntax.
|
||||
if ! ARG_LIST=$(${GETOPT} -q -o bfhno:qr:s \
|
||||
-l binary-only,force-download,help,no-checkout,options:,quick,repo:,source-only \
|
||||
if ! ARG_LIST=$(${GETOPT} -q -o abfhno:qr:s \
|
||||
-l all,binary-only,force-download,help,no-checkout,options:,quick,repo:,source-only \
|
||||
-- "${@}"); then
|
||||
usage >&2
|
||||
exit 1
|
||||
|
@ -98,6 +99,9 @@ eval set -- "${ARG_LIST[@]}"
|
|||
|
||||
while ((${#})); do
|
||||
case "${1}" in
|
||||
-a | --all )
|
||||
ALL=true
|
||||
shift 1 ;;
|
||||
-b | --binary-only )
|
||||
JAVA_ACC_COMMAND+=(-binary)
|
||||
shift 1 ;;
|
||||
|
@ -244,10 +248,12 @@ fi
|
|||
|
||||
# Generate annotation list dynamically; this way, there's no chance the file
|
||||
# gets stale and you have better visiblity into what classes are actually analyzed.
|
||||
declare -a ANNOTATION_LIST
|
||||
ANNOTATION_LIST+=(InterfaceAudience.Public)
|
||||
ANNOTATION_LIST+=(InterfaceAudience.LimitedPrivate)
|
||||
if ! [ -f ${SCRIPT_DIRECTORY}/target/compatibility/annotations ]; then
|
||||
cat > ${SCRIPT_DIRECTORY}/target/compatibility/annotations << __EOF
|
||||
$(tr " " "\n" <<< "${ANNOTATION_LIST}")
|
||||
$(tr " " "\n" <<< "${ANNOTATION_LIST[@]}")
|
||||
__EOF
|
||||
fi
|
||||
|
||||
|
@ -257,7 +263,9 @@ JAVA_ACC_COMMAND+=(-v1 ${COMMIT[1]} -v2 ${COMMIT[2]})
|
|||
JAVA_ACC_COMMAND+=(-d1 ${JARS[1]} -d2 ${JARS[2]})
|
||||
JAVA_ACC_COMMAND+=(-report-path \
|
||||
${SCRIPT_DIRECTORY}/target/compatibility/report/${COMMIT[1]}_${COMMIT[2]}_compat_report.html)
|
||||
JAVA_ACC_COMMAND+=(-annotations-list ${SCRIPT_DIRECTORY}/target/compatibility/annotations)
|
||||
if [ "${ALL}" != "true" ] ; then
|
||||
JAVA_ACC_COMMAND+=(-annotations-list ${SCRIPT_DIRECTORY}/target/compatibility/annotations)
|
||||
fi
|
||||
|
||||
# Delete any existing report folder under /dev-support/target/compatibility.
|
||||
rm -rf ${SCRIPT_DIRECTORY}/target/compatibility/report
|
||||
|
|
|
@ -25,7 +25,7 @@ export CLOVER_HOME=/home/jenkins/tools/clover/latest
|
|||
export MAVEN_HOME=/home/jenkins/tools/maven/latest
|
||||
|
||||
export PATH=$PATH:$JAVA_HOME/bin:$ANT_HOME/bin:
|
||||
export MAVEN_OPTS="-Xmx3100M -XX:-UsePerfData"
|
||||
export MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M -XX:-UsePerfData -XX:MaxPermSize=256m"}"
|
||||
|
||||
ulimit -n
|
||||
|
||||
|
|
|
@ -83,12 +83,12 @@ echo "Updating Git"
|
|||
git checkout master
|
||||
git pull
|
||||
|
||||
# Generate the site to ~/git/hbase/target/stage
|
||||
# Generate the site to ~/git/hbase/target/site
|
||||
if [ $INTERACTIVE ]; then
|
||||
read -p "Build the site? (y/n)" yn
|
||||
case $yn in
|
||||
[Yy]* )
|
||||
mvn clean package javadoc:aggregate site site:stage -DskipTests
|
||||
mvn clean package javadoc:aggregate site post-site site:stage -DskipTests
|
||||
status=$?
|
||||
if [ $status -ne 0 ]; then
|
||||
echo "The website does not build. Aborting."
|
||||
|
@ -101,7 +101,7 @@ if [ $INTERACTIVE ]; then
|
|||
esac
|
||||
else
|
||||
echo "Building the site in auto mode."
|
||||
mvn clean package javadoc:aggregate site site:stage -DskipTests
|
||||
mvn clean package javadoc:aggregate site post-site site:stage -DskipTests
|
||||
status=$?
|
||||
if [ $status != 0 ]; then
|
||||
echo "The website does not build. Aborting."
|
||||
|
@ -151,7 +151,7 @@ fi
|
|||
|
||||
# Delete known auto-generated content from trunk
|
||||
echo "Deleting known auto-generated content from SVN"
|
||||
rm -rf apidocs devapidocs xref xref-test book book.html java.html
|
||||
rm -rf apidocs devapidocs xref xref-test book book.html java.html apache_hbase_reference_guide.pdf*
|
||||
|
||||
# Copy generated site to svn -- cp takes different options on Darwin and GNU
|
||||
echo "Copying the generated site to SVN"
|
||||
|
@ -161,18 +161,20 @@ elif [ `uname` == "Linux" ]; then
|
|||
COPYOPTS='-au'
|
||||
fi
|
||||
|
||||
cp $COPYOPTS $GIT_DIR/target/site/* .
|
||||
cp $COPYOPTS $GIT_DIR/target/staging/* .
|
||||
|
||||
# Look for things we need to fix up in svn
|
||||
|
||||
echo "Untracked files: svn add"
|
||||
svn status |grep '?' |sed -e "s/[[:space:]]//g"|cut -d '?' -f 2|while read i
|
||||
do svn add $i
|
||||
svn status |grep '?' |sed -e "s/[[:space:]]//g"|cut -d '?' -f 2|while read i; do
|
||||
svn add $i
|
||||
echo "Added $i"
|
||||
done
|
||||
|
||||
echo "Locally deleted files: svn del"
|
||||
svn status |grep '!' |sed -e "s/[[:space:]]//g"|cut -d '!' -f 2|while read i
|
||||
do svn del $i
|
||||
svn status |grep '!' |sed -e "s/[[:space:]]//g"|cut -d '!' -f 2|while read i; do
|
||||
svn del $i
|
||||
echo "Deleted $i"
|
||||
done
|
||||
|
||||
# Display the proposed changes. I filtered out
|
||||
|
@ -196,12 +198,12 @@ SVN_NUM_DIFF=`expr $SVN_NEW_NUMFILES - $SVN_OLD_NUMFILES|sed 's/-//g'`
|
|||
# The whole site is only 500 MB so a difference of 10 MB is huge
|
||||
# In this case, we should abort because something is wrong
|
||||
# Leaving this commented out for now until we get some benchmarks
|
||||
#if [ $SVN_SIZE_DIFF > 10 -o $SVN_NUM_DIFF > 50 ]; then
|
||||
# echo "This commit would cause the website to change sizes by \
|
||||
# $SVN_DIFF MB and $SVN_NUM_DIFF files. There is likely a problem.
|
||||
# Aborting."
|
||||
# exit 1
|
||||
#fi
|
||||
if [ $SVN_SIZE_DIFF > 10 -o $SVN_NUM_DIFF > 50 ]; then
|
||||
echo "This commit would cause the website to change sizes by \
|
||||
$SVN_DIFF MB and $SVN_NUM_DIFF files. There is likely a problem.
|
||||
Aborting."
|
||||
exit 1
|
||||
fi
|
||||
|
||||
|
||||
if [ $INTERACTIVE ]; then
|
||||
|
|
|
@ -13,13 +13,12 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
MAVEN_OPTS="-Xmx3100M"
|
||||
MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
|
||||
|
||||
# The number of acceptable warning for *all* modules
|
||||
# Please update the per-module test-patch.properties if you update this file.
|
||||
|
||||
OK_RELEASEAUDIT_WARNINGS=0
|
||||
OK_FINDBUGS_WARNINGS=95
|
||||
# Allow two warnings. Javadoc complains about sun.misc.Unsafe use. See HBASE-7457
|
||||
OK_JAVADOC_WARNINGS=2
|
||||
|
||||
|
@ -27,4 +26,8 @@ MAX_LINE_LENGTH=100
|
|||
|
||||
# All supported branches for testing with precommit build
|
||||
# branch-1.x should apprear before branch-1 since the latter is a prefix
|
||||
BRANCH_NAMES="0.94 0.98 branch-1.0 branch-1 master"
|
||||
BRANCH_NAMES="0.94 0.98 branch-1.0 branch-1 master hbase-12439 hbase-11339"
|
||||
|
||||
# All supported Hadoop versions that we want to test the compilation with
|
||||
HADOOP2_VERSIONS="2.4.1 2.5.2 2.6.0"
|
||||
HADOOP3_VERSIONS="3.0.0-SNAPSHOT"
|
||||
|
|
|
@ -210,7 +210,7 @@ checkout () {
|
|||
findBranchNameFromPatchName() {
|
||||
local patchName=$1
|
||||
for LOCAL_BRANCH_NAME in $BRANCH_NAMES; do
|
||||
if [[ $patchName =~ .*$LOCAL_BRANCH_NAME.* ]]; then
|
||||
if [[ $patchName =~ /jira/secure/attachment/[0-9]*/.*$LOCAL_BRANCH_NAME ]]; then
|
||||
BRANCH_NAME=$LOCAL_BRANCH_NAME
|
||||
break
|
||||
fi
|
||||
|
@ -230,14 +230,42 @@ checkoutBranch() {
|
|||
echo ""
|
||||
if [[ $JENKINS == "true" ]] ; then
|
||||
if [[ "$BRANCH_NAME" != "master" ]]; then
|
||||
echo "${GIT} checkout ${BRANCH_NAME}"
|
||||
${GIT} checkout ${BRANCH_NAME}
|
||||
echo "origin/${BRANCH_NAME} HEAD is commit `${GIT} rev-list origin/${BRANCH_NAME} -1`"
|
||||
echo "${GIT} checkout -f `${GIT} rev-list origin/${BRANCH_NAME} -1`"
|
||||
${GIT} checkout -f `${GIT} rev-list origin/${BRANCH_NAME} -1`
|
||||
echo "${GIT} status"
|
||||
${GIT} status
|
||||
fi
|
||||
fi
|
||||
}
|
||||
|
||||
###############################################################################
|
||||
### Collect findbugs reports
|
||||
collectFindbugsReports() {
|
||||
name=$1
|
||||
basedir=$2
|
||||
patch_dir=$3
|
||||
for file in $(find $basedir -name findbugsXml.xml)
|
||||
do
|
||||
relative_file=${file#$basedir/} # strip leading $basedir prefix
|
||||
if [ ! $relative_file == "target/findbugsXml.xml" ]; then
|
||||
module_suffix=${relative_file%/target/findbugsXml.xml} # strip trailing path
|
||||
module_suffix=`basename ${module_suffix}`
|
||||
fi
|
||||
|
||||
cp $file $patch_dir/${name}FindbugsWarnings${module_suffix}.xml
|
||||
$FINDBUGS_HOME/bin/setBugDatabaseInfo -name $name \
|
||||
$patch_dir/${name}FindbugsWarnings${module_suffix}.xml \
|
||||
$patch_dir/${name}FindbugsWarnings${module_suffix}.xml
|
||||
done
|
||||
xml_file=$patch_dir/${name}FindbugsWarnings.xml
|
||||
html_file=$patch_dir/${name}FindbugsWarnings.html
|
||||
$FINDBUGS_HOME/bin/unionBugs -withMessages \
|
||||
-output $xml_file $patch_dir/${name}FindbugsWarnings*.xml
|
||||
$FINDBUGS_HOME/bin/convertXmlToText -html $xml_file $html_file
|
||||
file $xml_file $html_file
|
||||
}
|
||||
|
||||
###############################################################################
|
||||
setup () {
|
||||
### Download latest patch file (ignoring .htm and .html) when run from patch process
|
||||
|
@ -280,9 +308,8 @@ setup () {
|
|||
fi
|
||||
fi
|
||||
### exit if warnings are NOT defined in the properties file
|
||||
if [ -z "$OK_FINDBUGS_WARNINGS" ] || [[ -z "$OK_JAVADOC_WARNINGS" ]] || [[ -z $OK_RELEASEAUDIT_WARNINGS ]] ; then
|
||||
if [[ -z "$OK_JAVADOC_WARNINGS" ]] || [[ -z $OK_RELEASEAUDIT_WARNINGS ]] ; then
|
||||
echo "Please define the following properties in test-patch.properties file"
|
||||
echo "OK_FINDBUGS_WARNINGS"
|
||||
echo "OK_RELEASEAUDIT_WARNINGS"
|
||||
echo "OK_JAVADOC_WARNINGS"
|
||||
cleanupAndExit 1
|
||||
|
@ -296,10 +323,12 @@ setup () {
|
|||
echo "======================================================================"
|
||||
echo ""
|
||||
echo ""
|
||||
echo "$MVN clean package checkstyle:checkstyle-aggregate -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/trunkJavacWarnings.txt 2>&1"
|
||||
echo "$MVN clean package checkstyle:checkstyle-aggregate findbugs:findbugs -DskipTests \
|
||||
-D${PROJECT_NAME}PatchProcess > $PATCH_DIR/trunkJavacWarnings.txt 2>&1"
|
||||
export MAVEN_OPTS="${MAVEN_OPTS}"
|
||||
# build core and tests
|
||||
$MVN clean package checkstyle:checkstyle-aggregate -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/trunkJavacWarnings.txt 2>&1
|
||||
$MVN clean package checkstyle:checkstyle-aggregate findbugs:findbugs -DskipTests \
|
||||
-D${PROJECT_NAME}PatchProcess > $PATCH_DIR/trunkJavacWarnings.txt 2>&1
|
||||
if [[ $? != 0 ]] ; then
|
||||
ERR=`$GREP -A 5 'Compilation failure' $PATCH_DIR/trunkJavacWarnings.txt`
|
||||
echo "Trunk compilation is broken?
|
||||
|
@ -307,6 +336,7 @@ setup () {
|
|||
cleanupAndExit 1
|
||||
fi
|
||||
mv target/checkstyle-result.xml $PATCH_DIR/trunkCheckstyle.xml
|
||||
collectFindbugsReports trunk $BASEDIR $PATCH_DIR
|
||||
}
|
||||
|
||||
###############################################################################
|
||||
|
@ -360,6 +390,16 @@ checkTests () {
|
|||
return 0
|
||||
fi
|
||||
fi
|
||||
srcReferences=`${GREP} "diff --git" "${PATCH_DIR}/patch" | ${GREP} "src/main" | \
|
||||
${GREP} -v "src/main/asciidoc" | ${GREP} -v "src/main/site" -c`
|
||||
if [[ $srcReferences == 0 ]] ; then
|
||||
echo "The patch doesn't appear to alter any code that requires tests."
|
||||
JIRA_COMMENT="$JIRA_COMMENT
|
||||
|
||||
{color:green}+0 tests included{color}. The patch appears to be a documentation, build,
|
||||
or dev-support patch that doesn't require tests."
|
||||
return 0
|
||||
fi
|
||||
JIRA_COMMENT="$JIRA_COMMENT
|
||||
|
||||
{color:red}-1 tests included{color}. The patch doesn't appear to include any new or modified tests.
|
||||
|
@ -377,17 +417,21 @@ checkTests () {
|
|||
### Check there are no compilation errors, passing a file to be parsed.
|
||||
checkCompilationErrors() {
|
||||
local file=$1
|
||||
hadoopVersion=""
|
||||
if [ "$#" -ne 1 ]; then
|
||||
hadoopVersion="with Hadoop version $2"
|
||||
fi
|
||||
COMPILATION_ERROR=false
|
||||
eval $(awk '/ERROR/ {print "COMPILATION_ERROR=true"}' $file)
|
||||
if $COMPILATION_ERROR ; then
|
||||
ERRORS=$($AWK '/ERROR/ { print $0 }' $file)
|
||||
echo "======================================================================"
|
||||
echo "There are compilation errors."
|
||||
echo "There are compilation errors $hadoopVersion."
|
||||
echo "======================================================================"
|
||||
echo "$ERRORS"
|
||||
JIRA_COMMENT="$JIRA_COMMENT
|
||||
|
||||
{color:red}-1 javac{color}. The patch appears to cause mvn compile goal to fail.
|
||||
{color:red}-1 javac{color}. The patch appears to cause mvn compile goal to fail $hadoopVersion.
|
||||
|
||||
Compilation errors resume:
|
||||
$ERRORS
|
||||
|
@ -468,6 +512,29 @@ checkAntiPatterns () {
|
|||
return 0
|
||||
}
|
||||
|
||||
###############################################################################
|
||||
### Check that there are no incorrect annotations
|
||||
checkInterfaceAudience () {
|
||||
echo ""
|
||||
echo ""
|
||||
echo "======================================================================"
|
||||
echo "======================================================================"
|
||||
echo " Checking against hadoop InterfaceAudience."
|
||||
echo "======================================================================"
|
||||
echo "======================================================================"
|
||||
echo ""
|
||||
echo ""
|
||||
warnings=`$GREP 'import org.apache.hadoop.classification' $PATCH_DIR/patch`
|
||||
if [[ $warnings != "" ]]; then
|
||||
JIRA_COMMENT="$JIRA_COMMENT
|
||||
|
||||
{color:red}-1 InterfaceAudience{color}. The patch appears to contain InterfaceAudience from hadoop rather than hbase:
|
||||
$warnings."
|
||||
return 1
|
||||
fi
|
||||
return 0
|
||||
}
|
||||
|
||||
###############################################################################
|
||||
### Check there are no javadoc warnings
|
||||
checkJavadocWarnings () {
|
||||
|
@ -504,6 +571,31 @@ $JIRA_COMMENT_FOOTER"
|
|||
return 0
|
||||
}
|
||||
|
||||
checkBuildWithHadoopVersions() {
|
||||
echo ""
|
||||
echo ""
|
||||
echo "======================================================================"
|
||||
echo "======================================================================"
|
||||
echo " Building with all supported Hadoop versions ."
|
||||
echo "======================================================================"
|
||||
echo "======================================================================"
|
||||
echo ""
|
||||
echo ""
|
||||
export MAVEN_OPTS="${MAVEN_OPTS}"
|
||||
for HADOOP2_VERSION in $HADOOP2_VERSIONS ; do
|
||||
echo "$MVN clean install -DskipTests -D${PROJECT_NAME}PatchProcess -Dhadoop-two.version=$HADOOP2_VERSION > $PATCH_DIR/patchJavacWithHadoop-$HADOOP2_VERSION.txt 2>&1"
|
||||
$MVN clean install -DskipTests -D${PROJECT_NAME}PatchProcess -Dhadoop-two.version=$HADOOP2_VERSION > $PATCH_DIR/patchJavacWithHadoop-$HADOOP2_VERSION.txt 2>&1
|
||||
checkCompilationErrors $PATCH_DIR/patchJavacWithHadoop-$HADOOP2_VERSION.txt $HADOOP2_VERSION
|
||||
done
|
||||
|
||||
# TODO: add Hadoop3 versions and compilation here when we get the hadoop.profile=3.0 working
|
||||
|
||||
JIRA_COMMENT="$JIRA_COMMENT
|
||||
|
||||
{color:green}+1 hadoop versions{color}. The patch compiles with all supported hadoop versions ($HADOOP2_VERSIONS)"
|
||||
return 0
|
||||
}
|
||||
|
||||
###############################################################################
|
||||
### Check there are no changes in the number of Javac warnings
|
||||
checkJavacWarnings () {
|
||||
|
@ -596,7 +688,7 @@ checkProtocErrors () {
|
|||
checkProtocCompilationErrors $PATCH_DIR/patchProtocErrors.txt
|
||||
JIRA_COMMENT="$JIRA_COMMENT
|
||||
|
||||
{color:green}+1 javac{color}. The applied patch does not increase the total number of javac compiler warnings."
|
||||
{color:green}+1 protoc{color}. The applied patch does not increase the total number of protoc compiler warnings."
|
||||
return 0
|
||||
}
|
||||
|
||||
|
@ -665,41 +757,36 @@ checkFindbugsWarnings () {
|
|||
{color:red}-1 findbugs{color}. The patch appears to cause Findbugs (version ${findbugs_version}) to fail."
|
||||
return 1
|
||||
fi
|
||||
|
||||
findbugsWarnings=0
|
||||
for file in $(find $BASEDIR -name findbugsXml.xml)
|
||||
do
|
||||
relative_file=${file#$BASEDIR/} # strip leading $BASEDIR prefix
|
||||
if [ ! $relative_file == "target/findbugsXml.xml" ]; then
|
||||
module_suffix=${relative_file%/target/findbugsXml.xml} # strip trailing path
|
||||
module_suffix=`basename ${module_suffix}`
|
||||
fi
|
||||
|
||||
cp $file $PATCH_DIR/patchFindbugsWarnings${module_suffix}.xml
|
||||
$FINDBUGS_HOME/bin/setBugDatabaseInfo -timestamp "01/01/2000" \
|
||||
$PATCH_DIR/patchFindbugsWarnings${module_suffix}.xml \
|
||||
$PATCH_DIR/patchFindbugsWarnings${module_suffix}.xml
|
||||
newFindbugsWarnings=`$FINDBUGS_HOME/bin/filterBugs -first "01/01/2000" $PATCH_DIR/patchFindbugsWarnings${module_suffix}.xml \
|
||||
$PATCH_DIR/newPatchFindbugsWarnings${module_suffix}.xml | $AWK '{print $1}'`
|
||||
echo "Found $newFindbugsWarnings Findbugs warnings ($file)"
|
||||
findbugsWarnings=$((findbugsWarnings+newFindbugsWarnings))
|
||||
echo "$FINDBUGS_HOME/bin/convertXmlToText -html $PATCH_DIR/newPatchFindbugsWarnings${module_suffix}.xml $PATCH_DIR/newPatchFindbugsWarnings${module_suffix}.html"
|
||||
$FINDBUGS_HOME/bin/convertXmlToText -html $PATCH_DIR/newPatchFindbugsWarnings${module_suffix}.xml $PATCH_DIR/newPatchFindbugsWarnings${module_suffix}.html
|
||||
file $PATCH_DIR/newPatchFindbugsWarnings${module_suffix}.xml $PATCH_DIR/newPatchFindbugsWarnings${module_suffix}.html
|
||||
JIRA_COMMENT_FOOTER="Findbugs warnings: $BUILD_URL/artifact/patchprocess/newPatchFindbugsWarnings${module_suffix}.html
|
||||
$JIRA_COMMENT_FOOTER"
|
||||
done
|
||||
|
||||
### if current warnings greater than OK_FINDBUGS_WARNINGS
|
||||
if [[ $findbugsWarnings -gt $OK_FINDBUGS_WARNINGS ]] ; then
|
||||
collectFindbugsReports patch $BASEDIR $PATCH_DIR
|
||||
#this files are generated by collectFindbugsReports() named with its first argument
|
||||
patch_xml=$PATCH_DIR/patchFindbugsWarnings.xml
|
||||
trunk_xml=$PATCH_DIR/trunkFindbugsWarnings.xml
|
||||
# combine them to one database
|
||||
combined_xml=$PATCH_DIR/combinedFindbugsWarnings.xml
|
||||
new_xml=$PATCH_DIR/newFindbugsWarnings.xml
|
||||
new_html=$PATCH_DIR/newFindbugsWarnings.html
|
||||
$FINDBUGS_HOME/bin/computeBugHistory -useAnalysisTimes -withMessages \
|
||||
-output $combined_xml $trunk_xml $patch_xml
|
||||
findbugsWarnings=$($FINDBUGS_HOME/bin/filterBugs -first patch $combined_xml $new_xml)
|
||||
findbugsFixedWarnings=$($FINDBUGS_HOME/bin/filterBugs -fixed patch $combined_xml $new_xml)
|
||||
$FINDBUGS_HOME/bin/convertXmlToText -html $new_xml $new_html
|
||||
file $new_xml $new_html
|
||||
JIRA_COMMENT_FOOTER="Release Findbugs (version ${findbugs_version}) \
|
||||
warnings: $BUILD_URL/artifact/patchprocess/newFindbugsWarnings.html
|
||||
$JIRA_COMMENT_FOOTER"
|
||||
### if current warnings greater than 0, fail
|
||||
if [[ $findbugsWarnings -gt 0 ]] ; then
|
||||
JIRA_COMMENT="$JIRA_COMMENT
|
||||
|
||||
{color:red}-1 findbugs{color}. The patch appears to introduce `expr $(($findbugsWarnings-$OK_FINDBUGS_WARNINGS))` new Findbugs (version ${findbugs_version}) warnings."
|
||||
{color:red}-1 findbugs{color}. The patch appears to introduce $findbugsWarnings \
|
||||
new Findbugs (version ${findbugs_version}) warnings."
|
||||
return 1
|
||||
fi
|
||||
JIRA_COMMENT="$JIRA_COMMENT
|
||||
|
||||
{color:green}+1 findbugs{color}. The patch does not introduce any new Findbugs (version ${findbugs_version}) warnings."
|
||||
{color:green}+1 findbugs{color}. The patch does not introduce any \
|
||||
new Findbugs (version ${findbugs_version}) warnings."
|
||||
return 0
|
||||
}
|
||||
|
||||
|
@ -956,6 +1043,8 @@ fi
|
|||
|
||||
checkAntiPatterns
|
||||
(( RESULT = RESULT + $? ))
|
||||
checkBuildWithHadoopVersions
|
||||
(( RESULT = RESULT + $? ))
|
||||
checkJavacWarnings
|
||||
(( RESULT = RESULT + $? ))
|
||||
checkProtocErrors
|
||||
|
@ -964,6 +1053,8 @@ checkJavadocWarnings
|
|||
(( RESULT = RESULT + $? ))
|
||||
checkCheckstyleErrors
|
||||
(( RESULT = RESULT + $? ))
|
||||
checkInterfaceAudience
|
||||
(( RESULT = RESULT + $? ))
|
||||
checkFindbugsWarnings
|
||||
(( RESULT = RESULT + $? ))
|
||||
checkReleaseAuditWarnings
|
||||
|
|
|
@ -34,25 +34,6 @@
|
|||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>default-compile</id>
|
||||
<configuration>
|
||||
<compilerId>${java.default.compiler}</compilerId>
|
||||
<forceJavacCompilerUse>true</forceJavacCompilerUse>
|
||||
</configuration>
|
||||
</execution>
|
||||
<execution>
|
||||
<id>default-testCompile</id>
|
||||
<configuration>
|
||||
<compilerId>${java.default.compiler}</compilerId>
|
||||
<forceJavacCompilerUse>true</forceJavacCompilerUse>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-site-plugin</artifactId>
|
||||
|
@ -91,6 +72,36 @@
|
|||
<artifactId>maven-source-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
<pluginManagement>
|
||||
<plugins>
|
||||
<!--This plugin's configuration is used to store Eclipse m2e settings
|
||||
only. It has no influence on the Maven build itself.-->
|
||||
<plugin>
|
||||
<groupId>org.eclipse.m2e</groupId>
|
||||
<artifactId>lifecycle-mapping</artifactId>
|
||||
<version>1.0.0</version>
|
||||
<configuration>
|
||||
<lifecycleMappingMetadata>
|
||||
<pluginExecutions>
|
||||
<pluginExecution>
|
||||
<pluginExecutionFilter>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<versionRange>[3.2,)</versionRange>
|
||||
<goals>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
</pluginExecutionFilter>
|
||||
<action>
|
||||
<ignore></ignore>
|
||||
</action>
|
||||
</pluginExecution>
|
||||
</pluginExecutions>
|
||||
</lifecycleMappingMetadata>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</pluginManagement>
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -118,6 +118,9 @@ public class ClusterStatus extends VersionedWritable {
|
|||
* @return the names of region servers on the dead list
|
||||
*/
|
||||
public Collection<ServerName> getDeadServerNames() {
|
||||
if (deadServers == null) {
|
||||
return Collections.<ServerName>emptyList();
|
||||
}
|
||||
return Collections.unmodifiableCollection(deadServers);
|
||||
}
|
||||
|
||||
|
@ -125,14 +128,14 @@ public class ClusterStatus extends VersionedWritable {
|
|||
* @return the number of region servers in the cluster
|
||||
*/
|
||||
public int getServersSize() {
|
||||
return liveServers.size();
|
||||
return liveServers != null ? liveServers.size() : 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the number of dead region servers in the cluster
|
||||
*/
|
||||
public int getDeadServers() {
|
||||
return deadServers.size();
|
||||
return deadServers != null ? deadServers.size() : 0;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -148,8 +151,10 @@ public class ClusterStatus extends VersionedWritable {
|
|||
*/
|
||||
public int getRegionsCount() {
|
||||
int count = 0;
|
||||
for (Map.Entry<ServerName, ServerLoad> e: this.liveServers.entrySet()) {
|
||||
count += e.getValue().getNumberOfRegions();
|
||||
if (liveServers != null && !liveServers.isEmpty()) {
|
||||
for (Map.Entry<ServerName, ServerLoad> e: this.liveServers.entrySet()) {
|
||||
count += e.getValue().getNumberOfRegions();
|
||||
}
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
@ -159,8 +164,10 @@ public class ClusterStatus extends VersionedWritable {
|
|||
*/
|
||||
public int getRequestsCount() {
|
||||
int count = 0;
|
||||
for (Map.Entry<ServerName, ServerLoad> e: this.liveServers.entrySet()) {
|
||||
count += e.getValue().getNumberOfRequests();
|
||||
if (liveServers != null && !liveServers.isEmpty()) {
|
||||
for (Map.Entry<ServerName, ServerLoad> e: this.liveServers.entrySet()) {
|
||||
count += e.getValue().getNumberOfRequests();
|
||||
}
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
@ -222,6 +229,9 @@ public class ClusterStatus extends VersionedWritable {
|
|||
}
|
||||
|
||||
public Collection<ServerName> getServers() {
|
||||
if (liveServers == null) {
|
||||
return Collections.<ServerName>emptyList();
|
||||
}
|
||||
return Collections.unmodifiableCollection(this.liveServers.keySet());
|
||||
}
|
||||
|
||||
|
@ -237,13 +247,16 @@ public class ClusterStatus extends VersionedWritable {
|
|||
* @return the number of backup masters in the cluster
|
||||
*/
|
||||
public int getBackupMastersSize() {
|
||||
return this.backupMasters.size();
|
||||
return backupMasters != null ? backupMasters.size() : 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the names of backup masters
|
||||
*/
|
||||
public Collection<ServerName> getBackupMasters() {
|
||||
if (backupMasters == null) {
|
||||
return Collections.<ServerName>emptyList();
|
||||
}
|
||||
return Collections.unmodifiableCollection(this.backupMasters);
|
||||
}
|
||||
|
||||
|
@ -252,7 +265,7 @@ public class ClusterStatus extends VersionedWritable {
|
|||
* @return Server's load or null if not found.
|
||||
*/
|
||||
public ServerLoad getLoad(final ServerName sn) {
|
||||
return this.liveServers.get(sn);
|
||||
return liveServers != null ? liveServers.get(sn) : null;
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
|
@ -303,27 +316,41 @@ public class ClusterStatus extends VersionedWritable {
|
|||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder(1024);
|
||||
sb.append("Master: " + master);
|
||||
sb.append("\nNumber of backup masters: " + backupMasters.size());
|
||||
for (ServerName serverName: backupMasters) {
|
||||
sb.append("\n " + serverName);
|
||||
|
||||
int backupMastersSize = getBackupMastersSize();
|
||||
sb.append("\nNumber of backup masters: " + backupMastersSize);
|
||||
if (backupMastersSize > 0) {
|
||||
for (ServerName serverName: backupMasters) {
|
||||
sb.append("\n " + serverName);
|
||||
}
|
||||
}
|
||||
|
||||
sb.append("\nNumber of live region servers: " + liveServers.size());
|
||||
for (ServerName serverName: liveServers.keySet()) {
|
||||
sb.append("\n " + serverName.getServerName());
|
||||
int serversSize = getServersSize();
|
||||
sb.append("\nNumber of live region servers: " + serversSize);
|
||||
if (serversSize > 0) {
|
||||
for (ServerName serverName: liveServers.keySet()) {
|
||||
sb.append("\n " + serverName.getServerName());
|
||||
}
|
||||
}
|
||||
|
||||
sb.append("\nNumber of dead region servers: " + deadServers.size());
|
||||
for (ServerName serverName: deadServers) {
|
||||
sb.append("\n " + serverName);
|
||||
int deadServerSize = getDeadServers();
|
||||
sb.append("\nNumber of dead region servers: " + deadServerSize);
|
||||
if (deadServerSize > 0) {
|
||||
for (ServerName serverName: deadServers) {
|
||||
sb.append("\n " + serverName);
|
||||
}
|
||||
}
|
||||
|
||||
sb.append("\nAverage load: " + getAverageLoad());
|
||||
sb.append("\nNumber of requests: " + getRequestsCount());
|
||||
sb.append("\nNumber of regions: " + getRegionsCount());
|
||||
sb.append("\nNumber of regions in transition: " + intransition.size());
|
||||
for (RegionState state: intransition.values()) {
|
||||
sb.append("\n " + state.toDescriptiveString());
|
||||
|
||||
int ritSize = (intransition != null) ? intransition.size() : 0;
|
||||
sb.append("\nNumber of regions in transition: " + ritSize);
|
||||
if (ritSize > 0) {
|
||||
for (RegionState state: intransition.values()) {
|
||||
sb.append("\n " + state.toDescriptiveString());
|
||||
}
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
|
|
@ -20,7 +20,7 @@ import java.util.concurrent.ExecutorService;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.HTableInterface;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
|
||||
/**
|
||||
* Coprocessor environment state.
|
||||
|
@ -50,14 +50,14 @@ public interface CoprocessorEnvironment {
|
|||
* @return an interface for accessing the given table
|
||||
* @throws IOException
|
||||
*/
|
||||
HTableInterface getTable(TableName tableName) throws IOException;
|
||||
Table getTable(TableName tableName) throws IOException;
|
||||
|
||||
/**
|
||||
* @return an interface for accessing the given table using the passed executor to run batch
|
||||
* operations
|
||||
* @throws IOException
|
||||
*/
|
||||
HTableInterface getTable(TableName tableName, ExecutorService service) throws IOException;
|
||||
Table getTable(TableName tableName, ExecutorService service) throws IOException;
|
||||
|
||||
/**
|
||||
* @return the classloader for the loaded coprocessor instance
|
||||
|
|
|
@ -742,7 +742,7 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
String compressTagsStr = getValue(COMPRESS_TAGS);
|
||||
boolean compressTags = DEFAULT_COMPRESS_TAGS;
|
||||
if (compressTagsStr != null) {
|
||||
compressTags = Boolean.valueOf(compressTagsStr);
|
||||
compressTags = Boolean.parseBoolean(compressTagsStr);
|
||||
}
|
||||
return compressTags;
|
||||
}
|
||||
|
@ -755,7 +755,7 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
String compressTagsStr = getValue(COMPRESS_TAGS);
|
||||
boolean compressTags = DEFAULT_COMPRESS_TAGS;
|
||||
if (compressTagsStr != null) {
|
||||
compressTags = Boolean.valueOf(compressTagsStr);
|
||||
compressTags = Boolean.parseBoolean(compressTagsStr);
|
||||
}
|
||||
return compressTags;
|
||||
}
|
||||
|
@ -786,8 +786,9 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
*/
|
||||
public boolean isInMemory() {
|
||||
String value = getValue(HConstants.IN_MEMORY);
|
||||
if (value != null)
|
||||
return Boolean.valueOf(value).booleanValue();
|
||||
if (value != null) {
|
||||
return Boolean.parseBoolean(value);
|
||||
}
|
||||
return DEFAULT_IN_MEMORY;
|
||||
}
|
||||
|
||||
|
@ -835,7 +836,7 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
*/
|
||||
public int getTimeToLive() {
|
||||
String value = getValue(TTL);
|
||||
return (value != null)? Integer.valueOf(value).intValue(): DEFAULT_TTL;
|
||||
return (value != null)? Integer.parseInt(value) : DEFAULT_TTL;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -851,7 +852,7 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
*/
|
||||
public int getMinVersions() {
|
||||
String value = getValue(MIN_VERSIONS);
|
||||
return (value != null)? Integer.valueOf(value).intValue(): 0;
|
||||
return (value != null)? Integer.parseInt(value) : 0;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -869,8 +870,9 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
*/
|
||||
public boolean isBlockCacheEnabled() {
|
||||
String value = getValue(BLOCKCACHE);
|
||||
if (value != null)
|
||||
return Boolean.valueOf(value).booleanValue();
|
||||
if (value != null) {
|
||||
return Boolean.parseBoolean(value);
|
||||
}
|
||||
return DEFAULT_BLOCKCACHE;
|
||||
}
|
||||
|
||||
|
@ -908,7 +910,7 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
public int getScope() {
|
||||
byte[] value = getValue(REPLICATION_SCOPE_BYTES);
|
||||
if (value != null) {
|
||||
return Integer.valueOf(Bytes.toString(value));
|
||||
return Integer.parseInt(Bytes.toString(value));
|
||||
}
|
||||
return DEFAULT_REPLICATION_SCOPE;
|
||||
}
|
||||
|
@ -974,7 +976,9 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
|
||||
private boolean setAndGetBoolean(final String key, final boolean defaultSetting) {
|
||||
String value = getValue(key);
|
||||
if (value != null) return Boolean.valueOf(value).booleanValue();
|
||||
if (value != null) {
|
||||
return Boolean.parseBoolean(value);
|
||||
}
|
||||
return defaultSetting;
|
||||
}
|
||||
|
||||
|
@ -1213,7 +1217,7 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
@Override
|
||||
public int hashCode() {
|
||||
int result = Bytes.hashCode(this.name);
|
||||
result ^= Byte.valueOf(COLUMN_DESCRIPTOR_VERSION).hashCode();
|
||||
result ^= (int) COLUMN_DESCRIPTOR_VERSION;
|
||||
result ^= values.hashCode();
|
||||
result ^= configuration.hashCode();
|
||||
return result;
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
|
||||
|
@ -176,6 +177,14 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
|
|||
private static final Bytes REGION_REPLICATION_KEY =
|
||||
new Bytes(Bytes.toBytes(REGION_REPLICATION));
|
||||
|
||||
/**
|
||||
* <em>INTERNAL</em> flag to indicate whether or not the memstore should be replicated
|
||||
* for read-replicas (CONSISTENCY => TIMELINE).
|
||||
*/
|
||||
public static final String REGION_MEMSTORE_REPLICATION = "REGION_MEMSTORE_REPLICATION";
|
||||
private static final Bytes REGION_MEMSTORE_REPLICATION_KEY =
|
||||
new Bytes(Bytes.toBytes(REGION_MEMSTORE_REPLICATION));
|
||||
|
||||
/** Default durability for HTD is USE_DEFAULT, which defaults to HBase-global default value */
|
||||
private static final Durability DEFAULT_DURABLITY = Durability.USE_DEFAULT;
|
||||
|
||||
|
@ -210,6 +219,8 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
|
|||
|
||||
public static final int DEFAULT_REGION_REPLICATION = 1;
|
||||
|
||||
public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = true;
|
||||
|
||||
private final static Map<String, String> DEFAULT_VALUES
|
||||
= new HashMap<String, String>();
|
||||
private final static Set<Bytes> RESERVED_KEYWORDS
|
||||
|
@ -1073,6 +1084,31 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the read-replicas memstore replication is enabled.
|
||||
*/
|
||||
public boolean hasRegionMemstoreReplication() {
|
||||
return isSomething(REGION_MEMSTORE_REPLICATION_KEY, DEFAULT_REGION_MEMSTORE_REPLICATION);
|
||||
}
|
||||
|
||||
/**
|
||||
* Enable or Disable the memstore replication from the primary region to the replicas.
|
||||
* The replication will be used only for meta operations (e.g. flush, compaction, ...)
|
||||
*
|
||||
* @param memstoreReplication true if the new data written to the primary region
|
||||
* should be replicated.
|
||||
* false if the secondaries can tollerate to have new
|
||||
* data only when the primary flushes the memstore.
|
||||
*/
|
||||
public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
|
||||
setValue(REGION_MEMSTORE_REPLICATION_KEY, memstoreReplication ? TRUE : FALSE);
|
||||
// If the memstore replication is setup, we do not have to wait for observing a flush event
|
||||
// from primary before starting to serve reads, because gaps from replication is not applicable
|
||||
setConfiguration(RegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY,
|
||||
Boolean.toString(memstoreReplication));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the column family names of the current table. The map of
|
||||
* HTableDescriptor contains mapping of family name to HColumnDescriptors.
|
||||
|
|
|
@ -17,7 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -38,12 +40,11 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Consistency;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
|
@ -60,6 +61,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.PairOfSameType;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -118,6 +120,7 @@ public class MetaTableAccessor {
|
|||
*/
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(MetaTableAccessor.class);
|
||||
private static final Log METALOG = LogFactory.getLog("org.apache.hadoop.hbase.META");
|
||||
|
||||
static final byte [] META_REGION_PREFIX;
|
||||
static {
|
||||
|
@ -129,6 +132,37 @@ public class MetaTableAccessor {
|
|||
META_REGION_PREFIX, 0, len);
|
||||
}
|
||||
|
||||
/**
|
||||
* Lists all of the table regions currently in META.
|
||||
* Deprecated, keep there until some test use this.
|
||||
* @param connection what we will use
|
||||
* @param tableName table to list
|
||||
* @return Map of all user-space regions to servers
|
||||
* @throws java.io.IOException
|
||||
* @deprecated use {@link #getTableRegionsAndLocations}, region can have multiple locations
|
||||
*/
|
||||
@Deprecated
|
||||
public static NavigableMap<HRegionInfo, ServerName> allTableRegions(
|
||||
Connection connection, final TableName tableName) throws IOException {
|
||||
final NavigableMap<HRegionInfo, ServerName> regions =
|
||||
new TreeMap<HRegionInfo, ServerName>();
|
||||
Visitor visitor = new TableVisitorBase(tableName) {
|
||||
@Override
|
||||
public boolean visitInternal(Result result) throws IOException {
|
||||
RegionLocations locations = getRegionLocations(result);
|
||||
if (locations == null) return true;
|
||||
for (HRegionLocation loc : locations.getRegionLocations()) {
|
||||
if (loc != null) {
|
||||
HRegionInfo regionInfo = loc.getRegionInfo();
|
||||
regions.put(regionInfo, loc.getServerName());
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
};
|
||||
scanMetaForTableRegions(connection, visitor, tableName);
|
||||
return regions;
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public enum QueryType {
|
||||
|
@ -167,7 +201,7 @@ public class MetaTableAccessor {
|
|||
public static void fullScanRegions(Connection connection,
|
||||
final Visitor visitor)
|
||||
throws IOException {
|
||||
fullScan(connection, visitor, null, QueryType.REGION);
|
||||
scanMeta(connection, null, null, QueryType.REGION, visitor);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -189,20 +223,7 @@ public class MetaTableAccessor {
|
|||
public static void fullScanTables(Connection connection,
|
||||
final Visitor visitor)
|
||||
throws IOException {
|
||||
fullScan(connection, visitor, null, QueryType.TABLE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs a full scan of <code>hbase:meta</code>.
|
||||
* @param connection connection we're using
|
||||
* @param visitor Visitor invoked against each row.
|
||||
* @param type scanned part of meta
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void fullScan(Connection connection,
|
||||
final Visitor visitor, QueryType type)
|
||||
throws IOException {
|
||||
fullScan(connection, visitor, null, type);
|
||||
scanMeta(connection, null, null, QueryType.TABLE, visitor);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -215,7 +236,7 @@ public class MetaTableAccessor {
|
|||
public static List<Result> fullScan(Connection connection, QueryType type)
|
||||
throws IOException {
|
||||
CollectAllVisitor v = new CollectAllVisitor();
|
||||
fullScan(connection, v, null, type);
|
||||
scanMeta(connection, null, null, type, v);
|
||||
return v.getResults();
|
||||
}
|
||||
|
||||
|
@ -228,21 +249,10 @@ public class MetaTableAccessor {
|
|||
static Table getMetaHTable(final Connection connection)
|
||||
throws IOException {
|
||||
// We used to pass whole CatalogTracker in here, now we just pass in Connection
|
||||
if (connection == null || connection.isClosed()) {
|
||||
if (connection == null) {
|
||||
throw new NullPointerException("No connection");
|
||||
}
|
||||
// If the passed in 'connection' is 'managed' -- i.e. every second test uses
|
||||
// a Table or an HBaseAdmin with managed connections -- then doing
|
||||
// connection.getTable will throw an exception saying you are NOT to use
|
||||
// managed connections getting tables. Leaving this as it is for now. Will
|
||||
// revisit when inclined to change all tests. User code probaby makes use of
|
||||
// managed connections too so don't change it till post hbase 1.0.
|
||||
//
|
||||
// There should still be a way to use this method with an unmanaged connection.
|
||||
if (connection instanceof ClusterConnection) {
|
||||
if (((ClusterConnection) connection).isManaged()) {
|
||||
return new HTable(TableName.META_TABLE_NAME, connection);
|
||||
}
|
||||
} else if (connection.isClosed()) {
|
||||
throw new IOException("connection is closed");
|
||||
}
|
||||
return connection.getTable(TableName.META_TABLE_NAME);
|
||||
}
|
||||
|
@ -385,6 +395,28 @@ public class MetaTableAccessor {
|
|||
|| getTableState(connection, tableName) != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Lists all of the regions currently in META.
|
||||
*
|
||||
* @param connection to connect with
|
||||
* @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions,
|
||||
* true and we'll leave out offlined regions from returned list
|
||||
* @return List of all user-space regions.
|
||||
* @throws IOException
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static List<HRegionInfo> getAllRegions(Connection connection,
|
||||
boolean excludeOfflinedSplitParents)
|
||||
throws IOException {
|
||||
List<Pair<HRegionInfo, ServerName>> result;
|
||||
|
||||
result = getTableRegionsAndLocations(connection, null,
|
||||
excludeOfflinedSplitParents);
|
||||
|
||||
return getListOfHRegionInfos(result);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets all of the regions of the specified table. Do not use this method
|
||||
* to get meta table regions, use methods in MetaTableLocator instead.
|
||||
|
@ -441,15 +473,52 @@ public class MetaTableAccessor {
|
|||
|
||||
/**
|
||||
* @param tableName table we're working with
|
||||
* @return Place to start Scan in <code>hbase:meta</code> when passed a
|
||||
* <code>tableName</code>; returns <tableName&rt; <,&rt; <,&rt;
|
||||
* @return start row for scanning META according to query type
|
||||
*/
|
||||
static byte [] getTableStartRowForMeta(TableName tableName) {
|
||||
byte [] startRow = new byte[tableName.getName().length + 2];
|
||||
System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length);
|
||||
startRow[startRow.length - 2] = HConstants.DELIMITER;
|
||||
startRow[startRow.length - 1] = HConstants.DELIMITER;
|
||||
return startRow;
|
||||
public static byte[] getTableStartRowForMeta(TableName tableName, QueryType type) {
|
||||
if (tableName == null) {
|
||||
return null;
|
||||
}
|
||||
switch (type) {
|
||||
case REGION:
|
||||
byte[] startRow = new byte[tableName.getName().length + 2];
|
||||
System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length);
|
||||
startRow[startRow.length - 2] = HConstants.DELIMITER;
|
||||
startRow[startRow.length - 1] = HConstants.DELIMITER;
|
||||
return startRow;
|
||||
case ALL:
|
||||
case TABLE:
|
||||
default:
|
||||
return tableName.getName();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param tableName table we're working with
|
||||
* @return stop row for scanning META according to query type
|
||||
*/
|
||||
public static byte[] getTableStopRowForMeta(TableName tableName, QueryType type) {
|
||||
if (tableName == null) {
|
||||
return null;
|
||||
}
|
||||
final byte[] stopRow;
|
||||
switch (type) {
|
||||
case REGION:
|
||||
stopRow = new byte[tableName.getName().length + 3];
|
||||
System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length);
|
||||
stopRow[stopRow.length - 3] = ' ';
|
||||
stopRow[stopRow.length - 2] = HConstants.DELIMITER;
|
||||
stopRow[stopRow.length - 1] = HConstants.DELIMITER;
|
||||
break;
|
||||
case ALL:
|
||||
case TABLE:
|
||||
default:
|
||||
stopRow = new byte[tableName.getName().length + 1];
|
||||
System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length);
|
||||
stopRow[stopRow.length - 1] = ' ';
|
||||
break;
|
||||
}
|
||||
return stopRow;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -461,18 +530,39 @@ public class MetaTableAccessor {
|
|||
* @param tableName bytes of table's name
|
||||
* @return configured Scan object
|
||||
*/
|
||||
public static Scan getScanForTableName(TableName tableName) {
|
||||
String strName = tableName.getNameAsString();
|
||||
@Deprecated
|
||||
public static Scan getScanForTableName(Connection connection, TableName tableName) {
|
||||
// Start key is just the table name with delimiters
|
||||
byte[] startKey = Bytes.toBytes(strName + ",,");
|
||||
byte[] startKey = getTableStartRowForMeta(tableName, QueryType.REGION);
|
||||
// Stop key appends the smallest possible char to the table name
|
||||
byte[] stopKey = Bytes.toBytes(strName + " ,,");
|
||||
byte[] stopKey = getTableStopRowForMeta(tableName, QueryType.REGION);
|
||||
|
||||
Scan scan = new Scan(startKey);
|
||||
Scan scan = getMetaScan(connection);
|
||||
scan.setStartRow(startKey);
|
||||
scan.setStopRow(stopKey);
|
||||
return scan;
|
||||
}
|
||||
|
||||
private static Scan getMetaScan(Connection connection) {
|
||||
return getMetaScan(connection, Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
private static Scan getMetaScan(Connection connection, int rowUpperLimit) {
|
||||
Scan scan = new Scan();
|
||||
int scannerCaching = connection.getConfiguration()
|
||||
.getInt(HConstants.HBASE_META_SCANNER_CACHING,
|
||||
HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
|
||||
if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
|
||||
HConstants.DEFAULT_USE_META_REPLICAS)) {
|
||||
scan.setConsistency(Consistency.TIMELINE);
|
||||
}
|
||||
if (rowUpperLimit <= scannerCaching) {
|
||||
scan.setSmall(true);
|
||||
}
|
||||
int rows = Math.min(rowUpperLimit, scannerCaching);
|
||||
scan.setCaching(rows);
|
||||
return scan;
|
||||
}
|
||||
/**
|
||||
* Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
|
||||
* @param connection connection we're using
|
||||
|
@ -489,15 +579,15 @@ public class MetaTableAccessor {
|
|||
/**
|
||||
* Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
|
||||
* @param connection connection we're using
|
||||
* @param tableName table to work with
|
||||
* @param tableName table to work with, can be null for getting all regions
|
||||
* @param excludeOfflinedSplitParents don't return split parents
|
||||
* @return Return list of regioninfos and server addresses.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations(
|
||||
Connection connection, final TableName tableName,
|
||||
Connection connection, @Nullable final TableName tableName,
|
||||
final boolean excludeOfflinedSplitParents) throws IOException {
|
||||
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
||||
if (tableName != null && tableName.equals(TableName.META_TABLE_NAME)) {
|
||||
throw new IOException("This method can't be used to locate meta regions;"
|
||||
+ " use MetaTableLocator instead");
|
||||
}
|
||||
|
@ -514,7 +604,6 @@ public class MetaTableAccessor {
|
|||
return true;
|
||||
}
|
||||
HRegionInfo hri = current.getRegionLocation().getRegionInfo();
|
||||
if (!isInsideTable(hri, tableName)) return false;
|
||||
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
|
||||
// Else call super and add this Result to the collection.
|
||||
return super.visit(r);
|
||||
|
@ -533,7 +622,10 @@ public class MetaTableAccessor {
|
|||
}
|
||||
}
|
||||
};
|
||||
fullScan(connection, visitor, getTableStartRowForMeta(tableName), QueryType.REGION);
|
||||
scanMeta(connection,
|
||||
getTableStartRowForMeta(tableName, QueryType.REGION),
|
||||
getTableStopRowForMeta(tableName, QueryType.REGION),
|
||||
QueryType.REGION, visitor);
|
||||
return visitor.getResults();
|
||||
}
|
||||
|
||||
|
@ -565,7 +657,7 @@ public class MetaTableAccessor {
|
|||
}
|
||||
}
|
||||
};
|
||||
fullScan(connection, v, QueryType.REGION);
|
||||
scanMeta(connection, null, null, QueryType.REGION, v);
|
||||
return hris;
|
||||
}
|
||||
|
||||
|
@ -591,62 +683,140 @@ public class MetaTableAccessor {
|
|||
return true;
|
||||
}
|
||||
};
|
||||
fullScan(connection, v, QueryType.ALL);
|
||||
scanMeta(connection, null, null, QueryType.ALL, v);
|
||||
}
|
||||
|
||||
public static void scanMetaForTableRegions(Connection connection,
|
||||
Visitor visitor, TableName tableName) throws IOException {
|
||||
scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor);
|
||||
}
|
||||
|
||||
public static void scanMeta(Connection connection, TableName table,
|
||||
QueryType type, int maxRows, final Visitor visitor) throws IOException {
|
||||
scanMeta(connection, getTableStartRowForMeta(table, type), getTableStopRowForMeta(table, type),
|
||||
type, maxRows, visitor);
|
||||
}
|
||||
|
||||
public static void scanMeta(Connection connection,
|
||||
@Nullable final byte[] startRow, @Nullable final byte[] stopRow,
|
||||
QueryType type, final Visitor visitor) throws IOException {
|
||||
scanMeta(connection, startRow, stopRow, type, Integer.MAX_VALUE, visitor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs a full scan of a catalog table.
|
||||
* Performs a scan of META table for given table starting from
|
||||
* given row.
|
||||
*
|
||||
* @param connection connection we're using
|
||||
* @param visitor Visitor invoked against each row.
|
||||
* @param startrow Where to start the scan. Pass null if want to begin scan
|
||||
* at first row.
|
||||
* @param type scanned part of meta
|
||||
* <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
|
||||
* @param visitor visitor to call
|
||||
* @param tableName table withing we scan
|
||||
* @param row start scan from this row
|
||||
* @param rowLimit max number of rows to return
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void fullScan(Connection connection,
|
||||
final Visitor visitor, @Nullable final byte[] startrow, QueryType type) throws IOException {
|
||||
fullScan(connection, visitor, startrow, type, false);
|
||||
}
|
||||
public static void scanMeta(Connection connection,
|
||||
final Visitor visitor, final TableName tableName,
|
||||
final byte[] row, final int rowLimit)
|
||||
throws IOException {
|
||||
|
||||
/**
|
||||
* Performs a full scan of a catalog table.
|
||||
* @param connection connection we're using
|
||||
* @param visitor Visitor invoked against each row.
|
||||
* @param startrow Where to start the scan. Pass null if want to begin scan
|
||||
* at first row.
|
||||
* @param type scanned part of meta
|
||||
* @param raw read raw data including Delete tumbstones
|
||||
* <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void fullScan(Connection connection,
|
||||
final Visitor visitor, @Nullable final byte[] startrow, QueryType type, boolean raw)
|
||||
throws IOException {
|
||||
Scan scan = new Scan();
|
||||
scan.setRaw(raw);
|
||||
if (startrow != null) scan.setStartRow(startrow);
|
||||
if (startrow == null) {
|
||||
int caching = connection.getConfiguration()
|
||||
.getInt(HConstants.HBASE_META_SCANNER_CACHING, 100);
|
||||
scan.setCaching(caching);
|
||||
byte[] startRow = null;
|
||||
byte[] stopRow = null;
|
||||
if (tableName != null) {
|
||||
startRow =
|
||||
getTableStartRowForMeta(tableName, QueryType.REGION);
|
||||
if (row != null) {
|
||||
HRegionInfo closestRi =
|
||||
getClosestRegionInfo(connection, tableName, row);
|
||||
startRow = HRegionInfo
|
||||
.createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false);
|
||||
}
|
||||
stopRow =
|
||||
getTableStopRowForMeta(tableName, QueryType.REGION);
|
||||
}
|
||||
scanMeta(connection, startRow, stopRow, QueryType.REGION, rowLimit, visitor);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Performs a scan of META table.
|
||||
* @param connection connection we're using
|
||||
* @param startRow Where to start the scan. Pass null if want to begin scan
|
||||
* at first row.
|
||||
* @param stopRow Where to stop the scan. Pass null if want to scan all rows
|
||||
* from the start one
|
||||
* @param type scanned part of meta
|
||||
* @param maxRows maximum rows to return
|
||||
* @param visitor Visitor invoked against each row.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void scanMeta(Connection connection,
|
||||
@Nullable final byte[] startRow, @Nullable final byte[] stopRow,
|
||||
QueryType type, int maxRows, final Visitor visitor)
|
||||
throws IOException {
|
||||
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
|
||||
Scan scan = getMetaScan(connection, rowUpperLimit);
|
||||
|
||||
for (byte[] family : type.getFamilies()) {
|
||||
scan.addFamily(family);
|
||||
}
|
||||
Table metaTable = getMetaHTable(connection);
|
||||
ResultScanner scanner = null;
|
||||
try {
|
||||
scanner = metaTable.getScanner(scan);
|
||||
Result data;
|
||||
while((data = scanner.next()) != null) {
|
||||
if (data.isEmpty()) continue;
|
||||
// Break if visit returns false.
|
||||
if (!visitor.visit(data)) break;
|
||||
if (startRow != null) scan.setStartRow(startRow);
|
||||
if (stopRow != null) scan.setStopRow(stopRow);
|
||||
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Scanning META"
|
||||
+ " starting at row=" + Bytes.toStringBinary(startRow)
|
||||
+ " stopping at row=" + Bytes.toStringBinary(stopRow)
|
||||
+ " for max=" + rowUpperLimit
|
||||
+ " with caching=" + scan.getCaching());
|
||||
}
|
||||
|
||||
int currentRow = 0;
|
||||
try (Table metaTable = getMetaHTable(connection)) {
|
||||
try (ResultScanner scanner = metaTable.getScanner(scan)) {
|
||||
Result data;
|
||||
while ((data = scanner.next()) != null) {
|
||||
if (data.isEmpty()) continue;
|
||||
// Break if visit returns false.
|
||||
if (!visitor.visit(data)) break;
|
||||
if (++currentRow >= rowUpperLimit) break;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (scanner != null) scanner.close();
|
||||
metaTable.close();
|
||||
}
|
||||
if (visitor != null && visitor instanceof Closeable) {
|
||||
try {
|
||||
((Closeable) visitor).close();
|
||||
} catch (Throwable t) {
|
||||
ExceptionUtil.rethrowIfInterrupt(t);
|
||||
LOG.debug("Got exception in closing the meta scanner visitor", t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Get closest metatable region row to passed <code>row</code>
|
||||
* @throws java.io.IOException
|
||||
*/
|
||||
@Nonnull
|
||||
public static HRegionInfo getClosestRegionInfo(Connection connection,
|
||||
@Nonnull final TableName tableName,
|
||||
@Nonnull final byte[] row)
|
||||
throws IOException {
|
||||
byte[] searchRow = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
|
||||
Scan scan = getMetaScan(connection, 1);
|
||||
scan.setReversed(true);
|
||||
scan.setStartRow(searchRow);
|
||||
try (ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan)) {
|
||||
Result result = resultScanner.next();
|
||||
if (result == null) {
|
||||
throw new TableNotFoundException("Cannot find row in META " +
|
||||
" for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
|
||||
}
|
||||
HRegionInfo regionInfo = getHRegionInfo(result);
|
||||
if (regionInfo == null) {
|
||||
throw new IOException("HRegionInfo was null or empty in Meta for " +
|
||||
tableName + ", row=" + Bytes.toStringBinary(row));
|
||||
}
|
||||
return regionInfo;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -976,6 +1146,12 @@ public class MetaTableAccessor {
|
|||
boolean visit(final Result r) throws IOException;
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementations 'visit' a catalog table row but with close() at the end.
|
||||
*/
|
||||
public interface CloseableVisitor extends Visitor, Closeable {
|
||||
}
|
||||
|
||||
/**
|
||||
* A {@link Visitor} that collects content out of passed {@link Result}.
|
||||
*/
|
||||
|
@ -1009,6 +1185,59 @@ public class MetaTableAccessor {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A Visitor that skips offline regions and split parents
|
||||
*/
|
||||
public static abstract class DefaultVisitorBase implements Visitor {
|
||||
|
||||
public DefaultVisitorBase() {
|
||||
super();
|
||||
}
|
||||
|
||||
public abstract boolean visitInternal(Result rowResult) throws IOException;
|
||||
|
||||
@Override
|
||||
public boolean visit(Result rowResult) throws IOException {
|
||||
HRegionInfo info = getHRegionInfo(rowResult);
|
||||
if (info == null) {
|
||||
return true;
|
||||
}
|
||||
|
||||
//skip over offline and split regions
|
||||
if (!(info.isOffline() || info.isSplit())) {
|
||||
return visitInternal(rowResult);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A Visitor for a table. Provides a consistent view of the table's
|
||||
* hbase:meta entries during concurrent splits (see HBASE-5986 for details). This class
|
||||
* does not guarantee ordered traversal of meta entries, and can block until the
|
||||
* hbase:meta entries for daughters are available during splits.
|
||||
*/
|
||||
public static abstract class TableVisitorBase extends DefaultVisitorBase {
|
||||
private TableName tableName;
|
||||
|
||||
public TableVisitorBase(TableName tableName) {
|
||||
super();
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final boolean visit(Result rowResult) throws IOException {
|
||||
HRegionInfo info = getHRegionInfo(rowResult);
|
||||
if (info == null) {
|
||||
return true;
|
||||
}
|
||||
if (!(info.getTable().equals(tableName))) {
|
||||
return false;
|
||||
}
|
||||
return super.visit(rowResult);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Count regions in <code>hbase:meta</code> for passed table.
|
||||
* @param c Configuration object
|
||||
|
@ -1113,6 +1342,9 @@ public class MetaTableAccessor {
|
|||
*/
|
||||
private static void put(final Table t, final Put p) throws IOException {
|
||||
try {
|
||||
if (METALOG.isDebugEnabled()) {
|
||||
METALOG.debug(mutationToString(p));
|
||||
}
|
||||
t.put(p);
|
||||
} finally {
|
||||
t.close();
|
||||
|
@ -1129,6 +1361,9 @@ public class MetaTableAccessor {
|
|||
throws IOException {
|
||||
Table t = getMetaHTable(connection);
|
||||
try {
|
||||
if (METALOG.isDebugEnabled()) {
|
||||
METALOG.debug(mutationsToString(ps));
|
||||
}
|
||||
t.put(ps);
|
||||
} finally {
|
||||
t.close();
|
||||
|
@ -1158,6 +1393,9 @@ public class MetaTableAccessor {
|
|||
throws IOException {
|
||||
Table t = getMetaHTable(connection);
|
||||
try {
|
||||
if (METALOG.isDebugEnabled()) {
|
||||
METALOG.debug(mutationsToString(deletes));
|
||||
}
|
||||
t.delete(deletes);
|
||||
} finally {
|
||||
t.close();
|
||||
|
@ -1202,7 +1440,10 @@ public class MetaTableAccessor {
|
|||
throws IOException {
|
||||
Table t = getMetaHTable(connection);
|
||||
try {
|
||||
t.batch(mutations);
|
||||
if (METALOG.isDebugEnabled()) {
|
||||
METALOG.debug(mutationsToString(mutations));
|
||||
}
|
||||
t.batch(mutations, new Object[mutations.size()]);
|
||||
} catch (InterruptedException e) {
|
||||
InterruptedIOException ie = new InterruptedIOException(e.getMessage());
|
||||
ie.initCause(e);
|
||||
|
@ -1253,6 +1494,9 @@ public class MetaTableAccessor {
|
|||
Put put = makePutFromRegionInfo(regionInfo);
|
||||
addDaughtersToPut(put, splitA, splitB);
|
||||
meta.put(put);
|
||||
if (METALOG.isDebugEnabled()) {
|
||||
METALOG.debug(mutationToString(put));
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Added " + regionInfo.getRegionNameAsString());
|
||||
}
|
||||
|
@ -1464,6 +1708,9 @@ public class MetaTableAccessor {
|
|||
CoprocessorRpcChannel channel = table.coprocessorService(row);
|
||||
MultiRowMutationProtos.MutateRowsRequest.Builder mmrBuilder
|
||||
= MultiRowMutationProtos.MutateRowsRequest.newBuilder();
|
||||
if (METALOG.isDebugEnabled()) {
|
||||
METALOG.debug(mutationsToString(mutations));
|
||||
}
|
||||
for (Mutation mutation : mutations) {
|
||||
if (mutation instanceof Put) {
|
||||
mmrBuilder.addMutationRequest(ProtobufUtil.toMutation(
|
||||
|
@ -1659,4 +1906,28 @@ public class MetaTableAccessor {
|
|||
p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), now, null);
|
||||
return p;
|
||||
}
|
||||
|
||||
private static String mutationsToString(Mutation ... mutations) throws IOException {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
String prefix = "";
|
||||
for (Mutation mutation : mutations) {
|
||||
sb.append(prefix).append(mutationToString(mutation));
|
||||
prefix = ", ";
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
private static String mutationsToString(List<? extends Mutation> mutations) throws IOException {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
String prefix = "";
|
||||
for (Mutation mutation : mutations) {
|
||||
sb.append(prefix).append(mutationToString(mutation));
|
||||
prefix = ", ";
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
private static String mutationToString(Mutation p) throws IOException {
|
||||
return p.getClass().getSimpleName() + p.toJSON();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,9 +20,12 @@
|
|||
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Strings;
|
||||
|
||||
|
@ -153,6 +156,13 @@ public class RegionLoad {
|
|||
return regionLoadPB.getCompleteSequenceId();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return completed sequence id per store.
|
||||
*/
|
||||
public List<StoreSequenceId> getStoreCompleteSequenceId() {
|
||||
return regionLoadPB.getStoreCompleteSequenceIdList();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the uncompressed size of the storefiles in MB.
|
||||
*/
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase;
|
|||
import java.util.Collection;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
|
@ -200,9 +201,15 @@ public class RegionLocations {
|
|||
// in case of region replication going down, we might have a leak here.
|
||||
int max = other.locations.length;
|
||||
|
||||
HRegionInfo regionInfo = null;
|
||||
for (int i = 0; i < max; i++) {
|
||||
HRegionLocation thisLoc = this.getRegionLocation(i);
|
||||
HRegionLocation otherLoc = other.getRegionLocation(i);
|
||||
if (regionInfo == null && otherLoc != null && otherLoc.getRegionInfo() != null) {
|
||||
// regionInfo is the first non-null HRI from other RegionLocations. We use it to ensure that
|
||||
// all replica region infos belong to the same region with same region id.
|
||||
regionInfo = otherLoc.getRegionInfo();
|
||||
}
|
||||
|
||||
HRegionLocation selectedLoc = selectRegionLocation(thisLoc,
|
||||
otherLoc, true, false);
|
||||
|
@ -218,6 +225,18 @@ public class RegionLocations {
|
|||
}
|
||||
}
|
||||
|
||||
// ensure that all replicas share the same start code. Otherwise delete them
|
||||
if (newLocations != null && regionInfo != null) {
|
||||
for (int i=0; i < newLocations.length; i++) {
|
||||
if (newLocations[i] != null) {
|
||||
if (!RegionReplicaUtil.isReplicasForSameRegion(regionInfo,
|
||||
newLocations[i].getRegionInfo())) {
|
||||
newLocations[i] = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return newLocations == null ? this : new RegionLocations(newLocations);
|
||||
}
|
||||
|
||||
|
@ -264,6 +283,15 @@ public class RegionLocations {
|
|||
HRegionLocation[] newLocations = new HRegionLocation[Math.max(locations.length, replicaId +1)];
|
||||
System.arraycopy(locations, 0, newLocations, 0, locations.length);
|
||||
newLocations[replicaId] = location;
|
||||
// ensure that all replicas share the same start code. Otherwise delete them
|
||||
for (int i=0; i < newLocations.length; i++) {
|
||||
if (newLocations[i] != null) {
|
||||
if (!RegionReplicaUtil.isReplicasForSameRegion(location.getRegionInfo(),
|
||||
newLocations[i].getRegionInfo())) {
|
||||
newLocations[i] = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
return new RegionLocations(newLocations);
|
||||
}
|
||||
|
||||
|
|
|
@ -28,8 +28,11 @@ import java.util.TreeSet;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Strings;
|
||||
|
||||
|
@ -52,7 +55,7 @@ public class ServerLoad {
|
|||
private int totalStaticBloomSizeKB = 0;
|
||||
private long totalCompactingKVs = 0;
|
||||
private long currentCompactedKVs = 0;
|
||||
|
||||
|
||||
public ServerLoad(ClusterStatusProtos.ServerLoad serverLoad) {
|
||||
this.serverLoad = serverLoad;
|
||||
for (ClusterStatusProtos.RegionLoad rl: serverLoad.getRegionLoadsList()) {
|
||||
|
@ -70,7 +73,7 @@ public class ServerLoad {
|
|||
totalCompactingKVs += rl.getTotalCompactingKVs();
|
||||
currentCompactedKVs += rl.getCurrentCompactedKVs();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
// NOTE: Function name cannot start with "get" because then an OpenDataException is thrown because
|
||||
|
@ -177,6 +180,26 @@ public class ServerLoad {
|
|||
return serverLoad.getInfoServerPort();
|
||||
}
|
||||
|
||||
/**
|
||||
* Call directly from client such as hbase shell
|
||||
* @return the list of ReplicationLoadSource
|
||||
*/
|
||||
public List<ReplicationLoadSource> getReplicationLoadSourceList() {
|
||||
return ProtobufUtil.toReplicationLoadSourceList(serverLoad.getReplLoadSourceList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Call directly from client such as hbase shell
|
||||
* @return ReplicationLoadSink
|
||||
*/
|
||||
public ReplicationLoadSink getReplicationLoadSink() {
|
||||
if (serverLoad.hasReplLoadSink()) {
|
||||
return ProtobufUtil.toReplicationLoadSink(serverLoad.getReplLoadSink());
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Originally, this method factored in the effect of requests going to the
|
||||
* server as well. However, this does not interact very well with the current
|
||||
|
|
|
@ -23,14 +23,12 @@ import java.util.Iterator;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* Helper class for custom client scanners.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class AbstractClientScanner implements ResultScanner {
|
||||
|
||||
protected ScanMetrics scanMetrics;
|
||||
|
||||
/**
|
||||
|
@ -38,14 +36,19 @@ public abstract class AbstractClientScanner implements ResultScanner {
|
|||
*/
|
||||
protected void initScanMetrics(Scan scan) {
|
||||
// check if application wants to collect scan metrics
|
||||
byte[] enableMetrics = scan.getAttribute(
|
||||
Scan.SCAN_ATTRIBUTES_METRICS_ENABLE);
|
||||
if (enableMetrics != null && Bytes.toBoolean(enableMetrics)) {
|
||||
if (scan.isScanMetricsEnabled()) {
|
||||
scanMetrics = new ScanMetrics();
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: should this be at ResultScanner? ScanMetrics is not public API it seems.
|
||||
/**
|
||||
* Used internally accumulating metrics on scan. To
|
||||
* enable collection of metrics on a Scanner, call {@link Scan#setScanMetricsEnabled(boolean)}.
|
||||
* These metrics are cleared at key transition points. Metrics are accumulated in the
|
||||
* {@link Scan} object itself.
|
||||
* @see Scan#getScanMetrics()
|
||||
* @return Returns the running {@link ScanMetrics} instance or null if scan metrics not enabled.
|
||||
*/
|
||||
public ScanMetrics getScanMetrics() {
|
||||
return scanMetrics;
|
||||
}
|
||||
|
|
|
@ -702,6 +702,13 @@ public interface Admin extends Abortable, Closeable {
|
|||
*/
|
||||
boolean balancer() throws IOException;
|
||||
|
||||
/**
|
||||
* Query the current state of the balancer
|
||||
*
|
||||
* @return true if the balancer is enabled, false otherwise.
|
||||
*/
|
||||
boolean isBalancerEnabled() throws IOException;
|
||||
|
||||
/**
|
||||
* Enable/Disable the catalog janitor
|
||||
*
|
||||
|
|
|
@ -744,7 +744,7 @@ class AsyncProcess {
|
|||
|
||||
private final Batch.Callback<CResult> callback;
|
||||
private final BatchErrors errors;
|
||||
private final ConnectionManager.ServerErrorTracker errorsByServer;
|
||||
private final ConnectionImplementation.ServerErrorTracker errorsByServer;
|
||||
private final ExecutorService pool;
|
||||
private final Set<MultiServerCallable<Row>> callsInProgress;
|
||||
|
||||
|
@ -1743,8 +1743,8 @@ class AsyncProcess {
|
|||
* We may benefit from connection-wide tracking of server errors.
|
||||
* @return ServerErrorTracker to use, null if there is no ServerErrorTracker on this connection
|
||||
*/
|
||||
protected ConnectionManager.ServerErrorTracker createServerErrorTracker() {
|
||||
return new ConnectionManager.ServerErrorTracker(
|
||||
protected ConnectionImplementation.ServerErrorTracker createServerErrorTracker() {
|
||||
return new ConnectionImplementation.ServerErrorTracker(
|
||||
this.serverTrackerTimeout, this.numTries);
|
||||
}
|
||||
|
||||
|
|
|
@ -60,7 +60,7 @@ import java.util.List;
|
|||
* @see Connection
|
||||
* @since 1.0.0
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public interface BufferedMutator extends Closeable {
|
||||
/**
|
||||
|
@ -122,6 +122,8 @@ public interface BufferedMutator extends Closeable {
|
|||
/**
|
||||
* Listens for asynchronous exceptions on a {@link BufferedMutator}.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
interface ExceptionListener {
|
||||
public void onException(RetriesExhaustedWithDetailsException exception,
|
||||
BufferedMutator mutator) throws RetriesExhaustedWithDetailsException;
|
||||
|
|
|
@ -42,7 +42,7 @@ import java.util.concurrent.TimeUnit;
|
|||
* @since 1.0.0
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Stable
|
||||
@InterfaceStability.Evolving
|
||||
public class BufferedMutatorImpl implements BufferedMutator {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(BufferedMutatorImpl.class);
|
||||
|
|
|
@ -19,7 +19,10 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -44,8 +47,6 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import static org.apache.hadoop.hbase.client.ReversedClientScanner.createClosestRowBefore;
|
||||
|
||||
/**
|
||||
* Implements the scanner interface for the HBase client.
|
||||
* If there are multiple regions in a table, this scanner will iterate
|
||||
|
@ -54,6 +55,9 @@ import static org.apache.hadoop.hbase.client.ReversedClientScanner.createClosest
|
|||
@InterfaceAudience.Private
|
||||
public class ClientScanner extends AbstractClientScanner {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
// A byte array in which all elements are the max byte, and it is used to
|
||||
// construct closest front row
|
||||
static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);
|
||||
protected Scan scan;
|
||||
protected boolean closed = false;
|
||||
// Current region scanner is against. Gets cleared if current region goes
|
||||
|
@ -61,6 +65,18 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
protected HRegionInfo currentRegion = null;
|
||||
protected ScannerCallableWithReplicas callable = null;
|
||||
protected final LinkedList<Result> cache = new LinkedList<Result>();
|
||||
/**
|
||||
* A list of partial results that have been returned from the server. This list should only
|
||||
* contain results if this scanner does not have enough partial results to form the complete
|
||||
* result.
|
||||
*/
|
||||
protected final LinkedList<Result> partialResults = new LinkedList<Result>();
|
||||
/**
|
||||
* The row for which we are accumulating partial Results (i.e. the row of the Results stored
|
||||
* inside partialResults). Changes to partialResultsRow and partialResults are kept in sync
|
||||
* via the methods {@link #addToPartialResults(Result)} and {@link #clearPartialResults()}
|
||||
*/
|
||||
protected byte[] partialResultsRow = null;
|
||||
protected final int caching;
|
||||
protected long lastNext;
|
||||
// Keep lastResult returned successfully in case we have to reset scanner.
|
||||
|
@ -285,7 +301,7 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
return callable.isAnyRPCcancelled();
|
||||
}
|
||||
|
||||
static Result[] call(ScannerCallableWithReplicas callable,
|
||||
Result[] call(ScannerCallableWithReplicas callable,
|
||||
RpcRetryingCaller<Result[]> caller, int scannerTimeout)
|
||||
throws IOException, RuntimeException {
|
||||
if (Thread.interrupted()) {
|
||||
|
@ -317,9 +333,9 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
* machine; for scan/map reduce scenarios, we will have multiple scans running at the same time.
|
||||
*
|
||||
* By default, scan metrics are disabled; if the application wants to collect them, this
|
||||
* behavior can be turned on by calling calling:
|
||||
*
|
||||
* scan.setAttribute(SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE))
|
||||
* behavior can be turned on by calling calling {@link Scan#setScanMetricsEnabled(boolean)}
|
||||
*
|
||||
* <p>This invocation clears the scan metrics. Metrics are aggregated in the Scan instance.
|
||||
*/
|
||||
protected void writeScanMetrics() {
|
||||
if (this.scanMetrics == null || scanMetricsPublished) {
|
||||
|
@ -337,112 +353,7 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
return null;
|
||||
}
|
||||
if (cache.size() == 0) {
|
||||
Result [] values = null;
|
||||
long remainingResultSize = maxScannerResultSize;
|
||||
int countdown = this.caching;
|
||||
// We need to reset it if it's a new callable that was created
|
||||
// with a countdown in nextScanner
|
||||
callable.setCaching(this.caching);
|
||||
// This flag is set when we want to skip the result returned. We do
|
||||
// this when we reset scanner because it split under us.
|
||||
boolean retryAfterOutOfOrderException = true;
|
||||
do {
|
||||
try {
|
||||
// Server returns a null values if scanning is to stop. Else,
|
||||
// returns an empty array if scanning is to go on and we've just
|
||||
// exhausted current region.
|
||||
values = call(callable, caller, scannerTimeout);
|
||||
// When the replica switch happens, we need to do certain operations
|
||||
// again. The callable will openScanner with the right startkey
|
||||
// but we need to pick up from there. Bypass the rest of the loop
|
||||
// and let the catch-up happen in the beginning of the loop as it
|
||||
// happens for the cases where we see exceptions. Since only openScanner
|
||||
// would have happened, values would be null
|
||||
if (values == null && callable.switchedToADifferentReplica()) {
|
||||
this.currentRegion = callable.getHRegionInfo();
|
||||
continue;
|
||||
}
|
||||
retryAfterOutOfOrderException = true;
|
||||
} catch (DoNotRetryIOException e) {
|
||||
// DNRIOEs are thrown to make us break out of retries. Some types of DNRIOEs want us
|
||||
// to reset the scanner and come back in again.
|
||||
if (e instanceof UnknownScannerException) {
|
||||
long timeout = lastNext + scannerTimeout;
|
||||
// If we are over the timeout, throw this exception to the client wrapped in
|
||||
// a ScannerTimeoutException. Else, it's because the region moved and we used the old
|
||||
// id against the new region server; reset the scanner.
|
||||
if (timeout < System.currentTimeMillis()) {
|
||||
long elapsed = System.currentTimeMillis() - lastNext;
|
||||
ScannerTimeoutException ex = new ScannerTimeoutException(
|
||||
elapsed + "ms passed since the last invocation, " +
|
||||
"timeout is currently set to " + scannerTimeout);
|
||||
ex.initCause(e);
|
||||
throw ex;
|
||||
}
|
||||
} else {
|
||||
// If exception is any but the list below throw it back to the client; else setup
|
||||
// the scanner and retry.
|
||||
Throwable cause = e.getCause();
|
||||
if ((cause != null && cause instanceof NotServingRegionException) ||
|
||||
(cause != null && cause instanceof RegionServerStoppedException) ||
|
||||
e instanceof OutOfOrderScannerNextException) {
|
||||
// Pass
|
||||
// It is easier writing the if loop test as list of what is allowed rather than
|
||||
// as a list of what is not allowed... so if in here, it means we do not throw.
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
// Else, its signal from depths of ScannerCallable that we need to reset the scanner.
|
||||
if (this.lastResult != null) {
|
||||
// The region has moved. We need to open a brand new scanner at
|
||||
// the new location.
|
||||
// Reset the startRow to the row we've seen last so that the new
|
||||
// scanner starts at the correct row. Otherwise we may see previously
|
||||
// returned rows again.
|
||||
// (ScannerCallable by now has "relocated" the correct region)
|
||||
if(scan.isReversed()){
|
||||
scan.setStartRow(createClosestRowBefore(lastResult.getRow()));
|
||||
}else {
|
||||
scan.setStartRow(Bytes.add(lastResult.getRow(), new byte[1]));
|
||||
}
|
||||
}
|
||||
if (e instanceof OutOfOrderScannerNextException) {
|
||||
if (retryAfterOutOfOrderException) {
|
||||
retryAfterOutOfOrderException = false;
|
||||
} else {
|
||||
// TODO: Why wrap this in a DNRIOE when it already is a DNRIOE?
|
||||
throw new DoNotRetryIOException("Failed after retry of " +
|
||||
"OutOfOrderScannerNextException: was there a rpc timeout?", e);
|
||||
}
|
||||
}
|
||||
// Clear region.
|
||||
this.currentRegion = null;
|
||||
// Set this to zero so we don't try and do an rpc and close on remote server when
|
||||
// the exception we got was UnknownScanner or the Server is going down.
|
||||
callable = null;
|
||||
// This continue will take us to while at end of loop where we will set up new scanner.
|
||||
continue;
|
||||
}
|
||||
long currentTime = System.currentTimeMillis();
|
||||
if (this.scanMetrics != null) {
|
||||
this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime-lastNext);
|
||||
}
|
||||
lastNext = currentTime;
|
||||
if (values != null && values.length > 0) {
|
||||
for (Result rs : values) {
|
||||
cache.add(rs);
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
countdown--;
|
||||
this.lastResult = rs;
|
||||
}
|
||||
}
|
||||
// Values == null means server-side filter has determined we must STOP
|
||||
} while (remainingResultSize > 0 && countdown > 0 &&
|
||||
possiblyNextScanner(countdown, values == null));
|
||||
loadCache();
|
||||
}
|
||||
|
||||
if (cache.size() > 0) {
|
||||
|
@ -454,6 +365,309 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
return null;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public int getCacheSize() {
|
||||
return cache != null ? cache.size() : 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Contact the servers to load more {@link Result}s in the cache.
|
||||
*/
|
||||
protected void loadCache() throws IOException {
|
||||
Result[] values = null;
|
||||
long remainingResultSize = maxScannerResultSize;
|
||||
int countdown = this.caching;
|
||||
|
||||
// We need to reset it if it's a new callable that was created
|
||||
// with a countdown in nextScanner
|
||||
callable.setCaching(this.caching);
|
||||
// This flag is set when we want to skip the result returned. We do
|
||||
// this when we reset scanner because it split under us.
|
||||
boolean retryAfterOutOfOrderException = true;
|
||||
// We don't expect that the server will have more results for us if
|
||||
// it doesn't tell us otherwise. We rely on the size or count of results
|
||||
boolean serverHasMoreResults = false;
|
||||
do {
|
||||
try {
|
||||
// Server returns a null values if scanning is to stop. Else,
|
||||
// returns an empty array if scanning is to go on and we've just
|
||||
// exhausted current region.
|
||||
values = call(callable, caller, scannerTimeout);
|
||||
|
||||
// When the replica switch happens, we need to do certain operations
|
||||
// again. The callable will openScanner with the right startkey
|
||||
// but we need to pick up from there. Bypass the rest of the loop
|
||||
// and let the catch-up happen in the beginning of the loop as it
|
||||
// happens for the cases where we see exceptions. Since only openScanner
|
||||
// would have happened, values would be null
|
||||
if (values == null && callable.switchedToADifferentReplica()) {
|
||||
// Any accumulated partial results are no longer valid since the callable will
|
||||
// openScanner with the correct startkey and we must pick up from there
|
||||
clearPartialResults();
|
||||
this.currentRegion = callable.getHRegionInfo();
|
||||
continue;
|
||||
}
|
||||
retryAfterOutOfOrderException = true;
|
||||
} catch (DoNotRetryIOException e) {
|
||||
// An exception was thrown which makes any partial results that we were collecting
|
||||
// invalid. The scanner will need to be reset to the beginning of a row.
|
||||
clearPartialResults();
|
||||
|
||||
// DNRIOEs are thrown to make us break out of retries. Some types of DNRIOEs want us
|
||||
// to reset the scanner and come back in again.
|
||||
if (e instanceof UnknownScannerException) {
|
||||
long timeout = lastNext + scannerTimeout;
|
||||
// If we are over the timeout, throw this exception to the client wrapped in
|
||||
// a ScannerTimeoutException. Else, it's because the region moved and we used the old
|
||||
// id against the new region server; reset the scanner.
|
||||
if (timeout < System.currentTimeMillis()) {
|
||||
long elapsed = System.currentTimeMillis() - lastNext;
|
||||
ScannerTimeoutException ex =
|
||||
new ScannerTimeoutException(elapsed + "ms passed since the last invocation, "
|
||||
+ "timeout is currently set to " + scannerTimeout);
|
||||
ex.initCause(e);
|
||||
throw ex;
|
||||
}
|
||||
} else {
|
||||
// If exception is any but the list below throw it back to the client; else setup
|
||||
// the scanner and retry.
|
||||
Throwable cause = e.getCause();
|
||||
if ((cause != null && cause instanceof NotServingRegionException) ||
|
||||
(cause != null && cause instanceof RegionServerStoppedException) ||
|
||||
e instanceof OutOfOrderScannerNextException) {
|
||||
// Pass
|
||||
// It is easier writing the if loop test as list of what is allowed rather than
|
||||
// as a list of what is not allowed... so if in here, it means we do not throw.
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
// Else, its signal from depths of ScannerCallable that we need to reset the scanner.
|
||||
if (this.lastResult != null) {
|
||||
// The region has moved. We need to open a brand new scanner at
|
||||
// the new location.
|
||||
// Reset the startRow to the row we've seen last so that the new
|
||||
// scanner starts at the correct row. Otherwise we may see previously
|
||||
// returned rows again.
|
||||
// (ScannerCallable by now has "relocated" the correct region)
|
||||
if (scan.isReversed()) {
|
||||
scan.setStartRow(createClosestRowBefore(lastResult.getRow()));
|
||||
} else {
|
||||
scan.setStartRow(Bytes.add(lastResult.getRow(), new byte[1]));
|
||||
}
|
||||
}
|
||||
if (e instanceof OutOfOrderScannerNextException) {
|
||||
if (retryAfterOutOfOrderException) {
|
||||
retryAfterOutOfOrderException = false;
|
||||
} else {
|
||||
// TODO: Why wrap this in a DNRIOE when it already is a DNRIOE?
|
||||
throw new DoNotRetryIOException("Failed after retry of " +
|
||||
"OutOfOrderScannerNextException: was there a rpc timeout?", e);
|
||||
}
|
||||
}
|
||||
// Clear region.
|
||||
this.currentRegion = null;
|
||||
// Set this to zero so we don't try and do an rpc and close on remote server when
|
||||
// the exception we got was UnknownScanner or the Server is going down.
|
||||
callable = null;
|
||||
|
||||
// This continue will take us to while at end of loop where we will set up new scanner.
|
||||
continue;
|
||||
}
|
||||
long currentTime = System.currentTimeMillis();
|
||||
if (this.scanMetrics != null) {
|
||||
this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - lastNext);
|
||||
}
|
||||
lastNext = currentTime;
|
||||
// Groom the array of Results that we received back from the server before adding that
|
||||
// Results to the scanner's cache. If partial results are not allowed to be seen by the
|
||||
// caller, all book keeping will be performed within this method.
|
||||
List<Result> resultsToAddToCache = getResultsToAddToCache(values);
|
||||
if (!resultsToAddToCache.isEmpty()) {
|
||||
for (Result rs : resultsToAddToCache) {
|
||||
cache.add(rs);
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
countdown--;
|
||||
this.lastResult = rs;
|
||||
}
|
||||
}
|
||||
// We expect that the server won't have more results for us when we exhaust
|
||||
// the size (bytes or count) of the results returned. If the server *does* inform us that
|
||||
// there are more results, we want to avoid possiblyNextScanner(...). Only when we actually
|
||||
// get results is the moreResults context valid.
|
||||
if (null != values && values.length > 0 && callable.hasMoreResultsContext()) {
|
||||
// Only adhere to more server results when we don't have any partialResults
|
||||
// as it keeps the outer loop logic the same.
|
||||
serverHasMoreResults = callable.getServerHasMoreResults() & partialResults.isEmpty();
|
||||
}
|
||||
// Values == null means server-side filter has determined we must STOP
|
||||
// !partialResults.isEmpty() means that we are still accumulating partial Results for a
|
||||
// row. We should not change scanners before we receive all the partial Results for that
|
||||
// row.
|
||||
} while (remainingResultSize > 0 && countdown > 0 && !serverHasMoreResults
|
||||
&& (!partialResults.isEmpty() || possiblyNextScanner(countdown, values == null)));
|
||||
}
|
||||
|
||||
/**
|
||||
* This method ensures all of our book keeping regarding partial results is kept up to date. This
|
||||
* method should be called once we know that the results we received back from the RPC request do
|
||||
* not contain errors. We return a list of results that should be added to the cache. In general,
|
||||
* this list will contain all NON-partial results from the input array (unless the client has
|
||||
* specified that they are okay with receiving partial results)
|
||||
* @return the list of results that should be added to the cache.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected List<Result> getResultsToAddToCache(Result[] resultsFromServer) throws IOException {
|
||||
int resultSize = resultsFromServer != null ? resultsFromServer.length : 0;
|
||||
List<Result> resultsToAddToCache = new ArrayList<Result>(resultSize);
|
||||
|
||||
final boolean isBatchSet = scan != null && scan.getBatch() > 0;
|
||||
final boolean allowPartials = scan != null && scan.getAllowPartialResults();
|
||||
|
||||
// If the caller has indicated in their scan that they are okay with seeing partial results,
|
||||
// then simply add all results to the list. Note that since scan batching also returns results
|
||||
// for a row in pieces we treat batch being set as equivalent to allowing partials. The
|
||||
// implication of treating batching as equivalent to partial results is that it is possible
|
||||
// the caller will receive a result back where the number of cells in the result is less than
|
||||
// the batch size even though it may not be the last group of cells for that row.
|
||||
if (allowPartials || isBatchSet) {
|
||||
addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length);
|
||||
return resultsToAddToCache;
|
||||
}
|
||||
|
||||
// If no results were returned it indicates that we have the all the partial results necessary
|
||||
// to construct the complete result.
|
||||
if (resultsFromServer == null || resultsFromServer.length == 0) {
|
||||
if (!partialResults.isEmpty()) {
|
||||
resultsToAddToCache.add(Result.createCompleteResult(partialResults));
|
||||
clearPartialResults();
|
||||
}
|
||||
|
||||
return resultsToAddToCache;
|
||||
}
|
||||
|
||||
// In every RPC response there should be at most a single partial result. Furthermore, if
|
||||
// there is a partial result, it is guaranteed to be in the last position of the array.
|
||||
Result last = resultsFromServer[resultsFromServer.length - 1];
|
||||
Result partial = last.isPartial() ? last : null;
|
||||
|
||||
if (LOG.isTraceEnabled()) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("number results from RPC: ").append(resultsFromServer.length).append(",");
|
||||
sb.append("partial != null: ").append(partial != null).append(",");
|
||||
sb.append("number of partials so far: ").append(partialResults.size());
|
||||
LOG.trace(sb.toString());
|
||||
}
|
||||
|
||||
// There are three possibilities cases that can occur while handling partial results
|
||||
//
|
||||
// 1. (partial != null && partialResults.isEmpty())
|
||||
// This is the first partial result that we have received. It should be added to
|
||||
// the list of partialResults and await the next RPC request at which point another
|
||||
// portion of the complete result will be received
|
||||
//
|
||||
// 2. !partialResults.isEmpty()
|
||||
// Since our partialResults list is not empty it means that we have been accumulating partial
|
||||
// Results for a particular row. We cannot form the complete/whole Result for that row until
|
||||
// all partials for the row have been received. Thus we loop through all of the Results
|
||||
// returned from the server and determine whether or not all partial Results for the row have
|
||||
// been received. We know that we have received all of the partial Results for the row when:
|
||||
// i) We notice a row change in the Results
|
||||
// ii) We see a Result for the partial row that is NOT marked as a partial Result
|
||||
//
|
||||
// 3. (partial == null && partialResults.isEmpty())
|
||||
// Business as usual. We are not accumulating partial results and there wasn't a partial result
|
||||
// in the RPC response. This means that all of the results we received from the server are
|
||||
// complete and can be added directly to the cache
|
||||
if (partial != null && partialResults.isEmpty()) {
|
||||
addToPartialResults(partial);
|
||||
|
||||
// Exclude the last result, it's a partial
|
||||
addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length - 1);
|
||||
} else if (!partialResults.isEmpty()) {
|
||||
for (int i = 0; i < resultsFromServer.length; i++) {
|
||||
Result result = resultsFromServer[i];
|
||||
|
||||
// This result is from the same row as the partial Results. Add it to the list of partials
|
||||
// and check if it was the last partial Result for that row
|
||||
if (Bytes.equals(partialResultsRow, result.getRow())) {
|
||||
addToPartialResults(result);
|
||||
|
||||
// If the result is not a partial, it is a signal to us that it is the last Result we
|
||||
// need to form the complete Result client-side
|
||||
if (!result.isPartial()) {
|
||||
resultsToAddToCache.add(Result.createCompleteResult(partialResults));
|
||||
clearPartialResults();
|
||||
}
|
||||
} else {
|
||||
// The row of this result differs from the row of the partial results we have received so
|
||||
// far. If our list of partials isn't empty, this is a signal to form the complete Result
|
||||
// since the row has now changed
|
||||
if (!partialResults.isEmpty()) {
|
||||
resultsToAddToCache.add(Result.createCompleteResult(partialResults));
|
||||
clearPartialResults();
|
||||
}
|
||||
|
||||
// It's possible that in one response from the server we receive the final partial for
|
||||
// one row and receive a partial for a different row. Thus, make sure that all Results
|
||||
// are added to the proper list
|
||||
if (result.isPartial()) {
|
||||
addToPartialResults(result);
|
||||
} else {
|
||||
resultsToAddToCache.add(result);
|
||||
}
|
||||
}
|
||||
}
|
||||
} else { // partial == null && partialResults.isEmpty() -- business as usual
|
||||
addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length);
|
||||
}
|
||||
|
||||
return resultsToAddToCache;
|
||||
}
|
||||
|
||||
/**
|
||||
* A convenience method for adding a Result to our list of partials. This method ensure that only
|
||||
* Results that belong to the same row as the other partials can be added to the list.
|
||||
* @param result The result that we want to add to our list of partial Results
|
||||
* @throws IOException
|
||||
*/
|
||||
private void addToPartialResults(final Result result) throws IOException {
|
||||
final byte[] row = result.getRow();
|
||||
if (partialResultsRow != null && !Bytes.equals(row, partialResultsRow)) {
|
||||
throw new IOException("Partial result row does not match. All partial results must come "
|
||||
+ "from the same row. partialResultsRow: " + Bytes.toString(partialResultsRow) + "row: "
|
||||
+ Bytes.toString(row));
|
||||
}
|
||||
partialResultsRow = row;
|
||||
partialResults.add(result);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience method for clearing the list of partials and resetting the partialResultsRow.
|
||||
*/
|
||||
private void clearPartialResults() {
|
||||
partialResults.clear();
|
||||
partialResultsRow = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method for adding results between the indices [start, end) to the outputList
|
||||
* @param outputList the list that results will be added to
|
||||
* @param inputArray the array that results are taken from
|
||||
* @param start beginning index (inclusive)
|
||||
* @param end ending index (exclusive)
|
||||
*/
|
||||
private void addResultsToList(List<Result> outputList, Result[] inputArray, int start, int end) {
|
||||
if (inputArray == null || start < 0 || end > inputArray.length) return;
|
||||
|
||||
for (int i = start; i < end; i++) {
|
||||
outputList.add(inputArray[i]);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (!scanMetricsPublished) writeScanMetrics();
|
||||
|
@ -476,4 +690,26 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
}
|
||||
closed = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the closest row before the specified row
|
||||
* @param row
|
||||
* @return a new byte array which is the closest front row of the specified one
|
||||
*/
|
||||
protected static byte[] createClosestRowBefore(byte[] row) {
|
||||
if (row == null) {
|
||||
throw new IllegalArgumentException("The passed row is empty");
|
||||
}
|
||||
if (Bytes.equals(row, HConstants.EMPTY_BYTE_ARRAY)) {
|
||||
return MAX_BYTE_ARRAY;
|
||||
}
|
||||
if (row[row.length - 1] == 0) {
|
||||
return Arrays.copyOf(row, row.length - 1);
|
||||
} else {
|
||||
byte[] closestFrontRow = Arrays.copyOf(row, row.length);
|
||||
closestFrontRow[row.length - 1] = (byte) ((closestFrontRow[row.length - 1] & 0xff) - 1);
|
||||
closestFrontRow = Bytes.add(closestFrontRow, MAX_BYTE_ARRAY);
|
||||
return closestFrontRow;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,9 +31,13 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallable;
|
||||
import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Client scanner for small reversed scan. Generally, only one RPC is called to fetch the
|
||||
* scan results, unless the results cross multiple regions or the row count of
|
||||
|
@ -45,34 +49,83 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
public class ClientSmallReversedScanner extends ReversedClientScanner {
|
||||
private static final Log LOG = LogFactory.getLog(ClientSmallReversedScanner.class);
|
||||
private ScannerCallableWithReplicas smallScanCallable = null;
|
||||
private byte[] skipRowOfFirstResult = null;
|
||||
private SmallScannerCallableFactory callableFactory;
|
||||
|
||||
/**
|
||||
* Create a new ReversibleClientScanner for the specified table Note that the
|
||||
* passed {@link Scan}'s start row maybe changed changed.
|
||||
* Create a new ReversibleClientScanner for the specified table. Take note that the passed
|
||||
* {@link Scan} 's start row maybe changed changed.
|
||||
*
|
||||
* @param conf The {@link Configuration} to use.
|
||||
* @param scan {@link Scan} to use in this scanner
|
||||
* @param tableName The table that we wish to rangeGet
|
||||
* @param connection Connection identifying the cluster
|
||||
* @param conf
|
||||
* The {@link Configuration} to use.
|
||||
* @param scan
|
||||
* {@link Scan} to use in this scanner
|
||||
* @param tableName
|
||||
* The table that we wish to rangeGet
|
||||
* @param connection
|
||||
* Connection identifying the cluster
|
||||
* @param rpcFactory
|
||||
* Factory used to create the {@link RpcRetryingCaller}
|
||||
* @param controllerFactory
|
||||
* Factory used to access RPC payloads
|
||||
* @param pool
|
||||
* Threadpool for RPC threads
|
||||
* @param primaryOperationTimeout
|
||||
* Call timeout
|
||||
* @throws IOException
|
||||
* If the remote call fails
|
||||
*/
|
||||
public ClientSmallReversedScanner(final Configuration conf, final Scan scan,
|
||||
final TableName tableName, ClusterConnection connection,
|
||||
RpcRetryingCallerFactory rpcFactory, RpcControllerFactory controllerFactory,
|
||||
ExecutorService pool, int primaryOperationTimeout) throws IOException {
|
||||
super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
|
||||
primaryOperationTimeout);
|
||||
final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
|
||||
RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
|
||||
throws IOException {
|
||||
this(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
|
||||
primaryOperationTimeout, new SmallScannerCallableFactory());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a scanner for following scan. Move to next region or continue from the
|
||||
* last result or start from the start row.
|
||||
* Create a new ReversibleClientScanner for the specified table. Take note that the passed
|
||||
* {@link Scan}'s start row may be changed.
|
||||
*
|
||||
* @param conf
|
||||
* The {@link Configuration} to use.
|
||||
* @param scan
|
||||
* {@link Scan} to use in this scanner
|
||||
* @param tableName
|
||||
* The table that we wish to rangeGet
|
||||
* @param connection
|
||||
* Connection identifying the cluster
|
||||
* @param rpcFactory
|
||||
* Factory used to create the {@link RpcRetryingCaller}
|
||||
* @param controllerFactory
|
||||
* Factory used to access RPC payloads
|
||||
* @param pool
|
||||
* Threadpool for RPC threads
|
||||
* @param primaryOperationTimeout
|
||||
* Call timeout
|
||||
* @param callableFactory
|
||||
* Factory used to create the {@link SmallScannerCallable}
|
||||
* @throws IOException
|
||||
* If the remote call fails
|
||||
*/
|
||||
@VisibleForTesting
|
||||
ClientSmallReversedScanner(final Configuration conf, final Scan scan, final TableName tableName,
|
||||
ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
|
||||
RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout,
|
||||
SmallScannerCallableFactory callableFactory) throws IOException {
|
||||
super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
|
||||
primaryOperationTimeout);
|
||||
this.callableFactory = callableFactory;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a scanner for following scan. Move to next region or continue from the last result or
|
||||
* start from the start row.
|
||||
*
|
||||
* @param nbRows
|
||||
* @param done true if Server-side says we're done scanning.
|
||||
* @param currentRegionDone true if scan is over on current region
|
||||
* @param done
|
||||
* true if Server-side says we're done scanning.
|
||||
* @param currentRegionDone
|
||||
* true if scan is over on current region
|
||||
* @return true if has next scanner
|
||||
* @throws IOException
|
||||
*/
|
||||
|
@ -81,7 +134,7 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
|
|||
// Where to start the next getter
|
||||
byte[] localStartKey;
|
||||
int cacheNum = nbRows;
|
||||
skipRowOfFirstResult = null;
|
||||
boolean regionChanged = true;
|
||||
// if we're at end of table, close and return false to stop iterating
|
||||
if (this.currentRegion != null && currentRegionDone) {
|
||||
byte[] startKey = this.currentRegion.getStartKey();
|
||||
|
@ -100,9 +153,8 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
|
|||
LOG.debug("Finished with region " + this.currentRegion);
|
||||
}
|
||||
} else if (this.lastResult != null) {
|
||||
localStartKey = this.lastResult.getRow();
|
||||
skipRowOfFirstResult = this.lastResult.getRow();
|
||||
cacheNum++;
|
||||
regionChanged = false;
|
||||
localStartKey = createClosestRowBefore(lastResult.getRow());
|
||||
} else {
|
||||
localStartKey = this.scan.getStartRow();
|
||||
}
|
||||
|
@ -112,12 +164,11 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
|
|||
+ Bytes.toStringBinary(localStartKey) + "'");
|
||||
}
|
||||
|
||||
smallScanCallable = ClientSmallScanner.getSmallScanCallable(
|
||||
getConnection(), getTable(), scan, getScanMetrics(), localStartKey, cacheNum,
|
||||
rpcControllerFactory, getPool(), getPrimaryOperationTimeout(),
|
||||
getRetries(), getScannerTimeout(), getConf(), caller);
|
||||
smallScanCallable = callableFactory.getCallable(getConnection(), getTable(), scan,
|
||||
getScanMetrics(), localStartKey, cacheNum, rpcControllerFactory, getPool(),
|
||||
getPrimaryOperationTimeout(), getRetries(), getScannerTimeout(), getConf(), caller);
|
||||
|
||||
if (this.scanMetrics != null && skipRowOfFirstResult == null) {
|
||||
if (this.scanMetrics != null && regionChanged) {
|
||||
this.scanMetrics.countOfRegions.incrementAndGet();
|
||||
}
|
||||
return true;
|
||||
|
@ -131,45 +182,7 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
|
|||
return null;
|
||||
}
|
||||
if (cache.size() == 0) {
|
||||
Result[] values = null;
|
||||
long remainingResultSize = maxScannerResultSize;
|
||||
int countdown = this.caching;
|
||||
boolean currentRegionDone = false;
|
||||
// Values == null means server-side filter has determined we must STOP
|
||||
while (remainingResultSize > 0 && countdown > 0
|
||||
&& nextScanner(countdown, values == null, currentRegionDone)) {
|
||||
// Server returns a null values if scanning is to stop. Else,
|
||||
// returns an empty array if scanning is to go on and we've just
|
||||
// exhausted current region.
|
||||
// callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas,
|
||||
// we do a callWithRetries
|
||||
values = this.caller.callWithoutRetries(smallScanCallable, scannerTimeout);
|
||||
this.currentRegion = smallScanCallable.getHRegionInfo();
|
||||
long currentTime = System.currentTimeMillis();
|
||||
if (this.scanMetrics != null) {
|
||||
this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime
|
||||
- lastNext);
|
||||
}
|
||||
lastNext = currentTime;
|
||||
if (values != null && values.length > 0) {
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
Result rs = values[i];
|
||||
if (i == 0 && this.skipRowOfFirstResult != null
|
||||
&& Bytes.equals(skipRowOfFirstResult, rs.getRow())) {
|
||||
// Skip the first result
|
||||
continue;
|
||||
}
|
||||
cache.add(rs);
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
countdown--;
|
||||
this.lastResult = rs;
|
||||
}
|
||||
}
|
||||
currentRegionDone = countdown > 0;
|
||||
}
|
||||
loadCache();
|
||||
}
|
||||
|
||||
if (cache.size() > 0) {
|
||||
|
@ -181,6 +194,47 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void loadCache() throws IOException {
|
||||
Result[] values = null;
|
||||
long remainingResultSize = maxScannerResultSize;
|
||||
int countdown = this.caching;
|
||||
boolean currentRegionDone = false;
|
||||
// Values == null means server-side filter has determined we must STOP
|
||||
while (remainingResultSize > 0 && countdown > 0
|
||||
&& nextScanner(countdown, values == null, currentRegionDone)) {
|
||||
// Server returns a null values if scanning is to stop. Else,
|
||||
// returns an empty array if scanning is to go on and we've just
|
||||
// exhausted current region.
|
||||
// callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas,
|
||||
// we do a callWithRetries
|
||||
values = this.caller.callWithoutRetries(smallScanCallable, scannerTimeout);
|
||||
this.currentRegion = smallScanCallable.getHRegionInfo();
|
||||
long currentTime = System.currentTimeMillis();
|
||||
if (this.scanMetrics != null) {
|
||||
this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime
|
||||
- lastNext);
|
||||
}
|
||||
lastNext = currentTime;
|
||||
if (values != null && values.length > 0) {
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
Result rs = values[i];
|
||||
cache.add(rs);
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
countdown--;
|
||||
this.lastResult = rs;
|
||||
}
|
||||
}
|
||||
if (smallScanCallable.hasMoreResultsContext()) {
|
||||
currentRegionDone = !smallScanCallable.getServerHasMoreResults();
|
||||
} else {
|
||||
currentRegionDone = countdown > 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void initializeScannerInConstruction() throws IOException {
|
||||
|
@ -194,4 +248,8 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
|
|||
closed = true;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected void setScannerCallableFactory(SmallScannerCallableFactory callableFactory) {
|
||||
this.callableFactory = callableFactory;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
|
@ -52,29 +53,72 @@ import com.google.protobuf.ServiceException;
|
|||
public class ClientSmallScanner extends ClientScanner {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
private ScannerCallableWithReplicas smallScanCallable = null;
|
||||
// When fetching results from server, skip the first result if it has the same
|
||||
// row with this one
|
||||
private byte[] skipRowOfFirstResult = null;
|
||||
private SmallScannerCallableFactory callableFactory;
|
||||
|
||||
/**
|
||||
* Create a new ShortClientScanner for the specified table Note that the
|
||||
* passed {@link Scan}'s start row maybe changed changed.
|
||||
* Create a new ShortClientScanner for the specified table. Take note that the passed {@link Scan}
|
||||
* 's start row maybe changed changed.
|
||||
*
|
||||
* @param conf The {@link Configuration} to use.
|
||||
* @param scan {@link Scan} to use in this scanner
|
||||
* @param tableName The table that we wish to rangeGet
|
||||
* @param connection Connection identifying the cluster
|
||||
* @param conf
|
||||
* The {@link Configuration} to use.
|
||||
* @param scan
|
||||
* {@link Scan} to use in this scanner
|
||||
* @param tableName
|
||||
* The table that we wish to rangeGet
|
||||
* @param connection
|
||||
* Connection identifying the cluster
|
||||
* @param rpcFactory
|
||||
* Factory used to create the {@link RpcRetryingCaller}
|
||||
* @param controllerFactory
|
||||
* Factory used to access RPC payloads
|
||||
* @param pool
|
||||
* Threadpool for RPC threads
|
||||
* @param primaryOperationTimeout
|
||||
* Call timeout
|
||||
* @throws IOException
|
||||
* If the remote call fails
|
||||
*/
|
||||
public ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName,
|
||||
ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
|
||||
RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
|
||||
throws IOException {
|
||||
this(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
|
||||
primaryOperationTimeout, new SmallScannerCallableFactory());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new ShortClientScanner for the specified table. Take note that the passed {@link Scan}
|
||||
* 's start row maybe changed changed. Intended for unit tests to provide their own
|
||||
* {@link SmallScannerCallableFactory} implementation/mock.
|
||||
*
|
||||
* @param conf
|
||||
* The {@link Configuration} to use.
|
||||
* @param scan
|
||||
* {@link Scan} to use in this scanner
|
||||
* @param tableName
|
||||
* The table that we wish to rangeGet
|
||||
* @param connection
|
||||
* Connection identifying the cluster
|
||||
* @param rpcFactory
|
||||
* Factory used to create the {@link RpcRetryingCaller}
|
||||
* @param controllerFactory
|
||||
* Factory used to access RPC payloads
|
||||
* @param pool
|
||||
* Threadpool for RPC threads
|
||||
* @param primaryOperationTimeout
|
||||
* Call timeout
|
||||
* @param callableFactory
|
||||
* Factory used to create the {@link SmallScannerCallable}
|
||||
* @throws IOException
|
||||
*/
|
||||
public ClientSmallScanner(final Configuration conf, final Scan scan,
|
||||
final TableName tableName, ClusterConnection connection,
|
||||
RpcRetryingCallerFactory rpcFactory, RpcControllerFactory controllerFactory,
|
||||
ExecutorService pool, int primaryOperationTimeout) throws IOException {
|
||||
@VisibleForTesting
|
||||
ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName,
|
||||
ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
|
||||
RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout,
|
||||
SmallScannerCallableFactory callableFactory) throws IOException {
|
||||
super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
|
||||
primaryOperationTimeout);
|
||||
primaryOperationTimeout);
|
||||
this.callableFactory = callableFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -97,26 +141,25 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
// Where to start the next getter
|
||||
byte[] localStartKey;
|
||||
int cacheNum = nbRows;
|
||||
skipRowOfFirstResult = null;
|
||||
boolean regionChanged = true;
|
||||
// if we're at end of table, close and return false to stop iterating
|
||||
if (this.currentRegion != null && currentRegionDone) {
|
||||
byte[] endKey = this.currentRegion.getEndKey();
|
||||
if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)
|
||||
|| checkScanStopRow(endKey) || done) {
|
||||
close();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Finished with small scan at " + this.currentRegion);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Finished with small scan at " + this.currentRegion);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
localStartKey = endKey;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Finished with region " + this.currentRegion);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Finished with region " + this.currentRegion);
|
||||
}
|
||||
} else if (this.lastResult != null) {
|
||||
localStartKey = this.lastResult.getRow();
|
||||
skipRowOfFirstResult = this.lastResult.getRow();
|
||||
cacheNum++;
|
||||
regionChanged = false;
|
||||
localStartKey = Bytes.add(lastResult.getRow(), new byte[1]);
|
||||
} else {
|
||||
localStartKey = this.scan.getStartRow();
|
||||
}
|
||||
|
@ -125,32 +168,15 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
LOG.trace("Advancing internal small scanner to startKey at '"
|
||||
+ Bytes.toStringBinary(localStartKey) + "'");
|
||||
}
|
||||
smallScanCallable = getSmallScanCallable(
|
||||
getConnection(), getTable(), scan, getScanMetrics(), localStartKey, cacheNum,
|
||||
rpcControllerFactory, getPool(), getPrimaryOperationTimeout(),
|
||||
getRetries(), getScannerTimeout(), getConf(), caller);
|
||||
if (this.scanMetrics != null && skipRowOfFirstResult == null) {
|
||||
smallScanCallable = callableFactory.getCallable(getConnection(), getTable(), scan,
|
||||
getScanMetrics(), localStartKey, cacheNum, rpcControllerFactory, getPool(),
|
||||
getPrimaryOperationTimeout(), getRetries(), getScannerTimeout(), getConf(), caller);
|
||||
if (this.scanMetrics != null && regionChanged) {
|
||||
this.scanMetrics.countOfRegions.incrementAndGet();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
static ScannerCallableWithReplicas getSmallScanCallable(
|
||||
ClusterConnection connection, TableName table, Scan scan,
|
||||
ScanMetrics scanMetrics, byte[] localStartKey, final int cacheNum,
|
||||
RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout,
|
||||
int retries, int scannerTimeout, Configuration conf, RpcRetryingCaller<Result []> caller) {
|
||||
scan.setStartRow(localStartKey);
|
||||
SmallScannerCallable s = new SmallScannerCallable(
|
||||
connection, table, scan, scanMetrics, controllerFactory, cacheNum, 0);
|
||||
ScannerCallableWithReplicas scannerCallableWithReplicas =
|
||||
new ScannerCallableWithReplicas(table, connection,
|
||||
s, pool, primaryOperationTimeout, scan, retries,
|
||||
scannerTimeout, cacheNum, conf, caller);
|
||||
return scannerCallableWithReplicas;
|
||||
}
|
||||
|
||||
static class SmallScannerCallable extends ScannerCallable {
|
||||
public SmallScannerCallable(
|
||||
ClusterConnection connection, TableName table, Scan scan,
|
||||
|
@ -173,8 +199,15 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
controller.setPriority(getTableName());
|
||||
controller.setCallTimeout(timeout);
|
||||
response = getStub().scan(controller, request);
|
||||
return ResponseConverter.getResults(controller.cellScanner(),
|
||||
Result[] results = ResponseConverter.getResults(controller.cellScanner(),
|
||||
response);
|
||||
if (response.hasMoreResultsInRegion()) {
|
||||
setHasMoreResultsContext(true);
|
||||
setServerHasMoreResults(response.getMoreResultsInRegion());
|
||||
} else {
|
||||
setHasMoreResultsContext(false);
|
||||
}
|
||||
return results;
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -195,45 +228,7 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
return null;
|
||||
}
|
||||
if (cache.size() == 0) {
|
||||
Result[] values = null;
|
||||
long remainingResultSize = maxScannerResultSize;
|
||||
int countdown = this.caching;
|
||||
boolean currentRegionDone = false;
|
||||
// Values == null means server-side filter has determined we must STOP
|
||||
while (remainingResultSize > 0 && countdown > 0
|
||||
&& nextScanner(countdown, values == null, currentRegionDone)) {
|
||||
// Server returns a null values if scanning is to stop. Else,
|
||||
// returns an empty array if scanning is to go on and we've just
|
||||
// exhausted current region.
|
||||
// callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas,
|
||||
// we do a callWithRetries
|
||||
values = this.caller.callWithoutRetries(smallScanCallable, scannerTimeout);
|
||||
this.currentRegion = smallScanCallable.getHRegionInfo();
|
||||
long currentTime = System.currentTimeMillis();
|
||||
if (this.scanMetrics != null) {
|
||||
this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime
|
||||
- lastNext);
|
||||
}
|
||||
lastNext = currentTime;
|
||||
if (values != null && values.length > 0) {
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
Result rs = values[i];
|
||||
if (i == 0 && this.skipRowOfFirstResult != null
|
||||
&& Bytes.equals(skipRowOfFirstResult, rs.getRow())) {
|
||||
// Skip the first result
|
||||
continue;
|
||||
}
|
||||
cache.add(rs);
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
countdown--;
|
||||
this.lastResult = rs;
|
||||
}
|
||||
}
|
||||
currentRegionDone = countdown > 0;
|
||||
}
|
||||
loadCache();
|
||||
}
|
||||
|
||||
if (cache.size() > 0) {
|
||||
|
@ -246,8 +241,75 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void loadCache() throws IOException {
|
||||
Result[] values = null;
|
||||
long remainingResultSize = maxScannerResultSize;
|
||||
int countdown = this.caching;
|
||||
boolean currentRegionDone = false;
|
||||
// Values == null means server-side filter has determined we must STOP
|
||||
while (remainingResultSize > 0 && countdown > 0
|
||||
&& nextScanner(countdown, values == null, currentRegionDone)) {
|
||||
// Server returns a null values if scanning is to stop. Else,
|
||||
// returns an empty array if scanning is to go on and we've just
|
||||
// exhausted current region.
|
||||
// callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas,
|
||||
// we do a callWithRetries
|
||||
values = this.caller.callWithoutRetries(smallScanCallable, scannerTimeout);
|
||||
this.currentRegion = smallScanCallable.getHRegionInfo();
|
||||
long currentTime = System.currentTimeMillis();
|
||||
if (this.scanMetrics != null) {
|
||||
this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime
|
||||
- lastNext);
|
||||
}
|
||||
lastNext = currentTime;
|
||||
if (values != null && values.length > 0) {
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
Result rs = values[i];
|
||||
cache.add(rs);
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
countdown--;
|
||||
this.lastResult = rs;
|
||||
}
|
||||
}
|
||||
if (smallScanCallable.hasMoreResultsContext()) {
|
||||
// If the server has more results, the current region is not done
|
||||
currentRegionDone = !smallScanCallable.getServerHasMoreResults();
|
||||
} else {
|
||||
// not guaranteed to get the context in older versions, fall back to checking countdown
|
||||
currentRegionDone = countdown > 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
if (!scanMetricsPublished) writeScanMetrics();
|
||||
closed = true;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected void setScannerCallableFactory(SmallScannerCallableFactory callableFactory) {
|
||||
this.callableFactory = callableFactory;
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
protected static class SmallScannerCallableFactory {
|
||||
|
||||
public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table,
|
||||
Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum,
|
||||
RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout,
|
||||
int retries, int scannerTimeout, Configuration conf, RpcRetryingCaller<Result[]> caller) {
|
||||
scan.setStartRow(localStartKey);
|
||||
SmallScannerCallable s = new SmallScannerCallable(
|
||||
connection, table, scan, scanMetrics, controllerFactory, cacheNum, 0);
|
||||
ScannerCallableWithReplicas scannerCallableWithReplicas =
|
||||
new ScannerCallableWithReplicas(table, connection,
|
||||
s, pool, primaryOperationTimeout, scan, retries,
|
||||
scannerTimeout, cacheNum, conf, caller);
|
||||
return scannerCallableWithReplicas;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -287,12 +287,6 @@ public interface ClusterConnection extends HConnection {
|
|||
*/
|
||||
RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf);
|
||||
|
||||
/**
|
||||
*
|
||||
* @return true if this is a managed connection.
|
||||
*/
|
||||
boolean isManaged();
|
||||
|
||||
/**
|
||||
* @return the current statistics tracker associated with this connection
|
||||
*/
|
||||
|
@ -302,4 +296,4 @@ public interface ClusterConnection extends HConnection {
|
|||
* @return the configured client backoff policy
|
||||
*/
|
||||
ClientBackoffPolicy getBackoffPolicy();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -74,10 +74,13 @@ public interface Connection extends Abortable, Closeable {
|
|||
* The returned Table is not thread safe, a new instance should be created for each using thread.
|
||||
* This is a lightweight operation, pooling or caching of the returned Table
|
||||
* is neither required nor desired.
|
||||
* <br>
|
||||
* <p>
|
||||
* The caller is responsible for calling {@link Table#close()} on the returned
|
||||
* table instance.
|
||||
*
|
||||
* <p>
|
||||
* Since 0.98.1 this method no longer checks table existence. An exception
|
||||
* will be thrown if the table does not exist only when the first operation is
|
||||
* attempted.
|
||||
* @param tableName the name of the table
|
||||
* @return a Table to use for interactions with this table
|
||||
*/
|
||||
|
@ -88,9 +91,13 @@ public interface Connection extends Abortable, Closeable {
|
|||
* The returned Table is not thread safe, a new instance should be created for each using thread.
|
||||
* This is a lightweight operation, pooling or caching of the returned Table
|
||||
* is neither required nor desired.
|
||||
* <br>
|
||||
* <p>
|
||||
* The caller is responsible for calling {@link Table#close()} on the returned
|
||||
* table instance.
|
||||
* <p>
|
||||
* Since 0.98.1 this method no longer checks table existence. An exception
|
||||
* will be thrown if the table does not exist only when the first operation is
|
||||
* attempted.
|
||||
*
|
||||
* @param tableName the name of the table
|
||||
* @param pool The thread pool to use for batch operations, null to use a default pool.
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
|||
* A convenience to override when customizing method implementations.
|
||||
*
|
||||
*
|
||||
* @see ConnectionUtils#createShortCircuitHConnection(HConnection, ServerName,
|
||||
* @see ConnectionUtils#createShortCircuitHConnection(Connection, ServerName,
|
||||
* AdminService.BlockingInterface, ClientService.BlockingInterface) for case where we make
|
||||
* Connections skip RPC if request is to local server.
|
||||
*/
|
||||
|
@ -455,11 +455,6 @@ abstract class ConnectionAdapter implements ClusterConnection {
|
|||
return wrappedConnection.getNewRpcRetryingCallerFactory(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isManaged() {
|
||||
return wrappedConnection.isManaged();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerStatisticTracker getStatisticsTracker() {
|
||||
return wrappedConnection.getStatisticsTracker();
|
||||
|
|
|
@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.security.UserProvider;
|
|||
* Similarly, {@link Connection} also returns {@link Admin} and {@link RegionLocator}
|
||||
* implementations.
|
||||
*
|
||||
* This class replaces {@link HConnectionManager}, which is now deprecated.
|
||||
* @see Connection
|
||||
* @since 0.99.0
|
||||
*/
|
||||
|
@ -215,15 +214,9 @@ public class ConnectionFactory {
|
|||
user = provider.getCurrent();
|
||||
}
|
||||
|
||||
return createConnection(conf, false, pool, user);
|
||||
}
|
||||
|
||||
static Connection createConnection(final Configuration conf, final boolean managed,
|
||||
final ExecutorService pool, final User user)
|
||||
throws IOException {
|
||||
String className = conf.get(HConnection.HBASE_CLIENT_CONNECTION_IMPL,
|
||||
ConnectionManager.HConnectionImplementation.class.getName());
|
||||
Class<?> clazz = null;
|
||||
ConnectionImplementation.class.getName());
|
||||
Class<?> clazz;
|
||||
try {
|
||||
clazz = Class.forName(className);
|
||||
} catch (ClassNotFoundException e) {
|
||||
|
@ -233,9 +226,9 @@ public class ConnectionFactory {
|
|||
// Default HCM#HCI is not accessible; make it so before invoking.
|
||||
Constructor<?> constructor =
|
||||
clazz.getDeclaredConstructor(Configuration.class,
|
||||
boolean.class, ExecutorService.class, User.class);
|
||||
ExecutorService.class, User.class);
|
||||
constructor.setAccessible(true);
|
||||
return (Connection) constructor.newInstance(conf, managed, pool, user);
|
||||
return (Connection) constructor.newInstance(conf, pool, user);
|
||||
} catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -82,7 +82,7 @@ public final class ConnectionUtils {
|
|||
*/
|
||||
public static NonceGenerator injectNonceGeneratorForTesting(
|
||||
ClusterConnection conn, NonceGenerator cnm) {
|
||||
return ConnectionManager.injectNonceGeneratorForTesting(conn, cnm);
|
||||
return ConnectionImplementation.injectNonceGeneratorForTesting(conn, cnm);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -94,6 +94,7 @@ public final class ConnectionUtils {
|
|||
*/
|
||||
public static void setServerSideHConnectionRetriesConfig(
|
||||
final Configuration c, final String sn, final Log log) {
|
||||
// TODO: Fix this. Not all connections from server side should have 10 times the retries.
|
||||
int hcRetries = c.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
|
||||
// Go big. Multiply by 10. If we can't get to meta after this many retries
|
||||
|
@ -145,10 +146,10 @@ public final class ConnectionUtils {
|
|||
* Some tests shut down the master. But table availability is a master RPC which is performed on
|
||||
* region re-lookups.
|
||||
*/
|
||||
static class MasterlessConnection extends ConnectionManager.HConnectionImplementation {
|
||||
MasterlessConnection(Configuration conf, boolean managed,
|
||||
static class MasterlessConnection extends ConnectionImplementation {
|
||||
MasterlessConnection(Configuration conf,
|
||||
ExecutorService pool, User user) throws IOException {
|
||||
super(conf, managed, pool, user);
|
||||
super(conf, pool, user);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -253,7 +253,9 @@ public class Delete extends Mutation implements Comparable<Row> {
|
|||
* @param family family name
|
||||
* @param timestamp version timestamp
|
||||
* @return this for invocation chaining
|
||||
* @deprecated Since hbase-1.0.0. Use {@link #addFamilyVersion(byte[], long)}
|
||||
*/
|
||||
@Deprecated
|
||||
public Delete deleteFamilyVersion(byte [] family, long timestamp) {
|
||||
return addFamilyVersion(family, timestamp);
|
||||
}
|
||||
|
@ -264,9 +266,7 @@ public class Delete extends Mutation implements Comparable<Row> {
|
|||
* @param family family name
|
||||
* @param timestamp version timestamp
|
||||
* @return this for invocation chaining
|
||||
* @deprecated Since hbase-1.0.0. Use {@link #addFamilyVersion(byte[], long)}
|
||||
*/
|
||||
@Deprecated
|
||||
public Delete addFamilyVersion(final byte [] family, final long timestamp) {
|
||||
List<Cell> list = familyMap.get(family);
|
||||
if(list == null) {
|
||||
|
|
|
@ -0,0 +1,102 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.mortbay.log.Log;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* A Callable for flushRegion() RPC.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionResponse> {
|
||||
|
||||
private final byte[] regionName;
|
||||
private final boolean writeFlushWalMarker;
|
||||
private boolean reload;
|
||||
|
||||
public FlushRegionCallable(ClusterConnection connection,
|
||||
RpcControllerFactory rpcControllerFactory, TableName tableName, byte[] regionName,
|
||||
byte[] regionStartKey, boolean writeFlushWalMarker) {
|
||||
super(connection, rpcControllerFactory, tableName, regionStartKey);
|
||||
this.regionName = regionName;
|
||||
this.writeFlushWalMarker = writeFlushWalMarker;
|
||||
}
|
||||
|
||||
public FlushRegionCallable(ClusterConnection connection,
|
||||
RpcControllerFactory rpcControllerFactory, HRegionInfo regionInfo,
|
||||
boolean writeFlushWalMarker) {
|
||||
this(connection, rpcControllerFactory, regionInfo.getTable(), regionInfo.getRegionName(),
|
||||
regionInfo.getStartKey(), writeFlushWalMarker);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlushRegionResponse call(int callTimeout) throws Exception {
|
||||
return flushRegion();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void prepare(boolean reload) throws IOException {
|
||||
super.prepare(reload);
|
||||
this.reload = reload;
|
||||
}
|
||||
|
||||
private FlushRegionResponse flushRegion() throws IOException {
|
||||
// check whether we should still do the flush to this region. If the regions are changed due
|
||||
// to splits or merges, etc return success
|
||||
if (!Bytes.equals(location.getRegionInfo().getRegionName(), regionName)) {
|
||||
if (!reload) {
|
||||
throw new IOException("Cached location seems to be different than requested region.");
|
||||
}
|
||||
Log.info("Skipping flush region, because the located region "
|
||||
+ Bytes.toStringBinary(location.getRegionInfo().getRegionName()) + " is different than "
|
||||
+ " requested region " + Bytes.toStringBinary(regionName));
|
||||
return FlushRegionResponse.newBuilder()
|
||||
.setLastFlushTime(EnvironmentEdgeManager.currentTime())
|
||||
.setFlushed(false)
|
||||
.setWroteFlushWalMarker(false)
|
||||
.build();
|
||||
}
|
||||
|
||||
FlushRegionRequest request =
|
||||
RequestConverter.buildFlushRegionRequest(regionName, writeFlushWalMarker);
|
||||
|
||||
try {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(tableName);
|
||||
return stub.flushRegion(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -98,6 +98,7 @@ public class Get extends Query
|
|||
* @param get
|
||||
*/
|
||||
public Get(Get get) {
|
||||
this(get.getRow());
|
||||
this.filter = get.getFilter();
|
||||
this.cacheBlocks = get.getCacheBlocks();
|
||||
this.maxVersions = get.getMaxVersions();
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -38,22 +38,16 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
|||
/**
|
||||
* A cluster connection. Knows how to find the master, locate regions out on the cluster,
|
||||
* keeps a cache of locations and then knows how to re-calibrate after they move. You need one
|
||||
* of these to talk to your HBase cluster. {@link HConnectionManager} manages instances of this
|
||||
* of these to talk to your HBase cluster. {@link ConnectionFactory} manages instances of this
|
||||
* class. See it for how to get one of these.
|
||||
*
|
||||
* <p>This is NOT a connection to a particular server but to ALL servers in the cluster. Individual
|
||||
* connections are managed at a lower level.
|
||||
*
|
||||
* <p>HConnections are used by {@link HTable} mostly but also by
|
||||
* {@link HBaseAdmin}, and {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}.
|
||||
* HConnection instances can be shared. Sharing
|
||||
* is usually what you want because rather than each HConnection instance
|
||||
* having to do its own discovery of regions out on the cluster, instead, all
|
||||
* clients get to share the one cache of locations. {@link HConnectionManager} does the
|
||||
* sharing for you if you go by it getting connections. Sharing makes cleanup of
|
||||
* HConnections awkward. See {@link HConnectionManager} for cleanup discussion.
|
||||
* {@link HBaseAdmin}, and {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}.
|
||||
*
|
||||
* @see HConnectionManager
|
||||
* @see ConnectionFactory
|
||||
* @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
|
@ -78,8 +72,7 @@ public interface HConnection extends Connection {
|
|||
* be created for each using thread.
|
||||
* This is a lightweight operation, pooling or caching of the returned HTableInterface
|
||||
* is neither required nor desired.
|
||||
* Note that the HConnection needs to be unmanaged
|
||||
* (created with {@link HConnectionManager#createConnection(Configuration)}).
|
||||
* (created with {@link ConnectionFactory#createConnection(Configuration)}).
|
||||
* @param tableName
|
||||
* @return an HTable to use for interactions with this table
|
||||
*/
|
||||
|
@ -91,8 +84,7 @@ public interface HConnection extends Connection {
|
|||
* be created for each using thread.
|
||||
* This is a lightweight operation, pooling or caching of the returned HTableInterface
|
||||
* is neither required nor desired.
|
||||
* Note that the HConnection needs to be unmanaged
|
||||
* (created with {@link HConnectionManager#createConnection(Configuration)}).
|
||||
* (created with {@link ConnectionFactory#createConnection(Configuration)}).
|
||||
* @param tableName
|
||||
* @return an HTable to use for interactions with this table
|
||||
*/
|
||||
|
@ -104,8 +96,7 @@ public interface HConnection extends Connection {
|
|||
* be created for each using thread.
|
||||
* This is a lightweight operation, pooling or caching of the returned HTableInterface
|
||||
* is neither required nor desired.
|
||||
* Note that the HConnection needs to be unmanaged
|
||||
* (created with {@link HConnectionManager#createConnection(Configuration)}).
|
||||
* (created with {@link ConnectionFactory#createConnection(Configuration)}).
|
||||
* @param tableName
|
||||
* @return an HTable to use for interactions with this table
|
||||
*/
|
||||
|
@ -118,8 +109,7 @@ public interface HConnection extends Connection {
|
|||
* be created for each using thread.
|
||||
* This is a lightweight operation, pooling or caching of the returned HTableInterface
|
||||
* is neither required nor desired.
|
||||
* Note that the HConnection needs to be unmanaged
|
||||
* (created with {@link HConnectionManager#createConnection(Configuration)}).
|
||||
* (created with {@link ConnectionFactory#createConnection(Configuration)}).
|
||||
* @param tableName
|
||||
* @param pool The thread pool to use for batch operations, null to use a default pool.
|
||||
* @return an HTable to use for interactions with this table
|
||||
|
@ -132,8 +122,7 @@ public interface HConnection extends Connection {
|
|||
* be created for each using thread.
|
||||
* This is a lightweight operation, pooling or caching of the returned HTableInterface
|
||||
* is neither required nor desired.
|
||||
* Note that the HConnection needs to be unmanaged
|
||||
* (created with {@link HConnectionManager#createConnection(Configuration)}).
|
||||
* (created with {@link ConnectionFactory#createConnection(Configuration)}).
|
||||
* @param tableName
|
||||
* @param pool The thread pool to use for batch operations, null to use a default pool.
|
||||
* @return an HTable to use for interactions with this table
|
||||
|
@ -146,9 +135,8 @@ public interface HConnection extends Connection {
|
|||
* be created for each using thread.
|
||||
* This is a lightweight operation, pooling or caching of the returned HTableInterface
|
||||
* is neither required nor desired.
|
||||
* Note that the HConnection needs to be unmanaged
|
||||
* (created with {@link HConnectionManager#createConnection(Configuration)}).
|
||||
* @param tableName
|
||||
* (created with {@link ConnectionFactory#createConnection(Configuration)}).
|
||||
* @param tableName table to get interface for
|
||||
* @param pool The thread pool to use for batch operations, null to use a default pool.
|
||||
* @return an HTable to use for interactions with this table
|
||||
*/
|
||||
|
@ -161,10 +149,6 @@ public interface HConnection extends Connection {
|
|||
*
|
||||
* This is a lightweight operation. Pooling or caching of the returned RegionLocator is neither
|
||||
* required nor desired.
|
||||
*
|
||||
* RegionLocator needs to be unmanaged
|
||||
* (created with {@link HConnectionManager#createConnection(Configuration)}).
|
||||
*
|
||||
* @param tableName Name of the table who's region is to be examined
|
||||
* @return A RegionLocator instance
|
||||
*/
|
||||
|
@ -175,7 +159,7 @@ public interface HConnection extends Connection {
|
|||
* Retrieve an Admin implementation to administer an HBase cluster.
|
||||
* The returned Admin is not guaranteed to be thread-safe. A new instance should be created for
|
||||
* each using thread. This is a lightweight operation. Pooling or caching of the returned
|
||||
* Admin is not recommended. Note that HConnection needs to be unmanaged
|
||||
* Admin is not recommended.
|
||||
*
|
||||
* @return an Admin instance for cluster administration
|
||||
*/
|
||||
|
|
|
@ -1,146 +0,0 @@
|
|||
/**
|
||||
* Copyright The Apache Software Foundation
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
|
||||
/**
|
||||
* Denotes a unique key to an {@link HConnection} instance.
|
||||
*
|
||||
* In essence, this class captures the properties in {@link Configuration}
|
||||
* that may be used in the process of establishing a connection. In light of
|
||||
* that, if any new such properties are introduced into the mix, they must be
|
||||
* added to the {@link HConnectionKey#properties} list.
|
||||
*
|
||||
*/
|
||||
class HConnectionKey {
|
||||
final static String[] CONNECTION_PROPERTIES = new String[] {
|
||||
HConstants.ZOOKEEPER_QUORUM, HConstants.ZOOKEEPER_ZNODE_PARENT,
|
||||
HConstants.ZOOKEEPER_CLIENT_PORT,
|
||||
HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME,
|
||||
HConstants.HBASE_CLIENT_PAUSE, HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
||||
HConstants.HBASE_RPC_TIMEOUT_KEY,
|
||||
HConstants.HBASE_META_SCANNER_CACHING,
|
||||
HConstants.HBASE_CLIENT_INSTANCE_ID,
|
||||
HConstants.RPC_CODEC_CONF_KEY,
|
||||
HConstants.USE_META_REPLICAS,
|
||||
RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY};
|
||||
|
||||
private Map<String, String> properties;
|
||||
private String username;
|
||||
|
||||
HConnectionKey(Configuration conf) {
|
||||
Map<String, String> m = new HashMap<String, String>();
|
||||
if (conf != null) {
|
||||
for (String property : CONNECTION_PROPERTIES) {
|
||||
String value = conf.get(property);
|
||||
if (value != null) {
|
||||
m.put(property, value);
|
||||
}
|
||||
}
|
||||
}
|
||||
this.properties = Collections.unmodifiableMap(m);
|
||||
|
||||
try {
|
||||
UserProvider provider = UserProvider.instantiate(conf);
|
||||
User currentUser = provider.getCurrent();
|
||||
if (currentUser != null) {
|
||||
username = currentUser.getName();
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
ConnectionManager.LOG.warn(
|
||||
"Error obtaining current user, skipping username in HConnectionKey", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
if (username != null) {
|
||||
result = username.hashCode();
|
||||
}
|
||||
for (String property : CONNECTION_PROPERTIES) {
|
||||
String value = properties.get(property);
|
||||
if (value != null) {
|
||||
result = prime * result + value.hashCode();
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings (value="ES_COMPARING_STRINGS_WITH_EQ",
|
||||
justification="Optimization")
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj)
|
||||
return true;
|
||||
if (obj == null)
|
||||
return false;
|
||||
if (getClass() != obj.getClass())
|
||||
return false;
|
||||
HConnectionKey that = (HConnectionKey) obj;
|
||||
if (this.username != null && !this.username.equals(that.username)) {
|
||||
return false;
|
||||
} else if (this.username == null && that.username != null) {
|
||||
return false;
|
||||
}
|
||||
if (this.properties == null) {
|
||||
if (that.properties != null) {
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
if (that.properties == null) {
|
||||
return false;
|
||||
}
|
||||
for (String property : CONNECTION_PROPERTIES) {
|
||||
String thisValue = this.properties.get(property);
|
||||
String thatValue = that.properties.get(property);
|
||||
//noinspection StringEquality
|
||||
if (thisValue == thatValue) {
|
||||
continue;
|
||||
}
|
||||
if (thisValue == null || !thisValue.equals(thatValue)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HConnectionKey{" +
|
||||
"properties=" + properties +
|
||||
", username='" + username + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -1,324 +0,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.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
|
||||
/**
|
||||
* A non-instantiable class that manages creation of {@link HConnection}s.
|
||||
* <p>The simplest way to use this class is by using {@link #createConnection(Configuration)}.
|
||||
* This creates a new {@link HConnection} to the cluster that is managed by the caller.
|
||||
* From this {@link HConnection} {@link HTableInterface} implementations are retrieved
|
||||
* with {@link HConnection#getTable(byte[])}. Example:
|
||||
* <pre>
|
||||
* HConnection connection = HConnectionManager.createConnection(config);
|
||||
* HTableInterface table = connection.getTable(TableName.valueOf("table1"));
|
||||
* try {
|
||||
* // Use the table as needed, for a single operation and a single thread
|
||||
* } finally {
|
||||
* table.close();
|
||||
* connection.close();
|
||||
* }
|
||||
* </pre>
|
||||
* <p>This class has a static Map of {@link HConnection} instances keyed by
|
||||
* {@link HConnectionKey}; A {@link HConnectionKey} is identified by a set of
|
||||
* {@link Configuration} properties. Invocations of {@link #getConnection(Configuration)}
|
||||
* that pass the same {@link Configuration} instance will return the same
|
||||
* {@link HConnection} instance ONLY WHEN the set of properties are the same
|
||||
* (i.e. if you change properties in your {@link Configuration} instance, such as RPC timeout,
|
||||
* the codec used, HBase will create a new {@link HConnection} instance. For more details on
|
||||
* how this is done see {@link HConnectionKey}).
|
||||
* <p>Sharing {@link HConnection} instances is usually what you want; all clients
|
||||
* of the {@link HConnection} instances share the HConnections' cache of Region
|
||||
* locations rather than each having to discover for itself the location of meta, etc.
|
||||
* But sharing connections makes clean up of {@link HConnection} instances a little awkward.
|
||||
* Currently, clients cleanup by calling {@link #deleteConnection(Configuration)}. This will
|
||||
* shutdown the zookeeper connection the HConnection was using and clean up all
|
||||
* HConnection resources as well as stopping proxies to servers out on the
|
||||
* cluster. Not running the cleanup will not end the world; it'll
|
||||
* just stall the closeup some and spew some zookeeper connection failed
|
||||
* messages into the log. Running the cleanup on a {@link HConnection} that is
|
||||
* subsequently used by another will cause breakage so be careful running
|
||||
* cleanup.
|
||||
* <p>To create a {@link HConnection} that is not shared by others, you can
|
||||
* set property "hbase.client.instance.id" to a unique value for your {@link Configuration}
|
||||
* instance, like the following:
|
||||
* <pre>
|
||||
* {@code
|
||||
* conf.set("hbase.client.instance.id", "12345");
|
||||
* HConnection connection = HConnectionManager.getConnection(conf);
|
||||
* // Use the connection to your hearts' delight and then when done...
|
||||
* conf.set("hbase.client.instance.id", "12345");
|
||||
* HConnectionManager.deleteConnection(conf, true);
|
||||
* }
|
||||
* </pre>
|
||||
* <p>Cleanup used to be done inside in a shutdown hook. On startup we'd
|
||||
* register a shutdown hook that called {@link #deleteAllConnections()}
|
||||
* on its way out but the order in which shutdown hooks run is not defined so
|
||||
* were problematic for clients of HConnection that wanted to register their
|
||||
* own shutdown hooks so we removed ours though this shifts the onus for
|
||||
* cleanup to the client.
|
||||
* @deprecated Please use ConnectionFactory instead
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
@Deprecated
|
||||
public final class HConnectionManager extends ConnectionFactory {
|
||||
|
||||
/** @deprecated connection caching is going away */
|
||||
@Deprecated
|
||||
public static final String RETRIES_BY_SERVER_KEY =
|
||||
ConnectionManager.RETRIES_BY_SERVER_KEY;
|
||||
|
||||
/** @deprecated connection caching is going away */
|
||||
@Deprecated
|
||||
public static final int MAX_CACHED_CONNECTION_INSTANCES =
|
||||
ConnectionManager.MAX_CACHED_CONNECTION_INSTANCES;
|
||||
|
||||
/*
|
||||
* Non-instantiable.
|
||||
*/
|
||||
private HConnectionManager() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the connection that goes with the passed <code>conf</code> configuration instance.
|
||||
* If no current connection exists, method creates a new connection and keys it using
|
||||
* connection-specific properties from the passed {@link Configuration}; see
|
||||
* {@link HConnectionKey}.
|
||||
* @param conf configuration
|
||||
* @return HConnection object for <code>conf</code>
|
||||
* @deprecated connection caching is going away
|
||||
*/
|
||||
@Deprecated
|
||||
public static HConnection getConnection(final Configuration conf) throws IOException {
|
||||
return ConnectionManager.getConnectionInternal(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new HConnection instance using the passed <code>conf</code> instance.
|
||||
* <p>Note: This bypasses the usual HConnection life cycle management done by
|
||||
* {@link #getConnection(Configuration)}. The caller is responsible for
|
||||
* calling {@link HConnection#close()} on the returned connection instance.
|
||||
*
|
||||
* This is the recommended way to create HConnections.
|
||||
* <pre>
|
||||
* HConnection connection = HConnectionManager.createConnection(conf);
|
||||
* HTableInterface table = connection.getTable("mytable");
|
||||
* try {
|
||||
* table.get(...);
|
||||
* ...
|
||||
* } finally {
|
||||
* table.close();
|
||||
* connection.close();
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* @param conf configuration
|
||||
* @return HConnection object for <code>conf</code>
|
||||
* @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
|
||||
*/
|
||||
@Deprecated
|
||||
public static HConnection createConnection(Configuration conf) throws IOException {
|
||||
return ConnectionManager.createConnectionInternal(conf);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a new HConnection instance using the passed <code>conf</code> instance.
|
||||
* <p>Note: This bypasses the usual HConnection life cycle management done by
|
||||
* {@link #getConnection(Configuration)}. The caller is responsible for
|
||||
* calling {@link HConnection#close()} on the returned connection instance.
|
||||
* This is the recommended way to create HConnections.
|
||||
* <pre>
|
||||
* ExecutorService pool = ...;
|
||||
* HConnection connection = HConnectionManager.createConnection(conf, pool);
|
||||
* HTableInterface table = connection.getTable("mytable");
|
||||
* table.get(...);
|
||||
* ...
|
||||
* table.close();
|
||||
* connection.close();
|
||||
* </pre>
|
||||
* @param conf configuration
|
||||
* @param pool the thread pool to use for batch operation in HTables used via this HConnection
|
||||
* @return HConnection object for <code>conf</code>
|
||||
* @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
|
||||
*/
|
||||
@Deprecated
|
||||
public static HConnection createConnection(Configuration conf, ExecutorService pool)
|
||||
throws IOException {
|
||||
return ConnectionManager.createConnection(conf, pool);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new HConnection instance using the passed <code>conf</code> instance.
|
||||
* <p>Note: This bypasses the usual HConnection life cycle management done by
|
||||
* {@link #getConnection(Configuration)}. The caller is responsible for
|
||||
* calling {@link HConnection#close()} on the returned connection instance.
|
||||
* This is the recommended way to create HConnections.
|
||||
* <pre>
|
||||
* ExecutorService pool = ...;
|
||||
* HConnection connection = HConnectionManager.createConnection(conf, pool);
|
||||
* HTableInterface table = connection.getTable("mytable");
|
||||
* table.get(...);
|
||||
* ...
|
||||
* table.close();
|
||||
* connection.close();
|
||||
* </pre>
|
||||
* @param conf configuration
|
||||
* @param user the user the connection is for
|
||||
* @return HConnection object for <code>conf</code>
|
||||
* @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
|
||||
*/
|
||||
@Deprecated
|
||||
public static HConnection createConnection(Configuration conf, User user)
|
||||
throws IOException {
|
||||
return ConnectionManager.createConnection(conf, user);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new HConnection instance using the passed <code>conf</code> instance.
|
||||
* <p>Note: This bypasses the usual HConnection life cycle management done by
|
||||
* {@link #getConnection(Configuration)}. The caller is responsible for
|
||||
* calling {@link HConnection#close()} on the returned connection instance.
|
||||
* This is the recommended way to create HConnections.
|
||||
* <pre>
|
||||
* ExecutorService pool = ...;
|
||||
* HConnection connection = HConnectionManager.createConnection(conf, pool);
|
||||
* HTableInterface table = connection.getTable("mytable");
|
||||
* table.get(...);
|
||||
* ...
|
||||
* table.close();
|
||||
* connection.close();
|
||||
* </pre>
|
||||
* @param conf configuration
|
||||
* @param pool the thread pool to use for batch operation in HTables used via this HConnection
|
||||
* @param user the user the connection is for
|
||||
* @return HConnection object for <code>conf</code>
|
||||
* @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
|
||||
*/
|
||||
@Deprecated
|
||||
public static HConnection createConnection(Configuration conf, ExecutorService pool, User user)
|
||||
throws IOException {
|
||||
return ConnectionManager.createConnection(conf, pool, user);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
|
||||
*/
|
||||
@Deprecated
|
||||
static HConnection createConnection(final Configuration conf, final boolean managed)
|
||||
throws IOException {
|
||||
return ConnectionManager.createConnection(conf, managed);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
|
||||
*/
|
||||
@Deprecated
|
||||
static ClusterConnection createConnection(final Configuration conf, final boolean managed,
|
||||
final ExecutorService pool, final User user) throws IOException {
|
||||
return ConnectionManager.createConnection(conf, managed, pool, user);
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete connection information for the instance specified by passed configuration.
|
||||
* If there are no more references to the designated connection connection, this method will
|
||||
* then close connection to the zookeeper ensemble and let go of all associated resources.
|
||||
*
|
||||
* @param conf configuration whose identity is used to find {@link HConnection} instance.
|
||||
* @deprecated connection caching is going away.
|
||||
*/
|
||||
@Deprecated
|
||||
public static void deleteConnection(Configuration conf) {
|
||||
ConnectionManager.deleteConnection(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanup a known stale connection.
|
||||
* This will then close connection to the zookeeper ensemble and let go of all resources.
|
||||
*
|
||||
* @param connection
|
||||
* @deprecated connection caching is going away.
|
||||
*/
|
||||
@Deprecated
|
||||
public static void deleteStaleConnection(HConnection connection) {
|
||||
ConnectionManager.deleteStaleConnection(connection);
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete information for all connections. Close or not the connection, depending on the
|
||||
* staleConnection boolean and the ref count. By default, you should use it with
|
||||
* staleConnection to true.
|
||||
* @deprecated connection caching is going away.
|
||||
*/
|
||||
@Deprecated
|
||||
public static void deleteAllConnections(boolean staleConnection) {
|
||||
ConnectionManager.deleteAllConnections(staleConnection);
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete information for all connections..
|
||||
* @deprecated kept for backward compatibility, but the behavior is broken. HBASE-8983
|
||||
*/
|
||||
@Deprecated
|
||||
public static void deleteAllConnections() {
|
||||
ConnectionManager.deleteAllConnections();
|
||||
}
|
||||
|
||||
/**
|
||||
* This convenience method invokes the given {@link HConnectable#connect}
|
||||
* implementation using a {@link HConnection} instance that lasts just for the
|
||||
* duration of the invocation.
|
||||
*
|
||||
* @param <T> the return type of the connect method
|
||||
* @param connectable the {@link HConnectable} instance
|
||||
* @return the value returned by the connect method
|
||||
* @throws IOException
|
||||
* @deprecated Internal method, do not use thru HConnectionManager.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@Deprecated
|
||||
public static <T> T execute(HConnectable<T> connectable) throws IOException {
|
||||
return ConnectionManager.execute(connectable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the number of retries to use serverside when trying to communicate
|
||||
* with another server over {@link HConnection}. Used updating catalog
|
||||
* tables, etc. Call this method before we create any Connections.
|
||||
* @param c The Configuration instance to set the retries into.
|
||||
* @param log Used to log what we set in here.
|
||||
* @deprecated Internal method, do not use.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@Deprecated
|
||||
public static void setServerSideHConnectionRetries(
|
||||
final Configuration c, final String sn, final Log log) {
|
||||
ConnectionUtils.setServerSideHConnectionRetriesConfig(c, sn, log);
|
||||
}
|
||||
}
|
|
@ -21,12 +21,12 @@ package org.apache.hadoop.hbase.client;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -34,8 +34,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* An implementation of {@link RegionLocator}. Used to view region location information for a single
|
||||
* HBase table. Lightweight. Get as needed and just close when done. Instances of this class SHOULD
|
||||
|
@ -85,11 +83,11 @@ public class HRegionLocator implements RegionLocator {
|
|||
|
||||
@Override
|
||||
public List<HRegionLocation> getAllRegionLocations() throws IOException {
|
||||
NavigableMap<HRegionInfo, ServerName> locations =
|
||||
MetaScanner.allTableRegions(this.connection, getName());
|
||||
List<Pair<HRegionInfo, ServerName>> locations =
|
||||
MetaTableAccessor.getTableRegionsAndLocations(this.connection, getName());
|
||||
ArrayList<HRegionLocation> regions = new ArrayList<>(locations.size());
|
||||
for (Entry<HRegionInfo, ServerName> entry : locations.entrySet()) {
|
||||
regions.add(new HRegionLocation(entry.getKey(), entry.getValue()));
|
||||
for (Pair<HRegionInfo, ServerName> entry : locations) {
|
||||
regions.add(new HRegionLocation(entry.getFirst(), entry.getSecond()));
|
||||
}
|
||||
return regions;
|
||||
}
|
||||
|
@ -139,7 +137,18 @@ public class HRegionLocator implements RegionLocator {
|
|||
|
||||
@VisibleForTesting
|
||||
List<RegionLocations> listRegionLocations() throws IOException {
|
||||
return MetaScanner.listTableRegionLocations(getConfiguration(), this.connection, getName());
|
||||
final List<RegionLocations> regions = new ArrayList<RegionLocations>();
|
||||
MetaTableAccessor.Visitor visitor = new MetaTableAccessor.TableVisitorBase(tableName) {
|
||||
@Override
|
||||
public boolean visitInternal(Result result) throws IOException {
|
||||
RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
|
||||
if (locations == null) return true;
|
||||
regions.add(locations);
|
||||
return true;
|
||||
}
|
||||
};
|
||||
MetaTableAccessor.scanMetaForTableRegions(connection, visitor, tableName);
|
||||
return regions;
|
||||
}
|
||||
|
||||
public Configuration getConfiguration() {
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
|
@ -127,78 +128,6 @@ public class HTable implements HTableInterface {
|
|||
private RpcRetryingCallerFactory rpcCallerFactory;
|
||||
private RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
/**
|
||||
* Creates an object to access a HBase table.
|
||||
* @param conf Configuration object to use.
|
||||
* @param tableName Name of the table.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated Constructing HTable objects manually has been deprecated. Please use
|
||||
* {@link Connection} to instantiate a {@link Table} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public HTable(Configuration conf, final String tableName)
|
||||
throws IOException {
|
||||
this(conf, TableName.valueOf(tableName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an object to access a HBase table.
|
||||
* @param conf Configuration object to use.
|
||||
* @param tableName Name of the table.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated Constructing HTable objects manually has been deprecated. Please use
|
||||
* {@link Connection} to instantiate a {@link Table} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public HTable(Configuration conf, final byte[] tableName)
|
||||
throws IOException {
|
||||
this(conf, TableName.valueOf(tableName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an object to access a HBase table.
|
||||
* @param conf Configuration object to use.
|
||||
* @param tableName table name pojo
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated Constructing HTable objects manually has been deprecated. Please use
|
||||
* {@link Connection} to instantiate a {@link Table} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public HTable(Configuration conf, final TableName tableName)
|
||||
throws IOException {
|
||||
this.tableName = tableName;
|
||||
this.cleanupPoolOnClose = true;
|
||||
this.cleanupConnectionOnClose = true;
|
||||
if (conf == null) {
|
||||
this.connection = null;
|
||||
return;
|
||||
}
|
||||
this.connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
|
||||
this.configuration = conf;
|
||||
|
||||
this.pool = getDefaultExecutor(conf);
|
||||
this.finishSetup();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an object to access a HBase table.
|
||||
* @param tableName Name of the table.
|
||||
* @param connection HConnection to be used.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated Do not use.
|
||||
*/
|
||||
@Deprecated
|
||||
public HTable(TableName tableName, Connection connection) throws IOException {
|
||||
this.tableName = tableName;
|
||||
this.cleanupPoolOnClose = true;
|
||||
this.cleanupConnectionOnClose = false;
|
||||
this.connection = (ClusterConnection)connection;
|
||||
this.configuration = connection.getConfiguration();
|
||||
|
||||
this.pool = getDefaultExecutor(this.configuration);
|
||||
this.finishSetup();
|
||||
}
|
||||
|
||||
// Marked Private @since 1.0
|
||||
@InterfaceAudience.Private
|
||||
public static ThreadPoolExecutor getDefaultExecutor(Configuration conf) {
|
||||
|
@ -218,68 +147,6 @@ public class HTable implements HTableInterface {
|
|||
return pool;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an object to access a HBase table.
|
||||
* @param conf Configuration object to use.
|
||||
* @param tableName Name of the table.
|
||||
* @param pool ExecutorService to be used.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated Constructing HTable objects manually has been deprecated. Please use
|
||||
* {@link Connection} to instantiate a {@link Table} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public HTable(Configuration conf, final byte[] tableName, final ExecutorService pool)
|
||||
throws IOException {
|
||||
this(conf, TableName.valueOf(tableName), pool);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an object to access a HBase table.
|
||||
* @param conf Configuration object to use.
|
||||
* @param tableName Name of the table.
|
||||
* @param pool ExecutorService to be used.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated Constructing HTable objects manually has been deprecated. Please use
|
||||
* {@link Connection} to instantiate a {@link Table} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public HTable(Configuration conf, final TableName tableName, final ExecutorService pool)
|
||||
throws IOException {
|
||||
this.connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
|
||||
this.configuration = conf;
|
||||
this.pool = pool;
|
||||
if (pool == null) {
|
||||
this.pool = getDefaultExecutor(conf);
|
||||
this.cleanupPoolOnClose = true;
|
||||
} else {
|
||||
this.cleanupPoolOnClose = false;
|
||||
}
|
||||
this.tableName = tableName;
|
||||
this.cleanupConnectionOnClose = true;
|
||||
this.finishSetup();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an object to access a HBase table.
|
||||
* @param tableName Name of the table.
|
||||
* @param connection HConnection to be used.
|
||||
* @param pool ExecutorService to be used.
|
||||
* @throws IOException if a remote or network exception occurs.
|
||||
* @deprecated Do not use, internal ctor.
|
||||
*/
|
||||
@Deprecated
|
||||
public HTable(final byte[] tableName, final Connection connection,
|
||||
final ExecutorService pool) throws IOException {
|
||||
this(TableName.valueOf(tableName), connection, pool);
|
||||
}
|
||||
|
||||
/** @deprecated Do not use, internal ctor. */
|
||||
@Deprecated
|
||||
public HTable(TableName tableName, final Connection connection,
|
||||
final ExecutorService pool) throws IOException {
|
||||
this(tableName, (ClusterConnection)connection, null, null, null, pool);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an object to access a HBase table.
|
||||
* Used by HBase internally. DO NOT USE. See {@link ConnectionFactory} class comment for how to
|
||||
|
@ -290,7 +157,7 @@ public class HTable implements HTableInterface {
|
|||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public HTable(TableName tableName, final ClusterConnection connection,
|
||||
protected HTable(TableName tableName, final ClusterConnection connection,
|
||||
final TableConfiguration tableConfig,
|
||||
final RpcRetryingCallerFactory rpcCallerFactory,
|
||||
final RpcControllerFactory rpcControllerFactory,
|
||||
|
@ -452,12 +319,9 @@ public class HTable implements HTableInterface {
|
|||
@Deprecated
|
||||
public static boolean isTableEnabled(Configuration conf,
|
||||
final TableName tableName) throws IOException {
|
||||
return HConnectionManager.execute(new HConnectable<Boolean>(conf) {
|
||||
@Override
|
||||
public Boolean connect(HConnection connection) throws IOException {
|
||||
return connection.isTableEnabled(tableName);
|
||||
}
|
||||
});
|
||||
try(Connection conn = ConnectionFactory.createConnection(conf)) {
|
||||
return conn.getAdmin().isTableEnabled(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -517,17 +381,6 @@ public class HTable implements HTableInterface {
|
|||
return this.connection;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of rows that a scanner will fetch at once.
|
||||
* <p>
|
||||
* The default value comes from {@code hbase.client.scanner.caching}.
|
||||
* @deprecated Use {@link Scan#setCaching(int)} and {@link Scan#getCaching()}
|
||||
*/
|
||||
@Deprecated
|
||||
public int getScannerCaching() {
|
||||
return scannerCaching;
|
||||
}
|
||||
|
||||
/**
|
||||
* Kept in 0.96 for backward compatibility
|
||||
* @deprecated since 0.96. This is an internal buffer that should not be read nor write.
|
||||
|
@ -537,22 +390,6 @@ public class HTable implements HTableInterface {
|
|||
return mutator == null ? null : mutator.getWriteBuffer();
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the number of rows that a scanner will fetch at once.
|
||||
* <p>
|
||||
* This will override the value specified by
|
||||
* {@code hbase.client.scanner.caching}.
|
||||
* Increasing this value will reduce the amount of work needed each time
|
||||
* {@code next()} is called on a scanner, at the expense of memory use
|
||||
* (since more rows will need to be maintained in memory by the scanners).
|
||||
* @param scannerCaching the number of rows a scanner will fetch at once.
|
||||
* @deprecated Use {@link Scan#setCaching(int)}
|
||||
*/
|
||||
@Deprecated
|
||||
public void setScannerCaching(int scannerCaching) {
|
||||
this.scannerCaching = scannerCaching;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
|
@ -625,11 +462,12 @@ public class HTable implements HTableInterface {
|
|||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated This is no longer a public API. Use {@link #getAllRegionLocations()} instead.
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
@Deprecated
|
||||
public NavigableMap<HRegionInfo, ServerName> getRegionLocations() throws IOException {
|
||||
// TODO: Odd that this returns a Map of HRI to SN whereas getRegionLocator, singular,
|
||||
// returns an HRegionLocation.
|
||||
return MetaScanner.allTableRegions(this.connection, getName());
|
||||
return MetaTableAccessor.allTableRegions(this.connection, getName());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -775,7 +613,7 @@ public class HTable implements HTableInterface {
|
|||
throw new IllegalArgumentException("Small scan should not be used with batching");
|
||||
}
|
||||
if (scan.getCaching() <= 0) {
|
||||
scan.setCaching(getScannerCaching());
|
||||
scan.setCaching(scannerCaching);
|
||||
}
|
||||
|
||||
if (scan.isReversed()) {
|
||||
|
@ -1038,7 +876,15 @@ public class HTable implements HTableInterface {
|
|||
regionMutationBuilder.setAtomic(true);
|
||||
MultiRequest request =
|
||||
MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
|
||||
getStub().multi(controller, request);
|
||||
ClientProtos.MultiResponse response = getStub().multi(controller, request);
|
||||
ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
|
||||
if (res.hasException()) {
|
||||
Throwable ex = ProtobufUtil.toException(res.getException());
|
||||
if(ex instanceof IOException) {
|
||||
throw (IOException)ex;
|
||||
}
|
||||
throw new IOException("Failed to mutate row: "+Bytes.toStringBinary(rm.getRow()), ex);
|
||||
}
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -1317,6 +1163,15 @@ public class HTable implements HTableInterface {
|
|||
getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
|
||||
new BinaryComparator(value), compareType, rm);
|
||||
ClientProtos.MultiResponse response = getStub().multi(controller, request);
|
||||
ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
|
||||
if (res.hasException()) {
|
||||
Throwable ex = ProtobufUtil.toException(res.getException());
|
||||
if(ex instanceof IOException) {
|
||||
throw (IOException)ex;
|
||||
}
|
||||
throw new IOException("Failed to checkAndMutate row: "+
|
||||
Bytes.toStringBinary(rm.getRow()), ex);
|
||||
}
|
||||
return Boolean.valueOf(response.getProcessed());
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
|
@ -1438,6 +1293,7 @@ public class HTable implements HTableInterface {
|
|||
terminated = this.pool.awaitTermination(60, TimeUnit.SECONDS);
|
||||
} while (!terminated);
|
||||
} catch (InterruptedException e) {
|
||||
this.pool.shutdownNow();
|
||||
LOG.warn("waitForTermination interrupted");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -53,6 +53,8 @@ import org.apache.hadoop.hbase.util.ClassSize;
|
|||
public class Increment extends Mutation implements Comparable<Row> {
|
||||
private static final long HEAP_OVERHEAD = ClassSize.REFERENCE + ClassSize.TIMERANGE;
|
||||
|
||||
private static final String RETURN_RESULTS = "_rr_";
|
||||
|
||||
private TimeRange tr = new TimeRange();
|
||||
|
||||
/**
|
||||
|
@ -161,6 +163,24 @@ public class Increment extends Mutation implements Comparable<Row> {
|
|||
tr = new TimeRange(minStamp, maxStamp);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param returnResults True (default) if the increment operation should return the results. A
|
||||
* client that is not interested in the result can save network bandwidth setting this
|
||||
* to false.
|
||||
*/
|
||||
public Increment setReturnResults(boolean returnResults) {
|
||||
setAttribute(RETURN_RESULTS, Bytes.toBytes(returnResults));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return current value for returnResults
|
||||
*/
|
||||
public boolean isReturnResults() {
|
||||
byte[] v = getAttribute(RETURN_RESULTS);
|
||||
return v == null ? true : Bytes.toBoolean(v);
|
||||
}
|
||||
|
||||
/**
|
||||
* Method for retrieving the number of families to increment from
|
||||
|
|
|
@ -1,425 +0,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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Scanner class that contains the <code>hbase:meta</code> table scanning logic.
|
||||
* Provided visitors will be called for each row.
|
||||
*
|
||||
* Although public visibility, this is not a public-facing API and may evolve in
|
||||
* minor releases.
|
||||
*
|
||||
* <p> Note that during concurrent region splits, the scanner might not see
|
||||
* hbase:meta changes across rows (for parent and daughter entries) consistently.
|
||||
* see HBASE-5986, and {@link DefaultMetaScannerVisitor} for details. </p>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
//TODO: merge this to MetaTableAccessor, get rid of it.
|
||||
public final class MetaScanner {
|
||||
private static final Log LOG = LogFactory.getLog(MetaScanner.class);
|
||||
|
||||
private MetaScanner() {}
|
||||
|
||||
/**
|
||||
* Scans the meta table and calls a visitor on each RowResult and uses a empty
|
||||
* start row value as table name.
|
||||
*
|
||||
* <p>Visible for testing. Use {@link
|
||||
* #metaScan(Connection, MetaScannerVisitor, TableName)} instead.
|
||||
*
|
||||
* @param visitor A custom visitor
|
||||
* @throws IOException e
|
||||
*/
|
||||
@VisibleForTesting // Do not use. Used by tests only and hbck.
|
||||
public static void metaScan(Connection connection,
|
||||
MetaScannerVisitor visitor) throws IOException {
|
||||
metaScan(connection, visitor, null, null, Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Scans the meta table and calls a visitor on each RowResult. Uses a table
|
||||
* name to locate meta regions.
|
||||
*
|
||||
* @param connection connection to use internally (null to use a new instance)
|
||||
* @param visitor visitor object
|
||||
* @param userTableName User table name in meta table to start scan at. Pass
|
||||
* null if not interested in a particular table.
|
||||
* @throws IOException e
|
||||
*/
|
||||
public static void metaScan(Connection connection,
|
||||
MetaScannerVisitor visitor, TableName userTableName) throws IOException {
|
||||
metaScan(connection, visitor, userTableName, null, Integer.MAX_VALUE,
|
||||
TableName.META_TABLE_NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
* Scans the meta table and calls a visitor on each RowResult. Uses a table
|
||||
* name and a row name to locate meta regions. And it only scans at most
|
||||
* <code>rowLimit</code> of rows.
|
||||
*
|
||||
* <p>Visible for testing. Use {@link
|
||||
* #metaScan(Connection, MetaScannerVisitor, TableName)} instead.
|
||||
*
|
||||
* @param connection to scan on
|
||||
* @param visitor Visitor object.
|
||||
* @param userTableName User table name in meta table to start scan at. Pass
|
||||
* null if not interested in a particular table.
|
||||
* @param row Name of the row at the user table. The scan will start from
|
||||
* the region row where the row resides.
|
||||
* @param rowLimit Max of processed rows. If it is less than 0, it
|
||||
* will be set to default value <code>Integer.MAX_VALUE</code>.
|
||||
* @throws IOException e
|
||||
*/
|
||||
@VisibleForTesting // Do not use. Used by Master but by a method that is used testing.
|
||||
public static void metaScan(Connection connection,
|
||||
MetaScannerVisitor visitor, TableName userTableName, byte[] row,
|
||||
int rowLimit)
|
||||
throws IOException {
|
||||
metaScan(connection, visitor, userTableName, row, rowLimit, TableName
|
||||
.META_TABLE_NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
* Scans the meta table and calls a visitor on each RowResult. Uses a table
|
||||
* name and a row name to locate meta regions. And it only scans at most
|
||||
* <code>rowLimit</code> of rows.
|
||||
*
|
||||
* @param connection connection to use internally (null to use a new instance)
|
||||
* @param visitor Visitor object. Closes the visitor before returning.
|
||||
* @param tableName User table name in meta table to start scan at. Pass
|
||||
* null if not interested in a particular table.
|
||||
* @param row Name of the row at the user table. The scan will start from
|
||||
* the region row where the row resides.
|
||||
* @param rowLimit Max of processed rows. If it is less than 0, it
|
||||
* will be set to default value <code>Integer.MAX_VALUE</code>.
|
||||
* @param metaTableName Meta table to scan, root or meta.
|
||||
* @throws IOException e
|
||||
*/
|
||||
static void metaScan(Connection connection,
|
||||
final MetaScannerVisitor visitor, final TableName tableName,
|
||||
final byte[] row, final int rowLimit, final TableName metaTableName)
|
||||
throws IOException {
|
||||
|
||||
int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE;
|
||||
// Calculate startrow for scan.
|
||||
byte[] startRow;
|
||||
// If the passed in 'connection' is 'managed' -- i.e. every second test uses
|
||||
// an HTable or an HBaseAdmin with managed connections -- then doing
|
||||
// connection.getTable will throw an exception saying you are NOT to use
|
||||
// managed connections getting tables. Leaving this as it is for now. Will
|
||||
// revisit when inclined to change all tests. User code probaby makes use of
|
||||
// managed connections too so don't change it till post hbase 1.0.
|
||||
try (Table metaTable = new HTable(TableName.META_TABLE_NAME, connection, null)) {
|
||||
if (row != null) {
|
||||
// Scan starting at a particular row in a particular table
|
||||
Result startRowResult = getClosestRowOrBefore(metaTable, tableName, row,
|
||||
connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
|
||||
HConstants.DEFAULT_USE_META_REPLICAS));
|
||||
if (startRowResult == null) {
|
||||
throw new TableNotFoundException("Cannot find row in " + metaTable.getName() +
|
||||
" for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
|
||||
}
|
||||
HRegionInfo regionInfo = getHRegionInfo(startRowResult);
|
||||
if (regionInfo == null) {
|
||||
throw new IOException("HRegionInfo was null or empty in Meta for " +
|
||||
tableName + ", row=" + Bytes.toStringBinary(row));
|
||||
}
|
||||
byte[] rowBefore = regionInfo.getStartKey();
|
||||
startRow = HRegionInfo.createRegionName(tableName, rowBefore, HConstants.ZEROES, false);
|
||||
} else if (tableName == null || tableName.getName().length == 0) {
|
||||
// Full hbase:meta scan
|
||||
startRow = HConstants.EMPTY_START_ROW;
|
||||
} else {
|
||||
// Scan hbase:meta for an entire table
|
||||
startRow = HRegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW,
|
||||
HConstants.ZEROES, false);
|
||||
}
|
||||
final Scan scan = new Scan(startRow).addFamily(HConstants.CATALOG_FAMILY);
|
||||
int scannerCaching = connection.getConfiguration()
|
||||
.getInt(HConstants.HBASE_META_SCANNER_CACHING,
|
||||
HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
|
||||
if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
|
||||
HConstants.DEFAULT_USE_META_REPLICAS)) {
|
||||
scan.setConsistency(Consistency.TIMELINE);
|
||||
}
|
||||
if (rowUpperLimit <= scannerCaching) {
|
||||
scan.setSmall(true);
|
||||
}
|
||||
int rows = Math.min(rowLimit, scannerCaching);
|
||||
scan.setCaching(rows);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Scanning " + metaTableName.getNameAsString() + " starting at row=" +
|
||||
Bytes.toStringBinary(startRow) + " for max=" + rowUpperLimit + " with caching=" + rows);
|
||||
}
|
||||
// Run the scan
|
||||
try (ResultScanner resultScanner = metaTable.getScanner(scan)) {
|
||||
Result result;
|
||||
int processedRows = 0;
|
||||
while ((result = resultScanner.next()) != null) {
|
||||
if (visitor != null) {
|
||||
if (!visitor.processRow(result)) break;
|
||||
}
|
||||
processedRows++;
|
||||
if (processedRows >= rowUpperLimit) break;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (visitor != null) {
|
||||
try {
|
||||
visitor.close();
|
||||
} catch (Throwable t) {
|
||||
ExceptionUtil.rethrowIfInterrupt(t);
|
||||
LOG.debug("Got exception in closing the meta scanner visitor", t);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Get closest metatable region row to passed <code>row</code>
|
||||
* @throws IOException
|
||||
*/
|
||||
private static Result getClosestRowOrBefore(final Table metaTable, final TableName userTableName,
|
||||
final byte [] row, boolean useMetaReplicas)
|
||||
throws IOException {
|
||||
byte[] searchRow = HRegionInfo.createRegionName(userTableName, row, HConstants.NINES, false);
|
||||
Scan scan = Scan.createGetClosestRowOrBeforeReverseScan(searchRow);
|
||||
if (useMetaReplicas) {
|
||||
scan.setConsistency(Consistency.TIMELINE);
|
||||
}
|
||||
try (ResultScanner resultScanner = metaTable.getScanner(scan)) {
|
||||
return resultScanner.next();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns HRegionInfo object from the column
|
||||
* HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
|
||||
* table Result.
|
||||
* @param data a Result object from the catalog table scan
|
||||
* @return HRegionInfo or null
|
||||
* @deprecated Use {@link org.apache.hadoop.hbase.MetaTableAccessor#getRegionLocations(Result)}
|
||||
*/
|
||||
@Deprecated
|
||||
public static HRegionInfo getHRegionInfo(Result data) {
|
||||
return HRegionInfo.getHRegionInfo(data);
|
||||
}
|
||||
|
||||
/**
|
||||
* Lists all of the regions currently in META.
|
||||
* @param conf configuration
|
||||
* @param connection to connect with
|
||||
* @param offlined True if we are to include offlined regions, false and we'll
|
||||
* leave out offlined regions from returned list.
|
||||
* @return List of all user-space regions.
|
||||
* @throws IOException
|
||||
*/
|
||||
@VisibleForTesting // And for hbck.
|
||||
public static List<HRegionInfo> listAllRegions(Configuration conf, Connection connection,
|
||||
final boolean offlined)
|
||||
throws IOException {
|
||||
final List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
|
||||
MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
|
||||
@Override
|
||||
public boolean processRow(Result result) throws IOException {
|
||||
if (result == null || result.isEmpty()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
|
||||
if (locations == null) return true;
|
||||
for (HRegionLocation loc : locations.getRegionLocations()) {
|
||||
if (loc != null) {
|
||||
HRegionInfo regionInfo = loc.getRegionInfo();
|
||||
// If region offline AND we are not to include offlined regions, return.
|
||||
if (regionInfo.isOffline() && !offlined) continue;
|
||||
regions.add(regionInfo);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
};
|
||||
metaScan(connection, visitor);
|
||||
return regions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Lists all of the table regions currently in META.
|
||||
* @param conf
|
||||
* @param offlined True if we are to include offlined regions, false and we'll
|
||||
* leave out offlined regions from returned list.
|
||||
* @return Map of all user-space regions to servers
|
||||
* @throws IOException
|
||||
* @deprecated Use {@link #allTableRegions(Connection, TableName)} instead
|
||||
*/
|
||||
@Deprecated
|
||||
public static NavigableMap<HRegionInfo, ServerName> allTableRegions(Configuration conf,
|
||||
Connection connection, final TableName tableName, boolean offlined) throws IOException {
|
||||
return allTableRegions(connection, tableName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Lists all of the table regions currently in META.
|
||||
* @param connection
|
||||
* @param tableName
|
||||
* @return Map of all user-space regions to servers
|
||||
* @throws IOException
|
||||
*/
|
||||
public static NavigableMap<HRegionInfo, ServerName> allTableRegions(
|
||||
Connection connection, final TableName tableName) throws IOException {
|
||||
final NavigableMap<HRegionInfo, ServerName> regions =
|
||||
new TreeMap<HRegionInfo, ServerName>();
|
||||
MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
|
||||
@Override
|
||||
public boolean processRowInternal(Result result) throws IOException {
|
||||
RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
|
||||
if (locations == null) return true;
|
||||
for (HRegionLocation loc : locations.getRegionLocations()) {
|
||||
if (loc != null) {
|
||||
HRegionInfo regionInfo = loc.getRegionInfo();
|
||||
regions.put(new UnmodifyableHRegionInfo(regionInfo), loc.getServerName());
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
};
|
||||
metaScan(connection, visitor, tableName);
|
||||
return regions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Lists table regions and locations grouped by region range from META.
|
||||
*/
|
||||
public static List<RegionLocations> listTableRegionLocations(Configuration conf,
|
||||
Connection connection, final TableName tableName) throws IOException {
|
||||
final List<RegionLocations> regions = new ArrayList<RegionLocations>();
|
||||
MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
|
||||
@Override
|
||||
public boolean processRowInternal(Result result) throws IOException {
|
||||
RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
|
||||
if (locations == null) return true;
|
||||
regions.add(locations);
|
||||
return true;
|
||||
}
|
||||
};
|
||||
metaScan(connection, visitor, tableName);
|
||||
return regions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Visitor class called to process each row of the hbase:meta table
|
||||
*/
|
||||
public interface MetaScannerVisitor extends Closeable {
|
||||
/**
|
||||
* Visitor method that accepts a RowResult and the meta region location.
|
||||
* Implementations can return false to stop the region's loop if it becomes
|
||||
* unnecessary for some reason.
|
||||
*
|
||||
* @param rowResult result
|
||||
* @return A boolean to know if it should continue to loop in the region
|
||||
* @throws IOException e
|
||||
*/
|
||||
boolean processRow(Result rowResult) throws IOException;
|
||||
}
|
||||
|
||||
public static abstract class MetaScannerVisitorBase implements MetaScannerVisitor {
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A MetaScannerVisitor that skips offline regions and split parents
|
||||
*/
|
||||
public static abstract class DefaultMetaScannerVisitor
|
||||
extends MetaScannerVisitorBase {
|
||||
|
||||
public DefaultMetaScannerVisitor() {
|
||||
super();
|
||||
}
|
||||
|
||||
public abstract boolean processRowInternal(Result rowResult) throws IOException;
|
||||
|
||||
@Override
|
||||
public boolean processRow(Result rowResult) throws IOException {
|
||||
HRegionInfo info = getHRegionInfo(rowResult);
|
||||
if (info == null) {
|
||||
return true;
|
||||
}
|
||||
|
||||
//skip over offline and split regions
|
||||
if (!(info.isOffline() || info.isSplit())) {
|
||||
return processRowInternal(rowResult);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A MetaScannerVisitor for a table. Provides a consistent view of the table's
|
||||
* hbase:meta entries during concurrent splits (see HBASE-5986 for details). This class
|
||||
* does not guarantee ordered traversal of meta entries, and can block until the
|
||||
* hbase:meta entries for daughters are available during splits.
|
||||
*/
|
||||
public static abstract class TableMetaScannerVisitor extends DefaultMetaScannerVisitor {
|
||||
private TableName tableName;
|
||||
|
||||
public TableMetaScannerVisitor(TableName tableName) {
|
||||
super();
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final boolean processRow(Result rowResult) throws IOException {
|
||||
HRegionInfo info = getHRegionInfo(rowResult);
|
||||
if (info == null) {
|
||||
return true;
|
||||
}
|
||||
if (!(info.getTable().equals(tableName))) {
|
||||
return false;
|
||||
}
|
||||
return super.processRow(rowResult);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
|
||||
/**
|
||||
* Container for Actions (i.e. Get, Delete, or Put), which are grouped by
|
||||
* regionName. Intended to be used with HConnectionManager.processBatch()
|
||||
* regionName. Intended to be used with {@link AsyncProcess}.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public final class MultiAction<R> {
|
||||
|
|
|
@ -137,9 +137,22 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
|
|||
* @param qualifier column qualifier
|
||||
* @param value column value
|
||||
* @return this
|
||||
* @deprecated Since 1.0.0. Use {@link #addColumn(byte[], byte[], byte[])}
|
||||
*/
|
||||
@Deprecated
|
||||
public Put add(byte [] family, byte [] qualifier, byte [] value) {
|
||||
return add(family, qualifier, this.ts, value);
|
||||
return addColumn(family, qualifier, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the specified column and value to this Put operation.
|
||||
* @param family family name
|
||||
* @param qualifier column qualifier
|
||||
* @param value column value
|
||||
* @return this
|
||||
*/
|
||||
public Put addColumn(byte [] family, byte [] qualifier, byte [] value) {
|
||||
return addColumn(family, qualifier, this.ts, value);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -167,8 +180,23 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
|
|||
* @param ts version timestamp
|
||||
* @param value column value
|
||||
* @return this
|
||||
* @deprecated Since 1.0.0. Use {@link #addColumn(byte[], byte[], long, byte[])}
|
||||
*/
|
||||
@Deprecated
|
||||
public Put add(byte [] family, byte [] qualifier, long ts, byte [] value) {
|
||||
return addColumn(family, qualifier, ts, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the specified column and value, with the specified timestamp as
|
||||
* its version to this Put operation.
|
||||
* @param family family name
|
||||
* @param qualifier column qualifier
|
||||
* @param ts version timestamp
|
||||
* @param value column value
|
||||
* @return this
|
||||
*/
|
||||
public Put addColumn(byte [] family, byte [] qualifier, long ts, byte [] value) {
|
||||
if (ts < 0) {
|
||||
throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + ts);
|
||||
}
|
||||
|
@ -199,7 +227,6 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
|
|||
* This expects that the underlying arrays won't change. It's intended
|
||||
* for usage internal HBase to and for advanced client applications.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public Put addImmutable(byte[] family, byte[] qualifier, long ts, byte[] value, Tag[] tag) {
|
||||
List<Cell> list = getCellList(family);
|
||||
KeyValue kv = createPutKeyValue(family, qualifier, ts, value, tag);
|
||||
|
@ -233,8 +260,23 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
|
|||
* @param ts version timestamp
|
||||
* @param value column value
|
||||
* @return this
|
||||
* @deprecated Since 1.0.0. Use {@link Put#addColumn(byte[], ByteBuffer, long, ByteBuffer)}
|
||||
*/
|
||||
@Deprecated
|
||||
public Put add(byte[] family, ByteBuffer qualifier, long ts, ByteBuffer value) {
|
||||
return addColumn(family, qualifier, ts, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the specified column and value, with the specified timestamp as
|
||||
* its version to this Put operation.
|
||||
* @param family family name
|
||||
* @param qualifier column qualifier
|
||||
* @param ts version timestamp
|
||||
* @param value column value
|
||||
* @return this
|
||||
*/
|
||||
public Put addColumn(byte[] family, ByteBuffer qualifier, long ts, ByteBuffer value) {
|
||||
if (ts < 0) {
|
||||
throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + ts);
|
||||
}
|
||||
|
|
|
@ -23,13 +23,17 @@ import java.io.InterruptedIOException;
|
|||
import java.net.ConnectException;
|
||||
import java.net.SocketTimeoutException;
|
||||
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* Similar to {@link RegionServerCallable} but for the AdminService interface. This service callable
|
||||
|
@ -42,25 +46,39 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
|
|||
|
||||
protected final ClusterConnection connection;
|
||||
|
||||
protected final RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
protected AdminService.BlockingInterface stub;
|
||||
|
||||
protected HRegionLocation location;
|
||||
|
||||
protected final TableName tableName;
|
||||
protected final byte[] row;
|
||||
protected final int replicaId;
|
||||
|
||||
protected final static int MIN_WAIT_DEAD_SERVER = 10000;
|
||||
|
||||
public RegionAdminServiceCallable(ClusterConnection connection, TableName tableName, byte[] row) {
|
||||
this(connection, null, tableName, row);
|
||||
public RegionAdminServiceCallable(ClusterConnection connection,
|
||||
RpcControllerFactory rpcControllerFactory, TableName tableName, byte[] row) {
|
||||
this(connection, rpcControllerFactory, null, tableName, row);
|
||||
}
|
||||
|
||||
public RegionAdminServiceCallable(ClusterConnection connection, HRegionLocation location,
|
||||
public RegionAdminServiceCallable(ClusterConnection connection,
|
||||
RpcControllerFactory rpcControllerFactory, HRegionLocation location,
|
||||
TableName tableName, byte[] row) {
|
||||
this(connection, rpcControllerFactory, location,
|
||||
tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID);
|
||||
}
|
||||
|
||||
public RegionAdminServiceCallable(ClusterConnection connection,
|
||||
RpcControllerFactory rpcControllerFactory, HRegionLocation location,
|
||||
TableName tableName, byte[] row, int replicaId) {
|
||||
this.connection = connection;
|
||||
this.rpcControllerFactory = rpcControllerFactory;
|
||||
this.location = location;
|
||||
this.tableName = tableName;
|
||||
this.row = row;
|
||||
this.replicaId = replicaId;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -85,7 +103,18 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
|
|||
this.stub = stub;
|
||||
}
|
||||
|
||||
public abstract HRegionLocation getLocation(boolean useCache) throws IOException;
|
||||
public HRegionLocation getLocation(boolean useCache) throws IOException {
|
||||
RegionLocations rl = getRegionLocations(connection, tableName, row, useCache, replicaId);
|
||||
if (rl == null) {
|
||||
throw new HBaseIOException(getExceptionMessage());
|
||||
}
|
||||
HRegionLocation location = rl.getRegionLocation(replicaId);
|
||||
if (location == null) {
|
||||
throw new HBaseIOException(getExceptionMessage());
|
||||
}
|
||||
|
||||
return location;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void throwable(Throwable t, boolean retrying) {
|
||||
|
@ -115,7 +144,8 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
|
|||
|
||||
//subclasses can override this.
|
||||
protected String getExceptionMessage() {
|
||||
return "There is no location";
|
||||
return "There is no location" + " table=" + tableName
|
||||
+ " ,replica=" + replicaId + ", row=" + Bytes.toStringBinary(row);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -132,4 +162,27 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
|
|||
}
|
||||
return sleep;
|
||||
}
|
||||
|
||||
public static RegionLocations getRegionLocations(
|
||||
ClusterConnection connection, TableName tableName, byte[] row,
|
||||
boolean useCache, int replicaId)
|
||||
throws RetriesExhaustedException, DoNotRetryIOException, InterruptedIOException {
|
||||
RegionLocations rl;
|
||||
try {
|
||||
rl = connection.locateRegion(tableName, row, useCache, true, replicaId);
|
||||
} catch (DoNotRetryIOException e) {
|
||||
throw e;
|
||||
} catch (RetriesExhaustedException e) {
|
||||
throw e;
|
||||
} catch (InterruptedIOException e) {
|
||||
throw e;
|
||||
} catch (IOException e) {
|
||||
throw new RetriesExhaustedException("Can't get the location", e);
|
||||
}
|
||||
if (rl == null) {
|
||||
throw new RetriesExhaustedException("Can't get the locations");
|
||||
}
|
||||
|
||||
return rl;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Iterator;
|
|||
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* Utility methods which contain the logic for regions and replicas.
|
||||
|
@ -30,6 +31,19 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
@InterfaceAudience.Private
|
||||
public class RegionReplicaUtil {
|
||||
|
||||
/**
|
||||
* Whether or not the secondary region will wait for observing a flush / region open event
|
||||
* from the primary region via async wal replication before enabling read requests. Since replayed
|
||||
* edits from async wal replication from primary is not persisted in WAL, the memstore of the
|
||||
* secondary region might be non-empty at the time of close or crash. For ensuring seqId's not
|
||||
* "going back in time" in the secondary region replica, this should be enabled. However, in some
|
||||
* cases the above semantics might be ok for some application classes.
|
||||
* See HBASE-11580 for more context.
|
||||
*/
|
||||
public static final String REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY
|
||||
= "hbase.region.replica.wait.for.primary.flush";
|
||||
protected static final boolean DEFAULT_REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH = true;
|
||||
|
||||
/**
|
||||
* The default replicaId for the region
|
||||
*/
|
||||
|
@ -92,4 +106,46 @@ public class RegionReplicaUtil {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean isReplicasForSameRegion(HRegionInfo regionInfoA, HRegionInfo regionInfoB) {
|
||||
return compareRegionInfosWithoutReplicaId(regionInfoA, regionInfoB) == 0;
|
||||
}
|
||||
|
||||
private static int compareRegionInfosWithoutReplicaId(HRegionInfo regionInfoA,
|
||||
HRegionInfo regionInfoB) {
|
||||
int result = regionInfoA.getTable().compareTo(regionInfoB.getTable());
|
||||
if (result != 0) {
|
||||
return result;
|
||||
}
|
||||
|
||||
// Compare start keys.
|
||||
result = Bytes.compareTo(regionInfoA.getStartKey(), regionInfoB.getStartKey());
|
||||
if (result != 0) {
|
||||
return result;
|
||||
}
|
||||
|
||||
// Compare end keys.
|
||||
result = Bytes.compareTo(regionInfoA.getEndKey(), regionInfoB.getEndKey());
|
||||
|
||||
if (result != 0) {
|
||||
if (regionInfoA.getStartKey().length != 0
|
||||
&& regionInfoA.getEndKey().length == 0) {
|
||||
return 1; // this is last region
|
||||
}
|
||||
if (regionInfoB.getStartKey().length != 0
|
||||
&& regionInfoB.getEndKey().length == 0) {
|
||||
return -1; // o is the last region
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
// regionId is usually milli timestamp -- this defines older stamps
|
||||
// to be "smaller" than newer stamps in sort order.
|
||||
if (regionInfoA.getRegionId() > regionInfoB.getRegionId()) {
|
||||
return 1;
|
||||
} else if (regionInfoA.getRegionId() < regionInfoB.getRegionId()) {
|
||||
return -1;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.BufferOverflowException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
|
@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -80,6 +82,17 @@ public class Result implements CellScannable, CellScanner {
|
|||
private Cell[] cells;
|
||||
private Boolean exists; // if the query was just to check existence.
|
||||
private boolean stale = false;
|
||||
|
||||
/**
|
||||
* Partial results do not contain the full row's worth of cells. The result had to be returned in
|
||||
* parts because the size of the cells in the row exceeded the RPC result size on the server.
|
||||
* Partial results must be combined client side with results representing the remainder of the
|
||||
* row's cells to form the complete result. Partial results and RPC result size allow us to avoid
|
||||
* OOME on the server when servicing requests for large rows. The Scan configuration used to
|
||||
* control the result size on the server is {@link Scan#setMaxResultSize(long)} and the default
|
||||
* value can be seen here: {@link HConstants#DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE}
|
||||
*/
|
||||
private boolean partial = false;
|
||||
// We're not using java serialization. Transient here is just a marker to say
|
||||
// that this is where we cache row if we're ever asked for it.
|
||||
private transient byte [] row = null;
|
||||
|
@ -89,7 +102,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
|
||||
private static ThreadLocal<byte[]> localBuffer = new ThreadLocal<byte[]>();
|
||||
private static final int PAD_WIDTH = 128;
|
||||
public static final Result EMPTY_RESULT = new Result();
|
||||
public static final Result EMPTY_RESULT = new Result(true);
|
||||
|
||||
private final static int INITIAL_CELLSCANNER_INDEX = -1;
|
||||
|
||||
|
@ -99,6 +112,8 @@ public class Result implements CellScannable, CellScanner {
|
|||
private int cellScannerIndex = INITIAL_CELLSCANNER_INDEX;
|
||||
private ClientProtos.RegionLoadStats stats;
|
||||
|
||||
private final boolean readonly;
|
||||
|
||||
/**
|
||||
* Creates an empty Result w/ no KeyValue payload; returns null if you call {@link #rawCells()}.
|
||||
* Use this to represent no results if {@code null} won't do or in old 'mapred' as opposed
|
||||
|
@ -106,7 +121,16 @@ public class Result implements CellScannable, CellScanner {
|
|||
* {@link #copyFrom(Result)} call.
|
||||
*/
|
||||
public Result() {
|
||||
super();
|
||||
this(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Allows to construct special purpose immutable Result objects,
|
||||
* such as EMPTY_RESULT.
|
||||
* @param readonly whether this Result instance is readonly
|
||||
*/
|
||||
private Result(boolean readonly) {
|
||||
this.readonly = readonly;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -115,7 +139,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
* @param cells List of cells
|
||||
*/
|
||||
public static Result create(List<Cell> cells) {
|
||||
return new Result(cells.toArray(new Cell[cells.size()]), null, false);
|
||||
return create(cells, null);
|
||||
}
|
||||
|
||||
public static Result create(List<Cell> cells, Boolean exists) {
|
||||
|
@ -123,10 +147,14 @@ public class Result implements CellScannable, CellScanner {
|
|||
}
|
||||
|
||||
public static Result create(List<Cell> cells, Boolean exists, boolean stale) {
|
||||
return create(cells, exists, stale, false);
|
||||
}
|
||||
|
||||
public static Result create(List<Cell> cells, Boolean exists, boolean stale, boolean partial) {
|
||||
if (exists != null){
|
||||
return new Result(null, exists, stale);
|
||||
return new Result(null, exists, stale, partial);
|
||||
}
|
||||
return new Result(cells.toArray(new Cell[cells.size()]), null, stale);
|
||||
return new Result(cells.toArray(new Cell[cells.size()]), null, stale, partial);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -135,21 +163,27 @@ public class Result implements CellScannable, CellScanner {
|
|||
* @param cells array of cells
|
||||
*/
|
||||
public static Result create(Cell[] cells) {
|
||||
return new Result(cells, null, false);
|
||||
return create(cells, null, false);
|
||||
}
|
||||
|
||||
public static Result create(Cell[] cells, Boolean exists, boolean stale) {
|
||||
return create(cells, exists, stale, false);
|
||||
}
|
||||
|
||||
public static Result create(Cell[] cells, Boolean exists, boolean stale, boolean partial) {
|
||||
if (exists != null){
|
||||
return new Result(null, exists, stale);
|
||||
return new Result(null, exists, stale, partial);
|
||||
}
|
||||
return new Result(cells, null, stale);
|
||||
return new Result(cells, null, stale, partial);
|
||||
}
|
||||
|
||||
/** Private ctor. Use {@link #create(Cell[])}. */
|
||||
private Result(Cell[] cells, Boolean exists, boolean stale) {
|
||||
private Result(Cell[] cells, Boolean exists, boolean stale, boolean partial) {
|
||||
this.cells = cells;
|
||||
this.exists = exists;
|
||||
this.stale = stale;
|
||||
this.partial = partial;
|
||||
this.readonly = false;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -361,6 +395,9 @@ public class Result implements CellScannable, CellScanner {
|
|||
|
||||
/**
|
||||
* Get the latest version of the specified column.
|
||||
* Note: this call clones the value content of the hosting Cell. See
|
||||
* {@link #getValueAsByteBuffer(byte[], byte[])}, etc., or {@link #listCells()} if you would
|
||||
* avoid the cloning.
|
||||
* @param family family name
|
||||
* @param qualifier column qualifier
|
||||
* @return value of latest version of column, null if none found
|
||||
|
@ -388,7 +425,8 @@ public class Result implements CellScannable, CellScanner {
|
|||
if (kv == null) {
|
||||
return null;
|
||||
}
|
||||
return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
|
||||
return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()).
|
||||
asReadOnlyBuffer();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -411,7 +449,8 @@ public class Result implements CellScannable, CellScanner {
|
|||
if (kv == null) {
|
||||
return null;
|
||||
}
|
||||
return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
|
||||
return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()).
|
||||
asReadOnlyBuffer();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -741,7 +780,59 @@ public class Result implements CellScannable, CellScanner {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get total size of raw cells
|
||||
* Forms a single result from the partial results in the partialResults list. This method is
|
||||
* useful for reconstructing partial results on the client side.
|
||||
* @param partialResults list of partial results
|
||||
* @return The complete result that is formed by combining all of the partial results together
|
||||
* @throws IOException A complete result cannot be formed because the results in the partial list
|
||||
* come from different rows
|
||||
*/
|
||||
public static Result createCompleteResult(List<Result> partialResults)
|
||||
throws IOException {
|
||||
List<Cell> cells = new ArrayList<Cell>();
|
||||
boolean stale = false;
|
||||
byte[] prevRow = null;
|
||||
byte[] currentRow = null;
|
||||
|
||||
if (partialResults != null && !partialResults.isEmpty()) {
|
||||
for (int i = 0; i < partialResults.size(); i++) {
|
||||
Result r = partialResults.get(i);
|
||||
currentRow = r.getRow();
|
||||
if (prevRow != null && !Bytes.equals(prevRow, currentRow)) {
|
||||
throw new IOException(
|
||||
"Cannot form complete result. Rows of partial results do not match." +
|
||||
" Partial Results: " + partialResults);
|
||||
}
|
||||
|
||||
// Ensure that all Results except the last one are marked as partials. The last result
|
||||
// may not be marked as a partial because Results are only marked as partials when
|
||||
// the scan on the server side must be stopped due to reaching the maxResultSize.
|
||||
// Visualizing it makes it easier to understand:
|
||||
// maxResultSize: 2 cells
|
||||
// (-x-) represents cell number x in a row
|
||||
// Example: row1: -1- -2- -3- -4- -5- (5 cells total)
|
||||
// How row1 will be returned by the server as partial Results:
|
||||
// Result1: -1- -2- (2 cells, size limit reached, mark as partial)
|
||||
// Result2: -3- -4- (2 cells, size limit reached, mark as partial)
|
||||
// Result3: -5- (1 cell, size limit NOT reached, NOT marked as partial)
|
||||
if (i != (partialResults.size() - 1) && !r.isPartial()) {
|
||||
throw new IOException(
|
||||
"Cannot form complete result. Result is missing partial flag. " +
|
||||
"Partial Results: " + partialResults);
|
||||
}
|
||||
prevRow = currentRow;
|
||||
stale = stale || r.isStale();
|
||||
for (Cell c : r.rawCells()) {
|
||||
cells.add(c);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return Result.create(cells, null, stale);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get total size of raw cells
|
||||
* @param result
|
||||
* @return Total size.
|
||||
*/
|
||||
|
@ -755,9 +846,12 @@ public class Result implements CellScannable, CellScanner {
|
|||
|
||||
/**
|
||||
* Copy another Result into this one. Needed for the old Mapred framework
|
||||
* @throws UnsupportedOperationException if invoked on instance of EMPTY_RESULT
|
||||
* (which is supposed to be immutable).
|
||||
* @param other
|
||||
*/
|
||||
public void copyFrom(Result other) {
|
||||
checkReadonly();
|
||||
this.row = null;
|
||||
this.familyMap = null;
|
||||
this.cells = other.cells;
|
||||
|
@ -787,6 +881,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
}
|
||||
|
||||
public void setExists(Boolean exists) {
|
||||
checkReadonly();
|
||||
this.exists = exists;
|
||||
}
|
||||
|
||||
|
@ -799,11 +894,34 @@ public class Result implements CellScannable, CellScanner {
|
|||
return stale;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether or not the result is a partial result. Partial results contain a subset of the cells
|
||||
* for a row and should be combined with a result representing the remaining cells in that row to
|
||||
* form a complete (non-partial) result.
|
||||
* @return Whether or not the result is a partial result
|
||||
*/
|
||||
public boolean isPartial() {
|
||||
return partial;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add load information about the region to the information about the result
|
||||
* @param loadStats statistics about the current region from which this was returned
|
||||
* @deprecated use {@link #setStatistics(ClientProtos.RegionLoadStats)} instead
|
||||
* @throws UnsupportedOperationException if invoked on instance of EMPTY_RESULT
|
||||
* (which is supposed to be immutable).
|
||||
*/
|
||||
@Deprecated
|
||||
public void addResults(ClientProtos.RegionLoadStats loadStats) {
|
||||
checkReadonly();
|
||||
this.stats = loadStats;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set load information about the region to the information about the result
|
||||
* @param loadStats statistics about the current region from which this was returned
|
||||
*/
|
||||
public void setStatistics(ClientProtos.RegionLoadStats loadStats) {
|
||||
this.stats = loadStats;
|
||||
}
|
||||
|
||||
|
@ -814,4 +932,14 @@ public class Result implements CellScannable, CellScanner {
|
|||
public ClientProtos.RegionLoadStats getStats() {
|
||||
return stats;
|
||||
}
|
||||
|
||||
/**
|
||||
* All methods modifying state of Result object must call this method
|
||||
* to ensure that special purpose immutable Results can't be accidentally modified.
|
||||
*/
|
||||
private void checkReadonly() {
|
||||
if (readonly == true) {
|
||||
throw new UnsupportedOperationException("Attempting to modify readonly EMPTY_RESULT!");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Factory implementation to provide the {@link HConnectionImplementation} with
|
||||
* Factory implementation to provide the {@link ConnectionImplementation} with
|
||||
* the implementation of the {@link RetryingCallerInterceptor} that we would use
|
||||
* to intercept the {@link RpcRetryingCaller} during the course of their calls.
|
||||
*
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -38,9 +37,6 @@ import org.apache.hadoop.hbase.util.ExceptionUtil;
|
|||
@InterfaceAudience.Private
|
||||
public class ReversedClientScanner extends ClientScanner {
|
||||
private static final Log LOG = LogFactory.getLog(ReversedClientScanner.class);
|
||||
// A byte array in which all elements are the max byte, and it is used to
|
||||
// construct closest front row
|
||||
static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);
|
||||
|
||||
/**
|
||||
* Create a new ReversibleClientScanner for the specified table Note that the
|
||||
|
@ -139,9 +135,10 @@ public class ReversedClientScanner extends ClientScanner {
|
|||
new ReversedScannerCallable(getConnection(), getTable(), scan, this.scanMetrics,
|
||||
locateStartRow, this.rpcControllerFactory);
|
||||
s.setCaching(nbRows);
|
||||
ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas(getTable(), getConnection(),
|
||||
s, pool, primaryOperationTimeout, scan,
|
||||
getRetries(), getScannerTimeout(), caching, getConf(), caller);
|
||||
ScannerCallableWithReplicas sr =
|
||||
new ScannerCallableWithReplicas(getTable(), getConnection(), s, pool,
|
||||
primaryOperationTimeout, scan, getRetries(), getScannerTimeout(), caching, getConf(),
|
||||
caller);
|
||||
return sr;
|
||||
}
|
||||
|
||||
|
@ -161,26 +158,4 @@ public class ReversedClientScanner extends ClientScanner {
|
|||
}
|
||||
return false; // unlikely.
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the closest row before the specified row
|
||||
* @param row
|
||||
* @return a new byte array which is the closest front row of the specified one
|
||||
*/
|
||||
protected static byte[] createClosestRowBefore(byte[] row) {
|
||||
if (row == null) {
|
||||
throw new IllegalArgumentException("The passed row is empty");
|
||||
}
|
||||
if (Bytes.equals(row, HConstants.EMPTY_BYTE_ARRAY)) {
|
||||
return MAX_BYTE_ARRAY;
|
||||
}
|
||||
if (row[row.length - 1] == 0) {
|
||||
return Arrays.copyOf(row, row.length - 1);
|
||||
} else {
|
||||
byte[] closestFrontRow = Arrays.copyOf(row, row.length);
|
||||
closestFrontRow[row.length - 1] = (byte) ((closestFrontRow[row.length - 1] & 0xff) - 1);
|
||||
closestFrontRow = Bytes.add(closestFrontRow, MAX_BYTE_ARRAY);
|
||||
return closestFrontRow;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -129,8 +129,6 @@ public class ReversedScannerCallable extends ScannerCallable {
|
|||
}
|
||||
|
||||
// check how often we retry.
|
||||
// HConnectionManager will call instantiateServer with reload==true
|
||||
// if and only if for retries.
|
||||
if (reload && this.scanMetrics != null) {
|
||||
this.scanMetrics.countOfRPCRetries.incrementAndGet();
|
||||
if (isRegionServerRemote) {
|
||||
|
|
|
@ -54,10 +54,7 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
|
|||
* When we started making calls.
|
||||
*/
|
||||
private long globalStartTime;
|
||||
/**
|
||||
* Start and end times for a single call.
|
||||
*/
|
||||
private final static int MIN_RPC_TIMEOUT = 2000;
|
||||
|
||||
/** How many retries are allowed before we start to log */
|
||||
private final int startLogErrorsCnt;
|
||||
|
||||
|
@ -87,11 +84,11 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
|
|||
if (callTimeout == Integer.MAX_VALUE) return Integer.MAX_VALUE;
|
||||
int remainingTime = (int) (callTimeout -
|
||||
(EnvironmentEdgeManager.currentTime() - this.globalStartTime));
|
||||
if (remainingTime < MIN_RPC_TIMEOUT) {
|
||||
if (remainingTime < 1) {
|
||||
// If there is no time left, we're trying anyway. It's too late.
|
||||
// 0 means no timeout, and it's not the intent here. So we secure both cases by
|
||||
// resetting to the minimum.
|
||||
remainingTime = MIN_RPC_TIMEOUT;
|
||||
remainingTime = 1;
|
||||
}
|
||||
return remainingTime;
|
||||
}
|
||||
|
@ -222,7 +219,7 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
|
|||
}
|
||||
// Don't let ServiceException out; its rpc specific.
|
||||
t = cause;
|
||||
// t could be a RemoteException so go aaround again.
|
||||
// t could be a RemoteException so go around again.
|
||||
translateException(t);
|
||||
} else if (t instanceof DoNotRetryIOException) {
|
||||
throw (DoNotRetryIOException)t;
|
||||
|
|
|
@ -34,9 +34,11 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.security.access.Permission;
|
||||
import org.apache.hadoop.hbase.security.visibility.Authorizations;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -91,35 +93,40 @@ public class Scan extends Query {
|
|||
|
||||
private static final String RAW_ATTR = "_raw_";
|
||||
|
||||
/**
|
||||
* EXPERT ONLY.
|
||||
* An integer (not long) indicating to the scanner logic how many times we attempt to retrieve the
|
||||
* next KV before we schedule a reseek.
|
||||
* The right value depends on the size of the average KV. A reseek is more efficient when
|
||||
* it can skip 5-10 KVs or 512B-1KB, or when the next KV is likely found in another HFile block.
|
||||
* Setting this only has any effect when columns were added with
|
||||
* {@link #addColumn(byte[], byte[])}
|
||||
* <pre>{@code
|
||||
* Scan s = new Scan(...);
|
||||
* s.addColumn(...);
|
||||
* s.setAttribute(Scan.HINT_LOOKAHEAD, Bytes.toBytes(2));
|
||||
* }</pre>
|
||||
* Default is 0 (always reseek).
|
||||
*/
|
||||
public static final String HINT_LOOKAHEAD = "_look_ahead_";
|
||||
|
||||
private byte [] startRow = HConstants.EMPTY_START_ROW;
|
||||
private byte [] stopRow = HConstants.EMPTY_END_ROW;
|
||||
private int maxVersions = 1;
|
||||
private int batch = -1;
|
||||
|
||||
/**
|
||||
* Partial {@link Result}s are {@link Result}s must be combined to form a complete {@link Result}.
|
||||
* The {@link Result}s had to be returned in fragments (i.e. as partials) because the size of the
|
||||
* cells in the row exceeded max result size on the server. Typically partial results will be
|
||||
* combined client side into complete results before being delivered to the caller. However, if
|
||||
* this flag is set, the caller is indicating that they do not mind seeing partial results (i.e.
|
||||
* they understand that the results returned from the Scanner may only represent part of a
|
||||
* particular row). In such a case, any attempt to combine the partials into a complete result on
|
||||
* the client side will be skipped, and the caller will be able to see the exact results returned
|
||||
* from the server.
|
||||
*/
|
||||
private boolean allowPartialResults = false;
|
||||
|
||||
private int storeLimit = -1;
|
||||
private int storeOffset = 0;
|
||||
private boolean getScan;
|
||||
|
||||
// If application wants to collect scan metrics, it needs to
|
||||
// call scan.setAttribute(SCAN_ATTRIBUTES_ENABLE, Bytes.toBytes(Boolean.TRUE))
|
||||
/**
|
||||
* @deprecated since 1.0.0. Use {@link #setScanMetricsEnabled(boolean)}
|
||||
*/
|
||||
// Make private or remove.
|
||||
@Deprecated
|
||||
static public final String SCAN_ATTRIBUTES_METRICS_ENABLE = "scan.attributes.metrics.enable";
|
||||
|
||||
/**
|
||||
* Use {@link #getScanMetrics()}
|
||||
*/
|
||||
// Make this private or remove.
|
||||
@Deprecated
|
||||
static public final String SCAN_ATTRIBUTES_METRICS_DATA = "scan.attributes.metrics.data";
|
||||
|
||||
// If an application wants to use multiple scans over different tables each scan must
|
||||
|
@ -680,6 +687,27 @@ public class Scan extends Query {
|
|||
return reversed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Setting whether the caller wants to see the partial results that may be returned from the
|
||||
* server. By default this value is false and the complete results will be assembled client side
|
||||
* before being delivered to the caller.
|
||||
* @param allowPartialResults
|
||||
* @return this
|
||||
*/
|
||||
public Scan setAllowPartialResults(final boolean allowPartialResults) {
|
||||
this.allowPartialResults = allowPartialResults;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when the constructor of this scan understands that the results they will see may
|
||||
* only represent a partial portion of a row. The entire row would be retrieved by
|
||||
* subsequent calls to {@link ResultScanner#next()}
|
||||
*/
|
||||
public boolean getAllowPartialResults() {
|
||||
return allowPartialResults;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the value indicating whether loading CFs on demand should be allowed (cluster
|
||||
* default is false). On-demand CF loading doesn't load column families until necessary, e.g.
|
||||
|
@ -916,4 +944,31 @@ public class Scan extends Query {
|
|||
scan.setCaching(1);
|
||||
return scan;
|
||||
}
|
||||
|
||||
/**
|
||||
* Enable collection of {@link ScanMetrics}. For advanced users.
|
||||
* @param enabled Set to true to enable accumulating scan metrics
|
||||
*/
|
||||
public Scan setScanMetricsEnabled(final boolean enabled) {
|
||||
setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.valueOf(enabled)));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return True if collection of scan metrics is enabled. For advanced users.
|
||||
*/
|
||||
public boolean isScanMetricsEnabled() {
|
||||
byte[] attr = getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE);
|
||||
return attr == null ? false : Bytes.toBoolean(attr);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Metrics on this Scan, if metrics were enabled.
|
||||
* @see #setScanMetricsEnabled(boolean)
|
||||
*/
|
||||
public ScanMetrics getScanMetrics() {
|
||||
byte [] bytes = getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA);
|
||||
if (bytes == null) return null;
|
||||
return ProtobufUtil.toScanMetrics(bytes);
|
||||
}
|
||||
}
|
|
@ -76,6 +76,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
private int logCutOffLatency = 1000;
|
||||
private static String myAddress;
|
||||
protected final int id;
|
||||
protected boolean serverHasMoreResultsContext;
|
||||
protected boolean serverHasMoreResults;
|
||||
static {
|
||||
try {
|
||||
myAddress = DNS.getDefaultHost("default", "default");
|
||||
|
@ -153,8 +155,6 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
}
|
||||
|
||||
// check how often we retry.
|
||||
// HConnectionManager will call instantiateServer with reload==true
|
||||
// if and only if for retries.
|
||||
if (reload && this.scanMetrics != null) {
|
||||
this.scanMetrics.countOfRPCRetries.incrementAndGet();
|
||||
if (isRegionServerRemote) {
|
||||
|
@ -177,7 +177,6 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("deprecation")
|
||||
public Result [] call(int callTimeout) throws IOException {
|
||||
if (Thread.interrupted()) {
|
||||
throw new InterruptedIOException();
|
||||
|
@ -223,12 +222,23 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
+ rows + " rows from scanner=" + scannerId);
|
||||
}
|
||||
}
|
||||
if (response.hasMoreResults()
|
||||
&& !response.getMoreResults()) {
|
||||
// moreResults is only used for the case where a filter exhausts all elements
|
||||
if (response.hasMoreResults() && !response.getMoreResults()) {
|
||||
scannerId = -1L;
|
||||
closed = true;
|
||||
// Implied that no results were returned back, either.
|
||||
return null;
|
||||
}
|
||||
// moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due
|
||||
// to size or quantity of results in the response.
|
||||
if (response.hasMoreResultsInRegion()) {
|
||||
// Set what the RS said
|
||||
setHasMoreResultsContext(true);
|
||||
setServerHasMoreResults(response.getMoreResultsInRegion());
|
||||
} else {
|
||||
// Server didn't respond whether it has more results or not.
|
||||
setHasMoreResultsContext(false);
|
||||
}
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -394,4 +404,30 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
s.setCaching(this.caching);
|
||||
return s;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should the client attempt to fetch more results from this region
|
||||
* @return True if the client should attempt to fetch more results, false otherwise.
|
||||
*/
|
||||
protected boolean getServerHasMoreResults() {
|
||||
assert serverHasMoreResultsContext;
|
||||
return this.serverHasMoreResults;
|
||||
}
|
||||
|
||||
protected void setServerHasMoreResults(boolean serverHasMoreResults) {
|
||||
this.serverHasMoreResults = serverHasMoreResults;
|
||||
}
|
||||
|
||||
/**
|
||||
* Did the server respond with information about whether more results might exist.
|
||||
* Not guaranteed to respond with older server versions
|
||||
* @return True if the server responded with information about more results.
|
||||
*/
|
||||
protected boolean hasMoreResultsContext() {
|
||||
return serverHasMoreResultsContext;
|
||||
}
|
||||
|
||||
protected void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
|
||||
this.serverHasMoreResultsContext = serverHasMoreResultsContext;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.client.ClientScanner.createClosestRowBefore;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -44,8 +46,6 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import static org.apache.hadoop.hbase.client.ReversedClientScanner.createClosestRowBefore;
|
||||
|
||||
/**
|
||||
* This class has the logic for handling scanners for regions with and without replicas.
|
||||
* 1. A scan is attempted on the default (primary) region
|
||||
|
@ -111,6 +111,22 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
return currentScannerCallable.getHRegionInfo();
|
||||
}
|
||||
|
||||
public boolean getServerHasMoreResults() {
|
||||
return currentScannerCallable.getServerHasMoreResults();
|
||||
}
|
||||
|
||||
public void setServerHasMoreResults(boolean serverHasMoreResults) {
|
||||
currentScannerCallable.setServerHasMoreResults(serverHasMoreResults);
|
||||
}
|
||||
|
||||
public boolean hasMoreResultsContext() {
|
||||
return currentScannerCallable.hasMoreResultsContext();
|
||||
}
|
||||
|
||||
public void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
|
||||
currentScannerCallable.setHasMoreResultsContext(serverHasMoreResultsContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result [] call(int timeout) throws IOException {
|
||||
// If the active replica callable was closed somewhere, invoke the RPC to
|
||||
|
@ -276,14 +292,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
continue; //this was already scheduled earlier
|
||||
}
|
||||
ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id);
|
||||
|
||||
if (this.lastResult != null) {
|
||||
if(s.getScan().isReversed()){
|
||||
s.getScan().setStartRow(createClosestRowBefore(this.lastResult.getRow()));
|
||||
}else {
|
||||
s.getScan().setStartRow(Bytes.add(this.lastResult.getRow(), new byte[1]));
|
||||
}
|
||||
}
|
||||
setStartRowForReplicaCallable(s);
|
||||
outstandingCallables.add(s);
|
||||
RetryingRPC retryingOnReplica = new RetryingRPC(s);
|
||||
cs.submit(retryingOnReplica, scannerTimeout, id);
|
||||
|
@ -291,6 +300,31 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
return max - min + 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the start row for the replica callable based on the state of the last result received.
|
||||
* @param callable The callable to set the start row on
|
||||
*/
|
||||
private void setStartRowForReplicaCallable(ScannerCallable callable) {
|
||||
if (this.lastResult == null || callable == null) return;
|
||||
|
||||
if (this.lastResult.isPartial()) {
|
||||
// The last result was a partial result which means we have not received all of the cells
|
||||
// for this row. Thus, use the last result's row as the start row. If a replica switch
|
||||
// occurs, the scanner will ensure that any accumulated partial results are cleared,
|
||||
// and the scan can resume from this row.
|
||||
callable.getScan().setStartRow(this.lastResult.getRow());
|
||||
} else {
|
||||
// The last result was not a partial result which means it contained all of the cells for
|
||||
// that row (we no longer need any information from it). Set the start row to the next
|
||||
// closest row that could be seen.
|
||||
if (callable.getScan().isReversed()) {
|
||||
callable.getScan().setStartRow(createClosestRowBefore(this.lastResult.getRow()));
|
||||
} else {
|
||||
callable.getScan().setStartRow(Bytes.add(this.lastResult.getRow(), new byte[1]));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
boolean isAnyRPCcancelled() {
|
||||
return someRPCcancelled;
|
||||
|
|
|
@ -498,7 +498,9 @@ public interface Table extends Closeable {
|
|||
* The default value comes from the configuration parameter
|
||||
* {@code hbase.client.write.buffer}.
|
||||
* @return The size of the write buffer in bytes.
|
||||
* @deprecated as of 1.0.1 (should not have been in 1.0.0). Replaced by {@link BufferedMutator#getWriteBufferSize()}
|
||||
*/
|
||||
@Deprecated
|
||||
long getWriteBufferSize();
|
||||
|
||||
/**
|
||||
|
@ -508,7 +510,10 @@ public interface Table extends Closeable {
|
|||
* write buffer, the buffer gets flushed.
|
||||
* @param writeBufferSize The new write buffer size, in bytes.
|
||||
* @throws IOException if a remote or network exception occurs.
|
||||
* @deprecated as of 1.0.1 (should not have been in 1.0.0). Replaced by {@link BufferedMutator} and
|
||||
* {@link BufferedMutatorParams#writeBufferSize(long)}
|
||||
*/
|
||||
@Deprecated
|
||||
void setWriteBufferSize(long writeBufferSize) throws IOException;
|
||||
|
||||
/**
|
||||
|
|
|
@ -39,14 +39,14 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
|||
class ZooKeeperKeepAliveConnection extends ZooKeeperWatcher{
|
||||
ZooKeeperKeepAliveConnection(
|
||||
Configuration conf, String descriptor,
|
||||
ConnectionManager.HConnectionImplementation conn) throws IOException {
|
||||
ConnectionImplementation conn) throws IOException {
|
||||
super(conf, descriptor, conn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (this.abortable != null) {
|
||||
((ConnectionManager.HConnectionImplementation)abortable).releaseZooKeeperWatcher(this);
|
||||
((ConnectionImplementation)abortable).releaseZooKeeperWatcher(this);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -37,14 +37,14 @@ import org.apache.zookeeper.KeeperException;
|
|||
class ZooKeeperRegistry implements Registry {
|
||||
static final Log LOG = LogFactory.getLog(ZooKeeperRegistry.class);
|
||||
// Needs an instance of hci to function. Set after construct this instance.
|
||||
ConnectionManager.HConnectionImplementation hci;
|
||||
ConnectionImplementation hci;
|
||||
|
||||
@Override
|
||||
public void init(Connection connection) {
|
||||
if (!(connection instanceof ConnectionManager.HConnectionImplementation)) {
|
||||
throw new RuntimeException("This registry depends on HConnectionImplementation");
|
||||
if (!(connection instanceof ConnectionImplementation)) {
|
||||
throw new RuntimeException("This registry depends on ConnectionImplementation");
|
||||
}
|
||||
this.hci = (ConnectionManager.HConnectionImplementation)connection;
|
||||
this.hci = (ConnectionImplementation)connection;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,15 +22,14 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
||||
|
||||
/**
|
||||
* Provides client-side metrics related to scan operations
|
||||
* Provides client-side metrics related to scan operations.
|
||||
* The data can be passed to mapreduce framework or other systems.
|
||||
* We use atomic longs so that one thread can increment,
|
||||
* while another atomically resets to zero after the values are reported
|
||||
|
@ -40,12 +39,10 @@ import com.google.common.collect.ImmutableMap;
|
|||
* However, there is no need for this. So they are defined under scan operation
|
||||
* for now.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class ScanMetrics {
|
||||
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ScanMetrics.class);
|
||||
|
||||
/**
|
||||
* Hash to hold the String -> Atomic Long mappings.
|
||||
*/
|
||||
|
|
|
@ -38,17 +38,25 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationException;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeer;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -114,16 +122,31 @@ public class ReplicationAdmin implements Closeable {
|
|||
"enable it in order to use replication");
|
||||
}
|
||||
this.connection = ConnectionFactory.createConnection(conf);
|
||||
zkw = createZooKeeperWatcher();
|
||||
try {
|
||||
this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
|
||||
this.replicationPeers.init();
|
||||
this.replicationQueuesClient =
|
||||
ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
|
||||
this.replicationQueuesClient.init();
|
||||
|
||||
} catch (ReplicationException e) {
|
||||
throw new IOException("Error initializing the replication admin client.", e);
|
||||
zkw = createZooKeeperWatcher();
|
||||
try {
|
||||
this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
|
||||
this.replicationPeers.init();
|
||||
this.replicationQueuesClient =
|
||||
ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
|
||||
this.replicationQueuesClient.init();
|
||||
} catch (Exception exception) {
|
||||
if (zkw != null) {
|
||||
zkw.close();
|
||||
}
|
||||
throw exception;
|
||||
}
|
||||
} catch (Exception exception) {
|
||||
if (connection != null) {
|
||||
connection.close();
|
||||
}
|
||||
if (exception instanceof IOException) {
|
||||
throw (IOException) exception;
|
||||
} else if (exception instanceof RuntimeException) {
|
||||
throw (RuntimeException) exception;
|
||||
} else {
|
||||
throw new IOException("Error initializing the replication admin client.", exception);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -501,4 +524,203 @@ public class ReplicationAdmin implements Closeable {
|
|||
|
||||
return replicationColFams;
|
||||
}
|
||||
|
||||
/**
|
||||
* Enable a table's replication switch.
|
||||
* @param tableName name of the table
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
public void enableTableRep(final TableName tableName) throws IOException {
|
||||
if (tableName == null) {
|
||||
throw new IllegalArgumentException("Table name cannot be null");
|
||||
}
|
||||
try (Admin admin = this.connection.getAdmin()) {
|
||||
if (!admin.tableExists(tableName)) {
|
||||
throw new TableNotFoundException("Table '" + tableName.getNameAsString()
|
||||
+ "' does not exists.");
|
||||
}
|
||||
}
|
||||
byte[][] splits = getTableSplitRowKeys(tableName);
|
||||
checkAndSyncTableDescToPeers(tableName, splits);
|
||||
setTableRep(tableName, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Disable a table's replication switch.
|
||||
* @param tableName name of the table
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
public void disableTableRep(final TableName tableName) throws IOException {
|
||||
if (tableName == null) {
|
||||
throw new IllegalArgumentException("Table name is null");
|
||||
}
|
||||
try (Admin admin = this.connection.getAdmin()) {
|
||||
if (!admin.tableExists(tableName)) {
|
||||
throw new TableNotFoundException("Table '" + tableName.getNamespaceAsString()
|
||||
+ "' does not exists.");
|
||||
}
|
||||
}
|
||||
setTableRep(tableName, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the split row keys of table
|
||||
* @param tableName table name
|
||||
* @return array of split row keys
|
||||
* @throws IOException
|
||||
*/
|
||||
private byte[][] getTableSplitRowKeys(TableName tableName) throws IOException {
|
||||
try (RegionLocator locator = connection.getRegionLocator(tableName);) {
|
||||
byte[][] startKeys = locator.getStartKeys();
|
||||
if (startKeys.length == 1) {
|
||||
return null;
|
||||
}
|
||||
byte[][] splits = new byte[startKeys.length - 1][];
|
||||
for (int i = 1; i < startKeys.length; i++) {
|
||||
splits[i - 1] = startKeys[i];
|
||||
}
|
||||
return splits;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Connect to peer and check the table descriptor on peer:
|
||||
* <ol>
|
||||
* <li>Create the same table on peer when not exist.</li>
|
||||
* <li>Throw exception if the table exists on peer cluster but descriptors are not same.</li>
|
||||
* </ol>
|
||||
* @param tableName name of the table to sync to the peer
|
||||
* @param splits table split keys
|
||||
* @throws IOException
|
||||
*/
|
||||
private void checkAndSyncTableDescToPeers(final TableName tableName, final byte[][] splits)
|
||||
throws IOException {
|
||||
List<ReplicationPeer> repPeers = listValidReplicationPeers();
|
||||
if (repPeers == null || repPeers.size() <= 0) {
|
||||
throw new IllegalArgumentException("Found no peer cluster for replication.");
|
||||
}
|
||||
for (ReplicationPeer repPeer : repPeers) {
|
||||
Configuration peerConf = repPeer.getConfiguration();
|
||||
HTableDescriptor htd = null;
|
||||
try (Connection conn = ConnectionFactory.createConnection(peerConf);
|
||||
Admin admin = this.connection.getAdmin();
|
||||
Admin repHBaseAdmin = conn.getAdmin()) {
|
||||
htd = admin.getTableDescriptor(tableName);
|
||||
HTableDescriptor peerHtd = null;
|
||||
if (!repHBaseAdmin.tableExists(tableName)) {
|
||||
repHBaseAdmin.createTable(htd, splits);
|
||||
} else {
|
||||
peerHtd = repHBaseAdmin.getTableDescriptor(tableName);
|
||||
if (peerHtd == null) {
|
||||
throw new IllegalArgumentException("Failed to get table descriptor for table "
|
||||
+ tableName.getNameAsString() + " from peer cluster " + repPeer.getId());
|
||||
} else if (!peerHtd.equals(htd)) {
|
||||
throw new IllegalArgumentException("Table " + tableName.getNameAsString()
|
||||
+ " exists in peer cluster " + repPeer.getId()
|
||||
+ ", but the table descriptors are not same when comapred with source cluster."
|
||||
+ " Thus can not enable the table's replication switch.");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private List<ReplicationPeer> listValidReplicationPeers() {
|
||||
Map<String, ReplicationPeerConfig> peers = listPeerConfigs();
|
||||
if (peers == null || peers.size() <= 0) {
|
||||
return null;
|
||||
}
|
||||
List<ReplicationPeer> validPeers = new ArrayList<ReplicationPeer>(peers.size());
|
||||
for (Entry<String, ReplicationPeerConfig> peerEntry : peers.entrySet()) {
|
||||
String peerId = peerEntry.getKey();
|
||||
String clusterKey = peerEntry.getValue().getClusterKey();
|
||||
Configuration peerConf = new Configuration(this.connection.getConfiguration());
|
||||
Stat s = null;
|
||||
try {
|
||||
ZKUtil.applyClusterKeyToConf(peerConf, clusterKey);
|
||||
Pair<ReplicationPeerConfig, Configuration> pair = this.replicationPeers.getPeerConf(peerId);
|
||||
ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst());
|
||||
s =
|
||||
zkw.getRecoverableZooKeeper().exists(peerConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT),
|
||||
null);
|
||||
if (null == s) {
|
||||
LOG.info(peerId + ' ' + clusterKey + " is invalid now.");
|
||||
continue;
|
||||
}
|
||||
validPeers.add(peer);
|
||||
} catch (ReplicationException e) {
|
||||
LOG.warn("Failed to get valid replication peers. "
|
||||
+ "Error connecting to peer cluster with peerId=" + peerId);
|
||||
LOG.debug("Failure details to get valid replication peers.", e);
|
||||
continue;
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn("Failed to get valid replication peers. KeeperException code="
|
||||
+ e.code().intValue());
|
||||
LOG.debug("Failure details to get valid replication peers.", e);
|
||||
continue;
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("Failed to get valid replication peers due to InterruptedException.");
|
||||
LOG.debug("Failure details to get valid replication peers.", e);
|
||||
Thread.currentThread().interrupt();
|
||||
continue;
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to get valid replication peers due to IOException.");
|
||||
LOG.debug("Failure details to get valid replication peers.", e);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
return validPeers;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the table's replication switch if the table's replication switch is already not set.
|
||||
* @param tableName name of the table
|
||||
* @param isRepEnabled is replication switch enable or disable
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
private void setTableRep(final TableName tableName, boolean isRepEnabled) throws IOException {
|
||||
Admin admin = null;
|
||||
try {
|
||||
admin = this.connection.getAdmin();
|
||||
HTableDescriptor htd = admin.getTableDescriptor(tableName);
|
||||
if (isTableRepEnabled(htd) ^ isRepEnabled) {
|
||||
boolean isOnlineSchemaUpdateEnabled =
|
||||
this.connection.getConfiguration()
|
||||
.getBoolean("hbase.online.schema.update.enable", true);
|
||||
if (!isOnlineSchemaUpdateEnabled) {
|
||||
admin.disableTable(tableName);
|
||||
}
|
||||
for (HColumnDescriptor hcd : htd.getFamilies()) {
|
||||
hcd.setScope(isRepEnabled ? HConstants.REPLICATION_SCOPE_GLOBAL
|
||||
: HConstants.REPLICATION_SCOPE_LOCAL);
|
||||
}
|
||||
admin.modifyTable(tableName, htd);
|
||||
if (!isOnlineSchemaUpdateEnabled) {
|
||||
admin.enableTable(tableName);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (admin != null) {
|
||||
try {
|
||||
admin.close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to close admin connection.");
|
||||
LOG.debug("Details on failure to close admin connection.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param htd table descriptor details for the table to check
|
||||
* @return true if table's replication switch is enabled
|
||||
*/
|
||||
private boolean isTableRepEnabled(HTableDescriptor htd) {
|
||||
for (HColumnDescriptor hcd : htd.getFamilies()) {
|
||||
if (hcd.getScope() != HConstants.REPLICATION_SCOPE_GLOBAL) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
* Thrown when the client believes that we are trying to communicate to has
|
||||
* been repeatedly unresponsive for a while.
|
||||
*
|
||||
* On receiving such an exception. The HConnectionManager will skip all
|
||||
* On receiving such an exception. The ConnectionManager will skip all
|
||||
* retries and fast fail the operation.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
* Thrown when the client believes that we are trying to communicate to has
|
||||
* been repeatedly unresponsive for a while.
|
||||
*
|
||||
* On receiving such an exception. The HConnectionManager will skip all
|
||||
* On receiving such an exception. The ConnectionManager will skip all
|
||||
* retries and fast fail the operation.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
|
|
|
@ -37,22 +37,22 @@ public enum EventType {
|
|||
// Messages originating from RS (NOTE: there is NO direct communication from
|
||||
// RS to Master). These are a result of RS updates into ZK.
|
||||
// RS_ZK_REGION_CLOSING (1), // It is replaced by M_ZK_REGION_CLOSING(HBASE-4739)
|
||||
|
||||
|
||||
/**
|
||||
* RS_ZK_REGION_CLOSED<br>
|
||||
*
|
||||
*
|
||||
* RS has finished closing a region.
|
||||
*/
|
||||
RS_ZK_REGION_CLOSED (2, ExecutorType.MASTER_CLOSE_REGION),
|
||||
/**
|
||||
* RS_ZK_REGION_OPENING<br>
|
||||
*
|
||||
*
|
||||
* RS is in process of opening a region.
|
||||
*/
|
||||
RS_ZK_REGION_OPENING (3, null),
|
||||
/**
|
||||
* RS_ZK_REGION_OPENED<br>
|
||||
*
|
||||
*
|
||||
* RS has finished opening a region.
|
||||
*/
|
||||
RS_ZK_REGION_OPENED (4, ExecutorType.MASTER_OPEN_REGION),
|
||||
|
@ -70,7 +70,7 @@ public enum EventType {
|
|||
RS_ZK_REGION_SPLIT (6, ExecutorType.MASTER_SERVER_OPERATIONS),
|
||||
/**
|
||||
* RS_ZK_REGION_FAILED_OPEN<br>
|
||||
*
|
||||
*
|
||||
* RS failed to open a region.
|
||||
*/
|
||||
RS_ZK_REGION_FAILED_OPEN (7, ExecutorType.MASTER_CLOSE_REGION),
|
||||
|
@ -217,7 +217,7 @@ public enum EventType {
|
|||
* Master adds this region as closing in ZK
|
||||
*/
|
||||
M_ZK_REGION_CLOSING (51, null),
|
||||
|
||||
|
||||
/**
|
||||
* Master controlled events to be executed on the master
|
||||
* M_SERVER_SHUTDOWN
|
||||
|
@ -232,14 +232,14 @@ public enum EventType {
|
|||
M_META_SERVER_SHUTDOWN (72, ExecutorType.MASTER_META_SERVER_OPERATIONS),
|
||||
/**
|
||||
* Master controlled events to be executed on the master.<br>
|
||||
*
|
||||
*
|
||||
* M_MASTER_RECOVERY<br>
|
||||
* Master is processing recovery of regions found in ZK RIT
|
||||
*/
|
||||
M_MASTER_RECOVERY (73, ExecutorType.MASTER_SERVER_OPERATIONS),
|
||||
/**
|
||||
* Master controlled events to be executed on the master.<br>
|
||||
*
|
||||
*
|
||||
* M_LOG_REPLAY<br>
|
||||
* Master is processing log replay of failed region server
|
||||
*/
|
||||
|
@ -247,18 +247,25 @@ public enum EventType {
|
|||
|
||||
/**
|
||||
* RS controlled events to be executed on the RS.<br>
|
||||
*
|
||||
*
|
||||
* RS_PARALLEL_SEEK
|
||||
*/
|
||||
RS_PARALLEL_SEEK (80, ExecutorType.RS_PARALLEL_SEEK),
|
||||
|
||||
|
||||
/**
|
||||
* RS wal recovery work items(either creating recover.edits or directly replay wals)
|
||||
* to be executed on the RS.<br>
|
||||
*
|
||||
*
|
||||
* RS_LOG_REPLAY
|
||||
*/
|
||||
RS_LOG_REPLAY (81, ExecutorType.RS_LOG_REPLAY_OPS);
|
||||
RS_LOG_REPLAY (81, ExecutorType.RS_LOG_REPLAY_OPS),
|
||||
|
||||
/**
|
||||
* RS flush triggering from secondary region replicas to primary region replica. <br>
|
||||
*
|
||||
* RS_REGION_REPLICA_FLUSH
|
||||
*/
|
||||
RS_REGION_REPLICA_FLUSH (82, ExecutorType.RS_REGION_REPLICA_FLUSH_OPS);
|
||||
|
||||
private final int code;
|
||||
private final ExecutorType executor;
|
||||
|
|
|
@ -45,7 +45,8 @@ public enum ExecutorType {
|
|||
RS_CLOSE_ROOT (24),
|
||||
RS_CLOSE_META (25),
|
||||
RS_PARALLEL_SEEK (26),
|
||||
RS_LOG_REPLAY_OPS (27);
|
||||
RS_LOG_REPLAY_OPS (27),
|
||||
RS_REGION_REPLICA_FLUSH_OPS (28);
|
||||
|
||||
ExecutorType(int value) {}
|
||||
|
||||
|
|
|
@ -133,7 +133,7 @@ public class ColumnRangeFilter extends FilterBase {
|
|||
}
|
||||
|
||||
if (!this.minColumnInclusive && cmpMin == 0) {
|
||||
return ReturnCode.SKIP;
|
||||
return ReturnCode.NEXT_COL;
|
||||
}
|
||||
|
||||
if (this.maxColumn == null) {
|
||||
|
|
|
@ -65,7 +65,7 @@ public class FamilyFilter extends CompareFilter {
|
|||
if (familyLength > 0) {
|
||||
if (doCompare(this.compareOp, this.comparator, v.getFamilyArray(),
|
||||
v.getFamilyOffset(), familyLength)) {
|
||||
return ReturnCode.SKIP;
|
||||
return ReturnCode.NEXT_ROW;
|
||||
}
|
||||
}
|
||||
return ReturnCode.INCLUDE;
|
||||
|
|
|
@ -72,7 +72,7 @@ public class AsyncCall extends DefaultPromise<Message> {
|
|||
this.responseDefaultType = responseDefaultType;
|
||||
|
||||
this.startTime = EnvironmentEdgeManager.currentTime();
|
||||
this.rpcTimeout = controller.getCallTimeout();
|
||||
this.rpcTimeout = controller.hasCallTimeout() ? controller.getCallTimeout() : 0;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -84,9 +84,10 @@ public class AsyncCall extends DefaultPromise<Message> {
|
|||
return this.startTime;
|
||||
}
|
||||
|
||||
@Override public String toString() {
|
||||
return "callId: " + this.id + " methodName: " + this.method.getName() + " param {" +
|
||||
(this.param != null ? ProtobufUtil.getShortTextFormat(this.param) : "") + "}";
|
||||
@Override
|
||||
public String toString() {
|
||||
return "callId: " + this.id + " methodName: " + this.method.getName() + " param {"
|
||||
+ (this.param != null ? ProtobufUtil.getShortTextFormat(this.param) : "") + "}";
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,9 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import io.netty.bootstrap.Bootstrap;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufOutputStream;
|
||||
|
@ -31,11 +28,29 @@ import io.netty.util.Timeout;
|
|||
import io.netty.util.TimerTask;
|
||||
import io.netty.util.concurrent.GenericFutureListener;
|
||||
import io.netty.util.concurrent.Promise;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.ConnectException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.SocketException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import javax.security.sasl.SaslException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.TracingProtos;
|
||||
|
@ -56,18 +71,9 @@ import org.apache.hadoop.security.token.TokenSelector;
|
|||
import org.apache.htrace.Span;
|
||||
import org.apache.htrace.Trace;
|
||||
|
||||
import javax.security.sasl.SaslException;
|
||||
import java.io.IOException;
|
||||
import java.net.ConnectException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.SocketException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.ConcurrentSkipListMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
|
||||
/**
|
||||
* Netty RPC channel
|
||||
|
@ -97,8 +103,6 @@ public class AsyncRpcChannel {
|
|||
final String serviceName;
|
||||
final InetSocketAddress address;
|
||||
|
||||
ConcurrentSkipListMap<Integer, AsyncCall> calls = new ConcurrentSkipListMap<>();
|
||||
|
||||
private int ioFailureCounter = 0;
|
||||
private int connectFailureCounter = 0;
|
||||
|
||||
|
@ -108,15 +112,18 @@ public class AsyncRpcChannel {
|
|||
private Token<? extends TokenIdentifier> token;
|
||||
private String serverPrincipal;
|
||||
|
||||
volatile boolean shouldCloseConnection = false;
|
||||
private IOException closeException;
|
||||
|
||||
// NOTE: closed and connected flags below are only changed when a lock on pendingCalls
|
||||
private final Map<Integer, AsyncCall> pendingCalls = new HashMap<Integer, AsyncCall>();
|
||||
private boolean connected = false;
|
||||
private boolean closed = false;
|
||||
|
||||
private Timeout cleanupTimer;
|
||||
|
||||
private final TimerTask timeoutTask = new TimerTask() {
|
||||
@Override public void run(Timeout timeout) throws Exception {
|
||||
cleanupTimer = null;
|
||||
cleanupCalls(false);
|
||||
@Override
|
||||
public void run(Timeout timeout) throws Exception {
|
||||
cleanupCalls();
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -183,10 +190,11 @@ public class AsyncRpcChannel {
|
|||
if (ticket == null) {
|
||||
throw new FatalConnectionException("ticket/user is null");
|
||||
}
|
||||
final UserGroupInformation realTicket = ticket;
|
||||
saslHandler = ticket.doAs(new PrivilegedExceptionAction<SaslClientHandler>() {
|
||||
@Override
|
||||
public SaslClientHandler run() throws IOException {
|
||||
return getSaslHandler(bootstrap);
|
||||
return getSaslHandler(realTicket, bootstrap);
|
||||
}
|
||||
});
|
||||
if (saslHandler != null) {
|
||||
|
@ -213,15 +221,20 @@ public class AsyncRpcChannel {
|
|||
ch.pipeline()
|
||||
.addLast("frameDecoder", new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4));
|
||||
ch.pipeline().addLast(new AsyncServerResponseHandler(this));
|
||||
|
||||
try {
|
||||
writeChannelHeader(ch).addListener(new GenericFutureListener<ChannelFuture>() {
|
||||
@Override public void operationComplete(ChannelFuture future) throws Exception {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
if (!future.isSuccess()) {
|
||||
close(future.cause());
|
||||
return;
|
||||
}
|
||||
for (AsyncCall call : calls.values()) {
|
||||
List<AsyncCall> callsToWrite;
|
||||
synchronized (pendingCalls) {
|
||||
connected = true;
|
||||
callsToWrite = new ArrayList<AsyncCall>(pendingCalls.values());
|
||||
}
|
||||
for (AsyncCall call : callsToWrite) {
|
||||
writeRequest(call);
|
||||
}
|
||||
}
|
||||
|
@ -233,24 +246,26 @@ public class AsyncRpcChannel {
|
|||
|
||||
/**
|
||||
* Get SASL handler
|
||||
*
|
||||
* @param bootstrap to reconnect to
|
||||
* @return new SASL handler
|
||||
* @throws java.io.IOException if handler failed to create
|
||||
*/
|
||||
private SaslClientHandler getSaslHandler(final Bootstrap bootstrap) throws IOException {
|
||||
return new SaslClientHandler(authMethod, token, serverPrincipal, client.fallbackAllowed,
|
||||
client.conf.get("hbase.rpc.protection",
|
||||
SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase()),
|
||||
private SaslClientHandler getSaslHandler(final UserGroupInformation realTicket,
|
||||
final Bootstrap bootstrap) throws IOException {
|
||||
return new SaslClientHandler(realTicket, authMethod, token, serverPrincipal,
|
||||
client.fallbackAllowed, client.conf.get("hbase.rpc.protection",
|
||||
SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase()),
|
||||
new SaslClientHandler.SaslExceptionHandler() {
|
||||
@Override public void handle(int retryCount, Random random, Throwable cause) {
|
||||
@Override
|
||||
public void handle(int retryCount, Random random, Throwable cause) {
|
||||
try {
|
||||
// Handle Sasl failure. Try to potentially get new credentials
|
||||
handleSaslConnectionFailure(retryCount, cause, ticket.getUGI());
|
||||
handleSaslConnectionFailure(retryCount, cause, realTicket);
|
||||
|
||||
// Try to reconnect
|
||||
AsyncRpcClient.WHEEL_TIMER.newTimeout(new TimerTask() {
|
||||
@Override public void run(Timeout timeout) throws Exception {
|
||||
client.newTimeout(new TimerTask() {
|
||||
@Override
|
||||
public void run(Timeout timeout) throws Exception {
|
||||
connect(bootstrap);
|
||||
}
|
||||
}, random.nextInt(reloginMaxBackoff) + 1, TimeUnit.MILLISECONDS);
|
||||
|
@ -259,10 +274,11 @@ public class AsyncRpcChannel {
|
|||
}
|
||||
}
|
||||
}, new SaslClientHandler.SaslSuccessfulConnectHandler() {
|
||||
@Override public void onSuccess(Channel channel) {
|
||||
startHBaseConnection(channel);
|
||||
}
|
||||
});
|
||||
@Override
|
||||
public void onSuccess(Channel channel) {
|
||||
startHBaseConnection(channel);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -274,7 +290,7 @@ public class AsyncRpcChannel {
|
|||
*/
|
||||
private void retryOrClose(final Bootstrap bootstrap, int connectCounter, Throwable e) {
|
||||
if (connectCounter < client.maxRetries) {
|
||||
AsyncRpcClient.WHEEL_TIMER.newTimeout(new TimerTask() {
|
||||
client.newTimeout(new TimerTask() {
|
||||
@Override public void run(Timeout timeout) throws Exception {
|
||||
connect(bootstrap);
|
||||
}
|
||||
|
@ -295,66 +311,50 @@ public class AsyncRpcChannel {
|
|||
public Promise<Message> callMethod(final Descriptors.MethodDescriptor method,
|
||||
final PayloadCarryingRpcController controller, final Message request,
|
||||
final Message responsePrototype) {
|
||||
if (shouldCloseConnection) {
|
||||
Promise<Message> promise = channel.eventLoop().newPromise();
|
||||
promise.setFailure(new ConnectException());
|
||||
return promise;
|
||||
}
|
||||
|
||||
final AsyncCall call = new AsyncCall(channel.eventLoop(), client.callIdCnt.getAndIncrement(),
|
||||
method, request, controller, responsePrototype);
|
||||
|
||||
final AsyncCall call =
|
||||
new AsyncCall(channel.eventLoop(), client.callIdCnt.getAndIncrement(), method, request,
|
||||
controller, responsePrototype);
|
||||
controller.notifyOnCancel(new RpcCallback<Object>() {
|
||||
@Override
|
||||
public void run(Object parameter) {
|
||||
calls.remove(call.id);
|
||||
// TODO: do not need to call AsyncCall.setFailed?
|
||||
synchronized (pendingCalls) {
|
||||
pendingCalls.remove(call.id);
|
||||
}
|
||||
}
|
||||
});
|
||||
// TODO: this should be handled by PayloadCarryingRpcController.
|
||||
if (controller.isCanceled()) {
|
||||
// To finish if the call was cancelled before we set the notification (race condition)
|
||||
call.cancel(true);
|
||||
return call;
|
||||
}
|
||||
|
||||
calls.put(call.id, call);
|
||||
|
||||
// check again, see https://issues.apache.org/jira/browse/HBASE-12951
|
||||
if (shouldCloseConnection) {
|
||||
Promise<Message> promise = channel.eventLoop().newPromise();
|
||||
promise.setFailure(new ConnectException());
|
||||
return promise;
|
||||
synchronized (pendingCalls) {
|
||||
if (closed) {
|
||||
Promise<Message> promise = channel.eventLoop().newPromise();
|
||||
promise.setFailure(new ConnectException());
|
||||
return promise;
|
||||
}
|
||||
pendingCalls.put(call.id, call);
|
||||
// Add timeout for cleanup if none is present
|
||||
if (cleanupTimer == null && call.getRpcTimeout() > 0) {
|
||||
cleanupTimer =
|
||||
client.newTimeout(timeoutTask, call.getRpcTimeout(),
|
||||
TimeUnit.MILLISECONDS);
|
||||
}
|
||||
if (!connected) {
|
||||
return call;
|
||||
}
|
||||
}
|
||||
|
||||
// Add timeout for cleanup if none is present
|
||||
if (cleanupTimer == null) {
|
||||
cleanupTimer = AsyncRpcClient.WHEEL_TIMER.newTimeout(timeoutTask, call.getRpcTimeout(),
|
||||
TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
if(channel.isActive()) {
|
||||
writeRequest(call);
|
||||
}
|
||||
|
||||
writeRequest(call);
|
||||
return call;
|
||||
}
|
||||
|
||||
/**
|
||||
* Calls method and returns a promise
|
||||
* @param method to call
|
||||
* @param controller to run call with
|
||||
* @param request to send
|
||||
* @param responsePrototype for response message
|
||||
* @return Promise to listen to result
|
||||
* @throws java.net.ConnectException on connection failures
|
||||
*/
|
||||
public Promise<Message> callMethodWithPromise(
|
||||
final Descriptors.MethodDescriptor method, final PayloadCarryingRpcController controller,
|
||||
final Message request, final Message responsePrototype) throws ConnectException {
|
||||
if (shouldCloseConnection || !channel.isOpen()) {
|
||||
throw new ConnectException();
|
||||
AsyncCall removePendingCall(int id) {
|
||||
synchronized (pendingCalls) {
|
||||
return pendingCalls.remove(id);
|
||||
}
|
||||
|
||||
return this.callMethod(method, controller, request, responsePrototype);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -380,6 +380,7 @@ public class AsyncRpcChannel {
|
|||
headerBuilder.setCellBlockCompressorClass(client.compressor.getClass().getCanonicalName());
|
||||
}
|
||||
|
||||
headerBuilder.setVersionInfo(ProtobufUtil.getVersionInfo());
|
||||
RPCProtos.ConnectionHeader header = headerBuilder.build();
|
||||
|
||||
|
||||
|
@ -400,10 +401,6 @@ public class AsyncRpcChannel {
|
|||
*/
|
||||
private void writeRequest(final AsyncCall call) {
|
||||
try {
|
||||
if (shouldCloseConnection) {
|
||||
return;
|
||||
}
|
||||
|
||||
final RPCProtos.RequestHeader.Builder requestHeaderBuilder = RPCProtos.RequestHeader
|
||||
.newBuilder();
|
||||
requestHeaderBuilder.setCallId(call.id)
|
||||
|
@ -439,25 +436,12 @@ public class AsyncRpcChannel {
|
|||
IPCUtil.write(out, rh, call.param, cellBlock);
|
||||
}
|
||||
|
||||
channel.writeAndFlush(b).addListener(new CallWriteListener(this,call));
|
||||
channel.writeAndFlush(b).addListener(new CallWriteListener(this, call.id));
|
||||
} catch (IOException e) {
|
||||
if (!shouldCloseConnection) {
|
||||
close(e);
|
||||
}
|
||||
close(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fail a call
|
||||
*
|
||||
* @param call to fail
|
||||
* @param cause of fail
|
||||
*/
|
||||
void failCall(AsyncCall call, IOException cause) {
|
||||
calls.remove(call.id);
|
||||
call.setFailed(cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set up server authorization
|
||||
*
|
||||
|
@ -550,18 +534,22 @@ public class AsyncRpcChannel {
|
|||
* @param e exception on close
|
||||
*/
|
||||
public void close(final Throwable e) {
|
||||
client.removeConnection(ConnectionId.hashCode(ticket,serviceName,address));
|
||||
client.removeConnection(this);
|
||||
|
||||
// Move closing from the requesting thread to the channel thread
|
||||
channel.eventLoop().execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
if (shouldCloseConnection) {
|
||||
return;
|
||||
List<AsyncCall> toCleanup;
|
||||
synchronized (pendingCalls) {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
closed = true;
|
||||
toCleanup = new ArrayList<AsyncCall>(pendingCalls.values());
|
||||
pendingCalls.clear();
|
||||
}
|
||||
|
||||
shouldCloseConnection = true;
|
||||
|
||||
IOException closeException = null;
|
||||
if (e != null) {
|
||||
if (e instanceof IOException) {
|
||||
closeException = (IOException) e;
|
||||
|
@ -569,16 +557,19 @@ public class AsyncRpcChannel {
|
|||
closeException = new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
// log the info
|
||||
if (LOG.isDebugEnabled() && closeException != null) {
|
||||
LOG.debug(name + ": closing ipc connection to " + address + ": " +
|
||||
closeException.getMessage());
|
||||
LOG.debug(name + ": closing ipc connection to " + address, closeException);
|
||||
}
|
||||
if (cleanupTimer != null) {
|
||||
cleanupTimer.cancel();
|
||||
cleanupTimer = null;
|
||||
}
|
||||
for (AsyncCall call : toCleanup) {
|
||||
call.setFailed(closeException != null ? closeException : new ConnectionClosingException(
|
||||
"Call id=" + call.id + " on server " + address + " aborted: connection is closing"));
|
||||
}
|
||||
|
||||
cleanupCalls(true);
|
||||
channel.disconnect().addListener(ChannelFutureListener.CLOSE);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(name + ": closed");
|
||||
}
|
||||
|
@ -591,64 +582,37 @@ public class AsyncRpcChannel {
|
|||
*
|
||||
* @param cleanAll true if all calls should be cleaned, false for only the timed out calls
|
||||
*/
|
||||
public void cleanupCalls(boolean cleanAll) {
|
||||
// Cancel outstanding timers
|
||||
if (cleanupTimer != null) {
|
||||
cleanupTimer.cancel();
|
||||
cleanupTimer = null;
|
||||
}
|
||||
|
||||
if (cleanAll) {
|
||||
for (AsyncCall call : calls.values()) {
|
||||
synchronized (call) {
|
||||
// Calls can be done on another thread so check before failing them
|
||||
if(!call.isDone()) {
|
||||
if (closeException == null) {
|
||||
failCall(call, new ConnectionClosingException("Call id=" + call.id +
|
||||
" on server " + address + " aborted: connection is closing"));
|
||||
} else {
|
||||
failCall(call, closeException);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (AsyncCall call : calls.values()) {
|
||||
long waitTime = EnvironmentEdgeManager.currentTime() - call.getStartTime();
|
||||
private void cleanupCalls() {
|
||||
List<AsyncCall> toCleanup = new ArrayList<AsyncCall>();
|
||||
long currentTime = EnvironmentEdgeManager.currentTime();
|
||||
long nextCleanupTaskDelay = -1L;
|
||||
synchronized (pendingCalls) {
|
||||
for (Iterator<AsyncCall> iter = pendingCalls.values().iterator(); iter.hasNext();) {
|
||||
AsyncCall call = iter.next();
|
||||
long timeout = call.getRpcTimeout();
|
||||
if (timeout > 0 && waitTime >= timeout) {
|
||||
synchronized (call) {
|
||||
// Calls can be done on another thread so check before failing them
|
||||
if (!call.isDone()) {
|
||||
closeException = new CallTimeoutException("Call id=" + call.id +
|
||||
", waitTime=" + waitTime + ", rpcTimeout=" + timeout);
|
||||
failCall(call, closeException);
|
||||
if (timeout > 0) {
|
||||
if (currentTime - call.getStartTime() >= timeout) {
|
||||
iter.remove();
|
||||
toCleanup.add(call);
|
||||
} else {
|
||||
if (nextCleanupTaskDelay < 0 || timeout < nextCleanupTaskDelay) {
|
||||
nextCleanupTaskDelay = timeout;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// We expect the call to be ordered by timeout. It may not be the case, but stopping
|
||||
// at the first valid call allows to be sure that we still have something to do without
|
||||
// spending too much time by reading the full list.
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!calls.isEmpty()) {
|
||||
AsyncCall firstCall = calls.firstEntry().getValue();
|
||||
|
||||
final long newTimeout;
|
||||
long maxWaitTime = EnvironmentEdgeManager.currentTime() - firstCall.getStartTime();
|
||||
if (maxWaitTime < firstCall.getRpcTimeout()) {
|
||||
newTimeout = firstCall.getRpcTimeout() - maxWaitTime;
|
||||
} else {
|
||||
newTimeout = 0;
|
||||
}
|
||||
|
||||
closeException = null;
|
||||
cleanupTimer = AsyncRpcClient.WHEEL_TIMER.newTimeout(timeoutTask,
|
||||
newTimeout, TimeUnit.MILLISECONDS);
|
||||
if (nextCleanupTaskDelay > 0) {
|
||||
cleanupTimer =
|
||||
client.newTimeout(timeoutTask, nextCleanupTaskDelay,
|
||||
TimeUnit.MILLISECONDS);
|
||||
} else {
|
||||
cleanupTimer = null;
|
||||
}
|
||||
}
|
||||
for (AsyncCall call : toCleanup) {
|
||||
call.setFailed(new CallTimeoutException("Call id=" + call.id + ", waitTime="
|
||||
+ (currentTime - call.getStartTime()) + ", rpcTimeout=" + call.getRpcTimeout()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -745,6 +709,10 @@ public class AsyncRpcChannel {
|
|||
});
|
||||
}
|
||||
|
||||
public int getConnectionHashCode() {
|
||||
return ConnectionId.hashCode(ticket, serviceName, address);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.address.toString() + "/" + this.serviceName + "/" + this.ticket;
|
||||
|
@ -755,20 +723,22 @@ public class AsyncRpcChannel {
|
|||
*/
|
||||
private static final class CallWriteListener implements ChannelFutureListener {
|
||||
private final AsyncRpcChannel rpcChannel;
|
||||
private final AsyncCall call;
|
||||
private final int id;
|
||||
|
||||
public CallWriteListener(AsyncRpcChannel asyncRpcChannel, AsyncCall call) {
|
||||
public CallWriteListener(AsyncRpcChannel asyncRpcChannel, int id) {
|
||||
this.rpcChannel = asyncRpcChannel;
|
||||
this.call = call;
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
@Override public void operationComplete(ChannelFuture future) throws Exception {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
if (!future.isSuccess()) {
|
||||
if(!this.call.isDone()) {
|
||||
AsyncCall call = rpcChannel.removePendingCall(id);
|
||||
if (call != null) {
|
||||
if (future.cause() instanceof IOException) {
|
||||
rpcChannel.failCall(call, (IOException) future.cause());
|
||||
call.setFailed((IOException) future.cause());
|
||||
} else {
|
||||
rpcChannel.failCall(call, new IOException(future.cause()));
|
||||
call.setFailed(new IOException(future.cause()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,14 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import com.google.protobuf.RpcController;
|
||||
import io.netty.bootstrap.Bootstrap;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelInitializer;
|
||||
import io.netty.channel.ChannelOption;
|
||||
|
@ -35,11 +28,24 @@ import io.netty.channel.nio.NioEventLoopGroup;
|
|||
import io.netty.channel.socket.SocketChannel;
|
||||
import io.netty.channel.socket.nio.NioSocketChannel;
|
||||
import io.netty.util.HashedWheelTimer;
|
||||
import io.netty.util.Timeout;
|
||||
import io.netty.util.TimerTask;
|
||||
import io.netty.util.concurrent.Future;
|
||||
import io.netty.util.concurrent.GenericFutureListener;
|
||||
import io.netty.util.concurrent.Promise;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.SocketAddress;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -49,25 +55,26 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
import org.apache.hadoop.hbase.util.PoolMap;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.SocketAddress;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
/**
|
||||
* Netty client for the requests and responses
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
|
||||
public class AsyncRpcClient extends AbstractRpcClient {
|
||||
|
||||
public static final String CLIENT_MAX_THREADS = "hbase.rpc.client.threads.max";
|
||||
public static final String USE_NATIVE_TRANSPORT = "hbase.rpc.client.useNativeTransport";
|
||||
public static final String USE_NATIVE_TRANSPORT = "hbase.rpc.client.nativetransport";
|
||||
public static final String USE_GLOBAL_EVENT_LOOP_GROUP = "hbase.rpc.client.globaleventloopgroup";
|
||||
|
||||
public static final HashedWheelTimer WHEEL_TIMER =
|
||||
new HashedWheelTimer(100, TimeUnit.MILLISECONDS);
|
||||
private static final HashedWheelTimer WHEEL_TIMER =
|
||||
new HashedWheelTimer(Threads.newDaemonThreadFactory("AsyncRpcChannel-timer"),
|
||||
100, TimeUnit.MILLISECONDS);
|
||||
|
||||
private static final ChannelInitializer<SocketChannel> DEFAULT_CHANNEL_INITIALIZER =
|
||||
new ChannelInitializer<SocketChannel>() {
|
||||
|
@ -79,12 +86,54 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
|
||||
protected final AtomicInteger callIdCnt = new AtomicInteger();
|
||||
|
||||
private final EventLoopGroup eventLoopGroup;
|
||||
private final PoolMap<Integer, AsyncRpcChannel> connections;
|
||||
|
||||
final FailedServers failedServers;
|
||||
|
||||
private final Bootstrap bootstrap;
|
||||
@VisibleForTesting
|
||||
final Bootstrap bootstrap;
|
||||
|
||||
private final boolean useGlobalEventLoopGroup;
|
||||
|
||||
@VisibleForTesting
|
||||
static Pair<EventLoopGroup, Class<? extends Channel>> GLOBAL_EVENT_LOOP_GROUP;
|
||||
|
||||
private synchronized static Pair<EventLoopGroup, Class<? extends Channel>>
|
||||
getGlobalEventLoopGroup(Configuration conf) {
|
||||
if (GLOBAL_EVENT_LOOP_GROUP == null) {
|
||||
GLOBAL_EVENT_LOOP_GROUP = createEventLoopGroup(conf);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Create global event loop group "
|
||||
+ GLOBAL_EVENT_LOOP_GROUP.getFirst().getClass().getSimpleName());
|
||||
}
|
||||
}
|
||||
return GLOBAL_EVENT_LOOP_GROUP;
|
||||
}
|
||||
|
||||
private static Pair<EventLoopGroup, Class<? extends Channel>> createEventLoopGroup(
|
||||
Configuration conf) {
|
||||
// Max amount of threads to use. 0 lets Netty decide based on amount of cores
|
||||
int maxThreads = conf.getInt(CLIENT_MAX_THREADS, 0);
|
||||
|
||||
// Config to enable native transport. Does not seem to be stable at time of implementation
|
||||
// although it is not extensively tested.
|
||||
boolean epollEnabled = conf.getBoolean(USE_NATIVE_TRANSPORT, false);
|
||||
|
||||
// Use the faster native epoll transport mechanism on linux if enabled
|
||||
if (epollEnabled && JVM.isLinux()) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Create EpollEventLoopGroup with maxThreads = " + maxThreads);
|
||||
}
|
||||
return new Pair<EventLoopGroup, Class<? extends Channel>>(new EpollEventLoopGroup(maxThreads,
|
||||
Threads.newDaemonThreadFactory("AsyncRpcChannel")), EpollSocketChannel.class);
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Create NioEventLoopGroup with maxThreads = " + maxThreads);
|
||||
}
|
||||
return new Pair<EventLoopGroup, Class<? extends Channel>>(new NioEventLoopGroup(maxThreads,
|
||||
Threads.newDaemonThreadFactory("AsyncRpcChannel")), NioSocketChannel.class);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor for tests
|
||||
|
@ -103,23 +152,16 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
LOG.debug("Starting async Hbase RPC client");
|
||||
}
|
||||
|
||||
// Max amount of threads to use. 0 lets Netty decide based on amount of cores
|
||||
int maxThreads = conf.getInt(CLIENT_MAX_THREADS, 0);
|
||||
|
||||
// Config to enable native transport. Does not seem to be stable at time of implementation
|
||||
// although it is not extensively tested.
|
||||
boolean epollEnabled = conf.getBoolean(USE_NATIVE_TRANSPORT, false);
|
||||
|
||||
// Use the faster native epoll transport mechanism on linux if enabled
|
||||
Class<? extends Channel> socketChannelClass;
|
||||
if (epollEnabled && JVM.isLinux()) {
|
||||
socketChannelClass = EpollSocketChannel.class;
|
||||
this.eventLoopGroup =
|
||||
new EpollEventLoopGroup(maxThreads, Threads.newDaemonThreadFactory("AsyncRpcChannel"));
|
||||
Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass;
|
||||
this.useGlobalEventLoopGroup = conf.getBoolean(USE_GLOBAL_EVENT_LOOP_GROUP, true);
|
||||
if (useGlobalEventLoopGroup) {
|
||||
eventLoopGroupAndChannelClass = getGlobalEventLoopGroup(configuration);
|
||||
} else {
|
||||
socketChannelClass = NioSocketChannel.class;
|
||||
this.eventLoopGroup =
|
||||
new NioEventLoopGroup(maxThreads, Threads.newDaemonThreadFactory("AsyncRpcChannel"));
|
||||
eventLoopGroupAndChannelClass = createEventLoopGroup(configuration);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Use " + (useGlobalEventLoopGroup ? "global" : "individual") + " event loop group "
|
||||
+ eventLoopGroupAndChannelClass.getFirst().getClass().getSimpleName());
|
||||
}
|
||||
|
||||
this.connections = new PoolMap<>(getPoolType(configuration), getPoolSize(configuration));
|
||||
|
@ -130,8 +172,8 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
|
||||
// Configure the default bootstrap.
|
||||
this.bootstrap = new Bootstrap();
|
||||
bootstrap.group(eventLoopGroup).channel(socketChannelClass)
|
||||
.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
|
||||
bootstrap.group(eventLoopGroupAndChannelClass.getFirst())
|
||||
.channel(eventLoopGroupAndChannelClass.getSecond())
|
||||
.option(ChannelOption.TCP_NODELAY, tcpNoDelay)
|
||||
.option(ChannelOption.SO_KEEPALIVE, tcpKeepAlive)
|
||||
.option(ChannelOption.CONNECT_TIMEOUT_MILLIS, operationTimeout);
|
||||
|
@ -169,16 +211,19 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
* @throws InterruptedException if call is interrupted
|
||||
* @throws java.io.IOException if a connection failure is encountered
|
||||
*/
|
||||
@Override protected Pair<Message, CellScanner> call(PayloadCarryingRpcController pcrc,
|
||||
@Override
|
||||
protected Pair<Message, CellScanner> call(PayloadCarryingRpcController pcrc,
|
||||
Descriptors.MethodDescriptor md, Message param, Message returnType, User ticket,
|
||||
InetSocketAddress addr) throws IOException, InterruptedException {
|
||||
|
||||
if (pcrc == null) {
|
||||
pcrc = new PayloadCarryingRpcController();
|
||||
}
|
||||
final AsyncRpcChannel connection = createRpcChannel(md.getService().getName(), addr, ticket);
|
||||
|
||||
Promise<Message> promise = connection.callMethodWithPromise(md, pcrc, param, returnType);
|
||||
|
||||
Promise<Message> promise = connection.callMethod(md, pcrc, param, returnType);
|
||||
long timeout = pcrc.hasCallTimeout() ? pcrc.getCallTimeout() : 0;
|
||||
try {
|
||||
Message response = promise.get();
|
||||
Message response = timeout > 0 ? promise.get(timeout, TimeUnit.MILLISECONDS) : promise.get();
|
||||
return new Pair<>(response, pcrc.cellScanner());
|
||||
} catch (ExecutionException e) {
|
||||
if (e.getCause() instanceof IOException) {
|
||||
|
@ -186,6 +231,8 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
} else {
|
||||
throw new IOException(e.getCause());
|
||||
}
|
||||
} catch (TimeoutException e) {
|
||||
throw new CallTimeoutException(promise.toString());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -231,6 +278,8 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
}
|
||||
}
|
||||
|
||||
private boolean closed = false;
|
||||
|
||||
/**
|
||||
* Close netty
|
||||
*/
|
||||
|
@ -240,12 +289,18 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
}
|
||||
|
||||
synchronized (connections) {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
closed = true;
|
||||
for (AsyncRpcChannel conn : connections.values()) {
|
||||
conn.close(null);
|
||||
}
|
||||
}
|
||||
|
||||
eventLoopGroup.shutdownGracefully();
|
||||
// do not close global EventLoopGroup.
|
||||
if (!useGlobalEventLoopGroup) {
|
||||
bootstrap.group().shutdownGracefully();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -282,10 +337,6 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
*/
|
||||
private AsyncRpcChannel createRpcChannel(String serviceName, InetSocketAddress location,
|
||||
User ticket) throws StoppedRpcClientException, FailedServerException {
|
||||
if (this.eventLoopGroup.isShuttingDown() || this.eventLoopGroup.isShutdown()) {
|
||||
throw new StoppedRpcClientException();
|
||||
}
|
||||
|
||||
// Check if server is failed
|
||||
if (this.failedServers.isFailedServer(location)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -300,6 +351,9 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
|
||||
AsyncRpcChannel rpcChannel;
|
||||
synchronized (connections) {
|
||||
if (closed) {
|
||||
throw new StoppedRpcClientException();
|
||||
}
|
||||
rpcChannel = connections.get(hashCode);
|
||||
if (rpcChannel == null) {
|
||||
rpcChannel = new AsyncRpcChannel(this.bootstrap, this, ticket, serviceName, location);
|
||||
|
@ -337,12 +391,20 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
|
||||
/**
|
||||
* Remove connection from pool
|
||||
*
|
||||
* @param connectionHashCode of connection
|
||||
*/
|
||||
public void removeConnection(int connectionHashCode) {
|
||||
public void removeConnection(AsyncRpcChannel connection) {
|
||||
int connectionHashCode = connection.getConnectionHashCode();
|
||||
synchronized (connections) {
|
||||
this.connections.remove(connectionHashCode);
|
||||
// we use address as cache key, so we should check here to prevent removing the
|
||||
// wrong connection
|
||||
AsyncRpcChannel connectionInPool = this.connections.get(connectionHashCode);
|
||||
if (connectionInPool == connection) {
|
||||
this.connections.remove(connectionHashCode);
|
||||
} else if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("%s already removed, expected instance %08x, actual %08x",
|
||||
connection.toString(), System.identityHashCode(connection),
|
||||
System.identityHashCode(connectionInPool)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -399,4 +461,8 @@ public class AsyncRpcClient extends AbstractRpcClient {
|
|||
this.rpcClient.callMethod(md, pcrc, param, returnType, this.ticket, this.isa, done);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Timeout newTimeout(TimerTask task, long delay, TimeUnit unit) {
|
||||
return WHEEL_TIMER.newTimeout(task, delay, unit);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,11 +17,13 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufInputStream;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelInboundHandlerAdapter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
|
@ -30,7 +32,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
import java.io.IOException;
|
||||
import com.google.protobuf.Message;
|
||||
|
||||
/**
|
||||
* Handles Hbase responses
|
||||
|
@ -53,16 +55,12 @@ public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter {
|
|||
@Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
|
||||
ByteBuf inBuffer = (ByteBuf) msg;
|
||||
ByteBufInputStream in = new ByteBufInputStream(inBuffer);
|
||||
|
||||
if (channel.shouldCloseConnection) {
|
||||
return;
|
||||
}
|
||||
int totalSize = inBuffer.readableBytes();
|
||||
try {
|
||||
// Read the header
|
||||
RPCProtos.ResponseHeader responseHeader = RPCProtos.ResponseHeader.parseDelimitedFrom(in);
|
||||
int id = responseHeader.getCallId();
|
||||
AsyncCall call = channel.calls.get(id);
|
||||
AsyncCall call = channel.removePendingCall(id);
|
||||
if (call == null) {
|
||||
// So we got a response for which we have no corresponding 'call' here on the client-side.
|
||||
// We probably timed out waiting, cleaned up all references, and now the server decides
|
||||
|
@ -86,7 +84,7 @@ public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter {
|
|||
equals(FatalConnectionException.class.getName())) {
|
||||
channel.close(re);
|
||||
} else {
|
||||
channel.failCall(call, re);
|
||||
call.setFailed(re);
|
||||
}
|
||||
} else {
|
||||
Message value = null;
|
||||
|
@ -105,13 +103,11 @@ public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter {
|
|||
}
|
||||
call.setSuccess(value, cellBlockScanner);
|
||||
}
|
||||
channel.calls.remove(id);
|
||||
} catch (IOException e) {
|
||||
// Treat this as a fatal condition and close this connection
|
||||
channel.close(e);
|
||||
} finally {
|
||||
inBuffer.release();
|
||||
channel.cleanupCalls(false);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.codec.Codec;
|
||||
import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
|
||||
import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -65,6 +66,7 @@ public class IPCUtil {
|
|||
this.conf = conf;
|
||||
this.cellBlockDecompressionMultiplier =
|
||||
conf.getInt("hbase.ipc.cellblock.decompression.buffersize.multiplier", 3);
|
||||
|
||||
// Guess that 16k is a good size for rpc buffer. Could go bigger. See the TODO below in
|
||||
// #buildCellBlock.
|
||||
this.cellBlockBuildingInitialBufferSize =
|
||||
|
@ -90,24 +92,49 @@ public class IPCUtil {
|
|||
@SuppressWarnings("resource")
|
||||
public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
|
||||
final CellScanner cellScanner)
|
||||
throws IOException {
|
||||
return buildCellBlock(codec, compressor, cellScanner, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Puts CellScanner Cells into a cell block using passed in <code>codec</code> and/or
|
||||
* <code>compressor</code>.
|
||||
* @param codec
|
||||
* @param compressor
|
||||
* @param cellScanner
|
||||
* @param pool Pool of ByteBuffers to make use of. Can be null and then we'll allocate
|
||||
* our own ByteBuffer.
|
||||
* @return Null or byte buffer filled with a cellblock filled with passed-in Cells encoded using
|
||||
* passed in <code>codec</code> and/or <code>compressor</code>; the returned buffer has been
|
||||
* flipped and is ready for reading. Use limit to find total size. If <code>pool</code> was not
|
||||
* null, then this returned ByteBuffer came from there and should be returned to the pool when
|
||||
* done.
|
||||
* @throws IOException
|
||||
*/
|
||||
@SuppressWarnings("resource")
|
||||
public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
|
||||
final CellScanner cellScanner, final BoundedByteBufferPool pool)
|
||||
throws IOException {
|
||||
if (cellScanner == null) return null;
|
||||
if (codec == null) throw new CellScannerButNoCodecException();
|
||||
int bufferSize = this.cellBlockBuildingInitialBufferSize;
|
||||
if (cellScanner instanceof HeapSize) {
|
||||
long longSize = ((HeapSize)cellScanner).heapSize();
|
||||
// Just make sure we don't have a size bigger than an int.
|
||||
if (longSize > Integer.MAX_VALUE) {
|
||||
throw new IOException("Size " + longSize + " > " + Integer.MAX_VALUE);
|
||||
ByteBufferOutputStream baos = null;
|
||||
if (pool != null) {
|
||||
ByteBuffer bb = pool.getBuffer();
|
||||
bufferSize = bb.capacity();
|
||||
baos = new ByteBufferOutputStream(bb);
|
||||
} else {
|
||||
// Then we need to make our own to return.
|
||||
if (cellScanner instanceof HeapSize) {
|
||||
long longSize = ((HeapSize)cellScanner).heapSize();
|
||||
// Just make sure we don't have a size bigger than an int.
|
||||
if (longSize > Integer.MAX_VALUE) {
|
||||
throw new IOException("Size " + longSize + " > " + Integer.MAX_VALUE);
|
||||
}
|
||||
bufferSize = ClassSize.align((int)longSize);
|
||||
}
|
||||
bufferSize = ClassSize.align((int)longSize);
|
||||
} // TODO: Else, get estimate on size of buffer rather than have the buffer resize.
|
||||
// See TestIPCUtil main for experiment where we spin through the Cells getting estimate of
|
||||
// total size before creating the buffer. It costs somw small percentage. If we are usually
|
||||
// within the estimated buffer size, then the cost is not worth it. If we are often well
|
||||
// outside the guesstimated buffer size, the processing can be done in half the time if we
|
||||
// go w/ the estimated size rather than let the buffer resize.
|
||||
ByteBufferOutputStream baos = new ByteBufferOutputStream(bufferSize);
|
||||
baos = new ByteBufferOutputStream(bufferSize);
|
||||
}
|
||||
OutputStream os = baos;
|
||||
Compressor poolCompressor = null;
|
||||
try {
|
||||
|
|
|
@ -263,7 +263,10 @@ public class RpcClientImpl extends AbstractRpcClient {
|
|||
try {
|
||||
Connection.this.tracedWriteRequest(cts.call, cts.priority, cts.span);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("call write error for call #" + cts.call.id + ", message =" + e.getMessage());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("call write error for call #" + cts.call.id
|
||||
+ ", message =" + e.getMessage());
|
||||
}
|
||||
cts.call.setException(e);
|
||||
markClosed(e);
|
||||
}
|
||||
|
@ -355,6 +358,7 @@ public class RpcClientImpl extends AbstractRpcClient {
|
|||
if (this.compressor != null) {
|
||||
builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
|
||||
}
|
||||
builder.setVersionInfo(ProtobufUtil.getVersionInfo());
|
||||
this.header = builder.build();
|
||||
|
||||
this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
|
||||
|
@ -1133,6 +1137,7 @@ public class RpcClientImpl extends AbstractRpcClient {
|
|||
* @throws InterruptedException
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
protected Pair<Message, CellScanner> call(PayloadCarryingRpcController pcrc, MethodDescriptor md,
|
||||
Message param, Message returnType, User ticket, InetSocketAddress addr)
|
||||
throws IOException, InterruptedException {
|
||||
|
|
|
@ -40,7 +40,6 @@ import java.util.NavigableSet;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.google.protobuf.*;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -56,6 +55,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Consistency;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -71,7 +71,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
|||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
|
@ -89,6 +88,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsReques
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
|
@ -107,6 +107,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Col
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||
|
@ -120,6 +121,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableReques
|
|||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
|
@ -133,6 +135,8 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
|||
import org.apache.hadoop.hbase.quotas.QuotaScope;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaType;
|
||||
import org.apache.hadoop.hbase.quotas.ThrottleType;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
|
||||
import org.apache.hadoop.hbase.security.access.Permission;
|
||||
import org.apache.hadoop.hbase.security.access.TablePermission;
|
||||
import org.apache.hadoop.hbase.security.access.UserPermission;
|
||||
|
@ -145,6 +149,7 @@ import org.apache.hadoop.hbase.util.DynamicClassLoader;
|
|||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
import org.apache.hadoop.hbase.util.Methods;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.VersionInfo;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
@ -1277,6 +1282,7 @@ public final class ProtobufUtil {
|
|||
}
|
||||
|
||||
builder.setStale(result.isStale());
|
||||
builder.setPartial(result.isPartial());
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
@ -1335,7 +1341,7 @@ public final class ProtobufUtil {
|
|||
for (CellProtos.Cell c : values) {
|
||||
cells.add(toCell(c));
|
||||
}
|
||||
return Result.create(cells, null, proto.getStale());
|
||||
return Result.create(cells, null, proto.getStale(), proto.getPartial());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1718,6 +1724,26 @@ public final class ProtobufUtil {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A helper to warmup a region given a region name
|
||||
* using admin protocol
|
||||
*
|
||||
* @param admin
|
||||
* @param regionInfo
|
||||
*
|
||||
*/
|
||||
public static void warmupRegion(final AdminService.BlockingInterface admin,
|
||||
final HRegionInfo regionInfo) throws IOException {
|
||||
|
||||
try {
|
||||
WarmupRegionRequest warmupRegionRequest =
|
||||
RequestConverter.buildWarmupRegionRequest(regionInfo);
|
||||
|
||||
admin.warmupRegion(null, warmupRegionRequest);
|
||||
} catch (ServiceException e) {
|
||||
throw getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A helper to open a region using admin protocol.
|
||||
|
@ -1736,6 +1762,7 @@ public final class ProtobufUtil {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A helper to get the all the online regions on a region
|
||||
* server using admin protocol.
|
||||
|
@ -2576,6 +2603,7 @@ public final class ProtobufUtil {
|
|||
FlushDescriptor.Builder desc = FlushDescriptor.newBuilder()
|
||||
.setAction(action)
|
||||
.setEncodedRegionName(ByteStringer.wrap(hri.getEncodedNameAsBytes()))
|
||||
.setRegionName(ByteStringer.wrap(hri.getRegionName()))
|
||||
.setFlushSequenceNumber(flushSeqId)
|
||||
.setTableName(ByteStringer.wrap(hri.getTable().getName()));
|
||||
|
||||
|
@ -2601,6 +2629,7 @@ public final class ProtobufUtil {
|
|||
.setEventType(eventType)
|
||||
.setTableName(ByteStringer.wrap(hri.getTable().getName()))
|
||||
.setEncodedRegionName(ByteStringer.wrap(hri.getEncodedNameAsBytes()))
|
||||
.setRegionName(ByteStringer.wrap(hri.getRegionName()))
|
||||
.setLogSequenceNumber(seqId)
|
||||
.setServer(toServerName(server));
|
||||
|
||||
|
@ -2991,7 +3020,6 @@ public final class ProtobufUtil {
|
|||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* This version of protobuf's mergeDelimitedFrom avoid the hard-coded 64MB limit for decoding
|
||||
* buffers
|
||||
|
@ -3071,4 +3099,40 @@ public final class ProtobufUtil {
|
|||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
public static ReplicationLoadSink toReplicationLoadSink(
|
||||
ClusterStatusProtos.ReplicationLoadSink cls) {
|
||||
return new ReplicationLoadSink(cls.getAgeOfLastAppliedOp(), cls.getTimeStampsOfLastAppliedOp());
|
||||
}
|
||||
|
||||
public static ReplicationLoadSource toReplicationLoadSource(
|
||||
ClusterStatusProtos.ReplicationLoadSource cls) {
|
||||
return new ReplicationLoadSource(cls.getPeerID(), cls.getAgeOfLastShippedOp(),
|
||||
cls.getSizeOfLogQueue(), cls.getTimeStampOfLastShippedOp(), cls.getReplicationLag());
|
||||
}
|
||||
|
||||
public static List<ReplicationLoadSource> toReplicationLoadSourceList(
|
||||
List<ClusterStatusProtos.ReplicationLoadSource> clsList) {
|
||||
ArrayList<ReplicationLoadSource> rlsList = new ArrayList<ReplicationLoadSource>();
|
||||
for (ClusterStatusProtos.ReplicationLoadSource cls : clsList) {
|
||||
rlsList.add(toReplicationLoadSource(cls));
|
||||
}
|
||||
return rlsList;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a protocol buffer VersionInfo
|
||||
*
|
||||
* @return the converted protocol buffer VersionInfo
|
||||
*/
|
||||
public static RPCProtos.VersionInfo getVersionInfo() {
|
||||
RPCProtos.VersionInfo.Builder builder = RPCProtos.VersionInfo.newBuilder();
|
||||
builder.setVersion(VersionInfo.getVersion());
|
||||
builder.setUrl(VersionInfo.getUrl());
|
||||
builder.setRevision(VersionInfo.getRevision());
|
||||
builder.setUser(VersionInfo.getUser());
|
||||
builder.setDate(VersionInfo.getDate());
|
||||
builder.setSrcChecksum(VersionInfo.getSrcChecksum());
|
||||
return builder.build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
|||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
|
||||
|
@ -91,6 +92,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterSta
|
|||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
|
||||
|
@ -486,6 +488,7 @@ public final class RequestConverter {
|
|||
builder.setCloseScanner(closeScanner);
|
||||
builder.setRegion(region);
|
||||
builder.setScan(ProtobufUtil.toScan(scan));
|
||||
builder.setClientHandlesPartials(true);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -503,6 +506,7 @@ public final class RequestConverter {
|
|||
builder.setNumberOfRows(numberOfRows);
|
||||
builder.setCloseScanner(closeScanner);
|
||||
builder.setScannerId(scannerId);
|
||||
builder.setClientHandlesPartials(true);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -522,6 +526,7 @@ public final class RequestConverter {
|
|||
builder.setCloseScanner(closeScanner);
|
||||
builder.setScannerId(scannerId);
|
||||
builder.setNextCallSeq(nextCallSeq);
|
||||
builder.setClientHandlesPartials(true);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -739,10 +744,22 @@ public final class RequestConverter {
|
|||
*/
|
||||
public static FlushRegionRequest
|
||||
buildFlushRegionRequest(final byte[] regionName) {
|
||||
return buildFlushRegionRequest(regionName, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a protocol buffer FlushRegionRequest for a given region name
|
||||
*
|
||||
* @param regionName the name of the region to get info
|
||||
* @return a protocol buffer FlushRegionRequest
|
||||
*/
|
||||
public static FlushRegionRequest
|
||||
buildFlushRegionRequest(final byte[] regionName, boolean writeFlushWALMarker) {
|
||||
FlushRegionRequest.Builder builder = FlushRegionRequest.newBuilder();
|
||||
RegionSpecifier region = buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
builder.setWriteFlushWalMarker(writeFlushWALMarker);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -834,6 +851,16 @@ public final class RequestConverter {
|
|||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a WarmupRegionRequest for a given region name
|
||||
*
|
||||
* @param regionInfo Region we are warming up
|
||||
*/
|
||||
public static WarmupRegionRequest buildWarmupRegionRequest(final HRegionInfo regionInfo) {
|
||||
WarmupRegionRequest.Builder builder = WarmupRegionRequest.newBuilder();
|
||||
builder.setRegionInfo(HRegionInfo.convert(regionInfo));
|
||||
return builder.build();
|
||||
}
|
||||
/**
|
||||
* Create a CloseRegionRequest for a given encoded region name
|
||||
*
|
||||
|
@ -1313,6 +1340,15 @@ public final class RequestConverter {
|
|||
return SetBalancerRunningRequest.newBuilder().setOn(on).setSynchronous(synchronous).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a protocol buffer IsBalancerEnabledRequest
|
||||
*
|
||||
* @return a IsBalancerEnabledRequest
|
||||
*/
|
||||
public static IsBalancerEnabledRequest buildIsBalancerEnabledRequest() {
|
||||
return IsBalancerEnabledRequest.newBuilder().build();
|
||||
}
|
||||
|
||||
/**
|
||||
* @see {@link #buildGetClusterStatusRequest}
|
||||
*/
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
|
||||
|
@ -300,8 +301,10 @@ public final class ResponseConverter {
|
|||
* @return A GetLastFlushedSequenceIdResponse
|
||||
*/
|
||||
public static GetLastFlushedSequenceIdResponse buildGetLastFlushedSequenceIdResponse(
|
||||
long seqId) {
|
||||
return GetLastFlushedSequenceIdResponse.newBuilder().setLastFlushedSequenceId(seqId).build();
|
||||
RegionStoreSequenceIds ids) {
|
||||
return GetLastFlushedSequenceIdResponse.newBuilder()
|
||||
.setLastFlushedSequenceId(ids.getLastFlushedSequenceId())
|
||||
.addAllStoreLastFlushedSequenceId(ids.getStoreSequenceIdList()).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -339,6 +342,9 @@ public final class ResponseConverter {
|
|||
// Cells are out in cellblocks. Group them up again as Results. How many to read at a
|
||||
// time will be found in getCellsLength -- length here is how many Cells in the i'th Result
|
||||
int noOfCells = response.getCellsPerResult(i);
|
||||
boolean isPartial =
|
||||
response.getPartialFlagPerResultCount() > i ?
|
||||
response.getPartialFlagPerResult(i) : false;
|
||||
List<Cell> cells = new ArrayList<Cell>(noOfCells);
|
||||
for (int j = 0; j < noOfCells; j++) {
|
||||
try {
|
||||
|
@ -361,7 +367,7 @@ public final class ResponseConverter {
|
|||
}
|
||||
cells.add(cellScanner.current());
|
||||
}
|
||||
results[i] = Result.create(cells, null, response.getStale());
|
||||
results[i] = Result.create(cells, null, response.getStale(), isPartial);
|
||||
} else {
|
||||
// Result is pure pb.
|
||||
results[i] = ProtobufUtil.toResult(response.getResults(i));
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
|
||||
<head />
|
||||
<body bgcolor="white">
|
||||
Holds classes generated from <a href="http://code.google.com/apis/protocolbuffers/">protobuf</a>
|
||||
Holds classes generated from <a href="https://developers.google.com/protocol-buffers/">protobuf</a>
|
||||
<code>src/main/protobuf</code> definition files.
|
||||
|
||||
<p>See under <code>src/main/protobuf</code> for instruction on how to generate the content under
|
||||
|
|
|
@ -21,8 +21,6 @@ package org.apache.hadoop.hbase.quotas;
|
|||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
|
@ -38,12 +36,11 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
public class ThrottlingException extends QuotaExceededException {
|
||||
private static final long serialVersionUID = 1406576492085155743L;
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ThrottlingException.class);
|
||||
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public enum Type {
|
||||
NumRequestsExceeded,
|
||||
RequestSizeExceeded,
|
||||
NumReadRequestsExceeded,
|
||||
NumWriteRequestsExceeded,
|
||||
WriteSizeExceeded,
|
||||
|
@ -52,6 +49,7 @@ public class ThrottlingException extends QuotaExceededException {
|
|||
|
||||
private static final String[] MSG_TYPE = new String[] {
|
||||
"number of requests exceeded",
|
||||
"request size limit exceeded",
|
||||
"number of read requests exceeded",
|
||||
"number of write requests exceeded",
|
||||
"write size limit exceeded",
|
||||
|
@ -98,6 +96,11 @@ public class ThrottlingException extends QuotaExceededException {
|
|||
throwThrottlingException(Type.NumRequestsExceeded, waitInterval);
|
||||
}
|
||||
|
||||
public static void throwRequestSizeExceeded(final long waitInterval)
|
||||
throws ThrottlingException {
|
||||
throwThrottlingException(Type.RequestSizeExceeded, waitInterval);
|
||||
}
|
||||
|
||||
public static void throwNumReadRequestsExceeded(final long waitInterval)
|
||||
throws ThrottlingException {
|
||||
throwThrottlingException(Type.NumReadRequestsExceeded, waitInterval);
|
||||
|
|
|
@ -0,0 +1,36 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license
|
||||
* agreements. See the NOTICE file distributed with this work for additional information regarding
|
||||
* copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
|
||||
* law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
|
||||
* BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
|
||||
* for the specific language governing permissions and limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.replication;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* A HBase ReplicationLoad to present MetricsSink information
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ReplicationLoadSink {
|
||||
private long ageOfLastAppliedOp;
|
||||
private long timeStampsOfLastAppliedOp;
|
||||
|
||||
public ReplicationLoadSink(long age, long timeStamp) {
|
||||
this.ageOfLastAppliedOp = age;
|
||||
this.timeStampsOfLastAppliedOp = timeStamp;
|
||||
}
|
||||
|
||||
public long getAgeOfLastAppliedOp() {
|
||||
return this.ageOfLastAppliedOp;
|
||||
}
|
||||
|
||||
public long getTimeStampsOfLastAppliedOp() {
|
||||
return this.timeStampsOfLastAppliedOp;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license
|
||||
* agreements. See the NOTICE file distributed with this work for additional information regarding
|
||||
* copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
|
||||
* law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
|
||||
* BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
|
||||
* for the specific language governing permissions and limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.replication;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* A HBase ReplicationLoad to present MetricsSource information
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ReplicationLoadSource {
|
||||
private String peerID;
|
||||
private long ageOfLastShippedOp;
|
||||
private int sizeOfLogQueue;
|
||||
private long timeStampOfLastShippedOp;
|
||||
private long replicationLag;
|
||||
|
||||
public ReplicationLoadSource(String id, long age, int size, long timeStamp, long lag) {
|
||||
this.peerID = id;
|
||||
this.ageOfLastShippedOp = age;
|
||||
this.sizeOfLogQueue = size;
|
||||
this.timeStampOfLastShippedOp = timeStamp;
|
||||
this.replicationLag = lag;
|
||||
}
|
||||
|
||||
public String getPeerID() {
|
||||
return this.peerID;
|
||||
}
|
||||
|
||||
public long getAgeOfLastShippedOp() {
|
||||
return this.ageOfLastShippedOp;
|
||||
}
|
||||
|
||||
public long getSizeOfLogQueue() {
|
||||
return this.sizeOfLogQueue;
|
||||
}
|
||||
|
||||
public long getTimeStampOfLastShippedOp() {
|
||||
return this.timeStampOfLastShippedOp;
|
||||
}
|
||||
|
||||
public long getReplicationLag() {
|
||||
return this.replicationLag;
|
||||
}
|
||||
}
|
|
@ -63,8 +63,7 @@ public class EncryptionUtil {
|
|||
|
||||
/**
|
||||
* Protect a key by encrypting it with the secret key of the given subject.
|
||||
* The configuration must be set up correctly for key alias resolution. Keys
|
||||
* are always wrapped using AES.
|
||||
* The configuration must be set up correctly for key alias resolution.
|
||||
* @param conf configuration
|
||||
* @param subject subject key alias
|
||||
* @param key the key
|
||||
|
@ -72,10 +71,12 @@ public class EncryptionUtil {
|
|||
*/
|
||||
public static byte[] wrapKey(Configuration conf, String subject, Key key)
|
||||
throws IOException {
|
||||
// Wrap the key with AES
|
||||
Cipher cipher = Encryption.getCipher(conf, "AES");
|
||||
// Wrap the key with the configured encryption algorithm.
|
||||
String algorithm =
|
||||
conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
|
||||
Cipher cipher = Encryption.getCipher(conf, algorithm);
|
||||
if (cipher == null) {
|
||||
throw new RuntimeException("Cipher 'AES' not available");
|
||||
throw new RuntimeException("Cipher '" + algorithm + "' not available");
|
||||
}
|
||||
EncryptionProtos.WrappedKey.Builder builder = EncryptionProtos.WrappedKey.newBuilder();
|
||||
builder.setAlgorithm(key.getAlgorithm());
|
||||
|
@ -100,8 +101,7 @@ public class EncryptionUtil {
|
|||
|
||||
/**
|
||||
* Unwrap a key by decrypting it with the secret key of the given subject.
|
||||
* The configuration must be set up correctly for key alias resolution. Keys
|
||||
* are always unwrapped using AES.
|
||||
* The configuration must be set up correctly for key alias resolution.
|
||||
* @param conf configuration
|
||||
* @param subject subject key alias
|
||||
* @param value the encrypted key bytes
|
||||
|
@ -113,10 +113,17 @@ public class EncryptionUtil {
|
|||
throws IOException, KeyException {
|
||||
EncryptionProtos.WrappedKey wrappedKey = EncryptionProtos.WrappedKey.PARSER
|
||||
.parseDelimitedFrom(new ByteArrayInputStream(value));
|
||||
Cipher cipher = Encryption.getCipher(conf, "AES");
|
||||
String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY,
|
||||
HConstants.CIPHER_AES);
|
||||
Cipher cipher = Encryption.getCipher(conf, algorithm);
|
||||
if (cipher == null) {
|
||||
throw new RuntimeException("Algorithm 'AES' not available");
|
||||
throw new RuntimeException("Cipher '" + algorithm + "' not available");
|
||||
}
|
||||
return getUnwrapKey(conf, subject, wrappedKey, cipher);
|
||||
}
|
||||
|
||||
private static Key getUnwrapKey(Configuration conf, String subject,
|
||||
EncryptionProtos.WrappedKey wrappedKey, Cipher cipher) throws IOException, KeyException {
|
||||
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
||||
byte[] iv = wrappedKey.hasIv() ? wrappedKey.getIv().toByteArray() : null;
|
||||
Encryption.decryptWithSubjectKey(out, wrappedKey.getData().newInput(),
|
||||
|
@ -130,4 +137,26 @@ public class EncryptionUtil {
|
|||
return new SecretKeySpec(keyBytes, wrappedKey.getAlgorithm());
|
||||
}
|
||||
|
||||
/**
|
||||
* Unwrap a wal key by decrypting it with the secret key of the given subject. The configuration
|
||||
* must be set up correctly for key alias resolution.
|
||||
* @param conf configuration
|
||||
* @param subject subject key alias
|
||||
* @param value the encrypted key bytes
|
||||
* @return the raw key bytes
|
||||
* @throws IOException if key is not found for the subject, or if some I/O error occurs
|
||||
* @throws KeyException if fail to unwrap the key
|
||||
*/
|
||||
public static Key unwrapWALKey(Configuration conf, String subject, byte[] value)
|
||||
throws IOException, KeyException {
|
||||
EncryptionProtos.WrappedKey wrappedKey =
|
||||
EncryptionProtos.WrappedKey.PARSER.parseDelimitedFrom(new ByteArrayInputStream(value));
|
||||
String algorithm = conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
|
||||
Cipher cipher = Encryption.getCipher(conf, algorithm);
|
||||
if (cipher == null) {
|
||||
throw new RuntimeException("Cipher '" + algorithm + "' not available");
|
||||
}
|
||||
return getUnwrapKey(conf, subject, wrappedKey, cipher);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -24,10 +24,12 @@ import io.netty.channel.ChannelFuture;
|
|||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
|
||||
|
@ -35,8 +37,10 @@ import javax.security.auth.callback.CallbackHandler;
|
|||
import javax.security.sasl.Sasl;
|
||||
import javax.security.sasl.SaslClient;
|
||||
import javax.security.sasl.SaslException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Random;
|
||||
|
||||
/**
|
||||
|
@ -48,6 +52,8 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
|
||||
private final boolean fallbackAllowed;
|
||||
|
||||
private final UserGroupInformation ticket;
|
||||
|
||||
/**
|
||||
* Used for client or server's token to send or receive from each other.
|
||||
*/
|
||||
|
@ -63,6 +69,7 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param ticket the ugi
|
||||
* @param method auth method
|
||||
* @param token for Sasl
|
||||
* @param serverPrincipal Server's Kerberos principal name
|
||||
|
@ -72,10 +79,11 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
* @param successfulConnectHandler handler for succesful connects
|
||||
* @throws java.io.IOException if handler could not be created
|
||||
*/
|
||||
public SaslClientHandler(AuthMethod method, Token<? extends TokenIdentifier> token,
|
||||
String serverPrincipal, boolean fallbackAllowed, String rpcProtection,
|
||||
SaslExceptionHandler exceptionHandler, SaslSuccessfulConnectHandler successfulConnectHandler)
|
||||
throws IOException {
|
||||
public SaslClientHandler(UserGroupInformation ticket, AuthMethod method,
|
||||
Token<? extends TokenIdentifier> token, String serverPrincipal, boolean fallbackAllowed,
|
||||
String rpcProtection, SaslExceptionHandler exceptionHandler,
|
||||
SaslSuccessfulConnectHandler successfulConnectHandler) throws IOException {
|
||||
this.ticket = ticket;
|
||||
this.fallbackAllowed = fallbackAllowed;
|
||||
|
||||
this.exceptionHandler = exceptionHandler;
|
||||
|
@ -109,8 +117,9 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
default:
|
||||
throw new IOException("Unknown authentication method " + method);
|
||||
}
|
||||
if (saslClient == null)
|
||||
if (saslClient == null) {
|
||||
throw new IOException("Unable to find SASL client implementation");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -144,14 +153,26 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
null);
|
||||
}
|
||||
|
||||
@Override public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
|
||||
@Override
|
||||
public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
|
||||
saslClient.dispose();
|
||||
}
|
||||
|
||||
@Override public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
this.saslToken = new byte[0];
|
||||
private byte[] evaluateChallenge(final byte[] challenge) throws Exception {
|
||||
return ticket.doAs(new PrivilegedExceptionAction<byte[]>() {
|
||||
|
||||
@Override
|
||||
public byte[] run() throws Exception {
|
||||
return saslClient.evaluateChallenge(challenge);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handlerAdded(final ChannelHandlerContext ctx) throws Exception {
|
||||
saslToken = new byte[0];
|
||||
if (saslClient.hasInitialResponse()) {
|
||||
saslToken = saslClient.evaluateChallenge(saslToken);
|
||||
saslToken = evaluateChallenge(saslToken);
|
||||
}
|
||||
if (saslToken != null) {
|
||||
writeSaslToken(ctx, saslToken);
|
||||
|
@ -161,7 +182,8 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
}
|
||||
}
|
||||
|
||||
@Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
|
||||
@Override
|
||||
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
|
||||
ByteBuf in = (ByteBuf) msg;
|
||||
|
||||
// If not complete, try to negotiate
|
||||
|
@ -187,15 +209,17 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
}
|
||||
}
|
||||
saslToken = new byte[len];
|
||||
if (LOG.isDebugEnabled())
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Will read input token of size " + saslToken.length
|
||||
+ " for processing by initSASLContext");
|
||||
}
|
||||
in.readBytes(saslToken);
|
||||
|
||||
saslToken = saslClient.evaluateChallenge(saslToken);
|
||||
saslToken = evaluateChallenge(saslToken);
|
||||
if (saslToken != null) {
|
||||
if (LOG.isDebugEnabled())
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Will send token of size " + saslToken.length + " from initSASLContext.");
|
||||
}
|
||||
writeSaslToken(ctx, saslToken);
|
||||
}
|
||||
}
|
||||
|
@ -246,8 +270,7 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
|
||||
/**
|
||||
* Write SASL token
|
||||
*
|
||||
* @param ctx to write to
|
||||
* @param ctx to write to
|
||||
* @param saslToken to write
|
||||
*/
|
||||
private void writeSaslToken(final ChannelHandlerContext ctx, byte[] saslToken) {
|
||||
|
@ -255,7 +278,8 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
b.writeInt(saslToken.length);
|
||||
b.writeBytes(saslToken, 0, saslToken.length);
|
||||
ctx.writeAndFlush(b).addListener(new ChannelFutureListener() {
|
||||
@Override public void operationComplete(ChannelFuture future) throws Exception {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
if (!future.isSuccess()) {
|
||||
exceptionCaught(ctx, future.cause());
|
||||
}
|
||||
|
@ -289,7 +313,8 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
exceptionHandler.handle(this.retryCount++, this.random, cause);
|
||||
}
|
||||
|
||||
@Override public void write(final ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
|
||||
@Override
|
||||
public void write(final ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
|
||||
throws Exception {
|
||||
// If not complete, try to negotiate
|
||||
if (!saslClient.isComplete()) {
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||
|
@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -60,7 +58,7 @@ public class AccessControlClient {
|
|||
|
||||
/**
|
||||
* Grants permission on the specified table for the specified user
|
||||
* @param conf
|
||||
* @param connection The Connection instance to use
|
||||
* @param tableName
|
||||
* @param userName
|
||||
* @param family
|
||||
|
@ -68,66 +66,51 @@ public class AccessControlClient {
|
|||
* @param actions
|
||||
* @throws Throwable
|
||||
*/
|
||||
public static void grant(Configuration conf, final TableName tableName,
|
||||
public static void grant(Connection connection, final TableName tableName,
|
||||
final String userName, final byte[] family, final byte[] qual,
|
||||
final Permission.Action... actions) throws Throwable {
|
||||
// TODO: Make it so caller passes in a Connection rather than have us do this expensive
|
||||
// setup each time. This class only used in test and shell at moment though.
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.grant(getAccessControlServiceStub(table), userName, tableName, family, qual,
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.grant(getAccessControlServiceStub(table), userName, tableName, family, qual,
|
||||
actions);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Grants permission on the specified namespace for the specified user.
|
||||
* @param conf
|
||||
* @param connection The Connection instance to use
|
||||
* @param namespace
|
||||
* @param userName
|
||||
* @param actions
|
||||
* @throws Throwable
|
||||
*/
|
||||
public static void grant(Configuration conf, final String namespace,
|
||||
public static void grant(Connection connection, final String namespace,
|
||||
final String userName, final Permission.Action... actions) throws Throwable {
|
||||
// TODO: Make it so caller passes in a Connection rather than have us do this expensive
|
||||
// setup each time. This class only used in test and shell at moment though.
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.grant(getAccessControlServiceStub(table), userName, namespace, actions);
|
||||
}
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.grant(getAccessControlServiceStub(table), userName, namespace, actions);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param connection The Connection instance to use
|
||||
* Grant global permissions for the specified user.
|
||||
*/
|
||||
public static void grant(Configuration conf, final String userName,
|
||||
public static void grant(Connection connection, final String userName,
|
||||
final Permission.Action... actions) throws Throwable {
|
||||
// TODO: Make it so caller passes in a Connection rather than have us do this expensive
|
||||
// setup each time. This class only used in test and shell at moment though.
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.grant(getAccessControlServiceStub(table), userName, actions);
|
||||
}
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.grant(getAccessControlServiceStub(table), userName, actions);
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean isAccessControllerRunning(Configuration conf)
|
||||
public static boolean isAccessControllerRunning(Connection connection)
|
||||
throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
|
||||
// TODO: Make it so caller passes in a Connection rather than have us do this expensive
|
||||
// setup each time. This class only used in test and shell at moment though.
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Admin admin = connection.getAdmin()) {
|
||||
return admin.isTableAvailable(ACL_TABLE_NAME);
|
||||
}
|
||||
try (Admin admin = connection.getAdmin()) {
|
||||
return admin.isTableAvailable(ACL_TABLE_NAME);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Revokes the permission on the table
|
||||
* @param conf
|
||||
* @param connection The Connection instance to use
|
||||
* @param tableName
|
||||
* @param username
|
||||
* @param family
|
||||
|
@ -135,81 +118,67 @@ public class AccessControlClient {
|
|||
* @param actions
|
||||
* @throws Throwable
|
||||
*/
|
||||
public static void revoke(Configuration conf, final TableName tableName,
|
||||
public static void revoke(Connection connection, final TableName tableName,
|
||||
final String username, final byte[] family, final byte[] qualifier,
|
||||
final Permission.Action... actions) throws Throwable {
|
||||
// TODO: Make it so caller passes in a Connection rather than have us do this expensive
|
||||
// setup each time. This class only used in test and shell at moment though.
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.revoke(getAccessControlServiceStub(table), username, tableName, family,
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.revoke(getAccessControlServiceStub(table), username, tableName, family,
|
||||
qualifier, actions);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Revokes the permission on the table for the specified user.
|
||||
* @param conf
|
||||
* @param connection The Connection instance to use
|
||||
* @param namespace
|
||||
* @param userName
|
||||
* @param actions
|
||||
* @throws Throwable
|
||||
*/
|
||||
public static void revoke(Configuration conf, final String namespace,
|
||||
final String userName, final Permission.Action... actions) throws Throwable {
|
||||
// TODO: Make it so caller passes in a Connection rather than have us do this expensive
|
||||
// setup each time. This class only used in test and shell at moment though.
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.revoke(getAccessControlServiceStub(table), userName, namespace, actions);
|
||||
}
|
||||
public static void revoke(Connection connection, final String namespace,
|
||||
final String userName, final Permission.Action... actions) throws Throwable {
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.revoke(getAccessControlServiceStub(table), userName, namespace, actions);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Revoke global permissions for the specified user.
|
||||
* @param connection The Connection instance to use
|
||||
*/
|
||||
public static void revoke(Configuration conf, final String userName,
|
||||
public static void revoke(Connection connection, final String userName,
|
||||
final Permission.Action... actions) throws Throwable {
|
||||
// TODO: Make it so caller passes in a Connection rather than have us do this expensive
|
||||
// setup each time. This class only used in test and shell at moment though.
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.revoke(getAccessControlServiceStub(table), userName, actions);
|
||||
}
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.revoke(getAccessControlServiceStub(table), userName, actions);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* List all the userPermissions matching the given pattern.
|
||||
* @param conf
|
||||
* @param connection The Connection instance to use
|
||||
* @param tableRegex The regular expression string to match against
|
||||
* @return - returns an array of UserPermissions
|
||||
* @throws Throwable
|
||||
*/
|
||||
public static List<UserPermission> getUserPermissions(Configuration conf, String tableRegex)
|
||||
public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex)
|
||||
throws Throwable {
|
||||
List<UserPermission> permList = new ArrayList<UserPermission>();
|
||||
// TODO: Make it so caller passes in a Connection rather than have us do this expensive
|
||||
// setup each time. This class only used in test and shell at moment though.
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
try (Admin admin = connection.getAdmin()) {
|
||||
CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
BlockingInterface protocol =
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
try (Admin admin = connection.getAdmin()) {
|
||||
CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
BlockingInterface protocol =
|
||||
AccessControlProtos.AccessControlService.newBlockingStub(service);
|
||||
HTableDescriptor[] htds = null;
|
||||
if (tableRegex == null || tableRegex.isEmpty()) {
|
||||
permList = ProtobufUtil.getUserPermissions(protocol);
|
||||
} else if (tableRegex.charAt(0) == '@') {
|
||||
String namespace = tableRegex.substring(1);
|
||||
permList = ProtobufUtil.getUserPermissions(protocol, Bytes.toBytes(namespace));
|
||||
} else {
|
||||
htds = admin.listTables(Pattern.compile(tableRegex), true);
|
||||
for (HTableDescriptor hd : htds) {
|
||||
permList.addAll(ProtobufUtil.getUserPermissions(protocol, hd.getTableName()));
|
||||
}
|
||||
HTableDescriptor[] htds = null;
|
||||
if (tableRegex == null || tableRegex.isEmpty()) {
|
||||
permList = ProtobufUtil.getUserPermissions(protocol);
|
||||
} else if (tableRegex.charAt(0) == '@') {
|
||||
String namespace = tableRegex.substring(1);
|
||||
permList = ProtobufUtil.getUserPermissions(protocol, Bytes.toBytes(namespace));
|
||||
} else {
|
||||
htds = admin.listTables(Pattern.compile(tableRegex), true);
|
||||
for (HTableDescriptor hd : htds) {
|
||||
permList.addAll(ProtobufUtil.getUserPermissions(protocol, hd.getTableName()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,9 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class SnapshotExistsException extends HBaseSnapshotException {
|
||||
public SnapshotExistsException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
/**
|
||||
* Failure due to the snapshot already existing
|
||||
|
|
|
@ -33,6 +33,11 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
|
||||
/**
|
||||
* Utility methods for reading, and building the ZooKeeper configuration.
|
||||
*
|
||||
* The order and priority for reading the config are as follows:
|
||||
* (1). zoo.cfg if ""hbase.config.read.zookeeper.config" is true
|
||||
* (2). Property with "hbase.zookeeper.property." prefix from HBase XML
|
||||
* (3). other zookeeper related properties in HBASE XML
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ZKConfig {
|
||||
|
@ -51,6 +56,24 @@ public class ZKConfig {
|
|||
* @return Properties holding mappings representing ZooKeeper config file.
|
||||
*/
|
||||
public static Properties makeZKProps(Configuration conf) {
|
||||
Properties zkProperties = makeZKPropsFromZooCfg(conf);
|
||||
|
||||
if (zkProperties == null) {
|
||||
// Otherwise, use the configuration options from HBase's XML files.
|
||||
zkProperties = makeZKPropsFromHbaseConfig(conf);
|
||||
}
|
||||
return zkProperties;
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses the corresponding config options from the zoo.cfg file
|
||||
* and make a Properties object holding the Zookeeper config.
|
||||
*
|
||||
* @param conf Configuration to read from.
|
||||
* @return Properties holding mappings representing the ZooKeeper config file or null if
|
||||
* the HBASE_CONFIG_READ_ZOOKEEPER_CONFIG is false or the file does not exist.
|
||||
*/
|
||||
private static Properties makeZKPropsFromZooCfg(Configuration conf) {
|
||||
if (conf.getBoolean(HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG, false)) {
|
||||
LOG.warn(
|
||||
"Parsing ZooKeeper's " + HConstants.ZOOKEEPER_CONFIG_NAME +
|
||||
|
@ -80,7 +103,18 @@ public class ZKConfig {
|
|||
}
|
||||
}
|
||||
|
||||
// Otherwise, use the configuration options from HBase's XML files.
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Make a Properties object holding ZooKeeper config.
|
||||
* Parses the corresponding config options from the HBase XML configs
|
||||
* and generates the appropriate ZooKeeper properties.
|
||||
*
|
||||
* @param conf Configuration to read from.
|
||||
* @return Properties holding mappings representing ZooKeeper config file.
|
||||
*/
|
||||
private static Properties makeZKPropsFromHbaseConfig(Configuration conf) {
|
||||
Properties zkProperties = new Properties();
|
||||
|
||||
// Directly map all of the hbase.zookeeper.property.KEY properties.
|
||||
|
@ -112,10 +146,17 @@ public class ZKConfig {
|
|||
|
||||
final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM,
|
||||
HConstants.LOCALHOST);
|
||||
String serverHost;
|
||||
String address;
|
||||
String key;
|
||||
for (int i = 0; i < serverHosts.length; ++i) {
|
||||
String serverHost = serverHosts[i];
|
||||
String address = serverHost + ":" + peerPort + ":" + leaderPort;
|
||||
String key = "server." + i;
|
||||
if (serverHosts[i].contains(":")) {
|
||||
serverHost = serverHosts[i].substring(0, serverHosts[i].indexOf(':'));
|
||||
} else {
|
||||
serverHost = serverHosts[i];
|
||||
}
|
||||
address = serverHost + ":" + peerPort + ":" + leaderPort;
|
||||
key = "server." + i;
|
||||
zkProperties.put(key, address);
|
||||
}
|
||||
|
||||
|
@ -177,7 +218,8 @@ public class ZKConfig {
|
|||
}
|
||||
// Special case for 'hbase.cluster.distributed' property being 'true'
|
||||
if (key.startsWith("server.")) {
|
||||
boolean mode = conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, HConstants.DEFAULT_CLUSTER_DISTRIBUTED);
|
||||
boolean mode =
|
||||
conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, HConstants.DEFAULT_CLUSTER_DISTRIBUTED);
|
||||
if (mode == HConstants.CLUSTER_IS_DISTRIBUTED && value.startsWith(HConstants.LOCALHOST)) {
|
||||
String msg = "The server in zoo.cfg cannot be set to localhost " +
|
||||
"in a fully-distributed setup because it won't be reachable. " +
|
||||
|
@ -198,7 +240,7 @@ public class ZKConfig {
|
|||
* @param properties
|
||||
* @return Quorum servers String
|
||||
*/
|
||||
public static String getZKQuorumServersString(Properties properties) {
|
||||
private static String getZKQuorumServersString(Properties properties) {
|
||||
String clientPort = null;
|
||||
List<String> servers = new ArrayList<String>();
|
||||
|
||||
|
@ -250,12 +292,59 @@ public class ZKConfig {
|
|||
return hostPortBuilder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the ZK Quorum servers string given the specified configuration
|
||||
*
|
||||
* @param conf
|
||||
* @return Quorum servers String
|
||||
*/
|
||||
private static String getZKQuorumServersStringFromHbaseConfig(Configuration conf) {
|
||||
String defaultClientPort = Integer.toString(
|
||||
conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT));
|
||||
|
||||
// Build the ZK quorum server string with "server:clientport" list, separated by ','
|
||||
final String[] serverHosts =
|
||||
conf.getStrings(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST);
|
||||
return buildQuorumServerString(serverHosts, defaultClientPort);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build the ZK quorum server string with "server:clientport" list, separated by ','
|
||||
*
|
||||
* @param serverHosts a list of servers for ZK quorum
|
||||
* @param clientPort the default client port
|
||||
* @return the string for a list of "server:port" separated by ","
|
||||
*/
|
||||
public static String buildQuorumServerString(String[] serverHosts, String clientPort) {
|
||||
StringBuilder quorumStringBuilder = new StringBuilder();
|
||||
String serverHost;
|
||||
for (int i = 0; i < serverHosts.length; ++i) {
|
||||
if (serverHosts[i].contains(":")) {
|
||||
serverHost = serverHosts[i]; // just use the port specified from the input
|
||||
} else {
|
||||
serverHost = serverHosts[i] + ":" + clientPort;
|
||||
}
|
||||
if (i > 0) {
|
||||
quorumStringBuilder.append(',');
|
||||
}
|
||||
quorumStringBuilder.append(serverHost);
|
||||
}
|
||||
return quorumStringBuilder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the ZK Quorum servers string given the specified configuration.
|
||||
* @param conf
|
||||
* @return Quorum servers
|
||||
*/
|
||||
public static String getZKQuorumServersString(Configuration conf) {
|
||||
return getZKQuorumServersString(makeZKProps(conf));
|
||||
// First try zoo.cfg; if not applicable, then try config XML.
|
||||
Properties zkProperties = makeZKPropsFromZooCfg(conf);
|
||||
|
||||
if (zkProperties != null) {
|
||||
return getZKQuorumServersString(zkProperties);
|
||||
}
|
||||
|
||||
return getZKQuorumServersStringFromHbaseConfig(conf);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.util.HashMap;
|
|||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
import javax.security.auth.login.AppConfigurationEntry;
|
||||
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
|
||||
|
@ -46,8 +45,9 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -74,6 +74,7 @@ import org.apache.zookeeper.proto.DeleteRequest;
|
|||
import org.apache.zookeeper.proto.SetDataRequest;
|
||||
import org.apache.zookeeper.server.ZooKeeperSaslServer;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
|
@ -93,6 +94,25 @@ public class ZKUtil {
|
|||
public static final char ZNODE_PATH_SEPARATOR = '/';
|
||||
private static int zkDumpConnectionTimeOut;
|
||||
|
||||
// The Quorum for the ZK cluster can have one the following format (see examples below):
|
||||
// (1). s1,s2,s3 (no client port in the list, the client port could be obtained from clientPort)
|
||||
// (2). s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server,
|
||||
// in this case, the clientPort would be ignored)
|
||||
// (3). s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use
|
||||
// the clientPort; otherwise, it would use the specified port)
|
||||
@VisibleForTesting
|
||||
public static class ZKClusterKey {
|
||||
public String quorumString;
|
||||
public int clientPort;
|
||||
public String znodeParent;
|
||||
|
||||
ZKClusterKey(String quorumString, int clientPort, String znodeParent) {
|
||||
this.quorumString = quorumString;
|
||||
this.clientPort = clientPort;
|
||||
this.znodeParent = znodeParent;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new connection to ZooKeeper, pulling settings and ensemble config
|
||||
* from the specified configuration object using methods from {@link ZKConfig}.
|
||||
|
@ -106,8 +126,7 @@ public class ZKUtil {
|
|||
*/
|
||||
public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher)
|
||||
throws IOException {
|
||||
Properties properties = ZKConfig.makeZKProps(conf);
|
||||
String ensemble = ZKConfig.getZKQuorumServersString(properties);
|
||||
String ensemble = ZKConfig.getZKQuorumServersString(conf);
|
||||
return connect(conf, ensemble, watcher);
|
||||
}
|
||||
|
||||
|
@ -381,10 +400,10 @@ public class ZKUtil {
|
|||
*/
|
||||
public static void applyClusterKeyToConf(Configuration conf, String key)
|
||||
throws IOException{
|
||||
String[] parts = transformClusterKey(key);
|
||||
conf.set(HConstants.ZOOKEEPER_QUORUM, parts[0]);
|
||||
conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, parts[1]);
|
||||
conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parts[2]);
|
||||
ZKClusterKey zkClusterKey = transformClusterKey(key);
|
||||
conf.set(HConstants.ZOOKEEPER_QUORUM, zkClusterKey.quorumString);
|
||||
conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClusterKey.clientPort);
|
||||
conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkClusterKey.znodeParent);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -395,14 +414,53 @@ public class ZKUtil {
|
|||
* @return the three configuration in the described order
|
||||
* @throws IOException
|
||||
*/
|
||||
public static String[] transformClusterKey(String key) throws IOException {
|
||||
public static ZKClusterKey transformClusterKey(String key) throws IOException {
|
||||
String[] parts = key.split(":");
|
||||
if (parts.length != 3) {
|
||||
throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" +
|
||||
HConstants.ZOOKEEPER_QUORUM + ":hbase.zookeeper.client.port:"
|
||||
+ HConstants.ZOOKEEPER_ZNODE_PARENT);
|
||||
|
||||
if (parts.length == 3) {
|
||||
return new ZKClusterKey(parts [0], Integer.parseInt(parts [1]), parts [2]);
|
||||
}
|
||||
return parts;
|
||||
|
||||
if (parts.length > 3) {
|
||||
// The quorum could contain client port in server:clientport format, try to transform more.
|
||||
String zNodeParent = parts [parts.length - 1];
|
||||
String clientPort = parts [parts.length - 2];
|
||||
|
||||
// The first part length is the total length minus the lengths of other parts and minus 2 ":"
|
||||
int endQuorumIndex = key.length() - zNodeParent.length() - clientPort.length() - 2;
|
||||
String quorumStringInput = key.substring(0, endQuorumIndex);
|
||||
String[] serverHosts = quorumStringInput.split(",");
|
||||
|
||||
// The common case is that every server has its own client port specified - this means
|
||||
// that (total parts - the ZNodeParent part - the ClientPort part) is equal to
|
||||
// (the number of "," + 1) - "+ 1" because the last server has no ",".
|
||||
if ((parts.length - 2) == (serverHosts.length + 1)) {
|
||||
return new ZKClusterKey(quorumStringInput, Integer.parseInt(clientPort), zNodeParent);
|
||||
}
|
||||
|
||||
// For the uncommon case that some servers has no port specified, we need to build the
|
||||
// server:clientport list using default client port for servers without specified port.
|
||||
return new ZKClusterKey(
|
||||
ZKConfig.buildQuorumServerString(serverHosts, clientPort),
|
||||
Integer.parseInt(clientPort),
|
||||
zNodeParent);
|
||||
}
|
||||
|
||||
throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" +
|
||||
HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":"
|
||||
+ HConstants.ZOOKEEPER_ZNODE_PARENT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Standardize the ZK quorum string: make it a "server:clientport" list, separated by ','
|
||||
* @param quorumStringInput a string contains a list of servers for ZK quorum
|
||||
* @param clientPort the default client port
|
||||
* @return the string for a list of "server:port" separated by ","
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static String standardizeQuorumServerString(String quorumStringInput, String clientPort) {
|
||||
String[] serverHosts = quorumStringInput.split(",");
|
||||
return ZKConfig.buildQuorumServerString(serverHosts, clientPort);
|
||||
}
|
||||
|
||||
//
|
||||
|
@ -939,7 +997,8 @@ public class ZKUtil {
|
|||
// Detection for embedded HBase client with jaas configuration
|
||||
// defined for third party programs.
|
||||
try {
|
||||
javax.security.auth.login.Configuration testConfig = javax.security.auth.login.Configuration.getConfiguration();
|
||||
javax.security.auth.login.Configuration testConfig =
|
||||
javax.security.auth.login.Configuration.getConfiguration();
|
||||
if(testConfig.getAppConfigurationEntry("Client") == null) {
|
||||
return false;
|
||||
}
|
||||
|
@ -1207,7 +1266,6 @@ public class ZKUtil {
|
|||
} catch (InterruptedException ie) {
|
||||
zkw.interruptedException(ie);
|
||||
}
|
||||
|
||||
} catch(InterruptedException ie) {
|
||||
zkw.interruptedException(ie);
|
||||
}
|
||||
|
@ -1334,8 +1392,8 @@ public class ZKUtil {
|
|||
deleteNodeRecursively(zkw, joinZNode(node, child));
|
||||
}
|
||||
}
|
||||
//Zookeeper Watches are one time triggers; When children of parent nodes are deleted recursively.
|
||||
//Must set another watch, get notified of delete node
|
||||
//Zookeeper Watches are one time triggers; When children of parent nodes are deleted
|
||||
//recursively, must set another watch, get notified of delete node
|
||||
if (zkw.getRecoverableZooKeeper().exists(node, zkw) != null){
|
||||
zkw.getRecoverableZooKeeper().delete(node, -1);
|
||||
}
|
||||
|
@ -1986,7 +2044,8 @@ public class ZKUtil {
|
|||
* @see #logZKTree(ZooKeeperWatcher, String)
|
||||
* @throws KeeperException if an unexpected exception occurs
|
||||
*/
|
||||
protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix) throws KeeperException {
|
||||
protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix)
|
||||
throws KeeperException {
|
||||
List<String> children = ZKUtil.listChildrenNoWatch(zkw, root);
|
||||
if (children == null) return;
|
||||
for (String child : children) {
|
||||
|
@ -2044,10 +2103,10 @@ public class ZKUtil {
|
|||
*/
|
||||
public static byte[] regionSequenceIdsToByteArray(final Long regionLastFlushedSequenceId,
|
||||
final Map<byte[], Long> storeSequenceIds) {
|
||||
ZooKeeperProtos.RegionStoreSequenceIds.Builder regionSequenceIdsBuilder =
|
||||
ZooKeeperProtos.RegionStoreSequenceIds.newBuilder();
|
||||
ZooKeeperProtos.StoreSequenceId.Builder storeSequenceIdBuilder =
|
||||
ZooKeeperProtos.StoreSequenceId.newBuilder();
|
||||
ClusterStatusProtos.RegionStoreSequenceIds.Builder regionSequenceIdsBuilder =
|
||||
ClusterStatusProtos.RegionStoreSequenceIds.newBuilder();
|
||||
ClusterStatusProtos.StoreSequenceId.Builder storeSequenceIdBuilder =
|
||||
ClusterStatusProtos.StoreSequenceId.newBuilder();
|
||||
if (storeSequenceIds != null) {
|
||||
for (Map.Entry<byte[], Long> e : storeSequenceIds.entrySet()){
|
||||
byte[] columnFamilyName = e.getKey();
|
||||
|
@ -2074,7 +2133,7 @@ public class ZKUtil {
|
|||
throw new DeserializationException("Unable to parse RegionStoreSequenceIds.");
|
||||
}
|
||||
RegionStoreSequenceIds.Builder regionSequenceIdsBuilder =
|
||||
ZooKeeperProtos.RegionStoreSequenceIds.newBuilder();
|
||||
ClusterStatusProtos.RegionStoreSequenceIds.newBuilder();
|
||||
int pblen = ProtobufUtil.lengthOfPBMagic();
|
||||
RegionStoreSequenceIds storeIds = null;
|
||||
try {
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue