now roll to r1416073
git-svn-id: https://svn.apache.org/repos/asf/hbase/branches/0.94-test@1425526 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/bin/hbase b/bin/hbase
index f231605..6270205 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -248,21 +248,6 @@
# restore ordinary behaviour
unset IFS
-#Set the right GC options based on the what we are running
-declare -a client_cmds=("shell" "hbck" "hlog" "hfile" "zkcli")
-for cmd in $client_cmds; do
- if [[ $cmd == $COMMAND ]]; then
- client=true
- break
- fi
-done
-
-if [[ $client ]]; then
- HBASE_OPTS="$HBASE_OPTS $CLIENT_GC_OPTS"
-else
- HBASE_OPTS="$HBASE_OPTS $SERVER_GC_OPTS"
-fi
-
# figure out which class to run
if [ "$COMMAND" = "shell" ] ; then
# eg export JRUBY_HOME=/usr/local/share/jruby
diff --git a/conf/hbase-env.sh b/conf/hbase-env.sh
index da53a27..5b5ba96 100644
--- a/conf/hbase-env.sh
+++ b/conf/hbase-env.sh
@@ -21,10 +21,6 @@
# Set environment variables here.
-# This script sets variables multiple times over the course of starting an hbase process,
-# so try to keep things idempotent unless you want to take an even deeper look
-# into the startup scripts (bin/hbase, etc.)
-
# The java implementation to use. Java 1.6 required.
# export JAVA_HOME=/usr/java/jdk1.6.0/
@@ -38,20 +34,12 @@
# Below are what we set by default. May only work with SUN JVM.
# For more on why as well as other possible settings,
# see http://wiki.apache.org/hadoop/PerformanceTuning
-export HBASE_OPTS="-XX:+UseConcMarkSweepGC"
+export HBASE_OPTS="$HBASE_OPTS -XX:+UseConcMarkSweepGC"
-# Uncomment below to enable java garbage collection logging for the server-side processes
-# this enables basic gc logging for the server processes to the .out file
-# export SERVER_GC_OPTS="-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps $HBASE_GC_OPTS"
+# Uncomment below to enable java garbage collection logging in the .out file.
+# export HBASE_OPTS="$HBASE_OPTS -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps $HBASE_GC_OPTS"
-# this enables gc logging using automatic GC log rolling. Only applies to jdk 1.6.0_34+ and 1.7.0_2+. Either use this set of options or the one above
-# export SERVER_GC_OPTS="-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=1 -XX:GCLogFileSize=512M $HBASE_GC_OPTS"
-
-# Uncomment below to enable java garbage collection logging for the client processes in the .out file.
-# export CLIENT_GC_OPTS="-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps $HBASE_GC_OPTS"
-
-# Uncomment below (along with above GC logging) to put GC information in its own logfile (will set HBASE_GC_OPTS).
-# This applies to both the server and client GC options above
+# Uncomment below (along with above GC logging) to put GC information in its own logfile (will set HBASE_GC_OPTS)
# export HBASE_USE_GC_LOGFILE=true
diff --git a/conf/log4j.properties b/conf/log4j.properties
index 24d7b2b..f6c3305 100644
--- a/conf/log4j.properties
+++ b/conf/log4j.properties
@@ -39,7 +39,6 @@
log4j.appender.DRFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
log4j.category.SecurityLogger=${hbase.security.logger}
log4j.additivity.SecurityLogger=false
-#log4j.logger.SecurityLogger.org.apache.hadoop.hbase.security.access.AccessController=TRACE
#
# Null Appender
diff --git a/pom.xml b/pom.xml
index c67fe64..d2c42e7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -36,7 +36,7 @@
<groupId>org.apache.hbase</groupId>
<artifactId>hbase</artifactId>
<packaging>jar</packaging>
- <version>0.94.4-SNAPSHOT</version>
+ <version>0.94.3</version>
<name>HBase</name>
<description>
HBase is the &lt;a href="http://hadoop.apache.org"&rt;Hadoop</a&rt; database. Use it when you need
@@ -767,7 +767,7 @@
I believe it is a failsafe bug, we may consider using surefire -->
<forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
<argLine>-enableassertions -Xmx1900m
- -Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true</argLine>
+ -Djava.security.egd=file:/dev/./urandom</argLine>
<testFailureIgnore>false</testFailureIgnore>
</configuration>
</plugin>
@@ -1017,9 +1017,8 @@
<protobuf.version>2.4.0a</protobuf.version>
<stax-api.version>1.0.1</stax-api.version>
<thrift.version>0.8.0</thrift.version>
- <zookeeper.version>3.4.5</zookeeper.version>
+ <zookeeper.version>3.4.3</zookeeper.version>
<hadoop-snappy.version>0.0.1-SNAPSHOT</hadoop-snappy.version>
- <clover.version>2.6.3</clover.version>
<package.prefix>/usr</package.prefix>
<package.conf.dir>/etc/hbase</package.conf.dir>
@@ -1715,7 +1714,7 @@
</property>
</activation>
<properties>
- <hadoop.version>1.1.1</hadoop.version>
+ <hadoop.version>1.1.0</hadoop.version>
<slf4j.version>1.4.3</slf4j.version>
</properties>
<dependencies>
@@ -2358,61 +2357,6 @@
<surefire.firstPartGroups></surefire.firstPartGroups>
</properties>
</profile>
-
- <!-- Profile for running clover. You need to have a clover license under ~/.clover.license for ${clover.version}
-or you can provide the license with -Dmaven.clover.licenseLocation=/path/to/license. Committers can find
-the license under https://svn.apache.org/repos/private/committers/donated-licenses/clover/
-Note that clover 2.6.3 does not run with maven 3, so you have to use maven2. The report will be generated
-under target/site/clover/index.html when you run
-MAVEN_OPTS=-Xmx2048m mvn clean test -Pclover site -->
- <profile>
- <id>clover</id>
- <activation>
- <activeByDefault>false</activeByDefault>
- <property>
- <name>clover</name>
- </property>
- </activation>
- <properties>
- <maven.clover.licenseLocation>${user.home}/.clover.license</maven.clover.licenseLocation>
- <clover.version>2.6.3</clover.version>
- </properties>
- <build>
- <plugins>
- <plugin>
- <groupId>com.atlassian.maven.plugins</groupId>
- <artifactId>maven-clover2-plugin</artifactId>
- <version>${clover.version}</version>
- <configuration>
- <includesAllSourceRoots>true</includesAllSourceRoots>
- <includesTestSourceRoots>true</includesTestSourceRoots>
- <targetPercentage>50%</targetPercentage>
- <generateHtml>true</generateHtml>
- <generateXml>true</generateXml>
- <excludes>
- <exclude>**/generated/**</exclude>
- </excludes>
- </configuration>
- <executions>
- <execution>
- <id>clover-setup</id>
- <phase>process-sources</phase>
- <goals>
- <goal>setup</goal>
- </goals>
- </execution>
- <execution>
- <id>clover</id>
- <phase>site</phase>
- <goals>
- <goal>clover</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
- </profile>
</profiles>
<!-- See http://jira.codehaus.org/browse/MSITE-443 why the settings need to be here and not in pluginManagement. -->
diff --git a/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java b/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java
index c0d10e0..4c7bcbb 100644
--- a/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java
+++ b/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java
@@ -98,7 +98,7 @@
User ticket = remoteId.getTicket();
Class<?> protocol = remoteId.getProtocol();
- this.useSasl = User.isHBaseSecurityEnabled(conf);
+ this.useSasl = User.isSecurityEnabled();
if (useSasl && protocol != null) {
TokenInfo tokenInfo = protocol.getAnnotation(TokenInfo.class);
if (tokenInfo != null) {
diff --git a/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureServer.java b/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureServer.java
index d1b8961..f9497a1 100644
--- a/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureServer.java
+++ b/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureServer.java
@@ -684,7 +684,7 @@
conf, serverName, highPriorityLevel);
this.authorize =
conf.getBoolean(HADOOP_SECURITY_AUTHORIZATION, false);
- this.isSecurityEnabled = User.isHBaseSecurityEnabled(this.conf);
+ this.isSecurityEnabled = UserGroupInformation.isSecurityEnabled();
if (isSecurityEnabled) {
HBaseSaslRpcServer.init(conf);
diff --git a/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index c9e4dee..b82ad5d 100644
--- a/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -46,8 +46,6 @@
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
@@ -64,7 +62,6 @@
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.Permission.Action;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
@@ -104,7 +101,7 @@
* </p>
*/
public class AccessController extends BaseRegionObserver
- implements MasterObserver, RegionServerObserver, AccessControllerProtocol {
+ implements MasterObserver, AccessControllerProtocol {
/**
* Represents the result of an authorization check for logging and error
* reporting.
@@ -115,14 +112,12 @@
private final byte[] family;
private final byte[] qualifier;
private final Permission.Action action;
- private final String request;
private final String reason;
private final User user;
- public AuthResult(boolean allowed, String request, String reason, User user,
+ public AuthResult(boolean allowed, String reason, User user,
Permission.Action action, byte[] table, byte[] family, byte[] qualifier) {
this.allowed = allowed;
- this.request = request;
this.reason = reason;
this.user = user;
this.table = table;
@@ -137,8 +132,6 @@
public String getReason() { return reason; }
- public String getRequest() { return request; }
-
public String toContextString() {
return "(user=" + (user != null ? user.getName() : "UNKNOWN") + ", " +
"scope=" + (table == null ? "GLOBAL" : Bytes.toString(table)) + ", " +
@@ -152,23 +145,23 @@
.append(toContextString()).toString();
}
- public static AuthResult allow(String request, String reason, User user, Permission.Action action,
+ public static AuthResult allow(String reason, User user, Permission.Action action,
byte[] table, byte[] family, byte[] qualifier) {
- return new AuthResult(true, request, reason, user, action, table, family, qualifier);
+ return new AuthResult(true, reason, user, action, table, family, qualifier);
}
- public static AuthResult allow(String request, String reason, User user, Permission.Action action, byte[] table) {
- return new AuthResult(true, request, reason, user, action, table, null, null);
+ public static AuthResult allow(String reason, User user, Permission.Action action, byte[] table) {
+ return new AuthResult(true, reason, user, action, table, null, null);
}
- public static AuthResult deny(String request, String reason, User user,
+ public static AuthResult deny(String reason, User user,
Permission.Action action, byte[] table) {
- return new AuthResult(false, request, reason, user, action, table, null, null);
+ return new AuthResult(false, reason, user, action, table, null, null);
}
- public static AuthResult deny(String request, String reason, User user,
+ public static AuthResult deny(String reason, User user,
Permission.Action action, byte[] table, byte[] family, byte[] qualifier) {
- return new AuthResult(false, request, reason, user, action, table, family, qualifier);
+ return new AuthResult(false, reason, user, action, table, family, qualifier);
}
}
@@ -259,7 +252,7 @@
* the request
* @return
*/
- AuthResult permissionGranted(String request, User user, TablePermission.Action permRequest,
+ AuthResult permissionGranted(User user, TablePermission.Action permRequest,
RegionCoprocessorEnvironment e,
Map<byte [], ? extends Collection<?>> families) {
HRegionInfo hri = e.getRegion().getRegionInfo();
@@ -269,12 +262,12 @@
// this is a very common operation, so deal with it quickly.
if (hri.isRootRegion() || hri.isMetaRegion()) {
if (permRequest == TablePermission.Action.READ) {
- return AuthResult.allow(request, "All users allowed", user, permRequest, tableName);
+ return AuthResult.allow("All users allowed", user, permRequest, tableName);
}
}
if (user == null) {
- return AuthResult.deny(request, "No user associated with request!", null, permRequest, tableName);
+ return AuthResult.deny("No user associated with request!", null, permRequest, tableName);
}
// Users with CREATE/ADMIN rights need to modify .META. and _acl_ table
@@ -288,12 +281,12 @@
(authManager.authorize(user, Permission.Action.CREATE) ||
authManager.authorize(user, Permission.Action.ADMIN)))
{
- return AuthResult.allow(request, "Table permission granted", user, permRequest, tableName);
+ return AuthResult.allow("Table permission granted", user, permRequest, tableName);
}
// 2. check for the table-level, if successful we can short-circuit
if (authManager.authorize(user, tableName, (byte[])null, permRequest)) {
- return AuthResult.allow(request, "Table permission granted", user, permRequest, tableName);
+ return AuthResult.allow("Table permission granted", user, permRequest, tableName);
}
// 3. check permissions against the requested families
@@ -314,7 +307,7 @@
for (byte[] qualifier : familySet) {
if (!authManager.authorize(user, tableName, family.getKey(),
qualifier, permRequest)) {
- return AuthResult.deny(request, "Failed qualifier check", user,
+ return AuthResult.deny("Failed qualifier check", user,
permRequest, tableName, family.getKey(), qualifier);
}
}
@@ -323,25 +316,25 @@
for (KeyValue kv : kvList) {
if (!authManager.authorize(user, tableName, family.getKey(),
kv.getQualifier(), permRequest)) {
- return AuthResult.deny(request, "Failed qualifier check", user,
+ return AuthResult.deny("Failed qualifier check", user,
permRequest, tableName, family.getKey(), kv.getQualifier());
}
}
}
} else {
// no qualifiers and family-level check already failed
- return AuthResult.deny(request, "Failed family check", user, permRequest,
+ return AuthResult.deny("Failed family check", user, permRequest,
tableName, family.getKey(), null);
}
}
// all family checks passed
- return AuthResult.allow(request, "All family checks passed", user, permRequest,
+ return AuthResult.allow("All family checks passed", user, permRequest,
tableName);
}
// 4. no families to check and table level access failed
- return AuthResult.deny(request, "No families to check and table permission failed",
+ return AuthResult.deny("No families to check and table permission failed",
user, permRequest, tableName);
}
@@ -356,7 +349,6 @@
" for user " + (result.getUser() != null ? result.getUser().getShortName() : "UNKNOWN") +
"; reason: " + result.getReason() +
"; remote address: " + (remoteAddr != null ? remoteAddr : "") +
- "; request: " + result.getRequest() +
"; context: " + result.toContextString());
}
}
@@ -385,20 +377,18 @@
* @throws IOException if obtaining the current user fails
* @throws AccessDeniedException if user has no authorization
*/
- private void requirePermission(String request, byte[] tableName, byte[] family, byte[] qualifier,
+ private void requirePermission(byte[] tableName, byte[] family, byte[] qualifier,
Action... permissions) throws IOException {
User user = getActiveUser();
AuthResult result = null;
for (Action permission : permissions) {
if (authManager.authorize(user, tableName, family, qualifier, permission)) {
- result = AuthResult.allow(request, "Table permission granted", user,
- permission, tableName, family, qualifier);
+ result = AuthResult.allow("Table permission granted", user, permission, tableName, family, qualifier);
break;
} else {
// rest of the world
- result = AuthResult.deny(request, "Insufficient permissions", user,
- permission, tableName, family, qualifier);
+ result = AuthResult.deny("Insufficient permissions", user, permission, tableName, family, qualifier);
}
}
logResult(result);
@@ -413,12 +403,12 @@
* @throws IOException if obtaining the current user fails
* @throws AccessDeniedException if authorization is denied
*/
- private void requirePermission(String request, Permission.Action perm) throws IOException {
+ private void requirePermission(Permission.Action perm) throws IOException {
User user = getActiveUser();
if (authManager.authorize(user, perm)) {
- logResult(AuthResult.allow(request, "Global check allowed", user, perm, null));
+ logResult(AuthResult.allow("Global check allowed", user, perm, null));
} else {
- logResult(AuthResult.deny(request, "Global check failed", user, perm, null));
+ logResult(AuthResult.deny("Global check failed", user, perm, null));
throw new AccessDeniedException("Insufficient permissions for user '" +
(user != null ? user.getShortName() : "null") +"' (global, action=" +
perm.toString() + ")");
@@ -433,7 +423,7 @@
* @param families The set of column families present/required in the request
* @throws AccessDeniedException if the authorization check failed
*/
- private void requirePermission(String request, Permission.Action perm,
+ private void requirePermission(Permission.Action perm,
RegionCoprocessorEnvironment env, Collection<byte[]> families)
throws IOException {
// create a map of family-qualifier
@@ -441,7 +431,7 @@
for (byte[] family : families) {
familyMap.put(family, null);
}
- requirePermission(request, perm, env, familyMap);
+ requirePermission(perm, env, familyMap);
}
/**
@@ -452,12 +442,12 @@
* @param families The map of column families-qualifiers.
* @throws AccessDeniedException if the authorization check failed
*/
- private void requirePermission(String request, Permission.Action perm,
+ private void requirePermission(Permission.Action perm,
RegionCoprocessorEnvironment env,
Map<byte[], ? extends Collection<?>> families)
throws IOException {
User user = getActiveUser();
- AuthResult result = permissionGranted(request, user, perm, env, families);
+ AuthResult result = permissionGranted(user, perm, env, families);
logResult(result);
if (!result.isAllowed()) {
@@ -521,31 +511,17 @@
/* ---- MasterObserver implementation ---- */
public void start(CoprocessorEnvironment env) throws IOException {
-
- ZooKeeperWatcher zk = null;
+ // if running on HMaster
if (env instanceof MasterCoprocessorEnvironment) {
- // if running on HMaster
- MasterCoprocessorEnvironment mEnv = (MasterCoprocessorEnvironment) env;
- zk = mEnv.getMasterServices().getZooKeeper();
- } else if (env instanceof RegionServerCoprocessorEnvironment) {
- RegionServerCoprocessorEnvironment rsEnv = (RegionServerCoprocessorEnvironment) env;
- zk = rsEnv.getRegionServerServices().getZooKeeper();
- } else if (env instanceof RegionCoprocessorEnvironment) {
- // if running at region
- regionEnv = (RegionCoprocessorEnvironment) env;
- zk = regionEnv.getRegionServerServices().getZooKeeper();
+ MasterCoprocessorEnvironment e = (MasterCoprocessorEnvironment)env;
+ this.authManager = TableAuthManager.get(
+ e.getMasterServices().getZooKeeper(),
+ e.getConfiguration());
}
- // If zk is null or IOException while obtaining auth manager,
- // throw RuntimeException so that the coprocessor is unloaded.
- if (zk != null) {
- try {
- this.authManager = TableAuthManager.get(zk, env.getConfiguration());
- } catch (IOException ioe) {
- throw new RuntimeException("Error obtaining TableAuthManager", ioe);
- }
- } else {
- throw new RuntimeException("Error obtaining TableAuthManager, zk found null.");
+ // if running at region
+ if (env instanceof RegionCoprocessorEnvironment) {
+ regionEnv = (RegionCoprocessorEnvironment)env;
}
}
@@ -556,7 +532,7 @@
@Override
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
- requirePermission("createTable", Permission.Action.CREATE);
+ requirePermission(Permission.Action.CREATE);
}
@Override
@@ -575,7 +551,7 @@
@Override
public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c, byte[] tableName)
throws IOException {
- requirePermission("deleteTable", tableName, null, null, Action.ADMIN, Action.CREATE);
+ requirePermission(tableName, null, null, Action.ADMIN, Action.CREATE);
}
@Override
@@ -587,7 +563,7 @@
@Override
public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, byte[] tableName,
HTableDescriptor htd) throws IOException {
- requirePermission("modifyTable", tableName, null, null, Action.ADMIN, Action.CREATE);
+ requirePermission(tableName, null, null, Action.ADMIN, Action.CREATE);
}
@Override
@@ -604,7 +580,7 @@
@Override
public void preAddColumn(ObserverContext<MasterCoprocessorEnvironment> c, byte[] tableName,
HColumnDescriptor column) throws IOException {
- requirePermission("addColumn", tableName, null, null, Action.ADMIN, Action.CREATE);
+ requirePermission(tableName, null, null, Action.ADMIN, Action.CREATE);
}
@Override
@@ -614,7 +590,7 @@
@Override
public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> c, byte[] tableName,
HColumnDescriptor descriptor) throws IOException {
- requirePermission("modifyColumn", tableName, null, null, Action.ADMIN, Action.CREATE);
+ requirePermission(tableName, null, null, Action.ADMIN, Action.CREATE);
}
@Override
@@ -624,7 +600,7 @@
@Override
public void preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> c, byte[] tableName,
byte[] col) throws IOException {
- requirePermission("deleteColumn", tableName, null, null, Action.ADMIN, Action.CREATE);
+ requirePermission(tableName, null, null, Action.ADMIN, Action.CREATE);
}
@Override
@@ -637,7 +613,7 @@
@Override
public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c, byte[] tableName)
throws IOException {
- requirePermission("enableTable", tableName, null, null, Action.ADMIN, Action.CREATE);
+ requirePermission(tableName, null, null, Action.ADMIN, Action.CREATE);
}
@Override
@@ -651,7 +627,7 @@
throw new AccessDeniedException("Not allowed to disable "
+ AccessControlLists.ACL_TABLE_NAME_STR + " table.");
}
- requirePermission("disableTable", tableName, null, null, Action.ADMIN, Action.CREATE);
+ requirePermission(tableName, null, null, Action.ADMIN, Action.CREATE);
}
@Override
@@ -661,7 +637,7 @@
@Override
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region,
ServerName srcServer, ServerName destServer) throws IOException {
- requirePermission("move", region.getTableName(), null, null, Action.ADMIN);
+ requirePermission(region.getTableName(), null, null, Action.ADMIN);
}
@Override
@@ -672,7 +648,7 @@
@Override
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo)
throws IOException {
- requirePermission("assign", regionInfo.getTableName(), null, null, Action.ADMIN);
+ requirePermission(regionInfo.getTableName(), null, null, Action.ADMIN);
}
@Override
@@ -682,7 +658,7 @@
@Override
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo,
boolean force) throws IOException {
- requirePermission("unassign", regionInfo.getTableName(), null, null, Action.ADMIN);
+ requirePermission(regionInfo.getTableName(), null, null, Action.ADMIN);
}
@Override
@@ -692,7 +668,7 @@
@Override
public void preBalance(ObserverContext<MasterCoprocessorEnvironment> c)
throws IOException {
- requirePermission("balance", Permission.Action.ADMIN);
+ requirePermission(Permission.Action.ADMIN);
}
@Override
public void postBalance(ObserverContext<MasterCoprocessorEnvironment> c)
@@ -701,7 +677,7 @@
@Override
public boolean preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c,
boolean newValue) throws IOException {
- requirePermission("balanceSwitch", Permission.Action.ADMIN);
+ requirePermission(Permission.Action.ADMIN);
return newValue;
}
@Override
@@ -711,13 +687,13 @@
@Override
public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> c)
throws IOException {
- requirePermission("shutdown", Permission.Action.ADMIN);
+ requirePermission(Permission.Action.ADMIN);
}
@Override
public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c)
throws IOException {
- requirePermission("stopMaster", Permission.Action.ADMIN);
+ requirePermission(Permission.Action.ADMIN);
}
@Override
@@ -731,34 +707,27 @@
/* ---- RegionObserver implementation ---- */
@Override
- public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
- RegionCoprocessorEnvironment env = e.getEnvironment();
- final HRegion region = env.getRegion();
- if (region == null) {
- LOG.error("NULL region from RegionCoprocessorEnvironment in preOpen()");
- return;
- } else {
- HRegionInfo regionInfo = region.getRegionInfo();
- if (isSpecialTable(regionInfo)) {
- isSystemOrSuperUser(regionEnv.getConfiguration());
- } else {
- requirePermission("open", Action.ADMIN);
- }
- }
- }
-
- @Override
public void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {
- RegionCoprocessorEnvironment env = c.getEnvironment();
- final HRegion region = env.getRegion();
+ RegionCoprocessorEnvironment e = c.getEnvironment();
+ final HRegion region = e.getRegion();
if (region == null) {
LOG.error("NULL region from RegionCoprocessorEnvironment in postOpen()");
return;
}
+
+ try {
+ this.authManager = TableAuthManager.get(
+ e.getRegionServerServices().getZooKeeper(),
+ regionEnv.getConfiguration());
+ } catch (IOException ioe) {
+ // pass along as a RuntimeException, so that the coprocessor is unloaded
+ throw new RuntimeException("Error obtaining TableAuthManager", ioe);
+ }
+
if (AccessControlLists.isAclRegion(region)) {
aclRegion = true;
try {
- initialize(env);
+ initialize(e);
} catch (IOException ex) {
// if we can't obtain permissions, it's better to fail
// than perform checks incorrectly
@@ -769,32 +738,32 @@
@Override
public void preFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
- requirePermission("flush", getTableName(e.getEnvironment()), null, null, Action.ADMIN);
+ requirePermission(getTableName(e.getEnvironment()), null, null, Action.ADMIN);
}
@Override
public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
- requirePermission("split", getTableName(e.getEnvironment()), null, null, Action.ADMIN);
+ requirePermission(getTableName(e.getEnvironment()), null, null, Action.ADMIN);
}
@Override
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
final Store store, final InternalScanner scanner) throws IOException {
- requirePermission("compact", getTableName(e.getEnvironment()), null, null, Action.ADMIN);
+ requirePermission(getTableName(e.getEnvironment()), null, null, Action.ADMIN);
return scanner;
}
@Override
public void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> e,
final Store store, final List<StoreFile> candidates) throws IOException {
- requirePermission("compactSelection", getTableName(e.getEnvironment()), null, null, Action.ADMIN);
+ requirePermission(getTableName(e.getEnvironment()), null, null, Action.ADMIN);
}
@Override
public void preGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> c,
final byte [] row, final byte [] family, final Result result)
throws IOException {
- requirePermission("getClosestRowBefore", TablePermission.Action.READ, c.getEnvironment(),
+ requirePermission(TablePermission.Action.READ, c.getEnvironment(),
(family != null ? Lists.newArrayList(family) : null));
}
@@ -807,7 +776,7 @@
*/
RegionCoprocessorEnvironment e = c.getEnvironment();
User requestUser = getActiveUser();
- AuthResult authResult = permissionGranted("get", requestUser,
+ AuthResult authResult = permissionGranted(requestUser,
TablePermission.Action.READ, e, get.getFamilyMap());
if (!authResult.isAllowed()) {
if (hasFamilyQualifierPermission(requestUser,
@@ -824,7 +793,7 @@
} else {
get.setFilter(filter);
}
- logResult(AuthResult.allow("get", "Access allowed with filter", requestUser,
+ logResult(AuthResult.allow("Access allowed with filter", requestUser,
TablePermission.Action.READ, authResult.table));
} else {
logResult(authResult);
@@ -840,7 +809,7 @@
@Override
public boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> c,
final Get get, final boolean exists) throws IOException {
- requirePermission("exists", TablePermission.Action.READ, c.getEnvironment(),
+ requirePermission(TablePermission.Action.READ, c.getEnvironment(),
get.familySet());
return exists;
}
@@ -849,7 +818,7 @@
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c,
final Put put, final WALEdit edit, final boolean writeToWAL)
throws IOException {
- requirePermission("put", TablePermission.Action.WRITE, c.getEnvironment(),
+ requirePermission(TablePermission.Action.WRITE, c.getEnvironment(),
put.getFamilyMap());
}
@@ -865,7 +834,7 @@
public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
final Delete delete, final WALEdit edit, final boolean writeToWAL)
throws IOException {
- requirePermission("delete", TablePermission.Action.WRITE, c.getEnvironment(),
+ requirePermission(TablePermission.Action.WRITE, c.getEnvironment(),
delete.getFamilyMap());
}
@@ -885,8 +854,8 @@
final WritableByteArrayComparable comparator, final Put put,
final boolean result) throws IOException {
Collection<byte[]> familyMap = Arrays.asList(new byte[][]{family});
- requirePermission("checkAndPut", TablePermission.Action.READ, c.getEnvironment(), familyMap);
- requirePermission("checkAndPut", TablePermission.Action.WRITE, c.getEnvironment(), familyMap);
+ requirePermission(TablePermission.Action.READ, c.getEnvironment(), familyMap);
+ requirePermission(TablePermission.Action.WRITE, c.getEnvironment(), familyMap);
return result;
}
@@ -897,8 +866,8 @@
final WritableByteArrayComparable comparator, final Delete delete,
final boolean result) throws IOException {
Collection<byte[]> familyMap = Arrays.asList(new byte[][]{family});
- requirePermission("checkAndDelete", TablePermission.Action.READ, c.getEnvironment(), familyMap);
- requirePermission("checkAndDelete", TablePermission.Action.WRITE, c.getEnvironment(), familyMap);
+ requirePermission(TablePermission.Action.READ, c.getEnvironment(), familyMap);
+ requirePermission(TablePermission.Action.WRITE, c.getEnvironment(), familyMap);
return result;
}
@@ -907,7 +876,7 @@
final byte [] row, final byte [] family, final byte [] qualifier,
final long amount, final boolean writeToWAL)
throws IOException {
- requirePermission("incrementColumnValue", TablePermission.Action.WRITE, c.getEnvironment(),
+ requirePermission(TablePermission.Action.WRITE, c.getEnvironment(),
Arrays.asList(new byte[][]{family}));
return -1;
}
@@ -915,7 +884,7 @@
@Override
public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
throws IOException {
- requirePermission("append", TablePermission.Action.WRITE, c.getEnvironment(), append.getFamilyMap());
+ requirePermission(TablePermission.Action.WRITE, c.getEnvironment(), append.getFamilyMap());
return null;
}
@@ -923,7 +892,7 @@
public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
final Increment increment)
throws IOException {
- requirePermission("increment", TablePermission.Action.WRITE, c.getEnvironment(),
+ requirePermission(TablePermission.Action.WRITE, c.getEnvironment(),
increment.getFamilyMap().keySet());
return null;
}
@@ -937,7 +906,7 @@
*/
RegionCoprocessorEnvironment e = c.getEnvironment();
User user = getActiveUser();
- AuthResult authResult = permissionGranted("scannerOpen", user, TablePermission.Action.READ, e,
+ AuthResult authResult = permissionGranted(user, TablePermission.Action.READ, e,
scan.getFamilyMap());
if (!authResult.isAllowed()) {
if (hasFamilyQualifierPermission(user, TablePermission.Action.READ, e,
@@ -954,7 +923,7 @@
} else {
scan.setFilter(filter);
}
- logResult(AuthResult.allow("scannerOpen", "Access allowed with filter", user,
+ logResult(AuthResult.allow("Access allowed with filter", user,
TablePermission.Action.READ, authResult.table));
} else {
// no table/family level perms and no qualifier level perms, reject
@@ -1030,7 +999,7 @@
LOG.debug("Received request to grant access permission " + perm.toString());
}
- requirePermission("grant", perm.getTable(), perm.getFamily(), perm.getQualifier(), Action.ADMIN);
+ requirePermission(perm.getTable(), perm.getFamily(), perm.getQualifier(), Action.ADMIN);
AccessControlLists.addUserPermission(regionEnv.getConfiguration(), perm);
if (AUDITLOG.isTraceEnabled()) {
@@ -1060,8 +1029,7 @@
LOG.debug("Received request to revoke access permission " + perm.toString());
}
- requirePermission("revoke", perm.getTable(), perm.getFamily(),
- perm.getQualifier(), Action.ADMIN);
+ requirePermission(perm.getTable(), perm.getFamily(), perm.getQualifier(), Action.ADMIN);
AccessControlLists.removeUserPermission(regionEnv.getConfiguration(), perm);
if (AUDITLOG.isTraceEnabled()) {
@@ -1087,7 +1055,7 @@
public List<UserPermission> getUserPermissions(final byte[] tableName) throws IOException {
// only allowed to be called on _acl_ region
if (aclRegion) {
- requirePermission("userPermissions", tableName, null, null, Action.ADMIN);
+ requirePermission(tableName, null, null, Action.ADMIN);
List<UserPermission> perms = AccessControlLists.getUserPermissions(
regionEnv.getConfiguration(), tableName);
@@ -1121,12 +1089,12 @@
}
}
- requirePermission("checkPermissions", action, regionEnv, familyMap);
+ requirePermission(action, regionEnv, familyMap);
}
} else {
for (Permission.Action action : permission.getActions()) {
- requirePermission("checkPermissions", action);
+ requirePermission(action);
}
}
}
@@ -1159,56 +1127,4 @@
}
return tableName;
}
-
-
- @Override
- public void preClose(ObserverContext<RegionCoprocessorEnvironment> e, boolean abortRequested)
- throws IOException {
- requirePermission("close", Permission.Action.ADMIN);
- }
-
- @Override
- public void preLockRow(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] regionName,
- byte[] row) throws IOException {
- requirePermission("lockRow", getTableName(ctx.getEnvironment()), null, null,
- Permission.Action.WRITE, Permission.Action.CREATE);
- }
-
- @Override
- public void preUnlockRow(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] regionName,
- long lockId) throws IOException {
- requirePermission("unlockRow", getTableName(ctx.getEnvironment()), null, null,
- Permission.Action.WRITE, Permission.Action.CREATE);
- }
-
- private void isSystemOrSuperUser(Configuration conf) throws IOException {
- User user = User.getCurrent();
- if (user == null) {
- throw new IOException("Unable to obtain the current user, "
- + "authorization checks for internal operations will not work correctly!");
- }
-
- String currentUser = user.getShortName();
- List<String> superusers = Lists.asList(currentUser,
- conf.getStrings(AccessControlLists.SUPERUSER_CONF_KEY, new String[0]));
-
- User activeUser = getActiveUser();
- if (!(superusers.contains(activeUser.getShortName()))) {
- throw new AccessDeniedException("User '" + (user != null ? user.getShortName() : "null")
- + "is not system or super user.");
- }
- }
-
- private boolean isSpecialTable(HRegionInfo regionInfo) {
- byte[] tableName = regionInfo.getTableName();
- return tableName.equals(AccessControlLists.ACL_TABLE_NAME)
- || tableName.equals(Bytes.toBytes("-ROOT-"))
- || tableName.equals(Bytes.toBytes(".META."));
- }
-
- @Override
- public void preStopRegionServer(ObserverContext<RegionServerCoprocessorEnvironment> env)
- throws IOException {
- requirePermission("stop", Permission.Action.ADMIN);
- }
}
diff --git a/security/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/security/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index 4aa0e18..1a087b6 100644
--- a/security/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/security/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -21,7 +21,6 @@
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.ipc.SecureRpcEngine;
import org.apache.hadoop.hbase.security.User;
@@ -33,9 +32,8 @@
conf.set("hadoop.security.authorization", "false");
conf.set("hadoop.security.authentication", "simple");
conf.set("hbase.rpc.engine", SecureRpcEngine.class.getName());
- conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
- conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
- conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+ conf.set("hbase.coprocessor.master.classes", AccessController.class.getName());
+ conf.set("hbase.coprocessor.region.classes", AccessController.class.getName());
// add the process running user to superusers
String currentUser = User.getCurrent().getName();
conf.set("hbase.superuser", "admin,"+currentUser);
diff --git a/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index ad6dda4..765f0af 100644
--- a/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -36,7 +36,6 @@
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.UnknownRowLockException;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@@ -52,11 +51,9 @@
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
-import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.Permission.Action;
@@ -82,8 +79,6 @@
private static User USER_ADMIN;
// user with rw permissions
private static User USER_RW;
- // user with rw permissions on table.
- private static User USER_RW_ON_TABLE;
// user with read-only permissions
private static User USER_RO;
// user is table owner. will have all permissions on table
@@ -98,7 +93,6 @@
private static MasterCoprocessorEnvironment CP_ENV;
private static RegionCoprocessorEnvironment RCP_ENV;
- private static RegionServerCoprocessorEnvironment RSCP_ENV;
private static AccessController ACCESS_CONTROLLER;
@BeforeClass
@@ -113,10 +107,6 @@
ACCESS_CONTROLLER = (AccessController) cpHost.findCoprocessor(AccessController.class.getName());
CP_ENV = cpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
Coprocessor.PRIORITY_HIGHEST, 1, conf);
- RegionServerCoprocessorHost rsHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
- .getCoprocessorHost();
- RSCP_ENV = rsHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
- Coprocessor.PRIORITY_HIGHEST, 1, conf);
// Wait for the ACL table to become available
TEST_UTIL.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME, 5000);
@@ -126,7 +116,6 @@
USER_ADMIN = User.createUserForTesting(conf, "admin2", new String[0]);
USER_RW = User.createUserForTesting(conf, "rwuser", new String[0]);
USER_RO = User.createUserForTesting(conf, "rouser", new String[0]);
- USER_RW_ON_TABLE = User.createUserForTesting(conf, "rwuser_1", new String[0]);
USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
USER_CREATE = User.createUserForTesting(conf, "tbl_create", new String[0]);
USER_NONE = User.createUserForTesting(conf, "nouser", new String[0]);
@@ -159,9 +148,6 @@
protocol.grant(new UserPermission(Bytes.toBytes(USER_CREATE.getShortName()), TEST_TABLE, null,
Permission.Action.CREATE));
-
- protocol.grant(new UserPermission(Bytes.toBytes(USER_RW_ON_TABLE.getShortName()), TEST_TABLE,
- null, Permission.Action.READ, Permission.Action.WRITE));
}
@AfterClass
@@ -175,8 +161,6 @@
user.runAs(action);
} catch (AccessDeniedException ade) {
fail("Expected action to pass for user '" + user.getShortName() + "' but was denied");
- } catch (UnknownRowLockException exp){
- //expected
}
}
}
@@ -1287,70 +1271,4 @@
}
}
-
- @Test
- public void testLockAction() throws Exception {
- PrivilegedExceptionAction lockAction = new PrivilegedExceptionAction() {
- public Object run() throws Exception {
- ACCESS_CONTROLLER.preLockRow(ObserverContext.createAndPrepare(RCP_ENV, null), null,
- Bytes.toBytes("random_row"));
- return null;
- }
- };
- verifyAllowed(lockAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW_ON_TABLE);
- verifyDenied(lockAction, USER_RO, USER_RW, USER_NONE);
- }
-
- @Test
- public void testUnLockAction() throws Exception {
- PrivilegedExceptionAction unLockAction = new PrivilegedExceptionAction() {
- public Object run() throws Exception {
- ACCESS_CONTROLLER.preUnlockRow(ObserverContext.createAndPrepare(RCP_ENV, null), null,
- 123456);
- return null;
- }
- };
- verifyAllowed(unLockAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_RW_ON_TABLE);
- verifyDenied(unLockAction, USER_NONE, USER_RO, USER_RW);
- }
-
- @Test
- public void testStopRegionServer() throws Exception {
- PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
- public Object run() throws Exception {
- ACCESS_CONTROLLER.preStopRegionServer(ObserverContext.createAndPrepare(RSCP_ENV, null));
- return null;
- }
- };
-
- verifyAllowed(action, SUPERUSER, USER_ADMIN);
- verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE);
- }
-
- @Test
- public void testOpenRegion() throws Exception {
- PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
- public Object run() throws Exception {
- ACCESS_CONTROLLER.preOpen(ObserverContext.createAndPrepare(RCP_ENV, null));
- return null;
- }
- };
-
- verifyAllowed(action, SUPERUSER, USER_ADMIN);
- verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
- }
-
- @Test
- public void testCloseRegion() throws Exception {
- PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
- public Object run() throws Exception {
- ACCESS_CONTROLLER.preClose(ObserverContext.createAndPrepare(RCP_ENV, null), false);
- return null;
- }
- };
-
- verifyAllowed(action, SUPERUSER, USER_ADMIN);
- verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
- }
-
}
diff --git a/src/main/java/org/apache/hadoop/hbase/HConstants.java b/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 3c276de..ba657e0 100644
--- a/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -157,9 +157,6 @@
/** Default value for ZooKeeper session timeout */
public static final int DEFAULT_ZK_SESSION_TIMEOUT = 180 * 1000;
- /** Configuration key for whether to use ZK.multi */
- public static final String ZOOKEEPER_USEMULTI = "hbase.zookeeper.useMulti";
-
/** Parameter name for port region server listens on. */
public static final String REGIONSERVER_PORT = "hbase.regionserver.port";
diff --git a/src/main/java/org/apache/hadoop/hbase/HServerLoad.java b/src/main/java/org/apache/hadoop/hbase/HServerLoad.java
index ffdbc6b..f683ebe 100644
--- a/src/main/java/org/apache/hadoop/hbase/HServerLoad.java
+++ b/src/main/java/org/apache/hadoop/hbase/HServerLoad.java
@@ -60,14 +60,30 @@
private int maxHeapMB = 0;
// Regionserver-level coprocessors, e.g., WALObserver implementations.
- private Set<String> coprocessors = new TreeSet<String>();
+ // Region-level coprocessors, on the other hand, are stored inside RegionLoad
+ // objects.
+ private Set<String> coprocessors =
+ new TreeSet<String>();
/**
* HBASE-4070: Improve region server metrics to report loaded coprocessors.
- * @return the set of all the server-wide coprocessors on this regionserver
+ *
+ * @return Returns the set of all coprocessors on this
+ * regionserver, where this set is the union of the
+ * regionserver-level coprocessors on one hand, and all of the region-level
+ * coprocessors, on the other.
+ *
+ * We must iterate through all regions loaded on this regionserver to
+ * obtain all of the region-level coprocessors.
*/
- public String[] getRsCoprocessors() {
- return coprocessors.toArray(new String[0]);
+ public String[] getCoprocessors() {
+ TreeSet<String> returnValue = new TreeSet<String>(coprocessors);
+ for (Map.Entry<byte[], RegionLoad> rls: getRegionsLoad().entrySet()) {
+ for (String coprocessor: rls.getValue().getCoprocessors()) {
+ returnValue.add(coprocessor);
+ }
+ }
+ return returnValue.toArray(new String[0]);
}
/** per-region load metrics */
@@ -129,6 +145,10 @@
*/
private int totalStaticBloomSizeKB;
+ // Region-level coprocessors.
+ Set<String> coprocessors =
+ new TreeSet<String>();
+
/**
* Constructor, for Writable
*/
@@ -148,6 +168,7 @@
* @param writeRequestsCount
* @param totalCompactingKVs
* @param currentCompactedKVs
+ * @param coprocessors
*/
public RegionLoad(final byte[] name, final int stores,
final int storefiles, final int storeUncompressedSizeMB,
@@ -156,7 +177,8 @@
final int rootIndexSizeKB, final int totalStaticIndexSizeKB,
final int totalStaticBloomSizeKB,
final long readRequestsCount, final long writeRequestsCount,
- final long totalCompactingKVs, final long currentCompactedKVs) {
+ final long totalCompactingKVs, final long currentCompactedKVs,
+ final Set<String> coprocessors) {
this.name = name;
this.stores = stores;
this.storefiles = storefiles;
@@ -171,6 +193,12 @@
this.writeRequestsCount = writeRequestsCount;
this.totalCompactingKVs = totalCompactingKVs;
this.currentCompactedKVs = currentCompactedKVs;
+ this.coprocessors = coprocessors;
+ }
+
+ // Getters
+ private String[] getCoprocessors() {
+ return coprocessors.toArray(new String[0]);
}
/**
@@ -372,9 +400,9 @@
this.totalCompactingKVs = in.readLong();
this.currentCompactedKVs = in.readLong();
int coprocessorsSize = in.readInt();
- // Backward compatibility - there may be coprocessors in the region load, ignore them.
+ coprocessors = new TreeSet<String>();
for (int i = 0; i < coprocessorsSize; i++) {
- in.readUTF();
+ coprocessors.add(in.readUTF());
}
}
@@ -403,9 +431,9 @@
this.totalCompactingKVs = WritableUtils.readVLong(in);
this.currentCompactedKVs = WritableUtils.readVLong(in);
int coprocessorsSize = WritableUtils.readVInt(in);
- // Backward compatibility - there may be coprocessors in the region load, ignore them.
+ coprocessors = new TreeSet<String>();
for (int i = 0; i < coprocessorsSize; i++) {
- in.readUTF();
+ coprocessors.add(in.readUTF());
}
}
@@ -426,9 +454,10 @@
WritableUtils.writeVInt(out, totalStaticBloomSizeKB);
WritableUtils.writeVLong(out, totalCompactingKVs);
WritableUtils.writeVLong(out, currentCompactedKVs);
- // Backward compatibility - write out 0 as coprocessor count,
- // we don't report region-level coprocessors anymore.
- WritableUtils.writeVInt(out, 0);
+ WritableUtils.writeVInt(out, coprocessors.size());
+ for (String coprocessor: coprocessors) {
+ out.writeUTF(coprocessor);
+ }
}
/**
@@ -474,6 +503,11 @@
}
sb = Strings.appendKeyValue(sb, "compactionProgressPct",
compactionProgressPct);
+ String coprocessors = Arrays.toString(getCoprocessors());
+ if (coprocessors != null) {
+ sb = Strings.appendKeyValue(sb, "coprocessors",
+ Arrays.toString(getCoprocessors()));
+ }
return sb.toString();
}
}
diff --git a/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index c9c962e..bd87c34 100644
--- a/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -216,6 +216,9 @@
private int offset = 0;
private int length = 0;
+ // the row cached
+ private volatile byte [] rowCache = null;
+
/**
* @return True if a delete type, a {@link KeyValue.Type#Delete} or
* a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
@@ -984,6 +987,7 @@
int tsOffset = getTimestampOffset();
System.arraycopy(now, 0, this.bytes, tsOffset, Bytes.SIZEOF_LONG);
// clear cache or else getTimestamp() possibly returns an old value
+ timestampCache = -1L;
return true;
}
return false;
@@ -1033,19 +1037,28 @@
* @return Row in a new byte array.
*/
public byte [] getRow() {
- int o = getRowOffset();
- short l = getRowLength();
- byte result[] = new byte[l];
- System.arraycopy(getBuffer(), o, result, 0, l);
- return result;
+ if (rowCache == null) {
+ int o = getRowOffset();
+ short l = getRowLength();
+ // initialize and copy the data into a local variable
+ // in case multiple threads race here.
+ byte local[] = new byte[l];
+ System.arraycopy(getBuffer(), o, local, 0, l);
+ rowCache = local; // volatile assign
+ }
+ return rowCache;
}
/**
*
* @return Timestamp
*/
+ private long timestampCache = -1;
public long getTimestamp() {
- return getTimestamp(getKeyLength());
+ if (timestampCache == -1) {
+ timestampCache = getTimestamp(getKeyLength());
+ }
+ return timestampCache;
}
/**
@@ -2247,17 +2260,21 @@
// HeapSize
public long heapSize() {
- return ClassSize.align(ClassSize.OBJECT + ClassSize.REFERENCE
- + ClassSize.align(ClassSize.ARRAY) + ClassSize.align(length)
- + (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_LONG);
+ return ClassSize.align(ClassSize.OBJECT + (2 * ClassSize.REFERENCE) +
+ ClassSize.align(ClassSize.ARRAY) + ClassSize.align(length) +
+ (3 * Bytes.SIZEOF_INT) +
+ ClassSize.align(ClassSize.ARRAY) +
+ (2 * Bytes.SIZEOF_LONG));
}
// this overload assumes that the length bytes have already been read,
// and it expects the length of the KeyValue to be explicitly passed
// to it.
public void readFields(int length, final DataInput in) throws IOException {
+ this.rowCache = null;
this.length = length;
this.offset = 0;
+ this.timestampCache = -1;
this.keyLength = 0;
this.bytes = new byte[this.length];
in.readFully(this.bytes, 0, this.length);
diff --git a/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java b/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
deleted file mode 100644
index f5217bc..0000000
--- a/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
+++ /dev/null
@@ -1,47 +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;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Thrown by a region server if it will block and wait to serve a request.
- * For example, the client wants to insert something to a region while the
- * region is compacting.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class RegionTooBusyException extends IOException {
- private static final long serialVersionUID = 1728345723728342L;
-
- /** default constructor */
- public RegionTooBusyException() {
- super();
- }
-
- /**
- * Constructor
- * @param msg message
- */
- public RegionTooBusyException(final String msg) {
- super(msg);
- }
-}
diff --git a/src/main/java/org/apache/hadoop/hbase/UnknownRowLockException.java b/src/main/java/org/apache/hadoop/hbase/UnknownRowLockException.java
index 675e6e2..8ca50a9 100644
--- a/src/main/java/org/apache/hadoop/hbase/UnknownRowLockException.java
+++ b/src/main/java/org/apache/hadoop/hbase/UnknownRowLockException.java
@@ -22,7 +22,6 @@
/**
* Thrown if a region server is passed an unknown row lock id
- * @deprecated row locks are deprecated (and thus so our associated exceptions).
*/
public class UnknownRowLockException extends DoNotRetryIOException {
private static final long serialVersionUID = 993179627856392526L;
diff --git a/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java b/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
index 0da7c69..ed787e4 100644
--- a/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
+++ b/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
@@ -147,38 +147,6 @@
}
/**
- * Remove from the specified region the store files of the specified column family,
- * either by archiving them or outright deletion
- * @param fs the filesystem where the store files live
- * @param conf {@link Configuration} to examine to determine the archive directory
- * @param parent Parent region hosting the store files
- * @param tableDir {@link Path} to where the table is being stored (for building the archive path)
- * @param family the family hosting the store files
- * @throws IOException if the files could not be correctly disposed.
- */
- public static void archiveFamily(FileSystem fs, Configuration conf,
- HRegionInfo parent, Path tableDir, byte[] family) throws IOException {
- Path familyDir = new Path(tableDir, new Path(parent.getEncodedName(), Bytes.toString(family)));
- FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir, null);
- if (storeFiles == null) {
- LOG.debug("No store files to dispose for region=" + parent.getRegionNameAsString() +
- ", family=" + Bytes.toString(family));
- return;
- }
-
- FileStatusConverter getAsFile = new FileStatusConverter(fs);
- Collection<File> toArchive = Lists.transform(Arrays.asList(storeFiles), getAsFile);
- Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, parent, tableDir, family);
-
- // do the actual archive
- if (!resolveAndArchive(fs, storeArchiveDir, toArchive)) {
- throw new IOException("Failed to archive/delete all the files for region:"
- + Bytes.toString(parent.getRegionName()) + ", family:" + Bytes.toString(family)
- + " into " + storeArchiveDir + ". Something is probably awry on the filesystem.");
- }
- }
-
- /**
* Remove the store files, either by archiving them or outright deletion
* @param fs the filesystem where the store files live
* @param parent Parent region hosting the store files
@@ -228,7 +196,7 @@
if (!resolveAndArchive(fs, storeArchiveDir, storeFiles)) {
throw new IOException("Failed to archive/delete all the files for region:"
+ Bytes.toString(parent.getRegionName()) + ", family:" + Bytes.toString(family)
- + " into " + storeArchiveDir + ". Something is probably awry on the filesystem.");
+ + " into " + storeArchiveDir + "Something is probably arwy on the filesystem.");
}
}
diff --git a/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java b/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java
index 780ed0a..1da3b27 100644
--- a/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java
+++ b/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java
@@ -432,6 +432,7 @@
return true;
}
if (!isInsideTable(this.current, tableNameBytes)) return false;
+ if (this.current.isSplitParent()) return true;
// Else call super and add this Result to the collection.
super.visit(r);
// Stop collecting regions from table after we get one.
diff --git a/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/src/main/java/org/apache/hadoop/hbase/client/Delete.java
index 26986e7..a806f8a 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/Delete.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/Delete.java
@@ -86,25 +86,6 @@
}
/**
- * Create a Delete operation for the specified row and timestamp.<p>
- *
- * If no further operations are done, this will delete all columns in all
- * families of the specified row with a timestamp less than or equal to the
- * specified timestamp.<p>
- *
- * This timestamp is ONLY used for a delete row operation. If specifying
- * families or columns, you must specify each timestamp individually.
- * @param row row key
- * @param timestamp maximum version timestamp (only for delete row)
- * @param rowLock previously acquired row lock, or null
- * @deprecated {@link RowLock} is deprecated, use #de
- */
- public Delete(byte [] row, long timestamp) {
- this.row = row;
- this.ts = timestamp;
- }
-
- /**
* Create a Delete operation for the specified row and timestamp, using
* an optional row lock.<p>
*
@@ -117,7 +98,6 @@
* @param row row key
* @param timestamp maximum version timestamp (only for delete row)
* @param rowLock previously acquired row lock, or null
- * @deprecated {@link RowLock} is deprecated, use {@link #Delete(byte[], long)}.
*/
public Delete(byte [] row, long timestamp, RowLock rowLock) {
this.row = row;
diff --git a/src/main/java/org/apache/hadoop/hbase/client/Get.java b/src/main/java/org/apache/hadoop/hbase/client/Get.java
index 0fb6838..3f35cae 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/Get.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/Get.java
@@ -20,7 +20,6 @@
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.TimeRange;
@@ -98,7 +97,6 @@
* all columns in all families of the specified row.
* @param row row key
* @param rowLock previously acquired row lock, or null
- * @deprecated {@link RowLock} is deprecated, use {@link #Get(byte[])}.
*/
public Get(byte [] row, RowLock rowLock) {
this.row = row;
@@ -133,9 +131,6 @@
if(set == null) {
set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
}
- if (qualifier == null) {
- qualifier = HConstants.EMPTY_BYTE_ARRAY;
- }
set.add(qualifier);
familyMap.put(family, set);
return this;
diff --git a/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
index 08fd2a9..e3f483a 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
@@ -494,7 +494,6 @@
private final Object masterLock = new Object();
private volatile boolean closed;
private volatile boolean aborted;
- private volatile boolean resetting;
private volatile HMasterInterface master;
// ZooKeeper reference
private volatile ZooKeeperWatcher zooKeeper;
@@ -506,8 +505,6 @@
private final Object metaRegionLock = new Object();
private final Object userRegionLock = new Object();
-
- private final Object resetLock = new Object();
private final Configuration conf;
// Known region HServerAddress.toString() -> HRegionInterface
@@ -576,7 +573,6 @@
HConstants.DEFAULT_HBASE_CLIENT_PREFETCH_LIMIT);
this.master = null;
- this.resetting = false;
}
private synchronized void ensureZookeeperTrackers()
@@ -1682,12 +1678,7 @@
LOG.info("ZK session expired. This disconnect could have been" +
" caused by a network partition or a long-running GC pause," +
" either way it's recommended that you verify your environment.");
- synchronized (resetLock) {
- if (resetting) return;
- this.resetting = true;
- }
resetZooKeeperTrackers();
- this.resetting = false;
}
return;
}
diff --git a/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java b/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
index 404567b..007f0a3 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
@@ -380,7 +380,6 @@
* @throws IOException if a remote or network exception occurs.
* @see RowLock
* @see #unlockRow
- * @deprecated {@link RowLock} and associated operations are deprecated
*/
RowLock lockRow(byte[] row) throws IOException;
@@ -391,7 +390,6 @@
* @throws IOException if a remote or network exception occurs.
* @see RowLock
* @see #unlockRow
- * @deprecated {@link RowLock} and associated operations are deprecated
*/
void unlockRow(RowLock rl) throws IOException;
diff --git a/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java b/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
index 698241b..398dc52 100755
--- a/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
@@ -458,17 +458,11 @@
returnTable(table);
}
- /**
- * @deprecated {@link RowLock} and associated operations are deprecated
- */
@Override
public RowLock lockRow(byte[] row) throws IOException {
return table.lockRow(row);
}
- /**
- * @deprecated {@link RowLock} and associated operations are deprecated
- */
@Override
public void unlockRow(RowLock rl) throws IOException {
table.unlockRow(rl);
diff --git a/src/main/java/org/apache/hadoop/hbase/client/Increment.java b/src/main/java/org/apache/hadoop/hbase/client/Increment.java
index 52fac04..ef0b654 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/Increment.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/Increment.java
@@ -73,8 +73,6 @@
* At least one column must be incremented.
* @param row row key
* @param rowLock previously acquired row lock, or null
- * @deprecated {@link RowLock} and associated operations are deprecated,
- * use {@link #Increment(byte[])}
*/
public Increment(byte [] row, RowLock rowLock) {
this.row = row;
@@ -116,7 +114,6 @@
/**
* Method for retrieving the increment's RowLock
* @return RowLock
- * @deprecated {@link RowLock} and associated operations are deprecated
*/
public RowLock getRowLock() {
return new RowLock(this.row, this.lockId);
@@ -125,7 +122,6 @@
/**
* Method for retrieving the increment's lockId
* @return lockId
- * @deprecated {@link RowLock} and associated operations are deprecated
*/
public long getLockId() {
return this.lockId;
diff --git a/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index 812b560..79a6d1f 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -164,7 +164,6 @@
/**
* Method for retrieving the delete's RowLock
* @return RowLock
- * @deprecated {@link RowLock} and associated operations are deprecated
*/
public RowLock getRowLock() {
return new RowLock(this.row, this.lockId);
@@ -174,7 +173,6 @@
* Method for retrieving the delete's lock ID.
*
* @return The lock ID.
- * @deprecated {@link RowLock} and associated operations are deprecated
*/
public long getLockId() {
return this.lockId;
@@ -193,7 +191,6 @@
* @param clusterId
*/
public void setClusterId(UUID clusterId) {
- if (clusterId == null) return;
byte[] val = new byte[2*Bytes.SIZEOF_LONG];
Bytes.putLong(val, 0, clusterId.getMostSignificantBits());
Bytes.putLong(val, Bytes.SIZEOF_LONG, clusterId.getLeastSignificantBits());
diff --git a/src/main/java/org/apache/hadoop/hbase/client/Put.java b/src/main/java/org/apache/hadoop/hbase/client/Put.java
index 4596e57..1ec81ff 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/Put.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/Put.java
@@ -67,7 +67,6 @@
* Create a Put operation for the specified row, using an existing row lock.
* @param row row key
* @param rowLock previously acquired row lock, or null
- * @deprecated {@link RowLock} and associated operations are deprecated, use {@link #Put(byte[])}
*/
public Put(byte [] row, RowLock rowLock) {
this(row, HConstants.LATEST_TIMESTAMP, rowLock);
@@ -88,8 +87,6 @@
* @param row row key
* @param ts timestamp
* @param rowLock previously acquired row lock, or null
- * @deprecated {@link RowLock} and associated operations are deprecated,
- * use {@link #Put(byte[], long)}
*/
public Put(byte [] row, long ts, RowLock rowLock) {
if(row == null || row.length > HConstants.MAX_ROW_LENGTH) {
diff --git a/src/main/java/org/apache/hadoop/hbase/client/Result.java b/src/main/java/org/apache/hadoop/hbase/client/Result.java
index bf8bda3..d80b8c9 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -96,7 +96,7 @@
* @param kvs List of KeyValues
*/
public Result(List<KeyValue> kvs) {
- this(kvs.toArray(new KeyValue[kvs.size()]));
+ this(kvs.toArray(new KeyValue[0]));
}
/**
@@ -649,15 +649,4 @@
}
}
}
-
- /**
- * Copy another Result into this one. Needed for the old Mapred framework
- * @param other
- */
- public void copyFrom(Result other) {
- this.row = other.row;
- this.bytes = other.bytes;
- this.familyMap = other.familyMap;
- this.kvs = other.kvs;
- }
}
diff --git a/src/main/java/org/apache/hadoop/hbase/client/RowLock.java b/src/main/java/org/apache/hadoop/hbase/client/RowLock.java
index 5888ec8..56b0787 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/RowLock.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/RowLock.java
@@ -21,7 +21,6 @@
/**
* Holds row name and lock id.
- * @deprecated {@link RowLock} and associated operations are deprecated.
*/
public class RowLock {
private byte [] row = null;
diff --git a/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/src/main/java/org/apache/hadoop/hbase/client/Scan.java
index 024059b..093c93b 100644
--- a/src/main/java/org/apache/hadoop/hbase/client/Scan.java
+++ b/src/main/java/org/apache/hadoop/hbase/client/Scan.java
@@ -215,9 +215,6 @@
if(set == null) {
set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
}
- if (qualifier == null) {
- qualifier = HConstants.EMPTY_BYTE_ARRAY;
- }
set.add(qualifier);
familyMap.put(family, set);
diff --git a/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java b/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
index abc972c..5889f32 100644
--- a/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
+++ b/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
@@ -58,14 +58,14 @@
public void stop(CoprocessorEnvironment e) throws IOException { }
@Override
- public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException { }
+ public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e) { }
@Override
public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) { }
@Override
- public void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
- throws IOException { }
+ public void preClose(ObserverContext<RegionCoprocessorEnvironment> e,
+ boolean abortRequested) { }
@Override
public void postClose(ObserverContext<RegionCoprocessorEnvironment> e,
@@ -320,20 +320,4 @@
List<Pair<byte[], String>> familyPaths, boolean hasLoaded) throws IOException {
return hasLoaded;
}
-
- @Override
- public void preLockRow(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] regionName,
- byte[] row) throws IOException { }
-
- @Override
- public void preUnlockRow(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] regionName,
- long lockId) throws IOException { }
-
- @Override
- public void postLockRow(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] regionName,
- byte[] row) throws IOException { }
-
- @Override
- public void postUnlockRow(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] regionName,
- long lockId) throws IOException { }
}
diff --git a/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorClassLoader.java b/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorClassLoader.java
index 42d7dfe..7d0ba92 100644
--- a/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorClassLoader.java
+++ b/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorClassLoader.java
@@ -65,13 +65,9 @@
"org.w3c",
"org.xml",
"sunw.",
- // logging
- "org.apache.commons.logging",
- "org.apache.log4j",
- "com.hadoop",
- // Hadoop/HBase/ZK:
+ // Hadoop/HBase:
"org.apache.hadoop",
- "org.apache.zookeeper",
+ "com.hadoop",
};
/**
@@ -84,12 +80,7 @@
new Pattern[] {
Pattern.compile("^[^-]+-default\\.xml$")
};
-
- /**
- * Parent classloader used to load any class not matching the exemption list.
- */
- private final ClassLoader parent;
-
+
/**
* Creates a CoprocessorClassLoader that loads classes from the given paths.
* @param paths paths from which to load classes.
@@ -97,12 +88,8 @@
*/
public CoprocessorClassLoader(List<URL> paths, ClassLoader parent) {
super(paths.toArray(new URL[]{}), parent);
- this.parent = parent;
- if (parent == null) {
- throw new IllegalArgumentException("No parent classloader!");
- }
}
-
+
@Override
synchronized public Class<?> loadClass(String name)
throws ClassNotFoundException {
@@ -112,9 +99,9 @@
LOG.debug("Skipping exempt class " + name +
" - delegating directly to parent");
}
- return parent.loadClass(name);
+ return super.loadClass(name);
}
-
+
// Check whether the class has already been loaded:
Class<?> clasz = findLoadedClass(name);
if (clasz != null) {
@@ -136,7 +123,7 @@
LOG.debug("Class " + name + " not found - delegating to parent");
}
try {
- clasz = parent.loadClass(name);
+ clasz = super.loadClass(name);
} catch (ClassNotFoundException e2) {
// Class not found in this ClassLoader or in the parent ClassLoader
// Log some debug output before rethrowing ClassNotFoundException
diff --git a/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
index bc2927d..2fdaf6f 100644
--- a/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
+++ b/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
@@ -20,8 +20,6 @@
package org.apache.hadoop.hbase.coprocessor;
-import com.google.common.collect.MapMaker;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -46,7 +44,6 @@
import java.io.IOException;
import java.net.URL;
import java.util.*;
-import java.util.concurrent.ConcurrentMap;
import java.util.jar.JarEntry;
import java.util.jar.JarFile;
@@ -58,13 +55,11 @@
*/
public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
public static final String REGION_COPROCESSOR_CONF_KEY =
- "hbase.coprocessor.region.classes";
- public static final String REGIONSERVER_COPROCESSOR_CONF_KEY =
- "hbase.coprocessor.regionserver.classes";
+ "hbase.coprocessor.region.classes";
public static final String USER_REGION_COPROCESSOR_CONF_KEY =
- "hbase.coprocessor.user.region.classes";
+ "hbase.coprocessor.user.region.classes";
public static final String MASTER_COPROCESSOR_CONF_KEY =
- "hbase.coprocessor.master.classes";
+ "hbase.coprocessor.master.classes";
public static final String WAL_COPROCESSOR_CONF_KEY =
"hbase.coprocessor.wal.classes";
@@ -77,15 +72,6 @@
protected String pathPrefix;
protected volatile int loadSequence;
- /*
- * External classloaders cache keyed by external jar path.
- * ClassLoader instance is stored as a weak-reference
- * to allow GC'ing when no CoprocessorHost is using it
- * (@see HBASE-7205)
- */
- static ConcurrentMap<Path, ClassLoader> classLoadersCache =
- new MapMaker().concurrencyLevel(3).weakValues().makeMap();
-
public CoprocessorHost() {
pathPrefix = UUID.randomUUID().toString();
}
@@ -173,27 +159,14 @@
LOG.debug("Loading coprocessor class " + className + " with path " +
path + " and priority " + priority);
- ClassLoader cl = null;
- if (path == null) {
- try {
- implClass = getClass().getClassLoader().loadClass(className);
- } catch (ClassNotFoundException e) {
- throw new IOException("No jar path specified for " + className);
- }
- } else {
- // Have we already loaded the class, perhaps from an earlier region open
- // for the same table?
- cl = classLoadersCache.get(path);
- if (cl != null){
- LOG.debug("Found classloader "+ cl + "for "+path.toString());
- try {
- implClass = cl.loadClass(className);
- } catch (ClassNotFoundException e) {
- LOG.info("Class " + className + " needs to be loaded from a file - " +
- path + ".");
- // go ahead to load from file system.
- }
- }
+ // Have we already loaded the class, perhaps from an earlier region open
+ // for the same table?
+ try {
+ implClass = getClass().getClassLoader().loadClass(className);
+ } catch (ClassNotFoundException e) {
+ LOG.info("Class " + className + " needs to be loaded from a file - " +
+ path + ".");
+ // go ahead to load from file system.
}
// If not, load
@@ -225,8 +198,7 @@
// unsurprisingly wants URLs, not URIs; so we will use the deprecated
// method which returns URLs for as long as it is available
List<URL> paths = new ArrayList<URL>();
- URL url = new File(dst.toString()).getCanonicalFile().toURL();
- paths.add(url);
+ paths.add(new File(dst.toString()).getCanonicalFile().toURL());
JarFile jarFile = new JarFile(dst.toString());
Enumeration<JarEntry> entries = jarFile.entries();
@@ -243,33 +215,17 @@
}
jarFile.close();
- cl = new CoprocessorClassLoader(paths, this.getClass().getClassLoader());
- // cache cp classloader as a weak value, will be GC'ed when no reference left
- ClassLoader prev = classLoadersCache.putIfAbsent(path, cl);
- if (prev != null) {
- //lost update race, use already added classloader
- cl = prev;
- }
-
+ ClassLoader cl = new CoprocessorClassLoader(paths,
+ this.getClass().getClassLoader());
+ Thread.currentThread().setContextClassLoader(cl);
try {
implClass = cl.loadClass(className);
} catch (ClassNotFoundException e) {
- throw new IOException("Cannot load external coprocessor class " + className, e);
+ throw new IOException(e);
}
}
- //load custom code for coprocessor
- Thread currentThread = Thread.currentThread();
- ClassLoader hostClassLoader = currentThread.getContextClassLoader();
- try{
- // switch temporarily to the thread classloader for custom CP
- currentThread.setContextClassLoader(cl);
- E cpInstance = loadInstance(implClass, priority, conf);
- return cpInstance;
- } finally {
- // restore the fresh (host) classloader
- currentThread.setContextClassLoader(hostClassLoader);
- }
+ return loadInstance(implClass, priority, conf);
}
/**
@@ -346,24 +302,6 @@
}
/**
- * Retrieves the set of classloaders used to instantiate Coprocessor classes defined in external
- * jar files.
- * @return A set of ClassLoader instances
- */
- Set<ClassLoader> getExternalClassLoaders() {
- Set<ClassLoader> externalClassLoaders = new HashSet<ClassLoader>();
- final ClassLoader systemClassLoader = this.getClass().getClassLoader();
- for (E env : coprocessors) {
- ClassLoader cl = env.getInstance().getClass().getClassLoader();
- if (cl != systemClassLoader ){
- //do not include system classloader
- externalClassLoaders.add(cl);
- }
- }
- return externalClassLoaders;
- }
-
- /**
* Find a coprocessor environment by class name
* @param className the class name
* @return the coprocessor, or null if not found
@@ -537,17 +475,11 @@
return tableName;
}
- /**
- * @deprecated {@link RowLock} and associated operations are deprecated.
- */
public RowLock lockRow(byte[] row) throws IOException {
throw new RuntimeException(
"row locking is not allowed within the coprocessor environment");
}
- /**
- * @deprecated {@link RowLock} and associated operations are deprecated.
- */
public void unlockRow(RowLock rl) throws IOException {
throw new RuntimeException(
"row locking is not allowed within the coprocessor environment");
diff --git a/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index fa65fed..3dc2909 100644
--- a/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ b/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
@@ -55,9 +55,8 @@
/**
* Called before the region is reported as open to the master.
* @param c the environment provided by the region server
- * @throws IOException if an error occurred on the coprocessor
*/
- void preOpen(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException;
+ void preOpen(final ObserverContext<RegionCoprocessorEnvironment> c);
/**
* Called after the region is reported as open to the master.
@@ -228,10 +227,9 @@
* Called before the region is reported as closed to the master.
* @param c the environment provided by the region server
* @param abortRequested true if the region server is aborting
- * @throws IOException
*/
void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
- boolean abortRequested) throws IOException;
+ boolean abortRequested);
/**
* Called after the region is reported as closed to the master.
@@ -773,55 +771,4 @@
*/
boolean postBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
List<Pair<byte[], String>> familyPaths, boolean hasLoaded) throws IOException;
-
- /**
- * Called before locking a row.
- *
- * @param ctx
- * @param regionName
- * @param row
- * @throws IOException Signals that an I/O exception has occurred.
- * @deprecated Will be removed in 0.96
- */
- @Deprecated
- void preLockRow(final ObserverContext<RegionCoprocessorEnvironment> ctx,
- final byte[] regionName, final byte[] row) throws IOException;
-
- /**
- * Called after locking a row.
- *
- * @param ctx
- * @param regionName the region name
- * @param row
- * @throws IOException Signals that an I/O exception has occurred.
- * @deprecated Will be removed in 0.96
- */
- @Deprecated
- void postLockRow(final ObserverContext<RegionCoprocessorEnvironment> ctx,
- final byte[] regionName, final byte[] row) throws IOException;
-
- /**
- * Called before unlocking a row.
- *
- * @param ctx
- * @param regionName
- * @param lockId the lock id
- * @throws IOException Signals that an I/O exception has occurred.
- * @deprecated Will be removed in 0.96
- */
- @Deprecated
- void preUnlockRow(final ObserverContext<RegionCoprocessorEnvironment> ctx,
- final byte[] regionName, final long lockId) throws IOException;
-
- /**
- * Called after unlocking a row.
- * @param ctx
- * @param regionName the region name
- * @param lockId the lock id
- * @throws IOException Signals that an I/O exception has occurred.
- * @deprecated Will be removed in 0.96
- */
- @Deprecated
- void postUnlockRow(final ObserverContext<RegionCoprocessorEnvironment> ctx,
- final byte[] regionName, final long lockId) throws IOException;
}
diff --git a/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerCoprocessorEnvironment.java b/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerCoprocessorEnvironment.java
deleted file mode 100644
index 6a07b8b..0000000
--- a/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerCoprocessorEnvironment.java
+++ /dev/null
@@ -1,29 +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.coprocessor;
-
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-
-public interface RegionServerCoprocessorEnvironment extends CoprocessorEnvironment {
-
- /** @return reference to the HMaster services */
- RegionServerServices getRegionServerServices();
-
-}
diff --git a/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java b/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
deleted file mode 100644
index 044d086..0000000
--- a/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
+++ /dev/null
@@ -1,35 +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.coprocessor;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.Coprocessor;
-
-public interface RegionServerObserver extends Coprocessor {
-
- /**
- * Called before stopping region server.
- * @param env An instance of RegionServerCoprocessorEnvironment
- * @throws IOException Signals that an I/O exception has occurred.
- */
- void preStopRegionServer(final ObserverContext<RegionServerCoprocessorEnvironment> env)
- throws IOException;
-
-}
diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index edc633f..d0904aa 100644
--- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -29,8 +29,6 @@
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -1257,8 +1255,6 @@
/** The path (if any) where this data is coming from */
protected Path path;
- private final Lock streamLock = new ReentrantLock();
-
/** The default buffer size for our buffered streams */
public static final int DEFAULT_BUFFER_SIZE = 1 << 20;
@@ -1333,9 +1329,23 @@
"-byte array at offset " + destOffset);
}
- if (!pread && streamLock.tryLock()) {
+ if (pread) {
+ // Positional read. Better for random reads.
+ int extraSize = peekIntoNextBlock ? hdrSize : 0;
+
+ int ret = istream.read(fileOffset, dest, destOffset, size + extraSize);
+ if (ret < size) {
+ throw new IOException("Positional read of " + size + " bytes " +
+ "failed at offset " + fileOffset + " (returned " + ret + ")");
+ }
+
+ if (ret == size || ret < size + extraSize) {
+ // Could not read the next block's header, or did not try.
+ return -1;
+ }
+ } else {
// Seek + read. Better for scanning.
- try {
+ synchronized (istream) {
istream.seek(fileOffset);
long realOffset = istream.getPos();
@@ -1353,22 +1363,6 @@
// Try to read the next block header.
if (!readWithExtra(istream, dest, destOffset, size, hdrSize))
return -1;
- } finally {
- streamLock.unlock();
- }
- } else {
- // Positional read. Better for random reads; or when the streamLock is already locked.
- int extraSize = peekIntoNextBlock ? hdrSize : 0;
-
- int ret = istream.read(fileOffset, dest, destOffset, size + extraSize);
- if (ret < size) {
- throw new IOException("Positional read of " + size + " bytes " +
- "failed at offset " + fileOffset + " (returned " + ret + ")");
- }
-
- if (ret == size || ret < size + extraSize) {
- // Could not read the next block's header, or did not try.
- return -1;
}
}
diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index 8611b11..d655d3d 100644
--- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -342,7 +342,7 @@
midKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset + keyLen);
} else {
// The middle of the root-level index.
- midKey = blockKeys[rootCount / 2];
+ midKey = blockKeys[(rootCount - 1) / 2];
}
this.midKey.set(midKey);
@@ -1429,4 +1429,5 @@
public static int getMaxChunkSize(Configuration conf) {
return conf.getInt(MAX_CHUNK_SIZE_KEY, DEFAULT_MAX_CHUNK_SIZE);
}
+
}
diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
index 5f6f652..e6987ff 100644
--- a/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
+++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
@@ -645,7 +645,7 @@
// Log size
long totalSize = heapSize();
long freeSize = maxSize - totalSize;
- LruBlockCache.LOG.debug("Stats: " +
+ LruBlockCache.LOG.debug("LRU Stats: " +
"total=" + StringUtils.byteDesc(totalSize) + ", " +
"free=" + StringUtils.byteDesc(freeSize) + ", " +
"max=" + StringUtils.byteDesc(this.maxSize) + ", " +
@@ -653,11 +653,11 @@
"accesses=" + stats.getRequestCount() + ", " +
"hits=" + stats.getHitCount() + ", " +
"hitRatio=" +
- (stats.getHitCount() == 0 ? "0" : (StringUtils.formatPercent(stats.getHitRatio(), 2)+ ", ")) + ", " +
+ (stats.getHitCount() == 0 ? "0" : (StringUtils.formatPercent(stats.getHitRatio(), 2)+ ", ")) +
"cachingAccesses=" + stats.getRequestCachingCount() + ", " +
"cachingHits=" + stats.getHitCachingCount() + ", " +
"cachingHitsRatio=" +
- (stats.getHitCachingCount() == 0 ? "0" : (StringUtils.formatPercent(stats.getHitCachingRatio(), 2)+ ", ")) + ", " +
+ (stats.getHitCachingCount() == 0 ? "0" : (StringUtils.formatPercent(stats.getHitCachingRatio(), 2)+ ", ")) +
"evictions=" + stats.getEvictionCount() + ", " +
"evicted=" + stats.getEvictedCount() + ", " +
"evictedPerRun=" + stats.evictedPerEviction());
diff --git a/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java b/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
index 72e5774..5212ccd 100644
--- a/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
+++ b/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
@@ -227,7 +227,7 @@
if (result != null && result.size() > 0) {
key.set(result.getRow());
lastSuccessfulRow = key.get();
- value.copyFrom(result);
+ Writables.copyWritable(result, value);
return true;
}
return false;
diff --git a/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index 31238de..03d3f8d 100644
--- a/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -22,32 +22,23 @@
import java.io.IOException;
import java.util.Map;
import java.util.TreeMap;
-import java.util.UUID;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Delete;
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.Result;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.GenericOptionsParser;
-import org.apache.zookeeper.KeeperException;
/**
* Import data written by {@link Export}.
@@ -56,7 +47,6 @@
final static String NAME = "import";
final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
- private static final Log LOG = LogFactory.getLog(Import.class);
/**
* A mapper that just writes out KeyValues.
@@ -98,7 +88,6 @@
static class Importer
extends TableMapper<ImmutableBytesWritable, Mutation> {
private Map<byte[], byte[]> cfRenameMap;
- private UUID clusterId;
/**
* @param row The current table row key.
@@ -139,32 +128,16 @@
}
}
if (put != null) {
- put.setClusterId(clusterId);
context.write(key, put);
}
if (delete != null) {
- delete.setClusterId(clusterId);
context.write(key, delete);
}
}
@Override
public void setup(Context context) {
- Configuration conf = context.getConfiguration();
- cfRenameMap = createCfRenameMap(conf);
- try {
- HConnection connection = HConnectionManager.getConnection(conf);
- ZooKeeperWatcher zkw = connection.getZooKeeperWatcher();
- ReplicationZookeeper zkHelper = new ReplicationZookeeper(connection, conf, zkw);
- clusterId = zkHelper.getUUIDForCluster(zkw);
- } catch (ZooKeeperConnectionException e) {
- LOG.error("Problem connecting to ZooKeper during task setup", e);
- } catch (KeeperException e) {
- LOG.error("Problem reading ZooKeeper data during task setup", e);
- } catch (IOException e) {
- LOG.error("Problem setting up task", e);
- }
-
+ cfRenameMap = createCfRenameMap(context.getConfiguration());
}
}
@@ -319,4 +292,4 @@
Job job = createSubmittableJob(conf, otherArgs);
System.exit(job.waitForCompletion(true) ? 0 : 1);
}
-}
+}
\ No newline at end of file
diff --git a/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index e0598a8..503061d 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -142,10 +142,9 @@
// store all the table names in disabling state
Set<String> disablingTables = new HashSet<String>(1);
- // store all the enabling state table names and corresponding online servers' regions.
- // This may be needed to avoid calling assign twice for the regions of the ENABLING table
- // that could have been assigned through processRIT.
- Map<String, List<HRegionInfo>> enablingTables = new HashMap<String, List<HRegionInfo>>(1);
+ // store all the enabling state tablenames.
+ Set<String> enablingTables = new HashSet<String>(1);
+
/**
* Server to regions assignment map.
* Contains the set of regions currently assigned to a given server.
@@ -275,16 +274,6 @@
}
/**
- * Gives enabling table regions.
- *
- * @param tableName
- * @return list of regionInfos
- */
- public List<HRegionInfo> getEnablingTableRegions(String tableName){
- return this.enablingTables.get(tableName);
- }
-
- /**
* Add a regionPlan for the specified region.
* @param encodedName
* @param plan
@@ -375,9 +364,7 @@
// Recover the tables that were not fully moved to DISABLED state.
// These tables are in DISABLING state when the master restarted/switched.
boolean isWatcherCreated = recoverTableInDisablingState(this.disablingTables);
- recoverTableInEnablingState(this.enablingTables.keySet(), isWatcherCreated);
- this.enablingTables.clear();
- this.disablingTables.clear();
+ recoverTableInEnablingState(this.enablingTables, isWatcherCreated);
}
/**
@@ -522,10 +509,6 @@
String encodedRegionName = regionInfo.getEncodedName();
LOG.info("Processing region " + regionInfo.getRegionNameAsString() +
" in state " + data.getEventType());
- List<HRegionInfo> hris = this.enablingTables.get(regionInfo.getTableNameAsString());
- if (hris != null && !hris.isEmpty()) {
- hris.remove(regionInfo);
- }
synchronized (regionsInTransition) {
RegionState regionState = regionsInTransition.get(encodedRegionName);
if (regionState != null ||
@@ -2323,12 +2306,11 @@
// Skip assignment for regions of tables in DISABLING state also because
// during clean cluster startup no RS is alive and regions map also doesn't
// have any information about the regions. See HBASE-6281.
- Set<String> disablingDisabledAndEnablingTables = new HashSet<String>(this.disablingTables);
- disablingDisabledAndEnablingTables.addAll(this.zkTable.getDisabledTables());
- disablingDisabledAndEnablingTables.addAll(this.enablingTables.keySet());
+ Set<String> disablingAndDisabledTables = new HashSet<String>(this.disablingTables);
+ disablingAndDisabledTables.addAll(this.zkTable.getDisabledTables());
// Scan META for all user regions, skipping any disabled tables
Map<HRegionInfo, ServerName> allRegions = MetaReader.fullScan(catalogTracker,
- disablingDisabledAndEnablingTables, true);
+ disablingAndDisabledTables, true);
if (allRegions == null || allRegions.isEmpty()) return;
// Get all available servers
@@ -2576,14 +2558,13 @@
// from ENABLED state when application calls disableTable.
// It can't be in DISABLED state, because DISABLED states transitions
// from DISABLING state.
- boolean enabling = checkIfRegionsBelongsToEnabling(regionInfo);
- addTheTablesInPartialState(regionInfo);
- if (enabling) {
- addToEnablingTableRegions(regionInfo);
- } else {
- LOG.warn("Region " + regionInfo.getEncodedName() + " has null regionLocation."
- + " But its table " + tableName + " isn't in ENABLING state.");
+ if (false == checkIfRegionsBelongsToEnabling(regionInfo)) {
+ LOG.warn("Region " + regionInfo.getEncodedName() +
+ " has null regionLocation." + " But its table " + tableName +
+ " isn't in ENABLING state.");
}
+ addTheTablesInPartialState(this.disablingTables, this.enablingTables, regionInfo,
+ tableName);
} else if (!onlineServers.contains(regionLocation)) {
// Region is located on a server that isn't online
List<Pair<HRegionInfo, Result>> offlineRegions =
@@ -2594,7 +2575,8 @@
}
offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
disabled = checkIfRegionBelongsToDisabled(regionInfo);
- disablingOrEnabling = addTheTablesInPartialState(regionInfo);
+ disablingOrEnabling = addTheTablesInPartialState(this.disablingTables,
+ this.enablingTables, regionInfo, tableName);
// need to enable the table if not disabled or disabling or enabling
// this will be used in rolling restarts
enableTableIfNotDisabledOrDisablingOrEnabling(disabled,
@@ -2615,18 +2597,16 @@
}
// Region is being served and on an active server
// add only if region not in disabled and enabling table
- boolean enabling = checkIfRegionsBelongsToEnabling(regionInfo);
- disabled = checkIfRegionBelongsToDisabled(regionInfo);
- if (!enabling && !disabled) {
+ if (false == checkIfRegionBelongsToDisabled(regionInfo)
+ && false == checkIfRegionsBelongsToEnabling(regionInfo)) {
synchronized (this.regions) {
regions.put(regionInfo, regionLocation);
addToServers(regionLocation, regionInfo);
}
}
- disablingOrEnabling = addTheTablesInPartialState(regionInfo);
- if (enabling) {
- addToEnablingTableRegions(regionInfo);
- }
+ disablingOrEnabling = addTheTablesInPartialState(this.disablingTables,
+ this.enablingTables, regionInfo, tableName);
+ disabled = checkIfRegionBelongsToDisabled(regionInfo);
// need to enable the table if not disabled or disabling or enabling
// this will be used in rolling restarts
enableTableIfNotDisabledOrDisablingOrEnabling(disabled,
@@ -2636,18 +2616,6 @@
return offlineServers;
}
- private void addToEnablingTableRegions(HRegionInfo regionInfo) {
- String tableName = regionInfo.getTableNameAsString();
- List<HRegionInfo> hris = this.enablingTables.get(tableName);
- if (!hris.contains(regionInfo)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Adding region" + regionInfo.getRegionNameAsString()
- + " to enabling table " + tableName + ".");
- }
- hris.add(regionInfo);
- }
- }
-
private void enableTableIfNotDisabledOrDisablingOrEnabling(boolean disabled,
boolean disablingOrEnabling, String tableName) {
if (!disabled && !disablingOrEnabling
@@ -2656,15 +2624,14 @@
}
}
- private Boolean addTheTablesInPartialState(HRegionInfo regionInfo) {
- String tableName = regionInfo.getTableNameAsString();
+ private Boolean addTheTablesInPartialState(Set<String> disablingTables,
+ Set<String> enablingTables, HRegionInfo regionInfo,
+ String disablingTableName) {
if (checkIfRegionBelongsToDisabling(regionInfo)) {
- this.disablingTables.add(tableName);
+ disablingTables.add(disablingTableName);
return true;
} else if (checkIfRegionsBelongsToEnabling(regionInfo)) {
- if (!this.enablingTables.containsKey(tableName)) {
- this.enablingTables.put(tableName, new ArrayList<HRegionInfo>());
- }
+ enablingTables.add(disablingTableName);
return true;
}
return false;
diff --git a/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 1b61299..0cf1eba 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -281,10 +281,6 @@
this.rsFatals = new MemoryBoundedLogMessageBuffer(
conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
- // login the zookeeper client principal (if using security)
- ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
- "hbase.zookeeper.client.kerberos.principal", this.isa.getHostName());
-
// initialize server principal (if using secure Hadoop)
User.login(conf, "hbase.master.keytab.file",
"hbase.master.kerberos.principal", this.isa.getHostName());
diff --git a/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java b/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java
index e6fcce5..ae9faf9 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java
@@ -39,7 +39,6 @@
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.ServerCommandLine;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.zookeeper.KeeperException;
public class HMasterCommandLine extends ServerCommandLine {
@@ -125,11 +124,6 @@
+ HConstants.ZOOKEEPER_CLIENT_PORT);
}
zooKeeperCluster.setDefaultClientPort(zkClientPort);
-
- // login the zookeeper server principal (if using security)
- ZKUtil.loginServer(conf, "hbase.zookeeper.server.keytab.file",
- "hbase.zookeeper.server.kerberos.principal", null);
-
int clientPort = zooKeeperCluster.startup(zkDataPath);
if (clientPort != zkClientPort) {
String errorMsg = "Could not start ZK at requested port of " +
diff --git a/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index 5999a53..3552c22 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -99,8 +99,6 @@
String fsUri = this.fs.getUri().toString();
conf.set("fs.default.name", fsUri);
conf.set("fs.defaultFS", fsUri);
- // make sure the fs has the same conf
- fs.setConf(conf);
this.distributedLogSplitting =
conf.getBoolean("hbase.master.distributed.log.splitting", true);
if (this.distributedLogSplitting) {
@@ -456,23 +454,6 @@
// @see HRegion.checkRegioninfoOnFilesystem()
}
- public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName)
- throws IOException {
- // archive family store files
- Path tableDir = new Path(rootdir, region.getTableNameAsString());
- HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
-
- // delete the family folder
- Path familyDir = new Path(tableDir,
- new Path(region.getEncodedName(), Bytes.toString(familyName)));
- if (fs.delete(familyDir, true) == false) {
- throw new IOException("Could not delete family "
- + Bytes.toString(familyName) + " from FileSystem for region "
- + region.getRegionNameAsString() + "(" + region.getEncodedName()
- + ")");
- }
- }
-
public void stop() {
if (splitLogManager != null) {
this.splitLogManager.stop();
diff --git a/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index eb0e540..8be3c75 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -21,12 +21,13 @@
import java.io.IOException;
-import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
+import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
/**
* Services Master supplies
@@ -70,63 +71,6 @@
throws IOException;
/**
- * Delete a table
- * @param tableName The table name
- * @throws IOException
- */
- public void deleteTable(final byte[] tableName) throws IOException;
-
- /**
- * Modify the descriptor of an existing table
- * @param tableName The table name
- * @param descriptor The updated table descriptor
- * @throws IOException
- */
- public void modifyTable(final byte[] tableName, final HTableDescriptor descriptor)
- throws IOException;
-
- /**
- * Enable an existing table
- * @param tableName The table name
- * @throws IOException
- */
- public void enableTable(final byte[] tableName) throws IOException;
-
- /**
- * Disable an existing table
- * @param tableName The table name
- * @throws IOException
- */
- public void disableTable(final byte[] tableName) throws IOException;
-
- /**
- * Add a new column to an existing table
- * @param tableName The table name
- * @param column The column definition
- * @throws IOException
- */
- public void addColumn(final byte[] tableName, final HColumnDescriptor column)
- throws IOException;
-
- /**
- * Modify the column descriptor of an existing column in an existing table
- * @param tableName The table name
- * @param descriptor The updated column definition
- * @throws IOException
- */
- public void modifyColumn(byte[] tableName, HColumnDescriptor descriptor)
- throws IOException;
-
- /**
- * Delete a column from an existing table
- * @param tableName The table name
- * @param columnName The column name
- * @throws IOException
- */
- public void deleteColumn(final byte[] tableName, final byte[] columnName)
- throws IOException;
-
- /**
* @return Return table descriptors implementation.
*/
public TableDescriptors getTableDescriptors();
diff --git a/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java b/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
index af25def..9575060 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
@@ -142,12 +142,16 @@
List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
final int batchSize =
this.conf.getInt("hbase.master.createtable.batchsize", 100);
+ HLog hlog = null;
for (int regionIdx = 0; regionIdx < this.newRegions.length; regionIdx++) {
HRegionInfo newRegion = this.newRegions[regionIdx];
// 1. Create HRegion
HRegion region = HRegion.createHRegion(newRegion,
this.fileSystemManager.getRootDir(), this.conf,
- this.hTableDescriptor, null, false, true);
+ this.hTableDescriptor, hlog);
+ if (hlog == null) {
+ hlog = region.getLog();
+ }
regionInfos.add(region.getRegionInfo());
if (regionIdx % batchSize == 0) {
@@ -159,6 +163,7 @@
// 3. Close the new region to flush to disk. Close log file too.
region.close();
}
+ hlog.closeAndDelete();
if (regionInfos.size() > 0) {
MetaEditor.addRegionsToMeta(this.catalogTracker, regionInfos);
}
diff --git a/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java b/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
index aa147b5..00f8e72 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
@@ -20,7 +20,6 @@
package org.apache.hadoop.hbase.master.handler;
import java.io.IOException;
-import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.ExecutorService;
@@ -28,7 +27,6 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
@@ -36,11 +34,7 @@
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.BulkAssigner;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
import org.apache.zookeeper.KeeperException;
/**
@@ -52,7 +46,6 @@
private final String tableNameStr;
private final AssignmentManager assignmentManager;
private final CatalogTracker ct;
- private boolean retainAssignment = false;
public EnableTableHandler(Server server, byte [] tableName,
CatalogTracker catalogTracker, AssignmentManager assignmentManager,
@@ -63,7 +56,6 @@
this.tableNameStr = Bytes.toString(tableName);
this.ct = catalogTracker;
this.assignmentManager = assignmentManager;
- this.retainAssignment = skipTableStateCheck;
// Check if table exists
if (!MetaReader.tableExists(catalogTracker, this.tableNameStr)) {
throw new TableNotFoundException(Bytes.toString(tableName));
@@ -107,12 +99,10 @@
LOG.error("Error trying to enable the table " + this.tableNameStr, e);
} catch (KeeperException e) {
LOG.error("Error trying to enable the table " + this.tableNameStr, e);
- } catch (InterruptedException e) {
- LOG.error("Error trying to enable the table " + this.tableNameStr, e);
}
}
- private void handleEnableTable() throws IOException, KeeperException, InterruptedException {
+ private void handleEnableTable() throws IOException, KeeperException {
// I could check table is disabling and if so, not enable but require
// that user first finish disabling but that might be obnoxious.
@@ -121,18 +111,18 @@
boolean done = false;
// Get the regions of this table. We're done when all listed
// tables are onlined.
- List<Pair<HRegionInfo, ServerName>> tableRegionsAndLocations = MetaReader
- .getTableRegionsAndLocations(this.ct, tableName, true);
- int countOfRegionsInTable = tableRegionsAndLocations.size();
- List<HRegionInfo> regions = regionsToAssignWithServerName(tableRegionsAndLocations);
+ List<HRegionInfo> regionsInMeta;
+ regionsInMeta = MetaReader.getTableRegions(this.ct, tableName, true);
+ int countOfRegionsInTable = regionsInMeta.size();
+ List<HRegionInfo> regions = regionsToAssign(regionsInMeta);
int regionsCount = regions.size();
if (regionsCount == 0) {
done = true;
}
LOG.info("Table has " + countOfRegionsInTable + " regions of which " +
regionsCount + " are offline.");
- BulkEnabler bd = new BulkEnabler(this.server, regions, countOfRegionsInTable,
- this.retainAssignment);
+ BulkEnabler bd = new BulkEnabler(this.server, regions,
+ countOfRegionsInTable);
try {
if (bd.bulkAssign()) {
done = true;
@@ -150,34 +140,17 @@
/**
* @param regionsInMeta This datastructure is edited by this method.
- * @return List of regions neither in transition nor assigned.
+ * @return The <code>regionsInMeta</code> list minus the regions that have
+ * been onlined; i.e. List of regions that need onlining.
* @throws IOException
*/
- private List<HRegionInfo> regionsToAssignWithServerName(
- final List<Pair<HRegionInfo, ServerName>> regionsInMeta) throws IOException {
- ServerManager serverManager = ((HMaster) this.server).getServerManager();
- List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
- List<HRegionInfo> enablingTableRegions = this.assignmentManager
- .getEnablingTableRegions(this.tableNameStr);
- final List<HRegionInfo> onlineRegions = this.assignmentManager.getRegionsOfTable(tableName);
- for (Pair<HRegionInfo, ServerName> regionLocation : regionsInMeta) {
- HRegionInfo hri = regionLocation.getFirst();
- ServerName sn = regionLocation.getSecond();
- if (this.retainAssignment) {
- // Region may be available in enablingTableRegions during master startup only.
- if (enablingTableRegions != null && enablingTableRegions.contains(hri)) {
- regions.add(hri);
- if (sn != null && serverManager.isServerOnline(sn)) {
- this.assignmentManager.addPlan(hri.getEncodedName(), new RegionPlan(hri, null, sn));
- }
- }
- } else if (onlineRegions.contains(hri)) {
- continue;
- } else {
- regions.add(hri);
- }
- }
- return regions;
+ private List<HRegionInfo> regionsToAssign(
+ final List<HRegionInfo> regionsInMeta)
+ throws IOException {
+ final List<HRegionInfo> onlineRegions =
+ this.assignmentManager.getRegionsOfTable(tableName);
+ regionsInMeta.removeAll(onlineRegions);
+ return regionsInMeta;
}
/**
@@ -187,14 +160,12 @@
private final List<HRegionInfo> regions;
// Count of regions in table at time this assign was launched.
private final int countOfRegionsInTable;
- private final boolean retainAssignment;
BulkEnabler(final Server server, final List<HRegionInfo> regions,
- final int countOfRegionsInTable,final boolean retainAssignment) {
+ final int countOfRegionsInTable) {
super(server);
this.regions = regions;
this.countOfRegionsInTable = countOfRegionsInTable;
- this.retainAssignment = retainAssignment;
}
@Override
@@ -202,7 +173,7 @@
boolean roundRobinAssignment = this.server.getConfiguration().getBoolean(
"hbase.master.enabletable.roundrobin", false);
- if (retainAssignment || !roundRobinAssignment) {
+ if (!roundRobinAssignment) {
for (HRegionInfo region : regions) {
if (assignmentManager.isRegionInTransition(region) != null) {
continue;
@@ -210,11 +181,7 @@
final HRegionInfo hri = region;
pool.execute(new Runnable() {
public void run() {
- if (retainAssignment) {
- assignmentManager.assign(hri, true, false, false);
- } else {
- assignmentManager.assign(hri, true);
- }
+ assignmentManager.assign(hri, true);
}
});
}
diff --git a/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java b/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
index 9322ba6..8988931 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
@@ -26,7 +26,6 @@
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.util.Bytes;
/**
@@ -50,12 +49,6 @@
this.masterServices.getMasterFileSystem().deleteColumn(tableName, familyName);
// Update in-memory descriptor cache
this.masterServices.getTableDescriptors().add(htd);
- // Remove the column family from the file system
- MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
- for (HRegionInfo hri : hris) {
- // Delete the family directory in FS for all the regions one by one
- mfs.deleteFamilyFromFS(hri, familyName);
- }
}
@Override
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
deleted file mode 100644
index e123c22..0000000
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
+++ /dev/null
@@ -1,468 +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.regionserver;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.LineReader;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.apache.hadoop.hbase.util.FSUtils;
-
-/*
- * The CompactionTool allows to execute a compaction specifying a:
- * <ul>
- * <li>table folder (all regions and families will be compacted)
- * <li>region folder (all families in the region will be compacted)
- * <li>family folder (the store files will be compacted)
- * </ul>
- */
-@InterfaceAudience.Public
-public class CompactionTool extends Configured implements Tool {
- private static final Log LOG = LogFactory.getLog(CompactionTool.class);
-
- private final static String CONF_TMP_DIR = "hbase.tmp.dir";
- private final static String CONF_COMPACT_ONCE = "hbase.compactiontool.compact.once";
- private final static String CONF_DELETE_COMPACTED = "hbase.compactiontool.delete";
- private final static String CONF_COMPLETE_COMPACTION = "hbase.hstore.compaction.complete";
-
- /**
- * Class responsible to execute the Compaction on the specified path.
- * The path can be a table, region or family directory.
- */
- private static class CompactionWorker {
- private final boolean keepCompactedFiles;
- private final boolean deleteCompacted;
- private final Configuration conf;
- private final FileSystem fs;
- private final Path tmpDir;
-
- public CompactionWorker(final FileSystem fs, final Configuration conf) {
- this.conf = conf;
- this.keepCompactedFiles = !conf.getBoolean(CONF_COMPLETE_COMPACTION, true);
- this.deleteCompacted = conf.getBoolean(CONF_DELETE_COMPACTED, false);
- this.tmpDir = new Path(conf.get(CONF_TMP_DIR));
- this.fs = fs;
- }
-
- /**
- * Execute the compaction on the specified path.
- *
- * @param path Directory path on which run a
- * @param compactOnce Execute just a single step of compaction.
- */
- public void compact(final Path path, final boolean compactOnce) throws IOException {
- if (isFamilyDir(fs, path)) {
- Path regionDir = path.getParent();
- Path tableDir = regionDir.getParent();
- HTableDescriptor htd = FSTableDescriptors.getTableDescriptor(fs, tableDir);
- HRegion region = loadRegion(fs, conf, htd, regionDir);
- compactStoreFiles(region, path, compactOnce);
- } else if (isRegionDir(fs, path)) {
- Path tableDir = path.getParent();
- HTableDescriptor htd = FSTableDescriptors.getTableDescriptor(fs, tableDir);
- compactRegion(htd, path, compactOnce);
- } else if (isTableDir(fs, path)) {
- compactTable(path, compactOnce);
- } else {
- throw new IOException(
- "Specified path is not a table, region or family directory. path=" + path);
- }
- }
-
- private void compactTable(final Path tableDir, final boolean compactOnce)
- throws IOException {
- HTableDescriptor htd = FSTableDescriptors.getTableDescriptor(fs, tableDir);
- LOG.info("Compact table=" + htd.getNameAsString());
- for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
- compactRegion(htd, regionDir, compactOnce);
- }
- }
-
- private void compactRegion(final HTableDescriptor htd, final Path regionDir,
- final boolean compactOnce) throws IOException {
- HRegion region = loadRegion(fs, conf, htd, regionDir);
- LOG.info("Compact table=" + htd.getNameAsString() +
- " region=" + region.getRegionNameAsString());
- for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
- compactStoreFiles(region, familyDir, compactOnce);
- }
- }
-
- /**
- * Execute the actual compaction job.
- * If the compact once flag is not specified, execute the compaction until
- * no more compactions are needed. Uses the Configuration settings provided.
- */
- private void compactStoreFiles(final HRegion region, final Path familyDir,
- final boolean compactOnce) throws IOException {
- LOG.info("Compact table=" + region.getTableDesc().getNameAsString() +
- " region=" + region.getRegionNameAsString() +
- " family=" + familyDir.getName());
- Store store = getStore(region, familyDir);
- do {
- CompactionRequest cr = store.requestCompaction();
- StoreFile storeFile = store.compact(cr);
- if (storeFile != null) {
- if (keepCompactedFiles && deleteCompacted) {
- fs.delete(storeFile.getPath(), false);
- }
- }
- } while (store.needsCompaction() && !compactOnce);
- }
-
- /**
- * Create a "mock" HStore that uses the tmpDir specified by the user and
- * the store dir to compact as source.
- */
- private Store getStore(final HRegion region, final Path storeDir) throws IOException {
- byte[] familyName = Bytes.toBytes(storeDir.getName());
- HColumnDescriptor hcd = region.getTableDesc().getFamily(familyName);
- // Create a Store w/ check of hbase.rootdir blanked out and return our
- // list of files instead of have Store search its home dir.
- return new Store(tmpDir, region, hcd, fs, conf) {
- @Override
- public FileStatus[] getStoreFiles() throws IOException {
- return this.fs.listStatus(getHomedir());
- }
-
- @Override
- Path createStoreHomeDir(FileSystem fs, Path homedir) throws IOException {
- return storeDir;
- }
- };
- }
-
- private static HRegion loadRegion(final FileSystem fs, final Configuration conf,
- final HTableDescriptor htd, final Path regionDir) throws IOException {
- Path rootDir = regionDir.getParent().getParent();
- HRegionInfo hri = HRegion.loadDotRegionInfoFileContent(fs, regionDir);
- return HRegion.createHRegion(hri, rootDir, conf, htd, null, false, true);
- }
- }
-
- private static boolean isRegionDir(final FileSystem fs, final Path path) throws IOException {
- Path regionInfo = new Path(path, HRegion.REGIONINFO_FILE);
- return fs.exists(regionInfo);
- }
-
- private static boolean isTableDir(final FileSystem fs, final Path path) throws IOException {
- return FSTableDescriptors.getTableInfoPath(fs, path) != null;
- }
-
- private static boolean isFamilyDir(final FileSystem fs, final Path path) throws IOException {
- return isRegionDir(fs, path.getParent());
- }
-
- private static class CompactionMapper
- extends Mapper<LongWritable, Text, NullWritable, NullWritable> {
- private CompactionWorker compactor = null;
- private boolean compactOnce = false;
-
- @Override
- public void setup(Context context) {
- Configuration conf = context.getConfiguration();
- compactOnce = conf.getBoolean(CONF_COMPACT_ONCE, false);
-
- try {
- FileSystem fs = FileSystem.get(conf);
- this.compactor = new CompactionWorker(fs, conf);
- } catch (IOException e) {
- throw new RuntimeException("Could not get the input FileSystem", e);
- }
- }
-
- @Override
- public void map(LongWritable key, Text value, Context context)
- throws InterruptedException, IOException {
- Path path = new Path(value.toString());
- this.compactor.compact(path, compactOnce);
- }
- }
-
- /**
- * Input format that uses store files block location as input split locality.
- */
- private static class CompactionInputFormat extends TextInputFormat {
- @Override
- protected boolean isSplitable(JobContext context, Path file) {
- return true;
- }
-
- /**
- * Returns a split for each store files directory using the block location
- * of each file as locality reference.
- */
- @Override
- public List<InputSplit> getSplits(JobContext job) throws IOException {
- List<InputSplit> splits = new ArrayList<InputSplit>();
- List<FileStatus> files = listStatus(job);
-
- Text key = new Text();
- for (FileStatus file: files) {
- Path path = file.getPath();
- FileSystem fs = path.getFileSystem(job.getConfiguration());
- LineReader reader = new LineReader(fs.open(path));
- long pos = 0;
- int n;
- try {
- while ((n = reader.readLine(key)) > 0) {
- String[] hosts = getStoreDirHosts(fs, path);
- splits.add(new FileSplit(path, pos, n, hosts));
- pos += n;
- }
- } finally {
- reader.close();
- }
- }
-
- return splits;
- }
-
- /**
- * return the top hosts of the store files, used by the Split
- */
- private static String[] getStoreDirHosts(final FileSystem fs, final Path path)
- throws IOException {
- FileStatus[] files = FSUtils.listStatus(fs, path, null);
- if (files == null) {
- return new String[] {};
- }
-
- HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
- for (FileStatus hfileStatus: files) {
- HDFSBlocksDistribution storeFileBlocksDistribution =
- FSUtils.computeHDFSBlocksDistribution(fs, hfileStatus, 0, hfileStatus.getLen());
- hdfsBlocksDistribution.add(storeFileBlocksDistribution);
- }
-
- List<String> hosts = hdfsBlocksDistribution.getTopHosts();
- return hosts.toArray(new String[hosts.size()]);
- }
-
- /**
- * Create the input file for the given directories to compact.
- * The file is a TextFile with each line corrisponding to a
- * store files directory to compact.
- */
- public static void createInputFile(final FileSystem fs, final Path path,
- final Set<Path> toCompactDirs) throws IOException {
- // Extract the list of store dirs
- List<Path> storeDirs = new LinkedList<Path>();
- for (Path compactDir: toCompactDirs) {
- if (isFamilyDir(fs, compactDir)) {
- storeDirs.add(compactDir);
- } else if (isRegionDir(fs, compactDir)) {
- for (Path familyDir: FSUtils.getFamilyDirs(fs, compactDir)) {
- storeDirs.add(familyDir);
- }
- } else if (isTableDir(fs, compactDir)) {
- // Lookup regions
- for (Path regionDir: FSUtils.getRegionDirs(fs, compactDir)) {
- for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
- storeDirs.add(familyDir);
- }
- }
- } else {
- throw new IOException(
- "Specified path is not a table, region or family directory. path=" + compactDir);
- }
- }
-
- // Write Input File
- FSDataOutputStream stream = fs.create(path);
- LOG.info("Create input file=" + path + " with " + storeDirs.size() + " dirs to compact.");
- try {
- final byte[] newLine = Bytes.toBytes("\n");
- for (Path storeDir: storeDirs) {
- stream.write(Bytes.toBytes(storeDir.toString()));
- stream.write(newLine);
- }
- } finally {
- stream.close();
- }
- }
- }
-
- /**
- * Execute compaction, using a Map-Reduce job.
- */
- private int doMapReduce(final FileSystem fs, final Set<Path> toCompactDirs,
- final boolean compactOnce) throws Exception {
- Configuration conf = getConf();
- conf.setBoolean(CONF_COMPACT_ONCE, compactOnce);
-
- Job job = new Job(conf);
- job.setJobName("CompactionTool");
- job.setJarByClass(CompactionTool.class);
- job.setMapperClass(CompactionMapper.class);
- job.setInputFormatClass(CompactionInputFormat.class);
- job.setOutputFormatClass(NullOutputFormat.class);
- job.setMapSpeculativeExecution(false);
- job.setNumReduceTasks(0);
-
- String stagingName = "compact-" + EnvironmentEdgeManager.currentTimeMillis();
- Path stagingDir = new Path(conf.get(CONF_TMP_DIR), stagingName);
- fs.mkdirs(stagingDir);
- try {
- // Create input file with the store dirs
- Path inputPath = new Path(stagingDir, stagingName);
- CompactionInputFormat.createInputFile(fs, inputPath, toCompactDirs);
- CompactionInputFormat.addInputPath(job, inputPath);
-
- // Initialize credential for secure cluster
- TableMapReduceUtil.initCredentials(job);
-
- // Start the MR Job and wait
- return job.waitForCompletion(true) ? 0 : 1;
- } finally {
- fs.delete(stagingDir, true);
- }
- }
-
- /**
- * Execute compaction, from this client, one path at the time.
- */
- private int doClient(final FileSystem fs, final Set<Path> toCompactDirs,
- final boolean compactOnce) throws IOException {
- CompactionWorker worker = new CompactionWorker(fs, getConf());
- for (Path path: toCompactDirs) {
- worker.compact(path, compactOnce);
- }
- return 0;
- }
-
- @Override
- public int run(String[] args) throws Exception {
- Set<Path> toCompactDirs = new HashSet<Path>();
- boolean compactOnce = false;
- boolean mapred = false;
-
- Configuration conf = getConf();
- FileSystem fs = FileSystem.get(conf);
-
- try {
- for (int i = 0; i < args.length; ++i) {
- String opt = args[i];
- if (opt.equals("-compactOnce")) {
- compactOnce = true;
- } else if (opt.equals("-mapred")) {
- mapred = true;
- } else if (!opt.startsWith("-")) {
- Path path = new Path(opt);
- FileStatus status = fs.getFileStatus(path);
- if (!status.isDir()) {
- printUsage("Specified path is not a directory. path=" + path);
- return 1;
- }
- toCompactDirs.add(path);
- } else {
- printUsage();
- }
- }
- } catch (Exception e) {
- printUsage(e.getMessage());
- return 1;
- }
-
- if (toCompactDirs.size() == 0) {
- printUsage("No directories to compact specified.");
- return 1;
- }
-
- // Execute compaction!
- if (mapred) {
- return doMapReduce(fs, toCompactDirs, compactOnce);
- } else {
- return doClient(fs, toCompactDirs, compactOnce);
- }
- }
-
- private void printUsage() {
- printUsage(null);
- }
-
- private void printUsage(final String message) {
- if (message != null && message.length() > 0) {
- System.err.println(message);
- }
- System.err.println("Usage: java " + this.getClass().getName() + " \\");
- System.err.println(" [-compactOnce] [-mapred] [-D<property=value>]* files...");
- System.err.println();
- System.err.println("Options:");
- System.err.println(" mapred Use MapReduce to run compaction.");
- System.err.println(" compactOnce Execute just one compaction step. (default: while needed)");
- System.err.println();
- System.err.println("Note: -D properties will be applied to the conf used. ");
- System.err.println("For example: ");
- System.err.println(" To preserve input files, pass -D"+CONF_COMPLETE_COMPACTION+"=false");
- System.err.println(" To stop delete of compacted file, pass -D"+CONF_DELETE_COMPACTED+"=false");
- System.err.println(" To set tmp dir, pass -D"+CONF_TMP_DIR+"=ALTERNATE_DIR");
- System.err.println();
- System.err.println("Examples:");
- System.err.println(" To compact the full 'TestTable' using MapReduce:");
- System.err.println(" $ bin/hbase " + this.getClass().getName() + " -mapred hdfs:///hbase/TestTable");
- System.err.println();
- System.err.println(" To compact column family 'x' of the table 'TestTable' region 'abc':");
- System.err.println(" $ bin/hbase " + this.getClass().getName() + " hdfs:///hbase/TestTable/abc/x");
- }
-
- public static void main(String[] args) throws Exception {
- System.exit(ToolRunner.run(HBaseConfiguration.create(), new CompactionTool(), args));
- }
-}
\ No newline at end of file
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/Compactor.java b/src/main/java/org/apache/hadoop/hbase/regionserver/Compactor.java
deleted file mode 100644
index ad13248..0000000
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/Compactor.java
+++ /dev/null
@@ -1,213 +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.regionserver;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.io.hfile.Compression;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.util.StringUtils;
-
-/**
- * Compact passed set of files.
- * Create an instance and then call {@ink #compact(Store, Collection, boolean, long)}.
- */
-@InterfaceAudience.Private
-class Compactor extends Configured {
- private static final Log LOG = LogFactory.getLog(Compactor.class);
- private CompactionProgress progress;
-
- Compactor(final Configuration c) {
- super(c);
- }
-
- /**
- * Do a minor/major compaction on an explicit set of storefiles from a Store.
- *
- * @param store Store the files belong to
- * @param filesToCompact which files to compact
- * @param majorCompaction true to major compact (prune all deletes, max versions, etc)
- * @param maxId Readers maximum sequence id.
- * @return Product of compaction or null if all cells expired or deleted and
- * nothing made it through the compaction.
- * @throws IOException
- */
- StoreFile.Writer compact(final Store store,
- final Collection<StoreFile> filesToCompact,
- final boolean majorCompaction, final long maxId)
- throws IOException {
- // Calculate maximum key count after compaction (for blooms)
- // Also calculate earliest put timestamp if major compaction
- int maxKeyCount = 0;
- long earliestPutTs = HConstants.LATEST_TIMESTAMP;
- for (StoreFile file : filesToCompact) {
- StoreFile.Reader r = file.getReader();
- if (r == null) {
- LOG.warn("Null reader for " + file.getPath());
- continue;
- }
- // NOTE: getFilterEntries could cause under-sized blooms if the user
- // switches bloom type (e.g. from ROW to ROWCOL)
- long keyCount = (r.getBloomFilterType() == store.getFamily().getBloomFilterType()) ?
- r.getFilterEntries() : r.getEntries();
- maxKeyCount += keyCount;
- // For major compactions calculate the earliest put timestamp
- // of all involved storefiles. This is used to remove
- // family delete marker during the compaction.
- if (majorCompaction) {
- byte[] tmp = r.loadFileInfo().get(StoreFile.EARLIEST_PUT_TS);
- if (tmp == null) {
- // There's a file with no information, must be an old one
- // assume we have very old puts
- earliestPutTs = HConstants.OLDEST_TIMESTAMP;
- } else {
- earliestPutTs = Math.min(earliestPutTs, Bytes.toLong(tmp));
- }
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("Compacting " + file +
- ", keycount=" + keyCount +
- ", bloomtype=" + r.getBloomFilterType().toString() +
- ", size=" + StringUtils.humanReadableInt(r.length()) +
- ", encoding=" + r.getHFileReader().getEncodingOnDisk() +
- (majorCompaction? ", earliestPutTs=" + earliestPutTs: ""));
- }
- }
-
- // keep track of compaction progress
- this.progress = new CompactionProgress(maxKeyCount);
- // Get some configs
- int compactionKVMax = getConf().getInt("hbase.hstore.compaction.kv.max", 10);
- Compression.Algorithm compression = store.getFamily().getCompression();
- // Avoid overriding compression setting for major compactions if the user
- // has not specified it separately
- Compression.Algorithm compactionCompression =
- (store.getFamily().getCompactionCompression() != Compression.Algorithm.NONE) ?
- store.getFamily().getCompactionCompression(): compression;
-
- // For each file, obtain a scanner:
- List<StoreFileScanner> scanners = StoreFileScanner
- .getScannersForStoreFiles(filesToCompact, false, false, true);
-
- // Make the instantiation lazy in case compaction produces no product; i.e.
- // where all source cells are expired or deleted.
- StoreFile.Writer writer = null;
- // Find the smallest read point across all the Scanners.
- long smallestReadPoint = store.getHRegion().getSmallestReadPoint();
- MultiVersionConsistencyControl.setThreadReadPoint(smallestReadPoint);
- try {
- InternalScanner scanner = null;
- try {
- if (store.getHRegion().getCoprocessorHost() != null) {
- scanner = store.getHRegion()
- .getCoprocessorHost()
- .preCompactScannerOpen(store, scanners,
- majorCompaction ? ScanType.MAJOR_COMPACT : ScanType.MINOR_COMPACT, earliestPutTs);
- }
- if (scanner == null) {
- Scan scan = new Scan();
- scan.setMaxVersions(store.getFamily().getMaxVersions());
- /* Include deletes, unless we are doing a major compaction */
- scanner = new StoreScanner(store, store.getScanInfo(), scan, scanners,
- majorCompaction? ScanType.MAJOR_COMPACT : ScanType.MINOR_COMPACT,
- smallestReadPoint, earliestPutTs);
- }
- if (store.getHRegion().getCoprocessorHost() != null) {
- InternalScanner cpScanner =
- store.getHRegion().getCoprocessorHost().preCompact(store, scanner);
- // NULL scanner returned from coprocessor hooks means skip normal processing
- if (cpScanner == null) {
- return null;
- }
- scanner = cpScanner;
- }
-
- int bytesWritten = 0;
- // since scanner.next() can return 'false' but still be delivering data,
- // we have to use a do/while loop.
- List<KeyValue> kvs = new ArrayList<KeyValue>();
- // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
- boolean hasMore;
- do {
- hasMore = scanner.next(kvs, compactionKVMax);
- if (writer == null && !kvs.isEmpty()) {
- writer = store.createWriterInTmp(maxKeyCount, compactionCompression, true);
- }
- if (writer != null) {
- // output to writer:
- for (KeyValue kv : kvs) {
- if (kv.getMemstoreTS() <= smallestReadPoint) {
- kv.setMemstoreTS(0);
- }
- writer.append(kv);
- // update progress per key
- ++progress.currentCompactedKVs;
-
- // check periodically to see if a system stop is requested
- if (Store.closeCheckInterval > 0) {
- bytesWritten += kv.getLength();
- if (bytesWritten > Store.closeCheckInterval) {
- bytesWritten = 0;
- isInterrupted(store, writer);
- }
- }
- }
- }
- kvs.clear();
- } while (hasMore);
- } finally {
- if (scanner != null) {
- scanner.close();
- }
- }
- } finally {
- if (writer != null) {
- writer.appendMetadata(maxId, majorCompaction);
- writer.close();
- }
- }
- return writer;
- }
-
- void isInterrupted(final Store store, final StoreFile.Writer writer)
- throws IOException {
- if (store.getHRegion().areWritesEnabled()) return;
- // Else cleanup.
- writer.close();
- store.getFileSystem().delete(writer.getPath(), false);
- throw new InterruptedIOException( "Aborting compaction of store " + store +
- " in region " + store.getHRegion() + " because user requested stop.");
- }
-
- CompactionProgress getProgress() {
- return this.progress;
- }
-}
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 8aed783..06f93e1 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -20,7 +20,6 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.EOFException;
-import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.UnsupportedEncodingException;
@@ -56,14 +55,12 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -79,7 +76,6 @@
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.client.Append;
@@ -232,34 +228,15 @@
* The directory for the table this region is part of.
* This directory contains the directory for this region.
*/
- private final Path tableDir;
+ final Path tableDir;
- private final HLog log;
- private final FileSystem fs;
- private final Configuration conf;
- private final int rowLockWaitDuration;
+ final HLog log;
+ final FileSystem fs;
+ final Configuration conf;
+ final int rowLockWaitDuration;
static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
-
- // The internal wait duration to acquire a lock before read/update
- // from the region. It is not per row. The purpose of this wait time
- // is to avoid waiting a long time while the region is busy, so that
- // we can release the IPC handler soon enough to improve the
- // availability of the region server. It can be adjusted by
- // tuning configuration "hbase.busy.wait.duration".
- final long busyWaitDuration;
- static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
-
- // If updating multiple rows in one call, wait longer,
- // i.e. waiting for busyWaitDuration * # of rows. However,
- // we can limit the max multiplier.
- final int maxBusyWaitMultiplier;
-
- // Max busy wait duration. There is no point to wait longer than the RPC
- // purge timeout, when a RPC call will be terminated by the RPC engine.
- final long maxBusyWaitDuration;
-
- private final HRegionInfo regionInfo;
- private final Path regiondir;
+ final HRegionInfo regionInfo;
+ final Path regiondir;
KeyValue.KVComparator comparator;
private ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
@@ -377,10 +354,6 @@
this.coprocessorHost = null;
this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
this.opMetrics = new OperationMetrics();
-
- this.maxBusyWaitDuration = 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
- this.busyWaitDuration = DEFAULT_BUSY_WAIT_DURATION;
- this.maxBusyWaitMultiplier = 2;
}
/**
@@ -427,17 +400,6 @@
this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
this.opMetrics = new OperationMetrics(conf, this.regionInfo);
- this.busyWaitDuration = conf.getLong(
- "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION);
- this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2);
- if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) {
- throw new IllegalArgumentException("Invalid hbase.busy.wait.duration ("
- + busyWaitDuration + ") or hbase.busy.wait.multiplier.max ("
- + maxBusyWaitMultiplier + "). Their product should be positive");
- }
- this.maxBusyWaitDuration = conf.getLong("ipc.client.call.purge.timeout",
- 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
-
/*
* timestamp.slop provides a server-side constraint on the timestamp. This
* assumes that you base your TS around currentTimeMillis(). In this case,
@@ -726,7 +688,7 @@
public long addAndGetGlobalMemstoreSize(long memStoreSize) {
if (this.rsAccounting != null) {
rsAccounting.addAndGetGlobalMemstoreSize(memStoreSize);
- }
+ }
return this.memstoreSize.getAndAdd(memStoreSize);
}
@@ -752,7 +714,7 @@
// and then create the file
Path tmpPath = new Path(getTmpDir(), REGIONINFO_FILE);
-
+
// if datanode crashes or if the RS goes down just before the close is called while trying to
// close the created regioninfo file in the .tmp directory then on next
// creation we will be getting AlreadyCreatedException.
@@ -760,7 +722,7 @@
if (FSUtils.isExists(fs, tmpPath)) {
FSUtils.delete(fs, tmpPath, true);
}
-
+
FSDataOutputStream out = FSUtils.create(fs, tmpPath, perms);
try {
@@ -777,26 +739,6 @@
}
}
- /**
- * @param fs
- * @param dir
- * @return An HRegionInfo instance gotten from the <code>.regioninfo</code> file under region dir
- * @throws IOException
- */
- public static HRegionInfo loadDotRegionInfoFileContent(final FileSystem fs, final Path dir)
- throws IOException {
- Path regioninfo = new Path(dir, HRegion.REGIONINFO_FILE);
- if (!fs.exists(regioninfo)) throw new FileNotFoundException(regioninfo.toString());
- FSDataInputStream in = fs.open(regioninfo);
- try {
- HRegionInfo hri = new HRegionInfo();
- hri.readFields(in);
- return hri;
- } finally {
- in.close();
- }
- }
-
/** @return a HRegionInfo object for this region */
public HRegionInfo getRegionInfo() {
return this.regionInfo;
@@ -941,7 +883,6 @@
this.closing.set(true);
status.setStatus("Disabling writes for close");
- // block waiting for the lock for closing
lock.writeLock().lock();
try {
if (this.isClosed()) {
@@ -1043,16 +984,19 @@
return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
}
- static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads,
+ private ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads,
final String threadNamePrefix) {
- return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
- new ThreadFactory() {
- private int count = 1;
+ ThreadPoolExecutor openAndCloseThreadPool = Threads
+ .getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
+ new ThreadFactory() {
+ private int count = 1;
- public Thread newThread(Runnable r) {
- return new Thread(r, threadNamePrefix + "-" + count++);
- }
- });
+ public Thread newThread(Runnable r) {
+ Thread t = new Thread(r, threadNamePrefix + "-" + count++);
+ return t;
+ }
+ });
+ return openAndCloseThreadPool;
}
/**
@@ -1248,7 +1192,6 @@
return false;
}
Preconditions.checkArgument(cr.getHRegion().equals(this));
- // block waiting for the lock for compaction
lock.readLock().lock();
MonitoredTask status = TaskMonitor.get().createStatus(
"Compacting " + cr.getStore() + " in " + this);
@@ -1328,7 +1271,6 @@
}
MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
status.setStatus("Acquiring readlock on region");
- // block waiting for the lock for flushing cache
lock.readLock().lock();
try {
if (this.closed.get()) {
@@ -1464,7 +1406,6 @@
// end up in both snapshot and memstore (makes it difficult to do atomic
// rows then)
status.setStatus("Obtaining lock to block concurrent updates");
- // block waiting for the lock for internal flush
this.updatesLock.writeLock().lock();
long flushsize = this.memstoreSize.get();
status.setStatus("Preparing to flush by snapshotting stores");
@@ -1723,23 +1664,11 @@
//////////////////////////////////////////////////////////////////////////////
// set() methods for client use.
//////////////////////////////////////////////////////////////////////////////
-
- /**
- * @param delete delete object
- * @param writeToWAL append to the write ahead lock or not
- * @throws IOException read exceptions
- */
- public void delete(Delete delete, boolean writeToWAL)
- throws IOException {
- delete(delete, null, writeToWAL);
- }
-
/**
* @param delete delete object
* @param lockid existing lock id, or null for grab a lock
* @param writeToWAL append to the write ahead lock or not
* @throws IOException read exceptions
- * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
*/
public void delete(Delete delete, Integer lockid, boolean writeToWAL)
throws IOException {
@@ -1855,7 +1784,7 @@
byte [] byteNow = Bytes.toBytes(now);
boolean flush = false;
- lock(updatesLock.readLock());
+ updatesLock.readLock().lock();
try {
prepareDeleteTimestamps(delete.getFamilyMap(), byteNow);
@@ -1914,7 +1843,6 @@
* @param put
* @param lockid
* @throws IOException
- * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
*/
public void put(Put put, Integer lockid) throws IOException {
this.put(put, lockid, put.getWriteToWAL());
@@ -1927,7 +1855,6 @@
* @param lockid
* @param writeToWAL
* @throws IOException
- * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
*/
public void put(Put put, Integer lockid, boolean writeToWAL)
throws IOException {
@@ -2013,7 +1940,7 @@
System.arraycopy(putsAndLocks, 0, mutationsAndLocks, 0, putsAndLocks.length);
return batchMutate(mutationsAndLocks);
}
-
+
/**
* Perform a batch of mutations.
* It supports only Put and Delete mutations and will ignore other types passed.
@@ -2236,7 +2163,7 @@
}
}
- lock(this.updatesLock.readLock(), numReadyToWrite);
+ this.updatesLock.readLock().lock();
locked = true;
//
@@ -2367,7 +2294,7 @@
// do after lock
final long netTimeMs = EnvironmentEdgeManager.currentTimeMillis()- startTimeMs;
-
+
// See if the column families were consistent through the whole thing.
// if they were then keep them. If they were not then pass a null.
// null will be treated as unknown.
@@ -2401,24 +2328,6 @@
//the getting of the lock happens before, so that you would just pass it into
//the methods. So in the case of checkAndMutate you could just do lockRow,
//get, put, unlockRow or something
- /**
- *
- * @param row
- * @param family
- * @param qualifier
- * @param compareOp
- * @param comparator
- * @param writeToWAL
- * @throws IOException
- * @return true if the new put was execute, false otherwise
- */
- public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
- CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
- boolean writeToWAL)
- throws IOException {
- return checkAndMutate(row, family, qualifier, compareOp, comparator, w, null, writeToWAL);
- }
-
/**
*
* @param row
@@ -2430,7 +2339,6 @@
* @param writeToWAL
* @throws IOException
* @return true if the new put was execute, false otherwise
- * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
*/
public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
@@ -2549,8 +2457,7 @@
* this and the synchronize on 'this' inside in internalFlushCache to send
* the notify.
*/
- private void checkResources()
- throws RegionTooBusyException, InterruptedIOException {
+ private void checkResources() {
// If catalog region, do not impose resource constraints or block updates.
if (this.getRegionInfo().isMetaRegion()) return;
@@ -2568,30 +2475,12 @@
" is >= than blocking " +
StringUtils.humanReadableInt(this.blockingMemStoreSize) + " size");
}
- long now = EnvironmentEdgeManager.currentTimeMillis();
- long timeToWait = startTime + busyWaitDuration - now;
- if (timeToWait <= 0L) {
- final long totalTime = now - startTime;
- this.updatesBlockedMs.add(totalTime);
- LOG.info("Failed to unblock updates for region " + this + " '"
- + Thread.currentThread().getName() + "' in " + totalTime
- + "ms. The region is still busy.");
- throw new RegionTooBusyException("region is flushing");
- }
blocked = true;
synchronized(this) {
try {
- wait(Math.min(timeToWait, threadWakeFrequency));
- } catch (InterruptedException ie) {
- final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
- if (totalTime > 0) {
- this.updatesBlockedMs.add(totalTime);
- }
- LOG.info("Interrupted while waiting to unblock updates for region "
- + this + " '" + Thread.currentThread().getName() + "'");
- InterruptedIOException iie = new InterruptedIOException();
- iie.initCause(ie);
- throw iie;
+ wait(threadWakeFrequency);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
}
}
}
@@ -2658,7 +2547,7 @@
byte[] byteNow = Bytes.toBytes(now);
boolean flush = false;
- lock(this.updatesLock.readLock());
+ this.updatesLock.readLock().lock();
try {
checkFamilies(familyMap.keySet());
checkTimestamps(familyMap, now);
@@ -2689,7 +2578,7 @@
// do after lock
final long after = EnvironmentEdgeManager.currentTimeMillis();
this.opMetrics.updatePutMetrics(familyMap.keySet(), after - now);
-
+
if (flush) {
// Request a cache flush. Do it outside update lock.
requestFlush();
@@ -3283,7 +3172,6 @@
* @param lockId The lock ID to release.
*/
public void releaseRowLock(final Integer lockId) {
- if (lockId == null) return; // null lock id, do nothing
HashedBytes rowKey = lockIds.remove(lockId);
if (rowKey == null) {
LOG.warn("Release unknown lockId: " + lockId);
@@ -3524,10 +3412,6 @@
this(scan, null);
}
- @Override
- public long getMvccReadPoint() {
- return this.readPt;
- }
/**
* Reset both the filter and the old filter.
*/
@@ -3538,7 +3422,7 @@
}
@Override
- public boolean next(List<KeyValue> outResults, int limit)
+ public synchronized boolean next(List<KeyValue> outResults, int limit)
throws IOException {
return next(outResults, limit, null);
}
@@ -3558,42 +3442,30 @@
// This could be a new thread from the last time we called next().
MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
- return nextRaw(outResults, limit, metric);
+ results.clear();
+
+ boolean returnResult = nextInternal(limit, metric);
+
+ outResults.addAll(results);
+ resetFilters();
+ if (isFilterDone()) {
+ return false;
+ }
+ return returnResult;
} finally {
closeRegionOperation();
}
}
@Override
- public boolean nextRaw(List<KeyValue> outResults, String metric)
- throws IOException {
- return nextRaw(outResults, batch, metric);
- }
-
- @Override
- public boolean nextRaw(List<KeyValue> outResults, int limit,
- String metric) throws IOException {
- results.clear();
-
- boolean returnResult = nextInternal(limit, metric);
-
- outResults.addAll(results);
- resetFilters();
- if (isFilterDone()) {
- return false;
- }
- return returnResult;
- }
-
- @Override
- public boolean next(List<KeyValue> outResults)
+ public synchronized boolean next(List<KeyValue> outResults)
throws IOException {
// apply the batching limit by default
return next(outResults, batch, null);
}
@Override
- public boolean next(List<KeyValue> outResults, String metric)
+ public synchronized boolean next(List<KeyValue> outResults, String metric)
throws IOException {
// apply the batching limit by default
return next(outResults, batch, metric);
@@ -3617,16 +3489,8 @@
rpcCall.throwExceptionIfCallerDisconnected();
}
- KeyValue current = this.storeHeap.peek();
- byte[] currentRow = null;
- int offset = 0;
- short length = 0;
- if (current != null) {
- currentRow = current.getBuffer();
- offset = current.getRowOffset();
- length = current.getRowLength();
- }
- if (isStopRow(currentRow, offset, length)) {
+ byte [] currentRow = peekRow();
+ if (isStopRow(currentRow)) {
if (filter != null && filter.hasFilterRow()) {
filter.filterRow(results);
}
@@ -3635,10 +3499,10 @@
}
return false;
- } else if (filterRowKey(currentRow, offset, length)) {
- nextRow(currentRow, offset, length);
+ } else if (filterRowKey(currentRow)) {
+ nextRow(currentRow);
} else {
- KeyValue nextKv;
+ byte [] nextRow;
do {
this.storeHeap.next(results, limit - results.size(), metric);
if (limit > 0 && results.size() == limit) {
@@ -3648,10 +3512,9 @@
}
return true; // we are expecting more yes, but also limited to how many we can return.
}
- nextKv = this.storeHeap.peek();
- } while (nextKv != null && nextKv.matchingRow(currentRow, offset, length));
+ } while (Bytes.equals(currentRow, nextRow = peekRow()));
- final boolean stopRow = nextKv == null || isStopRow(nextKv.getBuffer(), nextKv.getRowOffset(), nextKv.getRowLength());
+ final boolean stopRow = isStopRow(nextRow);
// now that we have an entire row, lets process with a filters:
@@ -3666,7 +3529,7 @@
// the reasons for calling this method are:
// 1. reset the filters.
// 2. provide a hook to fast forward the row (used by subclasses)
- nextRow(currentRow, offset, length);
+ nextRow(currentRow);
// This row was totally filtered out, if this is NOT the last row,
// we should continue on.
@@ -3682,25 +3545,29 @@
return filter != null
&& filter.filterRow();
}
- private boolean filterRowKey(byte[] row, int offset, short length) {
+ private boolean filterRowKey(byte[] row) {
return filter != null
- && filter.filterRowKey(row, offset, length);
+ && filter.filterRowKey(row, 0, row.length);
}
- protected void nextRow(byte [] currentRow, int offset, short length) throws IOException {
- KeyValue next;
- while((next = this.storeHeap.peek()) != null && next.matchingRow(currentRow, offset, length)) {
- this.storeHeap.next(MOCKED_LIST);
+ protected void nextRow(byte [] currentRow) throws IOException {
+ while (Bytes.equals(currentRow, peekRow())) {
+ this.storeHeap.next(MOCKED_LIST);
}
results.clear();
resetFilters();
}
- private boolean isStopRow(byte [] currentRow, int offset, short length) {
+ private byte[] peekRow() {
+ KeyValue kv = this.storeHeap.peek();
+ return kv == null ? null : kv.getRow();
+ }
+
+ private boolean isStopRow(byte [] currentRow) {
return currentRow == null ||
(stopRow != null &&
comparator.compareRows(stopRow, 0, stopRow.length,
- currentRow, offset, length) <= isScan);
+ currentRow, 0, currentRow.length) <= isScan);
}
@Override
@@ -3828,7 +3695,6 @@
* @param conf
* @param hTableDescriptor
* @param hlog shared HLog
- * @param boolean initialize - true to initialize the region
* @return new HRegion
*
* @throws IOException
@@ -3836,36 +3702,7 @@
public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
final Configuration conf,
final HTableDescriptor hTableDescriptor,
- final HLog hlog,
- final boolean initialize)
- throws IOException {
- return createHRegion(info, rootDir, conf, hTableDescriptor,
- hlog, initialize, false);
- }
-
- /**
- * Convenience method creating new HRegions. Used by createTable.
- * The {@link HLog} for the created region needs to be closed
- * explicitly, if it is not null.
- * Use {@link HRegion#getLog()} to get access.
- *
- * @param info Info for region to create.
- * @param rootDir Root directory for HBase instance
- * @param conf
- * @param hTableDescriptor
- * @param hlog shared HLog
- * @param boolean initialize - true to initialize the region
- * @param boolean ignoreHLog
- - true to skip generate new hlog if it is null, mostly for createTable
- * @return new HRegion
- *
- * @throws IOException
- */
- public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
- final Configuration conf,
- final HTableDescriptor hTableDescriptor,
- final HLog hlog,
- final boolean initialize, final boolean ignoreHLog)
+ final HLog hlog)
throws IOException {
LOG.info("creating HRegion " + info.getTableNameAsString()
+ " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
@@ -3877,26 +3714,16 @@
FileSystem fs = FileSystem.get(conf);
fs.mkdirs(regionDir);
HLog effectiveHLog = hlog;
- if (hlog == null && !ignoreHLog) {
+ if (hlog == null) {
effectiveHLog = new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf);
}
HRegion region = HRegion.newHRegion(tableDir,
effectiveHLog, fs, conf, info, hTableDescriptor, null);
- if (initialize) {
- region.initialize();
- }
+ region.initialize();
return region;
}
- public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
- final Configuration conf,
- final HTableDescriptor hTableDescriptor,
- final HLog hlog)
- throws IOException {
- return createHRegion(info, rootDir, conf, hTableDescriptor, hlog, true);
- }
-
/**
* Open a Region.
* @param info Info for region to be opened.
@@ -4351,19 +4178,9 @@
//
/**
* @param get get object
- * @return result
- * @throws IOException read exceptions
- */
- public Result get(final Get get) throws IOException {
- return get(get, null);
- }
-
- /**
- * @param get get object
* @param lockid existing lock id, or null for no previous lock
* @return result
* @throws IOException read exceptions
- * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
*/
public Result get(final Get get, final Integer lockid) throws IOException {
checkRow(get.getRow(), "Get");
@@ -4418,7 +4235,7 @@
// do after lock
final long after = EnvironmentEdgeManager.currentTimeMillis();
this.opMetrics.updateGetMetrics(get.familySet(), after - now);
-
+
return results;
}
@@ -4486,7 +4303,7 @@
}
// 3. acquire the region lock
- lock(this.updatesLock.readLock(), acquiredLocks.size());
+ this.updatesLock.readLock().lock();
locked = true;
// 4. Get a mvcc write number
@@ -4608,23 +4425,6 @@
// TODO: There's a lot of boiler plate code identical
// to increment... See how to better unify that.
-
- /**
- *
- * Perform one or more append operations on a row.
- * <p>
- * Appends performed are done under row lock but reads do not take locks out
- * so this can be seen partially complete by gets and scans.
- *
- * @param append
- * @param writeToWAL
- * @return new keyvalues after increment
- * @throws IOException
- */
- public Result append(Append append, boolean writeToWAL)
- throws IOException {
- return append(append, null, writeToWAL);
- }
/**
*
* Perform one or more append operations on a row.
@@ -4637,7 +4437,6 @@
* @param writeToWAL
* @return new keyvalues after increment
* @throws IOException
- * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
*/
public Result append(Append append, Integer lockid, boolean writeToWAL)
throws IOException {
@@ -4657,7 +4456,7 @@
this.writeRequestsCount.increment();
try {
Integer lid = getLock(lockid, row, true);
- lock(this.updatesLock.readLock());
+ this.updatesLock.readLock().lock();
try {
long now = EnvironmentEdgeManager.currentTimeMillis();
// Process each family
@@ -4764,10 +4563,10 @@
closeRegionOperation();
}
-
+
long after = EnvironmentEdgeManager.currentTimeMillis();
this.opMetrics.updateAppendMetrics(append.getFamilyMap().keySet(), after - before);
-
+
if (flush) {
// Request a cache flush. Do it outside update lock.
requestFlush();
@@ -4777,22 +4576,6 @@
}
/**
- *
- * Perform one or more increment operations on a row.
- * <p>
- * Increments performed are done under row lock but reads do not take locks
- * out so this can be seen partially complete by gets and scans.
- * @param increment
- * @param writeToWAL
- * @return new keyvalues after increment
- * @throws IOException
- */
- public Result increment(Increment increment, boolean writeToWAL)
- throws IOException {
- return increment(increment, null, writeToWAL);
- }
-
- /**
*
* Perform one or more increment operations on a row.
* <p>
@@ -4803,8 +4586,6 @@
* @param writeToWAL
* @return new keyvalues after increment
* @throws IOException
- * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
-
*/
public Result increment(Increment increment, Integer lockid,
boolean writeToWAL)
@@ -4826,7 +4607,7 @@
this.writeRequestsCount.increment();
try {
Integer lid = getLock(lockid, row, true);
- lock(this.updatesLock.readLock());
+ this.updatesLock.readLock().lock();
try {
long now = EnvironmentEdgeManager.currentTimeMillis();
// Process each family
@@ -4910,7 +4691,7 @@
long after = EnvironmentEdgeManager.currentTimeMillis();
this.opMetrics.updateIncrementMetrics(increment.getFamilyMap().keySet(), after - before);
}
-
+
if (flush) {
// Request a cache flush. Do it outside update lock.
requestFlush();
@@ -4944,7 +4725,7 @@
this.writeRequestsCount.increment();
try {
Integer lid = obtainRowLock(row);
- lock(this.updatesLock.readLock());
+ this.updatesLock.readLock().lock();
try {
Store store = stores.get(family);
@@ -5037,8 +4818,8 @@
public static final long FIXED_OVERHEAD = ClassSize.align(
ClassSize.OBJECT +
ClassSize.ARRAY +
- 35 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
- (7 * Bytes.SIZEOF_LONG) +
+ 35 * ClassSize.REFERENCE + Bytes.SIZEOF_INT +
+ (5 * Bytes.SIZEOF_LONG) +
Bytes.SIZEOF_BOOLEAN);
public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
@@ -5338,16 +5119,13 @@
* #closeRegionOperation needs to be called in the try's finally block
* Acquires a read lock and checks if the region is closing or closed.
* @throws NotServingRegionException when the region is closing or closed
- * @throws RegionTooBusyException if failed to get the lock in time
- * @throws InterruptedIOException if interrupted while waiting for a lock
*/
- public void startRegionOperation()
- throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
+ private void startRegionOperation() throws NotServingRegionException {
if (this.closing.get()) {
throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
" is closing");
}
- lock(lock.readLock());
+ lock.readLock().lock();
if (this.closed.get()) {
lock.readLock().unlock();
throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
@@ -5359,7 +5137,7 @@
* Closes the lock. This needs to be called in the finally block corresponding
* to the try block of #startRegionOperation
*/
- public void closeRegionOperation(){
+ private void closeRegionOperation(){
lock.readLock().unlock();
}
@@ -5369,17 +5147,15 @@
* #closeBulkRegionOperation needs to be called in the try's finally block
* Acquires a writelock and checks if the region is closing or closed.
* @throws NotServingRegionException when the region is closing or closed
- * @throws RegionTooBusyException if failed to get the lock in time
- * @throws InterruptedIOException if interrupted while waiting for a lock
*/
private void startBulkRegionOperation(boolean writeLockNeeded)
- throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
+ throws NotServingRegionException {
if (this.closing.get()) {
throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
" is closing");
}
- if (writeLockNeeded) lock(lock.writeLock());
- else lock(lock.readLock());
+ if (writeLockNeeded) lock.writeLock().lock();
+ else lock.readLock().lock();
if (this.closed.get()) {
if (writeLockNeeded) lock.writeLock().unlock();
else lock.readLock().unlock();
@@ -5392,7 +5168,7 @@
* Closes the lock. This needs to be called in the finally block corresponding
* to the try block of #startRegionOperation
*/
- private void closeBulkRegionOperation() {
+ private void closeBulkRegionOperation(){
if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
else lock.readLock().unlock();
}
@@ -5403,7 +5179,7 @@
*/
private void recordPutWithoutWal(final Map<byte [], List<KeyValue>> familyMap) {
if (numPutsWithoutWAL.getAndIncrement() == 0) {
- LOG.info("writing data to region " + this +
+ LOG.info("writing data to region " + this +
" with WAL disabled. Data may be lost in the event of a crash.");
}
@@ -5417,33 +5193,6 @@
dataInMemoryWithoutWAL.addAndGet(putSize);
}
- private void lock(final Lock lock)
- throws RegionTooBusyException, InterruptedIOException {
- lock(lock, 1);
- }
-
- /**
- * Try to acquire a lock. Throw RegionTooBusyException
- * if failed to get the lock in time. Throw InterruptedIOException
- * if interrupted while waiting for the lock.
- */
- private void lock(final Lock lock, final int multiplier)
- throws RegionTooBusyException, InterruptedIOException {
- try {
- final long waitTime = Math.min(maxBusyWaitDuration,
- busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
- if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
- throw new RegionTooBusyException(
- "failed to get a lock in " + waitTime + "ms");
- }
- } catch (InterruptedException ie) {
- LOG.info("Interrupted while waiting for a lock");
- InterruptedIOException iie = new InterruptedIOException();
- iie.initCause(ie);
- throw iie;
- }
- }
-
/**
* Calls sync with the given transaction ID if the region's table is not
* deferring it.
@@ -5483,6 +5232,7 @@
}
};
+
/**
* Facility for dumping and compacting catalog tables.
* Only does catalog tables since these are only tables we for sure know
@@ -5515,11 +5265,11 @@
final HLog log = new HLog(fs, logdir, oldLogDir, c);
try {
processTable(fs, tableDir, log, c, majorCompact);
- } finally {
+ } finally {
log.close();
// TODO: is this still right?
BlockCache bc = new CacheConfig(c).getBlockCache();
if (bc != null) bc.shutdown();
- }
+ }
}
}
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index b86d058..a4b4b81 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -44,7 +44,6 @@
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
-import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -97,7 +96,6 @@
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowLock;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.coprocessor.Exec;
@@ -167,7 +165,6 @@
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
import org.codehaus.jackson.map.ObjectMapper;
-import org.joda.time.field.MillisDurationField;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
@@ -236,7 +233,7 @@
// Server to handle client requests. Default access so can be accessed by
// unit tests.
RpcServer rpcServer;
-
+
// Server to handle client requests.
private HBaseServer server;
@@ -366,8 +363,6 @@
*/
private ClusterId clusterId = null;
- private RegionServerCoprocessorHost rsHost;
-
/**
* Starts a HRegionServer at the default location
*
@@ -439,10 +434,6 @@
this.rpcServer.setQosFunction(new QosFunction());
this.startcode = System.currentTimeMillis();
- // login the zookeeper client principal (if using security)
- ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
- "hbase.zookeeper.client.kerberos.principal", this.isa.getHostName());
-
// login the server principal (if using secure Hadoop)
User.login(this.conf, "hbase.regionserver.keytab.file",
"hbase.regionserver.kerberos.principal", this.isa.getHostName());
@@ -1022,7 +1013,6 @@
// Init in here rather than in constructor after thread name has been set
this.metrics = new RegionServerMetrics();
this.dynamicMetrics = RegionServerDynamicMetrics.newInstance(this);
- this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
startServiceThreads();
LOG.info("Serving as " + this.serverNameFromMasterPOV +
", RPC listening on " + this.isa +
@@ -1030,7 +1020,6 @@
Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
isOnline = true;
} catch (Throwable e) {
- LOG.warn("Exception in region server : ", e);
this.isOnline = false;
stop("Failed initialization");
throw convertThrowableToIOE(cleanup(e, "Failed init"),
@@ -1106,7 +1095,8 @@
storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, rootIndexSizeKB,
totalStaticIndexSizeKB, totalStaticBloomSizeKB,
(int) r.readRequestsCount.get(), (int) r.writeRequestsCount.get(),
- totalCompactingKVs, currentCompactedKVs);
+ totalCompactingKVs, currentCompactedKVs,
+ r.getCoprocessorHost().getCoprocessors());
}
/**
@@ -1586,7 +1576,6 @@
this.splitLogWorker = new SplitLogWorker(this.zooKeeper,
this.getConfiguration(), this.getServerName().toString());
splitLogWorker.start();
-
}
/**
@@ -1654,15 +1643,10 @@
@Override
public void stop(final String msg) {
- try {
- this.rsHost.preStop(msg);
- this.stopped = true;
- LOG.info("STOPPED: " + msg);
- // Wakes run() if it is sleeping
- sleeper.skipSleepCycle();
- } catch (IOException exp) {
- LOG.warn("The region server did not stop", exp);
- }
+ this.stopped = true;
+ LOG.info("STOPPED: " + msg);
+ // Wakes run() if it is sleeping
+ sleeper.skipSleepCycle();
}
public void waitForServerOnline(){
@@ -2446,32 +2430,23 @@
}
}
- MultiVersionConsistencyControl.setThreadReadPoint(s.getMvccReadPoint());
- region.startRegionOperation();
- try {
- int i = 0;
- synchronized(s) {
- for (; i < nbRows
- && currentScanResultSize < maxScannerResultSize; i++) {
- // Collect values to be returned here
- boolean moreRows = s.nextRaw(values, SchemaMetrics.METRIC_NEXTSIZE);
- if (!values.isEmpty()) {
- for (KeyValue kv : values) {
- currentScanResultSize += kv.heapSize();
- }
- results.add(new Result(values));
- }
- if (!moreRows) {
- break;
- }
- values.clear();
+ for (int i = 0; i < nbRows
+ && currentScanResultSize < maxScannerResultSize; i++) {
+ requestCount.incrementAndGet();
+ // Collect values to be returned here
+ boolean moreRows = s.next(values, SchemaMetrics.METRIC_NEXTSIZE);
+ if (!values.isEmpty()) {
+ for (KeyValue kv : values) {
+ currentScanResultSize += kv.heapSize();
}
+ results.add(new Result(values));
}
- requestCount.addAndGet(i);
- region.readRequestsCount.add(i);
- } finally {
- region.closeRegionOperation();
+ if (!moreRows) {
+ break;
+ }
+ values.clear();
}
+
// coprocessor postNext hook
if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postScannerNext(s, results, nbRows, true);
@@ -2614,9 +2589,6 @@
return -1;
}
- /**
- * @deprecated {@link RowLock} and associated operations are deprecated.
- */
public long lockRow(byte[] regionName, byte[] row) throws IOException {
checkOpen();
NullPointerException npe = null;
@@ -2633,9 +2605,6 @@
requestCount.incrementAndGet();
try {
HRegion region = getRegion(regionName);
- if (region.getCoprocessorHost() != null) {
- region.getCoprocessorHost().preLockRow(regionName, row);
- }
Integer r = region.obtainRowLock(row);
long lockId = addRowLock(r, region);
LOG.debug("Row lock " + lockId + " explicitly acquired by client");
@@ -2679,9 +2648,6 @@
return rl;
}
- /**
- * @deprecated {@link RowLock} and associated operations are deprecated.
- */
@Override
@QosPriority(priority=HConstants.HIGH_QOS)
public void unlockRow(byte[] regionName, long lockId) throws IOException {
@@ -2700,9 +2666,6 @@
requestCount.incrementAndGet();
try {
HRegion region = getRegion(regionName);
- if (region.getCoprocessorHost() != null) {
- region.getCoprocessorHost().preUnLockRow(regionName, lockId);
- }
String lockName = String.valueOf(lockId);
Integer r = rowlocks.remove(lockName);
if (r == null) {
@@ -2879,11 +2842,6 @@
final int versionOfClosingNode)
throws IOException {
checkOpen();
- //Check for permissions to close.
- HRegion actualRegion = this.getFromOnlineRegions(region.getEncodedName());
- if (actualRegion.getCoprocessorHost() != null) {
- actualRegion.getCoprocessorHost().preClose(false);
- }
LOG.info("Received close region: " + region.getRegionNameAsString() +
". Version of ZK closing node:" + versionOfClosingNode);
boolean hasit = this.onlineRegions.containsKey(region.getEncodedName());
@@ -2931,17 +2889,6 @@
*/
protected boolean closeRegion(HRegionInfo region, final boolean abort,
final boolean zk, final int versionOfClosingNode) {
-
- HRegion actualRegion = this.getFromOnlineRegions(region.getEncodedName());
- if ((actualRegion != null) && (actualRegion.getCoprocessorHost() !=null)){
- try {
- actualRegion.getCoprocessorHost().preClose(abort);
- } catch (IOException e) {
- LOG.warn(e);
- return false;
- }
- }
-
if (this.regionsInTransitionInRS.containsKey(region.getEncodedNameAsBytes())) {
LOG.warn("Received close for region we are already opening or closing; " +
region.getEncodedName());
@@ -3642,10 +3589,6 @@
return this.zooKeeper;
}
- public RegionServerCoprocessorHost getCoprocessorHost(){
- return this.rsHost;
- }
-
public ConcurrentSkipListMap<byte[], Boolean> getRegionsInTransitionInRS() {
return this.regionsInTransitionInRS;
@@ -3823,13 +3766,8 @@
// used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
public String[] getCoprocessors() {
- TreeSet<String> coprocessors = new TreeSet<String>(
- this.hlog.getCoprocessorHost().getCoprocessors());
- Collection<HRegion> regions = getOnlineRegionsLocalContext();
- for (HRegion region: regions) {
- coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
- }
- return coprocessors.toArray(new String[0]);
+ HServerLoad hsl = buildServerLoad();
+ return hsl == null? null: hsl.getCoprocessors();
}
/**
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
index 27e194f..0f61539 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
@@ -37,6 +37,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
@@ -264,7 +265,7 @@
/**
* Invoked before a region open
*/
- public void preOpen(){
+ public void preOpen() {
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
for (RegionEnvironment env: coprocessors) {
if (env.getInstance() instanceof RegionObserver) {
@@ -284,7 +285,7 @@
/**
* Invoked after a region open
*/
- public void postOpen(){
+ public void postOpen() {
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
for (RegionEnvironment env: coprocessors) {
if (env.getInstance() instanceof RegionObserver) {
@@ -305,7 +306,7 @@
* Invoked before a region is closed
* @param abortRequested true if the server is aborting
*/
- public void preClose(boolean abortRequested) throws IOException {
+ public void preClose(boolean abortRequested) {
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
for (RegionEnvironment env: coprocessors) {
if (env.getInstance() instanceof RegionObserver) {
@@ -313,7 +314,7 @@
try {
((RegionObserver)env.getInstance()).preClose(ctx, abortRequested);
} catch (Throwable e) {
- handleCoprocessorThrowable(env, e);
+ handleCoprocessorThrowableNoRethrow(env, e);
}
}
}
@@ -323,7 +324,7 @@
* Invoked after a region is closed
* @param abortRequested true if the server is aborting
*/
- public void postClose(boolean abortRequested){
+ public void postClose(boolean abortRequested) {
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
for (RegionEnvironment env: coprocessors) {
if (env.getInstance() instanceof RegionObserver) {
@@ -1482,31 +1483,5 @@
return hasLoaded;
}
-
- public void preLockRow(byte[] regionName, byte[] row) throws IOException {
- ObserverContext<RegionCoprocessorEnvironment> ctx = null;
- for (RegionEnvironment env : coprocessors) {
- if (env.getInstance() instanceof RegionObserver) {
- ctx = ObserverContext.createAndPrepare(env, ctx);
- ((RegionObserver) env.getInstance()).preLockRow(ctx, regionName, row);
- if (ctx.shouldComplete()) {
- break;
- }
- }
- }
- }
-
- public void preUnLockRow(byte[] regionName, long lockId) throws IOException {
- ObserverContext<RegionCoprocessorEnvironment> ctx = null;
- for (RegionEnvironment env : coprocessors) {
- if (env.getInstance() instanceof RegionObserver) {
- ctx = ObserverContext.createAndPrepare(env, ctx);
- ((RegionObserver) env.getInstance()).preUnlockRow(ctx, regionName, lockId);
- if (ctx.shouldComplete()) {
- break;
- }
- }
- }
- }
}
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java b/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
index 7b6762c..da95e90 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
@@ -20,10 +20,7 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
-import java.util.List;
-
import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.KeyValue;
/**
* RegionScanner describes iterators over rows in an HRegion.
@@ -52,50 +49,4 @@
*/
public boolean reseek(byte[] row) throws IOException;
- /**
- * @return The Scanner's MVCC readPt see {@link MultiVersionConsistencyControl}
- */
- public long getMvccReadPoint();
-
- /**
- * Grab the next row's worth of values with the default limit on the number of values
- * to return.
- * This is a special internal method to be called from coprocessor hooks to avoid expensive setup.
- * Caller must set the thread's readpoint, start and close a region operation, an synchronize on the scanner object.
- * See {@link #nextRaw(List, int, String)}
- * @param result return output array
- * @param metric the metric name
- * @return true if more rows exist after this one, false if scanner is done
- * @throws IOException e
- */
- public boolean nextRaw(List<KeyValue> result, String metric) throws IOException;
-
- /**
- * Grab the next row's worth of values with a limit on the number of values
- * to return.
- * This is a special internal method to be called from coprocessor hooks to avoid expensive setup.
- * Caller must set the thread's readpoint, start and close a region operation, an synchronize on the scanner object.
- * Example:
- * <code><pre>
- * HRegion region = ...;
- * RegionScanner scanner = ...
- * MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint());
- * region.startRegionOperation();
- * try {
- * synchronized(scanner) {
- * ...
- * boolean moreRows = scanner.nextRaw(values);
- * ...
- * }
- * } finally {
- * region.closeRegionOperation();
- * }
- * </pre></code>
- * @param result return output array
- * @param limit limit on row count to get
- * @param metric the metric name
- * @return true if more rows exist after this one, false if scanner is done
- * @throws IOException e
- */
- public boolean nextRaw(List<KeyValue> result, int limit, String metric) throws IOException;
}
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java b/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
deleted file mode 100644
index 2302f9e..0000000
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
+++ /dev/null
@@ -1,104 +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.regionserver;
-
-import java.io.IOException;
-import java.util.Comparator;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
-
-public class RegionServerCoprocessorHost extends
- CoprocessorHost<RegionServerCoprocessorHost.RegionServerEnvironment> {
-
- private RegionServerServices rsServices;
-
- public RegionServerCoprocessorHost(RegionServerServices rsServices, Configuration conf) {
- this.rsServices = rsServices;
- this.conf = conf;
- // load system default cp's from configuration.
- loadSystemCoprocessors(conf, REGIONSERVER_COPROCESSOR_CONF_KEY);
- }
-
- @Override
- public RegionServerEnvironment createEnvironment(Class<?> implClass, Coprocessor instance,
- int priority, int sequence, Configuration conf) {
- return new RegionServerEnvironment(implClass, instance, priority, sequence, conf,
- this.rsServices);
- }
-
- public void preStop(String message) throws IOException {
- ObserverContext<RegionServerCoprocessorEnvironment> ctx = null;
- for (RegionServerEnvironment env : coprocessors) {
- if (env.getInstance() instanceof RegionServerObserver) {
- ctx = ObserverContext.createAndPrepare(env, ctx);
- ((RegionServerObserver) env.getInstance()).preStopRegionServer(ctx);
- if (ctx.shouldComplete()) {
- break;
- }
- }
- }
- }
-
- /**
- * Coprocessor environment extension providing access to region server related services.
- */
- static class RegionServerEnvironment extends CoprocessorHost.Environment implements
- RegionServerCoprocessorEnvironment {
-
- private RegionServerServices regionServerServices;
-
- public RegionServerEnvironment(final Class<?> implClass, final Coprocessor impl,
- final int priority, final int seq, final Configuration conf,
- final RegionServerServices services) {
- super(impl, priority, seq, conf);
- this.regionServerServices = services;
- }
-
- @Override
- public RegionServerServices getRegionServerServices() {
- return regionServerServices;
- }
- }
-
- /**
- * Environment priority comparator. Coprocessors are chained in sorted order.
- */
- static class EnvironmentPriorityComparator implements Comparator<CoprocessorEnvironment> {
- public int compare(final CoprocessorEnvironment env1, final CoprocessorEnvironment env2) {
- if (env1.getPriority() < env2.getPriority()) {
- return -1;
- } else if (env1.getPriority() > env2.getPriority()) {
- return 1;
- }
- if (env1.getLoadSequence() < env2.getLoadSequence()) {
- return -1;
- } else if (env1.getLoadSequence() > env2.getLoadSequence()) {
- return 1;
- }
- return 0;
- }
- }
-
-}
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
index 4b52159..d1cab8e 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
@@ -82,8 +82,6 @@
/* row is not private for tests */
/** Row the query is on */
byte [] row;
- int rowOffset;
- short rowLength;
/**
* Oldest put in any of the involved store files
@@ -224,7 +222,7 @@
short rowLength = Bytes.toShort(bytes, offset, Bytes.SIZEOF_SHORT);
offset += Bytes.SIZEOF_SHORT;
- int ret = this.rowComparator.compareRows(row, this.rowOffset, this.rowLength,
+ int ret = this.rowComparator.compareRows(row, 0, row.length,
bytes, offset, rowLength);
if (ret <= -1) {
return MatchCode.DONE;
@@ -387,10 +385,8 @@
* Set current row
* @param row
*/
- public void setRow(byte [] row, int offset, short length) {
+ public void setRow(byte [] row) {
this.row = row;
- this.rowOffset = offset;
- this.rowLength = length;
reset();
}
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index 9a02c16..f9e1103 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
+import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -131,6 +132,9 @@
private volatile long totalUncompressedBytes = 0L;
private final Object flushLock = new Object();
final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+ private final String storeNameStr;
+ private CompactionProgress progress;
+ private final int compactionKVMax;
private final boolean verifyBulkLoads;
/* The default priority for user-specified compaction requests.
@@ -154,6 +158,10 @@
new CopyOnWriteArraySet<ChangedReadersObserver>();
private final int blocksize;
+ /** Compression algorithm for flush files and minor compaction */
+ private final Compression.Algorithm compression;
+ /** Compression algorithm for major compaction */
+ private final Compression.Algorithm compactionCompression;
private HFileDataBlockEncoder dataBlockEncoder;
/** Checksum configuration */
@@ -163,8 +171,6 @@
// Comparing KeyValues
final KeyValue.KVComparator comparator;
- private final Compactor compactor;
-
/**
* Constructor
* @param basedir qualified path under which the region directory lives;
@@ -179,16 +185,25 @@
protected Store(Path basedir, HRegion region, HColumnDescriptor family,
FileSystem fs, Configuration conf)
throws IOException {
- super(conf, region.getRegionInfo().getTableNameAsString(),
+ super(conf, region.getTableDesc().getNameAsString(),
Bytes.toString(family.getName()));
- HRegionInfo info = region.getRegionInfo();
+ HRegionInfo info = region.regionInfo;
this.fs = fs;
- Path p = getStoreHomedir(basedir, info.getEncodedName(), family.getName());
- this.homedir = createStoreHomeDir(this.fs, p);
+ this.homedir = getStoreHomedir(basedir, info.getEncodedName(), family.getName());
+ if (!this.fs.exists(this.homedir)) {
+ if (!this.fs.mkdirs(this.homedir))
+ throw new IOException("Failed create of: " + this.homedir.toString());
+ }
this.region = region;
this.family = family;
this.conf = conf;
this.blocksize = family.getBlocksize();
+ this.compression = family.getCompression();
+ // avoid overriding compression setting for major compactions if the user
+ // has not specified it separately
+ this.compactionCompression =
+ (family.getCompactionCompression() != Compression.Algorithm.NONE) ?
+ family.getCompactionCompression() : this.compression;
this.dataBlockEncoder =
new HFileDataBlockEncoderImpl(family.getDataBlockEncodingOnDisk(),
@@ -213,6 +228,7 @@
"ms in store " + this);
scanInfo = new ScanInfo(family, ttl, timeToPurgeDeletes, this.comparator);
this.memstore = new MemStore(conf, this.comparator);
+ this.storeNameStr = getColumnFamilyName();
// By default, compact if storefile.count >= minFilesToCompact
this.minFilesToCompact = Math.max(2,
@@ -229,8 +245,10 @@
this.region.memstoreFlushSize);
this.maxCompactSize
= conf.getLong("hbase.hstore.compaction.max.size", Long.MAX_VALUE);
+ this.compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, 10);
- this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
+ this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify",
+ false);
if (Store.closeCheckInterval == 0) {
Store.closeCheckInterval = conf.getInt(
@@ -242,47 +260,6 @@
this.checksumType = getChecksumType(conf);
// initilize bytes per checksum
this.bytesPerChecksum = getBytesPerChecksum(conf);
- // Create a compaction tool instance
- this.compactor = new Compactor(this.conf);
- }
-
- /**
- * @param family
- * @return
- */
- long getTTL(final HColumnDescriptor family) {
- // HCD.getTimeToLive returns ttl in seconds. Convert to milliseconds.
- long ttl = family.getTimeToLive();
- if (ttl == HConstants.FOREVER) {
- // Default is unlimited ttl.
- ttl = Long.MAX_VALUE;
- } else if (ttl == -1) {
- ttl = Long.MAX_VALUE;
- } else {
- // Second -> ms adjust for user data
- ttl *= 1000;
- }
- return ttl;
- }
-
- /**
- * Create this store's homedir
- * @param fs
- * @param homedir
- * @return Return <code>homedir</code>
- * @throws IOException
- */
- Path createStoreHomeDir(final FileSystem fs,
- final Path homedir) throws IOException {
- if (!fs.exists(homedir)) {
- if (!fs.mkdirs(homedir))
- throw new IOException("Failed create of: " + homedir.toString());
- }
- return homedir;
- }
-
- FileSystem getFileSystem() {
- return this.fs;
}
/**
@@ -343,7 +320,7 @@
* Return the directory in which this store stores its
* StoreFiles
*/
- Path getHomedir() {
+ public Path getHomedir() {
return homedir;
}
@@ -362,10 +339,6 @@
this.dataBlockEncoder = blockEncoder;
}
- FileStatus [] getStoreFiles() throws IOException {
- return FSUtils.listStatus(this.fs, this.homedir, null);
- }
-
/**
* Creates an unsorted list of StoreFile loaded in parallel
* from the given directory.
@@ -373,7 +346,7 @@
*/
private List<StoreFile> loadStoreFiles() throws IOException {
ArrayList<StoreFile> results = new ArrayList<StoreFile>();
- FileStatus files[] = getStoreFiles();
+ FileStatus files[] = FSUtils.listStatus(this.fs, this.homedir, null);
if (files == null || files.length == 0) {
return results;
@@ -664,7 +637,7 @@
storeFileCloserThreadPool.shutdownNow();
}
}
- LOG.info("Closed " + this);
+ LOG.debug("closed " + this.storeNameStr);
return result;
} finally {
this.lock.writeLock().unlock();
@@ -750,7 +723,6 @@
scanner = cpScanner;
}
try {
- int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, 10);
// TODO: We can fail in the below block before we complete adding this
// flush to list of store files. Add cleanup of anything put on filesystem
// if we fail.
@@ -764,7 +736,7 @@
List<KeyValue> kvs = new ArrayList<KeyValue>();
boolean hasMore;
do {
- hasMore = scanner.next(kvs, compactionKVMax);
+ hasMore = scanner.next(kvs, this.compactionKVMax);
if (!kvs.isEmpty()) {
for (KeyValue kv : kvs) {
// If we know that this KV is going to be included always, then let us
@@ -856,7 +828,7 @@
*/
private StoreFile.Writer createWriterInTmp(int maxKeyCount)
throws IOException {
- return createWriterInTmp(maxKeyCount, this.family.getCompression(), false);
+ return createWriterInTmp(maxKeyCount, this.compression, false);
}
/*
@@ -1009,12 +981,16 @@
* @param cr
* compaction details obtained from requestCompaction()
* @throws IOException
- * @return Storefile we compacted into or null if we failed or opted out early.
*/
- StoreFile compact(CompactionRequest cr) throws IOException {
- if (cr == null || cr.getFiles().isEmpty()) return null;
- Preconditions.checkArgument(cr.getStore().toString().equals(this.toString()));
+ void compact(CompactionRequest cr) throws IOException {
+ if (cr == null || cr.getFiles().isEmpty()) {
+ return;
+ }
+ Preconditions.checkArgument(cr.getStore().toString()
+ .equals(this.toString()));
+
List<StoreFile> filesToCompact = cr.getFiles();
+
synchronized (filesCompacting) {
// sanity check: we're compacting files that this store knows about
// TODO: change this to LOG.error() after more debugging
@@ -1026,26 +1002,19 @@
// Ready to go. Have list of files to compact.
LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
- + this + " of "
+ + this.storeNameStr + " of "
+ this.region.getRegionInfo().getRegionNameAsString()
+ " into tmpdir=" + region.getTmpDir() + ", seqid=" + maxId + ", totalSize="
+ StringUtils.humanReadableInt(cr.getSize()));
StoreFile sf = null;
try {
- StoreFile.Writer writer =
- this.compactor.compact(this, filesToCompact, cr.isMajor(), maxId);
+ StoreFile.Writer writer = compactStore(filesToCompact, cr.isMajor(),
+ maxId);
// Move the compaction into place.
- if (this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
- sf = completeCompaction(filesToCompact, writer);
- if (region.getCoprocessorHost() != null) {
- region.getCoprocessorHost().postCompact(this, sf);
- }
- } else {
- // Create storefile around what we wrote with a reader on it.
- sf = new StoreFile(this.fs, writer.getPath(), this.conf, this.cacheConf,
- this.family.getBloomFilterType(), this.dataBlockEncoder);
- sf.createReader();
+ sf = completeCompaction(filesToCompact, writer);
+ if (region.getCoprocessorHost() != null) {
+ region.getCoprocessorHost().postCompact(this, sf);
}
} finally {
synchronized (filesCompacting) {
@@ -1054,7 +1023,7 @@
}
LOG.info("Completed" + (cr.isMajor() ? " major " : " ") + "compaction of "
- + filesToCompact.size() + " file(s) in " + this + " of "
+ + filesToCompact.size() + " file(s) in " + this.storeNameStr + " of "
+ this.region.getRegionInfo().getRegionNameAsString()
+ " into " +
(sf == null ? "none" : sf.getPath().getName()) +
@@ -1062,7 +1031,6 @@
StringUtils.humanReadableInt(sf.getReader().length()))
+ "; total size for store is "
+ StringUtils.humanReadableInt(storeSize));
- return sf;
}
/**
@@ -1102,8 +1070,7 @@
try {
// Ready to go. Have list of files to compact.
- StoreFile.Writer writer =
- this.compactor.compact(this, filesToCompact, isMajor, maxId);
+ StoreFile.Writer writer = compactStore(filesToCompact, isMajor, maxId);
// Move the compaction into place.
StoreFile sf = completeCompaction(filesToCompact, writer);
if (region.getCoprocessorHost() != null) {
@@ -1152,10 +1119,10 @@
}
/** getter for CompactionProgress object
- * @return CompactionProgress object; can be null
+ * @return CompactionProgress object
*/
public CompactionProgress getCompactionProgress() {
- return this.compactor.getProgress();
+ return this.progress;
}
/*
@@ -1207,19 +1174,19 @@
if (sf.isMajorCompaction() &&
(this.ttl == HConstants.FOREVER || oldest < this.ttl)) {
if (LOG.isDebugEnabled()) {
- LOG.debug("Skipping major compaction of " + this +
+ LOG.debug("Skipping major compaction of " + this.storeNameStr +
" because one (major) compacted file only and oldestTime " +
oldest + "ms is < ttl=" + this.ttl);
}
} else if (this.ttl != HConstants.FOREVER && oldest > this.ttl) {
- LOG.debug("Major compaction triggered on store " + this +
+ LOG.debug("Major compaction triggered on store " + this.storeNameStr +
", because keyvalues outdated; time since last major compaction " +
(now - lowTimestamp) + "ms");
result = true;
}
} else {
if (LOG.isDebugEnabled()) {
- LOG.debug("Major compaction triggered on store " + this +
+ LOG.debug("Major compaction triggered on store " + this.storeNameStr +
"; time since last major compaction " + (now - lowTimestamp) + "ms");
}
result = true;
@@ -1409,12 +1376,12 @@
compactSelection.getFilesToCompact().get(pos).getReader().length()
> maxCompactSize &&
!compactSelection.getFilesToCompact().get(pos).isReference()) ++pos;
- if (pos != 0) compactSelection.clearSubList(0, pos);
+ compactSelection.clearSubList(0, pos);
}
if (compactSelection.getFilesToCompact().isEmpty()) {
LOG.debug(this.getHRegionInfo().getEncodedName() + " - " +
- this + ": no store files to compact");
+ this.storeNameStr + ": no store files to compact");
compactSelection.emptyFileList();
return compactSelection;
}
@@ -1501,7 +1468,7 @@
// if we don't have enough files to compact, just wait
if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) {
if (LOG.isDebugEnabled()) {
- LOG.debug("Skipped compaction of " + this
+ LOG.debug("Skipped compaction of " + this.storeNameStr
+ ". Only " + (end - start) + " file(s) of size "
+ StringUtils.humanReadableInt(totalSize)
+ " have met compaction criteria.");
@@ -1528,6 +1495,149 @@
}
/**
+ * Do a minor/major compaction on an explicit set of storefiles in a Store.
+ * Uses the scan infrastructure to make it easy.
+ *
+ * @param filesToCompact which files to compact
+ * @param majorCompaction true to major compact (prune all deletes, max versions, etc)
+ * @param maxId Readers maximum sequence id.
+ * @return Product of compaction or null if all cells expired or deleted and
+ * nothing made it through the compaction.
+ * @throws IOException
+ */
+ StoreFile.Writer compactStore(final Collection<StoreFile> filesToCompact,
+ final boolean majorCompaction, final long maxId)
+ throws IOException {
+ // calculate maximum key count after compaction (for blooms)
+ int maxKeyCount = 0;
+ long earliestPutTs = HConstants.LATEST_TIMESTAMP;
+ for (StoreFile file : filesToCompact) {
+ StoreFile.Reader r = file.getReader();
+ if (r != null) {
+ // NOTE: getFilterEntries could cause under-sized blooms if the user
+ // switches bloom type (e.g. from ROW to ROWCOL)
+ long keyCount = (r.getBloomFilterType() == family.getBloomFilterType())
+ ? r.getFilterEntries() : r.getEntries();
+ maxKeyCount += keyCount;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Compacting " + file +
+ ", keycount=" + keyCount +
+ ", bloomtype=" + r.getBloomFilterType().toString() +
+ ", size=" + StringUtils.humanReadableInt(r.length()) +
+ ", encoding=" + r.getHFileReader().getEncodingOnDisk());
+ }
+ }
+ // For major compactions calculate the earliest put timestamp
+ // of all involved storefiles. This is used to remove
+ // family delete marker during the compaction.
+ if (majorCompaction) {
+ byte[] tmp = r.loadFileInfo().get(StoreFile.EARLIEST_PUT_TS);
+ if (tmp == null) {
+ // there's a file with no information, must be an old one
+ // assume we have very old puts
+ earliestPutTs = HConstants.OLDEST_TIMESTAMP;
+ } else {
+ earliestPutTs = Math.min(earliestPutTs, Bytes.toLong(tmp));
+ }
+ }
+ }
+
+ // keep track of compaction progress
+ progress = new CompactionProgress(maxKeyCount);
+
+ // For each file, obtain a scanner:
+ List<StoreFileScanner> scanners = StoreFileScanner
+ .getScannersForStoreFiles(filesToCompact, false, false, true);
+
+ // Make the instantiation lazy in case compaction produces no product; i.e.
+ // where all source cells are expired or deleted.
+ StoreFile.Writer writer = null;
+ // Find the smallest read point across all the Scanners.
+ long smallestReadPoint = region.getSmallestReadPoint();
+ MultiVersionConsistencyControl.setThreadReadPoint(smallestReadPoint);
+ try {
+ InternalScanner scanner = null;
+ try {
+ if (getHRegion().getCoprocessorHost() != null) {
+ scanner = getHRegion()
+ .getCoprocessorHost()
+ .preCompactScannerOpen(this, scanners,
+ majorCompaction ? ScanType.MAJOR_COMPACT : ScanType.MINOR_COMPACT, earliestPutTs);
+ }
+ if (scanner == null) {
+ Scan scan = new Scan();
+ scan.setMaxVersions(getFamily().getMaxVersions());
+ /* Include deletes, unless we are doing a major compaction */
+ scanner = new StoreScanner(this, getScanInfo(), scan, scanners,
+ majorCompaction? ScanType.MAJOR_COMPACT : ScanType.MINOR_COMPACT,
+ smallestReadPoint, earliestPutTs);
+ }
+ if (getHRegion().getCoprocessorHost() != null) {
+ InternalScanner cpScanner =
+ getHRegion().getCoprocessorHost().preCompact(this, scanner);
+ // NULL scanner returned from coprocessor hooks means skip normal processing
+ if (cpScanner == null) {
+ return null;
+ }
+ scanner = cpScanner;
+ }
+
+ int bytesWritten = 0;
+ // since scanner.next() can return 'false' but still be delivering data,
+ // we have to use a do/while loop.
+ ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
+ // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
+ boolean hasMore;
+ do {
+ hasMore = scanner.next(kvs, this.compactionKVMax);
+ if (writer == null && !kvs.isEmpty()) {
+ writer = createWriterInTmp(maxKeyCount, this.compactionCompression,
+ true);
+ }
+ if (writer != null) {
+ // output to writer:
+ for (KeyValue kv : kvs) {
+ if (kv.getMemstoreTS() <= smallestReadPoint) {
+ kv.setMemstoreTS(0);
+ }
+ writer.append(kv);
+ // update progress per key
+ ++progress.currentCompactedKVs;
+
+ // check periodically to see if a system stop is requested
+ if (Store.closeCheckInterval > 0) {
+ bytesWritten += kv.getLength();
+ if (bytesWritten > Store.closeCheckInterval) {
+ bytesWritten = 0;
+ if (!this.region.areWritesEnabled()) {
+ writer.close();
+ fs.delete(writer.getPath(), false);
+ throw new InterruptedIOException(
+ "Aborting compaction of store " + this +
+ " in region " + this.region +
+ " because user requested stop.");
+ }
+ }
+ }
+ }
+ }
+ kvs.clear();
+ } while (hasMore);
+ } finally {
+ if (scanner != null) {
+ scanner.close();
+ }
+ }
+ } finally {
+ if (writer != null) {
+ writer.appendMetadata(maxId, majorCompaction);
+ writer.close();
+ }
+ }
+ return writer;
+ }
+
+ /**
* Validates a store file by opening and closing it. In HFileV2 this should
* not be an expensive operation.
*
@@ -1631,7 +1741,7 @@
} catch (IOException e) {
e = RemoteExceptionHandler.checkIOException(e);
- LOG.error("Failed replacing compacted files in " + this +
+ LOG.error("Failed replacing compacted files in " + this.storeNameStr +
". Compacted file is " + (result == null? "none": result.toString()) +
". Files replaced " + compactedFiles.toString() +
" some of which may have been already removed", e);
@@ -1917,7 +2027,7 @@
return mk.getRow();
}
} catch(IOException e) {
- LOG.warn("Failed getting store size for " + this, e);
+ LOG.warn("Failed getting store size for " + this.storeNameStr, e);
} finally {
this.lock.readLock().unlock();
}
@@ -1970,7 +2080,7 @@
@Override
public String toString() {
- return getColumnFamilyName();
+ return this.storeNameStr;
}
/**
@@ -2086,7 +2196,7 @@
}
HRegionInfo getHRegionInfo() {
- return this.region.getRegionInfo();
+ return this.region.regionInfo;
}
/**
@@ -2214,8 +2324,8 @@
public static final long FIXED_OVERHEAD =
ClassSize.align(SchemaConfigured.SCHEMA_CONFIGURED_UNALIGNED_HEAP_SIZE +
- + (17 * ClassSize.REFERENCE) + (6 * Bytes.SIZEOF_LONG)
- + (5 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN);
+ + (20 * ClassSize.REFERENCE) + (6 * Bytes.SIZEOF_LONG)
+ + (6 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN);
public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
+ ClassSize.OBJECT + ClassSize.REENTRANT_LOCK
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index 40df899..af3ab83 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -320,7 +320,7 @@
* @return Calculated path to parent region file.
* @throws IOException
*/
- public static Path getReferredToFile(final Path p) {
+ static Path getReferredToFile(final Path p) {
Matcher m = REF_NAME_PARSER.matcher(p.getName());
if (m == null || !m.matches()) {
LOG.warn("Failed match of store file name " + p.toString());
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index bed4ad2..a87d326 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -340,11 +340,8 @@
// only call setRow if the row changes; avoids confusing the query matcher
// if scanning intra-row
- byte[] row = peeked.getBuffer();
- int offset = peeked.getRowOffset();
- short length = peeked.getRowLength();
- if ((matcher.row == null) || !Bytes.equals(row, offset, length, matcher.row, matcher.rowOffset, matcher.rowLength)) {
- matcher.setRow(row, offset, length);
+ if ((matcher.row == null) || !peeked.matchingRow(matcher.row)) {
+ matcher.setRow(peeked.getRow());
}
KeyValue kv;
@@ -524,12 +521,9 @@
if (kv == null) {
kv = lastTopKey;
}
- byte[] row = kv.getBuffer();
- int offset = kv.getRowOffset();
- short length = kv.getRowLength();
- if ((matcher.row == null) || !Bytes.equals(row, offset, length, matcher.row, matcher.rowOffset, matcher.rowLength)) {
+ if ((matcher.row == null) || !kv.matchingRow(matcher.row)) {
matcher.reset();
- matcher.setRow(row, offset, length);
+ matcher.setRow(kv.getRow());
}
}
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java b/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
index f91b782..9bc66e1 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
@@ -49,4 +49,5 @@
public float getProgressPct() {
return currentCompactedKVs / totalCompactingKVs;
}
+
}
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java b/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java
index c19d5b3..8193e1f 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Compressor.java
@@ -143,12 +143,7 @@
// the status byte also acts as the higher order byte of the dictionary
// entry
short dictIdx = toShort(status, in.readByte());
- byte[] entry;
- try {
- entry = dict.getEntry(dictIdx);
- } catch (Exception ex) {
- throw new IOException("Unable to uncompress the log entry", ex);
- }
+ byte[] entry = dict.getEntry(dictIdx);
if (entry == null) {
throw new IOException("Missing dictionary entry for index "
+ dictIdx);
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java b/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
index cbe3e25..2bb5ea1 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
@@ -167,7 +167,6 @@
Entry next(Entry reuse) throws IOException;
void seek(long pos) throws IOException;
long getPosition() throws IOException;
- void reset() throws IOException;
}
public interface Writer {
@@ -696,18 +695,15 @@
/**
* Get a reader for the WAL.
- * The proper way to tail a log that can be under construction is to first use this method
- * to get a reader then call {@link HLog.Reader#reset()} to see the new data. It will also
- * take care of keeping implementation-specific context (like compression).
* @param fs
* @param path
* @param conf
* @return A WAL reader. Close when done with it.
* @throws IOException
*/
- public static Reader getReader(final FileSystem fs, final Path path,
- Configuration conf)
- throws IOException {
+ public static Reader getReader(final FileSystem fs,
+ final Path path, Configuration conf)
+ throws IOException {
try {
if (logReaderClass == null) {
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java b/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
index 9d7928b..f5fb00f 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
@@ -139,17 +139,15 @@
Configuration conf;
WALReader reader;
- FileSystem fs;
// Needed logging exceptions
Path path;
int edit = 0;
long entryStart = 0;
- boolean emptyCompressionContext = true;
/**
* Compression context to use reading. Can be null if no compression.
*/
- protected CompressionContext compressionContext = null;
+ private CompressionContext compressionContext = null;
protected Class<? extends HLogKey> keyClass;
@@ -175,7 +173,6 @@
this.conf = conf;
this.path = path;
reader = new WALReader(fs, path, conf);
- this.fs = fs;
// If compression is enabled, new dictionaries are created here.
boolean compression = reader.isWALCompressionEnabled();
@@ -240,22 +237,11 @@
throw addFileInfoToException(ioe);
}
edit++;
- if (compressionContext != null && emptyCompressionContext) {
- emptyCompressionContext = false;
- }
return b? e: null;
}
@Override
public void seek(long pos) throws IOException {
- if (compressionContext != null && emptyCompressionContext) {
- while (next() != null) {
- if (getPosition() == pos) {
- emptyCompressionContext = false;
- break;
- }
- }
- }
try {
reader.seek(pos);
} catch (IOException ioe) {
@@ -300,11 +286,4 @@
return ioe;
}
-
- @Override
- public void reset() throws IOException {
- // Resetting the reader lets us see newly added data if the file is being written to
- // We also keep the same compressionContext which was previously populated for this file
- reader = new WALReader(fs, path, conf);
- }
}
\ No newline at end of file
diff --git a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java
deleted file mode 100644
index ce8585c..0000000
--- a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java
+++ /dev/null
@@ -1,136 +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.replication.regionserver;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
-
-import java.io.IOException;
-
-/**
- * Wrapper class around HLog to help manage the implementation details
- * such as compression.
- */
-@InterfaceAudience.Private
-public class ReplicationHLogReaderManager {
-
- private static final Log LOG = LogFactory.getLog(ReplicationHLogReaderManager.class);
- private final FileSystem fs;
- private final Configuration conf;
- private long position = 0;
- private HLog.Reader reader;
- private Path lastPath;
-
- /**
- * Creates the helper but doesn't open any file
- * Use setInitialPosition after using the constructor if some content needs to be skipped
- * @param fs
- * @param conf
- */
- public ReplicationHLogReaderManager(FileSystem fs, Configuration conf) {
- this.fs = fs;
- this.conf = conf;
- }
-
- /**
- * Opens the file at the current position
- * @param path
- * @return
- * @throws IOException
- */
- public HLog.Reader openReader(Path path) throws IOException {
- // Detect if this is a new file, if so get a new reader else
- // reset the current reader so that we see the new data
- if (this.reader == null || !this.lastPath.equals(path)) {
- this.reader = HLog.getReader(this.fs, path, this.conf);
- this.lastPath = path;
- } else {
- this.reader.reset();
- }
- return this.reader;
- }
-
- /**
- * Get the next entry, returned and also added in the array
- * @param entriesArray
- * @param currentNbEntries
- * @return a new entry or null
- * @throws IOException
- */
- public HLog.Entry readNextAndSetPosition(HLog.Entry[] entriesArray,
- int currentNbEntries) throws IOException {
- HLog.Entry entry = this.reader.next(entriesArray[currentNbEntries]);
- // Store the position so that in the future the reader can start
- // reading from here. If the above call to next() throws an
- // exception, the position won't be changed and retry will happen
- // from the last known good position
- this.position = this.reader.getPosition();
- // We need to set the CC to null else it will be compressed when sent to the sink
- if (entry != null) {
- entry.setCompressionContext(null);
- }
- return entry;
- }
-
- /**
- * Advance the reader to the current position
- * @throws IOException
- */
- public void seek() throws IOException {
- if (this.position != 0) {
- this.reader.seek(this.position);
- }
- }
-
- /**
- * Get the position that we stopped reading at
- * @return current position, cannot be negative
- */
- public long getPosition() {
- return this.position;
- }
-
- public void setPosition(long pos) {
- this.position = pos;
- }
-
- /**
- * Close the current reader
- * @throws IOException
- */
- public void closeReader() throws IOException {
- if (this.reader != null) {
- this.reader.close();
- }
- }
-
- /**
- * Tell the helper to reset internal state
- */
- public void finishCurrentFile() {
- this.position = 0;
- this.reader = null;
- }
-
-}
diff --git a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 9f26e50..09b8797 100644
--- a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -105,6 +105,8 @@
private int replicationQueueNbCapacity;
// Our reader for the current log
private HLog.Reader reader;
+ // Current position in the log
+ private long position = 0;
// Last position in the log that we sent to ZooKeeper
private long lastLoggedPosition = -1;
// Path of the current log
@@ -130,15 +132,10 @@
private int currentNbEntries = 0;
// Current number of operations (Put/Delete) that we need to replicate
private int currentNbOperations = 0;
- // Current size of data we need to replicate
- private int currentSize = 0;
// Indicates if this particular source is running
private volatile boolean running = true;
// Metrics for this source
private ReplicationSourceMetrics metrics;
- // Handle on the log reader helper
- private ReplicationHLogReaderManager repLogReader;
-
/**
* Instantiation method used by region servers
@@ -186,7 +183,7 @@
this.conf.getLong("replication.source.sleepforretries", 1000);
this.fs = fs;
this.metrics = new ReplicationSourceMetrics(peerClusterZnode);
- this.repLogReader = new ReplicationHLogReaderManager(this.fs, this.conf);
+
try {
this.clusterId = zkHelper.getUUIDForCluster(zkHelper.getZookeeperWatcher());
} catch (KeeperException ke) {
@@ -266,8 +263,8 @@
// normally has a position (unless the RS failed between 2 logs)
if (this.queueRecovered) {
try {
- this.repLogReader.setPosition(this.zkHelper.getHLogRepPosition(
- this.peerClusterZnode, this.queue.peek().getName()));
+ this.position = this.zkHelper.getHLogRepPosition(
+ this.peerClusterZnode, this.queue.peek().getName());
} catch (KeeperException e) {
this.terminate("Couldn't get the position of this recovered queue " +
peerClusterZnode, e);
@@ -325,7 +322,6 @@
boolean gotIOE = false;
currentNbEntries = 0;
- currentSize = 0;
try {
if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo)) {
continue;
@@ -361,7 +357,9 @@
}
} finally {
try {
- this.repLogReader.closeReader();
+ if (this.reader != null) {
+ this.reader.close();
+ }
} catch (IOException e) {
gotIOE = true;
LOG.warn("Unable to finalize the tailing of a file", e);
@@ -372,10 +370,10 @@
// wait a bit and retry.
// But if we need to stop, don't bother sleeping
if (this.isActive() && (gotIOE || currentNbEntries == 0)) {
- if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
+ if (this.lastLoggedPosition != this.position) {
this.manager.logPositionAndCleanOldLogs(this.currentPath,
- this.peerClusterZnode, this.repLogReader.getPosition(), queueRecovered, currentWALisBeingWrittenTo);
- this.lastLoggedPosition = this.repLogReader.getPosition();
+ this.peerClusterZnode, this.position, queueRecovered, currentWALisBeingWrittenTo);
+ this.lastLoggedPosition = this.position;
}
if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
sleepMultiplier++;
@@ -407,9 +405,11 @@
protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo)
throws IOException{
long seenEntries = 0;
- this.repLogReader.seek();
- HLog.Entry entry =
- this.repLogReader.readNextAndSetPosition(this.entriesArray, this.currentNbEntries);
+ if (this.position != 0) {
+ this.reader.seek(this.position);
+ }
+ long startPosition = this.position;
+ HLog.Entry entry = readNextAndSetPosition();
while (entry != null) {
WALEdit edit = entry.getEdit();
this.metrics.logEditsReadRate.inc(1);
@@ -433,18 +433,18 @@
}
currentNbOperations += countDistinctRowKeys(edit);
currentNbEntries++;
- currentSize += entry.getEdit().heapSize();
} else {
this.metrics.logEditsFilteredRate.inc(1);
}
}
// Stop if too many entries or too big
- if (currentSize >= this.replicationQueueSizeCapacity ||
+ if ((this.reader.getPosition() - startPosition)
+ >= this.replicationQueueSizeCapacity ||
currentNbEntries >= this.replicationQueueNbCapacity) {
break;
}
try {
- entry = this.repLogReader.readNextAndSetPosition(this.entriesArray, this.currentNbEntries);
+ entry = readNextAndSetPosition();
} catch (IOException ie) {
LOG.debug("Break on IOE: " + ie.getMessage());
break;
@@ -452,7 +452,7 @@
}
LOG.debug("currentNbOperations:" + currentNbOperations +
" and seenEntries:" + seenEntries +
- " and size: " + this.currentSize);
+ " and size: " + (this.reader.getPosition() - startPosition));
if (currentWALisBeingWrittenTo) {
return false;
}
@@ -461,6 +461,16 @@
return seenEntries == 0 && processEndOfFile();
}
+ private HLog.Entry readNextAndSetPosition() throws IOException {
+ HLog.Entry entry = this.reader.next(entriesArray[currentNbEntries]);
+ // Store the position so that in the future the reader can start
+ // reading from here. If the above call to next() throws an
+ // exception, the position won't be changed and retry will happen
+ // from the last known good position
+ this.position = this.reader.getPosition();
+ return entry;
+ }
+
private void connectToPeers() {
// Connect to peer cluster first, unless we have to stop
while (this.isActive() && this.currentPeers.size() == 0) {
@@ -499,9 +509,10 @@
protected boolean openReader(int sleepMultiplier) {
try {
LOG.debug("Opening log for replication " + this.currentPath.getName() +
- " at " + this.repLogReader.getPosition());
+ " at " + this.position);
try {
- this.reader = repLogReader.openReader(this.currentPath);
+ this.reader = null;
+ this.reader = HLog.getReader(this.fs, this.currentPath, this.conf);
} catch (FileNotFoundException fnfe) {
if (this.queueRecovered) {
// We didn't find the log in the archive directory, look if it still
@@ -637,10 +648,10 @@
HRegionInterface rrs = getRS();
LOG.debug("Replicating " + currentNbEntries);
rrs.replicateLogEntries(Arrays.copyOf(this.entriesArray, currentNbEntries));
- if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
+ if (this.lastLoggedPosition != this.position) {
this.manager.logPositionAndCleanOldLogs(this.currentPath,
- this.peerClusterZnode, this.repLogReader.getPosition(), queueRecovered, currentWALisBeingWrittenTo);
- this.lastLoggedPosition = this.repLogReader.getPosition();
+ this.peerClusterZnode, this.position, queueRecovered, currentWALisBeingWrittenTo);
+ this.lastLoggedPosition = this.position;
}
this.totalReplicatedEdits += currentNbEntries;
this.metrics.shippedBatchesRate.inc(1);
@@ -710,8 +721,7 @@
protected boolean processEndOfFile() {
if (this.queue.size() != 0) {
this.currentPath = null;
- this.repLogReader.finishCurrentFile();
- this.reader = null;
+ this.position = 0;
return true;
} else if (this.queueRecovered) {
this.manager.closeRecoveredQueue(this);
diff --git a/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java b/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
index e3be8e1..ba63898 100644
--- a/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
+++ b/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
@@ -72,6 +72,7 @@
final Client client;
final Configuration conf;
final byte[] name;
+ final String accessToken;
final int maxRetries;
final long sleepTime;
@@ -80,6 +81,10 @@
final long startTime, final long endTime, final int maxVersions) {
StringBuffer sb = new StringBuffer();
sb.append('/');
+ if (accessToken != null) {
+ sb.append(accessToken);
+ sb.append('/');
+ }
sb.append(Bytes.toStringBinary(name));
sb.append('/');
sb.append(Bytes.toStringBinary(row));
@@ -137,29 +142,6 @@
return sb.toString();
}
- protected String buildMultiRowSpec(final byte[][] rows, int maxVersions) {
- StringBuilder sb = new StringBuilder();
- sb.append('/');
- sb.append(Bytes.toStringBinary(name));
- sb.append("/multiget/");
- if (rows == null || rows.length == 0) {
- return sb.toString();
- }
- sb.append("?");
- for(int i=0; i<rows.length; i++) {
- byte[] rk = rows[i];
- if (i != 0) {
- sb.append('&');
- }
- sb.append("row=");
- sb.append(Bytes.toStringBinary(rk));
- }
- sb.append("&v=");
- sb.append(maxVersions);
-
- return sb.toString();
- }
-
protected Result[] buildResultFromModel(final CellSetModel model) {
List<Result> results = new ArrayList<Result>();
for (RowModel row: model.getRows()) {
@@ -205,9 +187,7 @@
* @param client
* @param name
* @param accessToken
- * @deprecated accessToken is not used and will be removed
*/
- @Deprecated
public RemoteHTable(Client client, String name, String accessToken) {
this(client, HBaseConfiguration.create(), Bytes.toBytes(name), accessToken);
}
@@ -217,20 +197,8 @@
* @param client
* @param conf
* @param name
- */
- public RemoteHTable(Client client, Configuration conf, String name) {
- this(client, conf, Bytes.toBytes(name), null);
- }
-
- /**
- * Constructor
- * @param client
- * @param conf
- * @param name
* @param accessToken
- * @deprecated accessToken is not used and will be removed
*/
- @Deprecated
public RemoteHTable(Client client, Configuration conf, String name,
String accessToken) {
this(client, conf, Bytes.toBytes(name), accessToken);
@@ -238,28 +206,14 @@
/**
* Constructor
- * @param client
* @param conf
- * @param name
*/
- public RemoteHTable(Client client, Configuration conf, byte[] name) {
- this(client, conf, name, null);
- }
-
- /**
- * Constructor
- * @param client
- * @param conf
- * @param name
- * @param accessToken
- * @deprecated accessToken is not used and will be removed
- */
- @Deprecated
public RemoteHTable(Client client, Configuration conf, byte[] name,
String accessToken) {
this.client = client;
this.conf = conf;
this.name = name;
+ this.accessToken = accessToken;
this.maxRetries = conf.getInt("hbase.rest.client.max.retries", 10);
this.sleepTime = conf.getLong("hbase.rest.client.sleep", 1000);
}
@@ -275,6 +229,10 @@
public HTableDescriptor getTableDescriptor() throws IOException {
StringBuilder sb = new StringBuilder();
sb.append('/');
+ if (accessToken != null) {
+ sb.append(accessToken);
+ sb.append('/');
+ }
sb.append(Bytes.toStringBinary(name));
sb.append('/');
sb.append("schema");
@@ -309,68 +267,30 @@
if (get.getFilter() != null) {
LOG.warn("filters not supported on gets");
}
- Result[] results = getResults(spec);
- if (results.length > 0) {
- if (results.length > 1) {
- LOG.warn("too many results for get (" + results.length + ")");
- }
- return results[0];
- } else {
- return new Result();
- }
- }
-
- public Result[] get(List<Get> gets) throws IOException {
- byte[][] rows = new byte[gets.size()][];
- int maxVersions = 1;
- int count = 0;
-
- for (Get g : gets) {
-
- if (count == 0) {
- maxVersions = g.getMaxVersions();
- } else if (g.getMaxVersions() != maxVersions) {
- LOG.warn("MaxVersions on Gets do not match, using the first in the list ("
- + maxVersions +")");
- }
-
- if (g.getFilter() != null) {
- LOG.warn("filters not supported on gets");
- }
-
- rows[count] = g.getRow();
- count++;
- }
-
- String spec = buildMultiRowSpec(rows, maxVersions);
-
- return getResults(spec);
- }
-
- private Result[] getResults(String spec) throws IOException {
for (int i = 0; i < maxRetries; i++) {
Response response = client.get(spec, Constants.MIMETYPE_PROTOBUF);
int code = response.getCode();
switch (code) {
- case 200:
- CellSetModel model = new CellSetModel();
- model.getObjectFromMessage(response.getBody());
- Result[] results = buildResultFromModel(model);
- if (results.length > 0) {
- return results;
+ case 200:
+ CellSetModel model = new CellSetModel();
+ model.getObjectFromMessage(response.getBody());
+ Result[] results = buildResultFromModel(model);
+ if (results.length > 0) {
+ if (results.length > 1) {
+ LOG.warn("too many results for get (" + results.length + ")");
}
- // fall through
- case 404:
- return new Result[0];
-
- case 509:
- try {
- Thread.sleep(sleepTime);
- } catch (InterruptedException e) {
- }
- break;
- default:
- throw new IOException("get request returned " + code);
+ return results[0];
+ }
+ // fall through
+ case 404:
+ return new Result();
+ case 509:
+ try {
+ Thread.sleep(sleepTime);
+ } catch (InterruptedException e) { }
+ break;
+ default:
+ throw new IOException("get request returned " + code);
}
}
throw new IOException("get request timed out");
@@ -386,6 +306,10 @@
CellSetModel model = buildModelFromPut(put);
StringBuilder sb = new StringBuilder();
sb.append('/');
+ if (accessToken != null) {
+ sb.append(accessToken);
+ sb.append('/');
+ }
sb.append(Bytes.toStringBinary(name));
sb.append('/');
sb.append(Bytes.toStringBinary(put.getRow()));
@@ -440,6 +364,10 @@
// build path for multiput
StringBuilder sb = new StringBuilder();
sb.append('/');
+ if (accessToken != null) {
+ sb.append(accessToken);
+ sb.append('/');
+ }
sb.append(Bytes.toStringBinary(name));
sb.append("/$multiput"); // can be any nonexistent row
for (int i = 0; i < maxRetries; i++) {
@@ -505,6 +433,10 @@
}
StringBuffer sb = new StringBuffer();
sb.append('/');
+ if (accessToken != null) {
+ sb.append(accessToken);
+ sb.append('/');
+ }
sb.append(Bytes.toStringBinary(name));
sb.append('/');
sb.append("scanner");
@@ -643,16 +575,10 @@
throw new IOException("getRowOrBefore not supported");
}
- /**
- * @deprecated {@link RowLock} and associated operations are deprecated
- */
public RowLock lockRow(byte[] row) throws IOException {
throw new IOException("lockRow not implemented");
}
- /**
- * @deprecated {@link RowLock} and associated operations are deprecated
- */
public void unlockRow(RowLock rl) throws IOException {
throw new IOException("unlockRow not implemented");
}
@@ -665,6 +591,10 @@
CellSetModel model = buildModelFromPut(put);
StringBuilder sb = new StringBuilder();
sb.append('/');
+ if (accessToken != null) {
+ sb.append(accessToken);
+ sb.append('/');
+ }
sb.append(Bytes.toStringBinary(name));
sb.append('/');
sb.append(Bytes.toStringBinary(put.getRow()));
@@ -700,6 +630,10 @@
CellSetModel model = buildModelFromPut(put);
StringBuilder sb = new StringBuilder();
sb.append('/');
+ if (accessToken != null) {
+ sb.append(accessToken);
+ sb.append('/');
+ }
sb.append(Bytes.toStringBinary(name));
sb.append('/');
sb.append(Bytes.toStringBinary(row));
@@ -756,6 +690,11 @@
}
@Override
+ public Result[] get(List<Get> gets) throws IOException {
+ throw new IOException("get(List<Get>) not supported");
+ }
+
+ @Override
public <T extends CoprocessorProtocol> T coprocessorProxy(Class<T> protocol,
byte[] row) {
throw new
diff --git a/src/main/java/org/apache/hadoop/hbase/security/User.java b/src/main/java/org/apache/hadoop/hbase/security/User.java
index f15c470..eb516a4 100644
--- a/src/main/java/org/apache/hadoop/hbase/security/User.java
+++ b/src/main/java/org/apache/hadoop/hbase/security/User.java
@@ -22,7 +22,6 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.util.Methods;
import org.apache.hadoop.mapred.JobConf;
@@ -217,15 +216,12 @@
}
/**
- * Returns whether or not secure authentication is enabled for HBase. Note that
- * HBase security requires HDFS security to provide any guarantees, so this requires that
- * both <code>hbase.security.authentication</code> and <code>hadoop.security.authentication</code>
- * are set to <code>kerberos</code>.
+ * Returns whether or not secure authentication is enabled for HBase
+ * (whether <code>hbase.security.authentication</code> is set to
+ * <code>kerberos</code>.
*/
public static boolean isHBaseSecurityEnabled(Configuration conf) {
- return "kerberos".equalsIgnoreCase(conf.get(HBASE_SECURITY_CONF_KEY)) &&
- "kerberos".equalsIgnoreCase(
- conf.get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION));
+ return "kerberos".equalsIgnoreCase(conf.get(HBASE_SECURITY_CONF_KEY));
}
/* Concrete implementations */
diff --git a/src/main/java/org/apache/hadoop/hbase/util/ChecksumType.java b/src/main/java/org/apache/hadoop/hbase/util/ChecksumType.java
index 885625b..d2329e1 100644
--- a/src/main/java/org/apache/hadoop/hbase/util/ChecksumType.java
+++ b/src/main/java/org/apache/hadoop/hbase/util/ChecksumType.java
@@ -25,6 +25,9 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.util.ChecksumFactory;
+
/**
* Checksum types. The Checksum type is a one byte number
* that stores a representation of the checksum algorithm
@@ -67,7 +70,7 @@
ctor = ChecksumFactory.newConstructor(PURECRC32);
LOG.info("Checksum using " + PURECRC32);
} catch (Exception e) {
- LOG.trace(PURECRC32 + " not available.");
+ LOG.info(PURECRC32 + " not available.");
}
try {
// The default checksum class name is java.util.zip.CRC32.
@@ -77,7 +80,7 @@
LOG.info("Checksum can use " + JDKCRC);
}
} catch (Exception e) {
- LOG.trace(JDKCRC + " not available.");
+ LOG.warn(JDKCRC + " not available. ", e);
}
}
@@ -110,7 +113,7 @@
ctor = ChecksumFactory.newConstructor(PURECRC32C);
LOG.info("Checksum can use " + PURECRC32C);
} catch (Exception e) {
- LOG.trace(PURECRC32C + " not available.");
+ LOG.info(PURECRC32C + " not available. ");
}
}
diff --git a/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 18074ef..27b61a2 100644
--- a/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -151,7 +151,7 @@
*/
public static FSDataOutputStream create(FileSystem fs, Path path,
FsPermission perm, boolean overwrite) throws IOException {
- LOG.debug("Creating file=" + path + " with permission=" + perm);
+ LOG.debug("Creating file:" + path + "with permission:" + perm);
return fs.create(path, perm, overwrite,
fs.getConf().getInt("io.file.buffer.size", 4096),
@@ -1013,25 +1013,6 @@
}
/**
- * Given a particular region dir, return all the familydirs inside it
- *
- * @param fs A file system for the Path
- * @param regionDir Path to a specific region directory
- * @return List of paths to valid family directories in region dir.
- * @throws IOException
- */
- public static List<Path> getFamilyDirs(final FileSystem fs, final Path regionDir) throws IOException {
- // assumes we are in a region dir.
- FileStatus[] fds = fs.listStatus(regionDir, new FamilyDirFilter(fs));
- List<Path> familyDirs = new ArrayList<Path>(fds.length);
- for (FileStatus fdfs: fds) {
- Path fdPath = fdfs.getPath();
- familyDirs.add(fdPath);
- }
- return familyDirs;
- }
-
- /**
* Filter for HFiles that excludes reference files.
*/
public static class HFileFilter implements PathFilter {
diff --git a/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 8ef8be2..7fe81ca 100644
--- a/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -83,7 +83,6 @@
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
@@ -187,7 +186,6 @@
private boolean fixTableOrphans = false; // fix fs holes (missing .tableinfo)
private boolean fixVersionFile = false; // fix missing hbase.version file in hdfs
private boolean fixSplitParents = false; // fix lingering split parents
- private boolean fixReferenceFiles = false; // fix lingering reference store file
// limit checking/fixes to listed tables, if empty attempt to check/fix all
// -ROOT- and .META. are always checked
@@ -439,8 +437,6 @@
admin.setBalancerRunning(oldBalancer, false);
}
- offlineReferenceFileRepair();
-
// Print table summary
printTableSummary(tablesInfo);
return errors.summarize();
@@ -597,67 +593,6 @@
}
/**
- * Scan all the store file names to find any lingering reference files,
- * which refer to some none-exiting files. If "fix" option is enabled,
- * any lingering reference file will be sidelined if found.
- * <p>
- * Lingering reference file prevents a region from opening. It has to
- * be fixed before a cluster can start properly.
- */
- private void offlineReferenceFileRepair() throws IOException {
- Configuration conf = getConf();
- Path hbaseRoot = FSUtils.getRootDir(conf);
- FileSystem fs = hbaseRoot.getFileSystem(conf);
- Map<String, Path> allFiles = FSUtils.getTableStoreFilePathMap(fs, hbaseRoot);
- for (Path path: allFiles.values()) {
- boolean isReference = false;
- try {
- isReference = StoreFile.isReference(path);
- } catch (Throwable t) {
- // Ignore. Some files may not be store files at all.
- // For example, files under .oldlogs folder in .META.
- // Warning message is already logged by
- // StoreFile#isReference.
- }
- if (!isReference) continue;
-
- Path referredToFile = StoreFile.getReferredToFile(path);
- if (fs.exists(referredToFile)) continue; // good, expected
-
- // Found a lingering reference file
- errors.reportError(ERROR_CODE.LINGERING_REFERENCE_HFILE,
- "Found lingering reference file " + path);
- if (!shouldFixReferenceFiles()) continue;
-
- // Now, trying to fix it since requested
- boolean success = false;
- String pathStr = path.toString();
-
- // A reference file path should be like
- // ${hbase.rootdir}/table_name/region_id/family_name/referred_file.region_name
- // Up 3 directories to get the table folder.
- // So the file will be sidelined to a similar folder structure.
- int index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR);
- for (int i = 0; index > 0 && i < 3; i++) {
- index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR, index);
- }
- if (index > 0) {
- Path rootDir = getSidelineDir();
- Path dst = new Path(rootDir, pathStr.substring(index));
- fs.mkdirs(dst.getParent());
- LOG.info("Trying to sildeline reference file"
- + path + " to " + dst);
- setShouldRerun();
-
- success = fs.rename(path, dst);
- }
- if (!success) {
- LOG.error("Failed to sideline reference file " + path);
- }
- }
- }
-
- /**
* TODO -- need to add tests for this.
*/
private void reportEmptyMetaCells() {
@@ -2844,7 +2779,7 @@
MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE,
FIRST_REGION_STARTKEY_NOT_EMPTY, LAST_REGION_ENDKEY_NOT_EMPTY, DUPE_STARTKEYS,
HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE, DEGENERATE_REGION,
- ORPHAN_HDFS_REGION, LINGERING_SPLIT_PARENT, NO_TABLEINFO_FILE, LINGERING_REFERENCE_HFILE
+ ORPHAN_HDFS_REGION, LINGERING_SPLIT_PARENT, NO_TABLEINFO_FILE
}
public void clear();
public void report(String message);
@@ -3276,14 +3211,6 @@
return fixSplitParents;
}
- public void setFixReferenceFiles(boolean shouldFix) {
- fixReferenceFiles = shouldFix;
- }
-
- boolean shouldFixReferenceFiles() {
- return fixReferenceFiles;
- }
-
public boolean shouldIgnorePreCheckPermission() {
return ignorePreCheckPermission;
}
@@ -3395,7 +3322,6 @@
System.err.println(" -maxOverlapsToSideline <n> When fixing region overlaps, allow at most <n> regions to sideline per group. (n=" + DEFAULT_OVERLAPS_TO_SIDELINE +" by default)");
System.err.println(" -fixSplitParents Try to force offline split parents to be online.");
System.err.println(" -ignorePreCheckPermission ignore filesystem permission pre-check");
- System.err.println(" -fixReferenceFiles Try to offline lingering reference store files");
System.err.println("");
System.err.println(" Datafile Repair options: (expert features, use with caution!)");
@@ -3405,7 +3331,7 @@
System.err.println("");
System.err.println(" Metadata Repair shortcuts");
System.err.println(" -repair Shortcut for -fixAssignments -fixMeta -fixHdfsHoles " +
- "-fixHdfsOrphans -fixHdfsOverlaps -fixVersionFile -sidelineBigOverlaps -fixReferenceFiles");
+ "-fixHdfsOrphans -fixHdfsOverlaps -fixVersionFile -sidelineBigOverlaps");
System.err.println(" -repairHoles Shortcut for -fixAssignments -fixMeta -fixHdfsHoles");
setRetCode(-2);
@@ -3512,8 +3438,6 @@
checkCorruptHFiles = true;
} else if (cmd.equals("-sidelineCorruptHFiles")) {
sidelineCorruptHFiles = true;
- } else if (cmd.equals("-fixReferenceFiles")) {
- setFixReferenceFiles(true);
} else if (cmd.equals("-repair")) {
// this attempts to merge overlapping hdfs regions, needs testing
// under load
@@ -3526,7 +3450,6 @@
setSidelineBigOverlaps(true);
setFixSplitParents(false);
setCheckHdfs(true);
- setFixReferenceFiles(true);
} else if (cmd.equals("-repairHoles")) {
// this will make all missing hdfs regions available but may lose data
setFixHdfsHoles(true);
diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
index 43a9e97..84b79a6 100644
--- a/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
+++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
@@ -61,12 +61,6 @@
writeMyID(zkProperties);
QuorumPeerConfig zkConfig = new QuorumPeerConfig();
zkConfig.parseProperties(zkProperties);
-
- // login the zookeeper server principal (if using security)
- ZKUtil.loginServer(conf, "hbase.zookeeper.server.keytab.file",
- "hbase.zookeeper.server.kerberos.principal",
- zkConfig.getClientPortAddress().getHostName());
-
runZKServer(zkConfig);
} catch (Exception e) {
e.printStackTrace();
diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
index 3ca866c..57ffe86 100644
--- a/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -22,7 +22,6 @@
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.util.ArrayList;
-import java.util.LinkedList;
import java.util.List;
import org.apache.commons.logging.Log;
@@ -33,16 +32,11 @@
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooKeeper.States;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.CreateRequest;
-import org.apache.zookeeper.proto.SetDataRequest;
/**
* A zookeeper that can handle 'recoverable' errors.
@@ -496,61 +490,6 @@
}
}
- /**
- * Convert Iterable of {@link ZKOp} we got into the ZooKeeper.Op
- * instances to actually pass to multi (need to do this in order to appendMetaData).
- */
- private Iterable<Op> prepareZKMulti(Iterable<Op> ops)
- throws UnsupportedOperationException {
- if(ops == null) return null;
-
- List<Op> preparedOps = new LinkedList<Op>();
- for (Op op : ops) {
- if (op.getType() == ZooDefs.OpCode.create) {
- CreateRequest create = (CreateRequest)op.toRequestRecord();
- preparedOps.add(Op.create(create.getPath(), appendMetaData(create.getData()),
- create.getAcl(), create.getFlags()));
- } else if (op.getType() == ZooDefs.OpCode.delete) {
- // no need to appendMetaData for delete
- preparedOps.add(op);
- } else if (op.getType() == ZooDefs.OpCode.setData) {
- SetDataRequest setData = (SetDataRequest)op.toRequestRecord();
- preparedOps.add(Op.setData(setData.getPath(), appendMetaData(setData.getData()),
- setData.getVersion()));
- } else {
- throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName());
- }
- }
- return preparedOps;
- }
-
- /**
- * Run multiple operations in a transactional manner. Retry before throwing exception
- */
- public List<OpResult> multi(Iterable<Op> ops)
- throws KeeperException, InterruptedException {
- RetryCounter retryCounter = retryCounterFactory.create();
- Iterable<Op> multiOps = prepareZKMulti(ops);
- while (true) {
- try {
- return zk.multi(multiOps);
- } catch (KeeperException e) {
- switch (e.code()) {
- case CONNECTIONLOSS:
- case SESSIONEXPIRED:
- case OPERATIONTIMEOUT:
- retryOrThrow(retryCounter, e, "multi");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- retryCounter.useRetry();
- }
- }
-
private String findPreviousSequentialNode(String path)
throws KeeperException, InterruptedException {
int lastSlashIdx = path.lastIndexOf('/');
diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java
index 41cd25f..2ac04e9 100644
--- a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java
+++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java
@@ -21,7 +21,6 @@
import java.util.HashMap;
import java.util.HashSet;
-import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -30,7 +29,6 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
import org.apache.zookeeper.KeeperException;
/**
@@ -230,19 +228,16 @@
}
}
synchronized (this.cache) {
- List<ZKUtilOp> ops = new LinkedList<ZKUtilOp>();
if (settingToEnabled) {
- ops.add(ZKUtilOp.deleteNodeFailSilent(znode92));
+ ZKUtil.deleteNodeFailSilent(this.watcher, znode92);
}
else {
- ops.add(ZKUtilOp.setData(znode92, Bytes.toBytes(state.toString())));
+ ZKUtil.setData(this.watcher, znode92, Bytes.toBytes(state.toString()));
}
- // If not running multi-update either because of configuration or failure,
- // set the current format znode after the 0.92 format znode.
+ // Set the current format znode after the 0.92 format znode.
// This is so in the case of failure, the AssignmentManager is guaranteed to
// see the state was not applied, since it uses the current format znode internally.
- ops.add(ZKUtilOp.setData(znode, Bytes.toBytes(state.toString())));
- ZKUtil.multiOrSequential(this.watcher, ops, true);
+ ZKUtil.setData(this.watcher, znode, Bytes.toBytes(state.toString()));
this.cache.put(tableName, state);
}
}
@@ -297,16 +292,13 @@
public void setDeletedTable(final String tableName)
throws KeeperException {
synchronized (this.cache) {
- List<ZKUtilOp> ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.deleteNodeFailSilent(
- ZKUtil.joinZNode(this.watcher.masterTableZNode92, tableName)));
- // If not running multi-update either because of configuration or failure,
- // delete the current format znode after the 0.92 format znode. This is so in the case of
- // failure, the AssignmentManager is guaranteed to see the table was not deleted, since it
- // uses the current format znode internally.
- ops.add(ZKUtilOp.deleteNodeFailSilent(
- ZKUtil.joinZNode(this.watcher.masterTableZNode, tableName)));
- ZKUtil.multiOrSequential(this.watcher, ops, true);
+ ZKUtil.deleteNodeFailSilent(this.watcher,
+ ZKUtil.joinZNode(this.watcher.masterTableZNode92, tableName));
+ // Delete the current format znode after the 0.92 format znode.
+ // This is so in the case of failure, the AssignmentManager is guaranteed to
+ // see the table was not deleted, since it uses the current format znode internally.
+ ZKUtil.deleteNodeFailSilent(this.watcher,
+ ZKUtil.joinZNode(this.watcher.masterTableZNode, tableName));
if (this.cache.remove(tableName) == null) {
LOG.warn("Moving table " + tableName + " state to deleted but was " +
"already deleted");
diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 4de15b0..b95dd3a 100644
--- a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -24,21 +24,10 @@
import java.io.InputStreamReader;
import java.io.PrintWriter;
import java.net.InetSocketAddress;
-import java.net.InetAddress;
import java.net.Socket;
import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.LinkedList;
import java.util.List;
import java.util.Properties;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.security.auth.login.LoginException;
-import javax.security.auth.login.AppConfigurationEntry;
-import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.authentication.util.KerberosUtil;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
@@ -50,24 +39,15 @@
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.apache.zookeeper.Op;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.client.ZooKeeperSaslClient;
-import org.apache.zookeeper.server.ZooKeeperSaslServer;
-import org.apache.zookeeper.proto.CreateRequest;
-import org.apache.zookeeper.proto.DeleteRequest;
-import org.apache.zookeeper.proto.SetDataRequest;
/**
* Internal HBase utility class for ZooKeeper.
@@ -128,170 +108,6 @@
retry, retryIntervalMillis);
}
- /**
- * Log in the current zookeeper server process using the given configuration
- * keys for the credential file and login principal.
- *
- * <p><strong>This is only applicable when running on secure hbase</strong>
- * On regular HBase (without security features), this will safely be ignored.
- * </p>
- *
- * @param conf The configuration data to use
- * @param keytabFileKey Property key used to configure the path to the credential file
- * @param userNameKey Property key used to configure the login principal
- * @param hostname Current hostname to use in any credentials
- * @throws IOException underlying exception from SecurityUtil.login() call
- */
- public static void loginServer(Configuration conf, String keytabFileKey,
- String userNameKey, String hostname) throws IOException {
- login(conf, keytabFileKey, userNameKey, hostname,
- ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY,
- JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME);
- }
-
- /**
- * Log in the current zookeeper client using the given configuration
- * keys for the credential file and login principal.
- *
- * <p><strong>This is only applicable when running on secure hbase</strong>
- * On regular HBase (without security features), this will safely be ignored.
- * </p>
- *
- * @param conf The configuration data to use
- * @param keytabFileKey Property key used to configure the path to the credential file
- * @param userNameKey Property key used to configure the login principal
- * @param hostname Current hostname to use in any credentials
- * @throws IOException underlying exception from SecurityUtil.login() call
- */
- public static void loginClient(Configuration conf, String keytabFileKey,
- String userNameKey, String hostname) throws IOException {
- login(conf, keytabFileKey, userNameKey, hostname,
- ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
- JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME);
- }
-
- /**
- * Log in the current process using the given configuration keys for the
- * credential file and login principal.
- *
- * <p><strong>This is only applicable when running on secure hbase</strong>
- * On regular HBase (without security features), this will safely be ignored.
- * </p>
- *
- * @param conf The configuration data to use
- * @param keytabFileKey Property key used to configure the path to the credential file
- * @param userNameKey Property key used to configure the login principal
- * @param hostname Current hostname to use in any credentials
- * @param loginContextProperty property name to expose the entry name
- * @param loginContextName jaas entry name
- * @throws IOException underlying exception from SecurityUtil.login() call
- */
- private static void login(Configuration conf, String keytabFileKey,
- String userNameKey, String hostname,
- String loginContextProperty, String loginContextName)
- throws IOException {
- if (!isSecureZooKeeper(conf))
- return;
-
- // User has specified a jaas.conf, keep this one as the good one.
- // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf"
- if (System.getProperty("java.security.auth.login.config") != null)
- return;
-
- String keytabFilename = conf.get(keytabFileKey);
- String principalConfig = conf.get(userNameKey, System.getProperty("user.name"));
- String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname);
-
- // Initialize the "jaas.conf" for keyTab/principal,
- // If keyTab is not specified use the Ticket Cache.
- // and set the zookeeper login context name.
- JaasConfiguration jaasConf = new JaasConfiguration(loginContextName,
- keytabFilename, principalName);
- javax.security.auth.login.Configuration.setConfiguration(jaasConf);
- System.setProperty(loginContextProperty, loginContextName);
- }
-
- /**
- * A JAAS configuration that defines the login modules that we want to use for login.
- */
- private static class JaasConfiguration extends javax.security.auth.login.Configuration {
- private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME =
- "zookeeper-server-keytab-kerberos";
- private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME =
- "zookeeper-client-keytab-kerberos";
-
- private static final Map<String, String> BASIC_JAAS_OPTIONS =
- new HashMap<String,String>();
- static {
- String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG");
- if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
- BASIC_JAAS_OPTIONS.put("debug", "true");
- }
- }
-
- private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS =
- new HashMap<String,String>();
- static {
- KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
- KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
- KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
- KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
- }
-
- private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
- new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
- LoginModuleControlFlag.REQUIRED,
- KEYTAB_KERBEROS_OPTIONS);
-
- private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF =
- new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN};
-
- private javax.security.auth.login.Configuration baseConfig;
- private final String loginContextName;
- private final boolean useTicketCache;
- private final String keytabFile;
- private final String principal;
-
- public JaasConfiguration(String loginContextName, String principal) {
- this(loginContextName, principal, null, true);
- }
-
- public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
- this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
- }
-
- private JaasConfiguration(String loginContextName, String principal,
- String keytabFile, boolean useTicketCache) {
- try {
- this.baseConfig = javax.security.auth.login.Configuration.getConfiguration();
- } catch (SecurityException e) {
- this.baseConfig = null;
- }
- this.loginContextName = loginContextName;
- this.useTicketCache = useTicketCache;
- this.keytabFile = keytabFile;
- this.principal = principal;
- LOG.info("JaasConfiguration loginContextName=" + loginContextName +
- " principal=" + principal + " useTicketCache=" + useTicketCache +
- " keytabFile=" + keytabFile);
- }
-
- @Override
- public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
- if (loginContextName.equals(appName)) {
- if (!useTicketCache) {
- KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
- KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
- }
- KEYTAB_KERBEROS_OPTIONS.put("principal", principal);
- KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false");
- return KEYTAB_KERBEROS_CONF;
- }
- if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName);
- return(null);
- }
- }
-
//
// Helper methods
//
@@ -433,6 +249,9 @@
/**
* Check if the specified node exists. Sets no watches.
*
+ * Returns true if node exists, false if not. Returns an exception if there
+ * is an unexpected zookeeper exception.
+ *
* @param zkw zk reference
* @param znode path of node to watch
* @return version of the node if it exists, -1 if does not exist
@@ -881,29 +700,19 @@
*/
public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data)
throws KeeperException, KeeperException.NoNodeException {
- setData(zkw, (SetData)ZKUtilOp.setData(znode, data));
+ setData(zkw, znode, data, -1);
}
- private static void setData(ZooKeeperWatcher zkw, SetData setData)
- throws KeeperException, KeeperException.NoNodeException {
- SetDataRequest sd = (SetDataRequest)toZooKeeperOp(zkw, setData).toRequestRecord();
- setData(zkw, sd.getPath(), sd.getData(), sd.getVersion());
- }
-
- /**
- * Returns whether or not secure authentication is enabled
- * (whether <code>hbase.security.authentication</code> is set to
- * <code>kerberos</code>.
- */
public static boolean isSecureZooKeeper(Configuration conf) {
- // hbase shell need to use:
- // -Djava.security.auth.login.config=user-jaas.conf
- // since each user has a different jaas.conf
- if (System.getProperty("java.security.auth.login.config") != null)
- return true;
-
- // Master & RSs uses hbase.zookeeper.client.*
- return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication"));
+ // TODO: We need a better check for security enabled ZooKeeper. Currently
+ // the secure ZooKeeper client is set up using a supplied JaaS
+ // configuration file. But if the system property for the JaaS
+ // configuration file is set, this may not be an exclusive indication
+ // that HBase should set ACLs on znodes. As an alternative, we could do
+ // this more like Hadoop and build a JaaS configuration programmatically
+ // based on a site conf setting. The scope of such a change will be
+ // addressed in HBASE-4791.
+ return (System.getProperty("java.security.auth.login.config") != null);
}
private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {
@@ -1087,20 +896,14 @@
* @throws KeeperException if unexpected zookeeper exception
*/
public static void createAndFailSilent(ZooKeeperWatcher zkw,
- String znode) throws KeeperException {
- createAndFailSilent(zkw,
- (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, new byte[0]));
- }
-
- private static void createAndFailSilent(ZooKeeperWatcher zkw, CreateAndFailSilent cafs)
+ String znode)
throws KeeperException {
- CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord();
- String znode = create.getPath();
try {
RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper();
waitForZKConnectionIfAuthenticating(zkw);
if (zk.exists(znode, false) == null) {
- zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags()));
+ zk.create(znode, new byte[0], createACL(zkw,znode),
+ CreateMode.PERSISTENT);
}
} catch(KeeperException.NodeExistsException nee) {
} catch(KeeperException.NoAuthException nee){
@@ -1186,15 +989,8 @@
*/
public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node)
throws KeeperException {
- deleteNodeFailSilent(zkw,
- (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node));
- }
-
- private static void deleteNodeFailSilent(ZooKeeperWatcher zkw,
- DeleteNodeFailSilent dnfs) throws KeeperException {
- DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord();
try {
- zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion());
+ zkw.getRecoverableZooKeeper().delete(node, -1);
} catch(KeeperException.NoNodeException nne) {
} catch(InterruptedException ie) {
zkw.interruptedException(ie);
@@ -1242,209 +1038,6 @@
}
}
- /**
- * Represents an action taken by ZKUtil, e.g. createAndFailSilent.
- * These actions are higher-level than {@link ZKOp} actions, which represent
- * individual actions in the ZooKeeper API, like create.
- */
- public abstract static class ZKUtilOp {
- private String path;
-
- private ZKUtilOp(String path) {
- this.path = path;
- }
-
- /**
- * @return a createAndFailSilent ZKUtilOp
- */
- public static ZKUtilOp createAndFailSilent(String path, byte[] data) {
- return new CreateAndFailSilent(path, data);
- }
-
- /**
- * @return a deleteNodeFailSilent ZKUtilOP
- */
- public static ZKUtilOp deleteNodeFailSilent(String path) {
- return new DeleteNodeFailSilent(path);
- }
-
- /**
- * @return a setData ZKUtilOp
- */
- public static ZKUtilOp setData(String path, byte [] data) {
- return new SetData(path, data);
- }
-
- /**
- * @return path to znode where the ZKOp will occur
- */
- public String getPath() {
- return path;
- }
-
- /**
- * ZKUtilOp representing createAndFailSilent in ZooKeeper
- * (attempt to create node, ignore error if already exists)
- */
- public static class CreateAndFailSilent extends ZKUtilOp {
- private byte [] data;
-
- private CreateAndFailSilent(String path, byte [] data) {
- super(path);
- this.data = data;
- }
-
- public byte[] getData() {
- return data;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof CreateAndFailSilent)) return false;
-
- CreateAndFailSilent op = (CreateAndFailSilent) o;
- return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
- }
- }
-
- /**
- * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper
- * (attempt to delete node, ignore error if node doesn't exist)
- */
- public static class DeleteNodeFailSilent extends ZKUtilOp {
- private DeleteNodeFailSilent(String path) {
- super(path);
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof DeleteNodeFailSilent)) return false;
-
- return super.equals(o);
- }
- }
-
- /**
- * @return ZKUtilOp representing setData in ZooKeeper
- */
- public static class SetData extends ZKUtilOp {
- private byte [] data;
-
- private SetData(String path, byte [] data) {
- super(path);
- this.data = data;
- }
-
- public byte[] getData() {
- return data;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof SetData)) return false;
-
- SetData op = (SetData) o;
- return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
- }
- }
- }
-
- /**
- * Convert from ZKUtilOp to ZKOp
- */
- private static Op toZooKeeperOp(ZooKeeperWatcher zkw, ZKUtilOp op)
- throws UnsupportedOperationException {
- if(op == null) return null;
-
- if (op instanceof CreateAndFailSilent) {
- CreateAndFailSilent cafs = (CreateAndFailSilent)op;
- return Op.create(cafs.getPath(), cafs.getData(), createACL(zkw, cafs.getPath()),
- CreateMode.PERSISTENT);
- } else if (op instanceof DeleteNodeFailSilent) {
- DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op;
- return Op.delete(dnfs.getPath(), -1);
- } else if (op instanceof SetData) {
- SetData sd = (SetData)op;
- return Op.setData(sd.getPath(), sd.getData(), -1);
- } else {
- throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
- + op.getClass().getName());
- }
- }
-
- /**
- * If hbase.zookeeper.useMulti is true, use ZooKeeper's multi-update functionality.
- * Otherwise, run the list of operations sequentially.
- *
- * If all of the following are true:
- * - runSequentialOnMultiFailure is true
- * - hbase.zookeeper.useMulti is true
- * - on calling multi, we get a ZooKeeper exception that can be handled by a sequential call(*)
- * Then:
- * - we retry the operations one-by-one (sequentially)
- *
- * Note *: an example is receiving a NodeExistsException from a "create" call. Without multi,
- * a user could call "createAndFailSilent" to ensure that a node exists if they don't care who
- * actually created the node (i.e. the NodeExistsException from ZooKeeper is caught).
- * This will cause all operations in the multi to fail, however, because
- * the NodeExistsException that zk.create throws will fail the multi transaction.
- * In this case, if the previous conditions hold, the commands are run sequentially, which should
- * result in the correct final state, but means that the operations will not run atomically.
- *
- * @throws KeeperException
- */
- public static void multiOrSequential(ZooKeeperWatcher zkw, List<ZKUtilOp> ops,
- boolean runSequentialOnMultiFailure) throws KeeperException {
- if (ops == null) return;
- boolean useMulti = zkw.getConfiguration().getBoolean(HConstants.ZOOKEEPER_USEMULTI, false);
-
- if (useMulti) {
- List<Op> zkOps = new LinkedList<Op>();
- for (ZKUtilOp op : ops) {
- zkOps.add(toZooKeeperOp(zkw, op));
- }
- try {
- zkw.getRecoverableZooKeeper().multi(zkOps);
- } catch (KeeperException ke) {
- switch (ke.code()) {
- case NODEEXISTS:
- case NONODE:
- case BADVERSION:
- case NOAUTH:
- // if we get an exception that could be solved by running sequentially
- // (and the client asked us to), then break out and run sequentially
- if (runSequentialOnMultiFailure) {
- LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "."
- + " Attempting to run operations sequentially because"
- + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + ".");
- break;
- }
- default:
- throw ke;
- }
- } catch (InterruptedException ie) {
- zkw.interruptedException(ie);
- }
- }
-
- // run sequentially
- for (ZKUtilOp op : ops) {
- if (op instanceof CreateAndFailSilent) {
- createAndFailSilent(zkw, (CreateAndFailSilent)op);
- } else if (op instanceof DeleteNodeFailSilent) {
- deleteNodeFailSilent(zkw, (DeleteNodeFailSilent)op);
- } else if (op instanceof SetData) {
- setData(zkw, (SetData)op);
- } else {
- throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
- + op.getClass().getName());
- }
- }
- }
-
//
// ZooKeeper cluster information
//
diff --git a/src/main/resources/hbase-default.xml b/src/main/resources/hbase-default.xml
index 3efa851..f73c477 100644
--- a/src/main/resources/hbase-default.xml
+++ b/src/main/resources/hbase-default.xml
@@ -690,17 +690,6 @@
for more information.
</description>
</property>
- <property>
- <name>hbase.zookeeper.useMulti</name>
- <value>false</value>
- <description>Instructs HBase to make use of ZooKeeper's multi-update functionality.
- This allows certain ZooKeeper operations to complete more quickly and prevents some issues
- with rare ZooKeeper failure scenarios (see the release note of HBASE-6710 for an example).
- IMPORTANT: only set this to true if all ZooKeeper servers in the cluster are on version 3.4+
- and will not be downgraded. ZooKeeper versions before 3.4 do not support multi-update and will
- not fail gracefully if multi-update is invoked (see ZOOKEEPER-1495).
- </description>
- </property>
<!-- End of properties used to generate ZooKeeper host:port quorum list. -->
<!--
diff --git a/src/test/java/org/apache/hadoop/hbase/ClassTestFinder.java b/src/test/java/org/apache/hadoop/hbase/ClassTestFinder.java
index f40aa79..0fefcd5 100644
--- a/src/test/java/org/apache/hadoop/hbase/ClassTestFinder.java
+++ b/src/test/java/org/apache/hadoop/hbase/ClassTestFinder.java
@@ -50,7 +50,7 @@
return new Class<?>[0];
}
- public static class TestFileNameFilter implements FileNameFilter {
+ private static class TestFileNameFilter implements FileNameFilter {
private static final Pattern hadoopCompactRe =
Pattern.compile("hbase-hadoop\\d?-compat");
@@ -68,7 +68,7 @@
* - one or more of its methods is annotated with org.junit.Test OR
* - the class is annotated with Suite.SuiteClasses
* */
- public static class TestClassFilter implements ClassFilter {
+ private static class TestClassFilter implements ClassFilter {
private Class<?> categoryAnnotation = null;
public TestClassFilter(Class<?> categoryAnnotation) {
this.categoryAnnotation = categoryAnnotation;
diff --git a/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 8ecdeb2..998ad15 100644
--- a/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -39,6 +39,7 @@
import java.util.NavigableSet;
import java.util.Random;
import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -1014,36 +1015,6 @@
}
/**
- * Load table of multiple column families with rows from 'aaa' to 'zzz'.
- * @param t Table
- * @param f Array of Families to load
- * @return Count of rows loaded.
- * @throws IOException
- */
- public int loadTable(final HTable t, final byte[][] f) throws IOException {
- t.setAutoFlush(false);
- byte[] k = new byte[3];
- int rowCount = 0;
- for (byte b1 = 'a'; b1 <= 'z'; b1++) {
- for (byte b2 = 'a'; b2 <= 'z'; b2++) {
- for (byte b3 = 'a'; b3 <= 'z'; b3++) {
- k[0] = b1;
- k[1] = b2;
- k[2] = b3;
- Put put = new Put(k);
- for (int i = 0; i < f.length; i++) {
- put.add(f[i], null, k);
- }
- t.put(put);
- rowCount++;
- }
- }
- }
- t.flushCommits();
- return rowCount;
- }
-
- /**
* Load region with rows from 'aaa' to 'zzz'.
* @param r Region
* @param f Family
@@ -1109,7 +1080,7 @@
*/
public int createMultiRegions(HTable table, byte[] columnFamily)
throws IOException {
- return createMultiRegions(table, columnFamily, true);
+ return createMultiRegions(getConfiguration(), table, columnFamily);
}
public static final byte[][] KEYS = {
@@ -1126,16 +1097,16 @@
/**
* Creates many regions names "aaa" to "zzz".
- *
+ * @param c Configuration to use.
* @param table The table to use for the data.
* @param columnFamily The family to insert the data into.
- * @param cleanupFS True if a previous region should be remove from the FS
* @return count of regions created.
* @throws IOException When creating the regions fails.
*/
- public int createMultiRegions(HTable table, byte[] columnFamily, boolean cleanupFS)
+ public int createMultiRegions(final Configuration c, final HTable table,
+ final byte[] columnFamily)
throws IOException {
- return createMultiRegions(getConfiguration(), table, columnFamily, KEYS, cleanupFS);
+ return createMultiRegions(c, table, columnFamily, KEYS);
}
/**
@@ -1163,12 +1134,7 @@
}
public int createMultiRegions(final Configuration c, final HTable table,
- final byte[] columnFamily, byte [][] startKeys) throws IOException {
- return createMultiRegions(c, table, columnFamily, startKeys, true);
- }
-
- public int createMultiRegions(final Configuration c, final HTable table,
- final byte[] columnFamily, byte [][] startKeys, boolean cleanupFS)
+ final byte[] columnFamily, byte [][] startKeys)
throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
@@ -1182,9 +1148,6 @@
// and end key. Adding the custom regions below adds those blindly,
// including the new start region from empty to "bbb". lg
List<byte[]> rows = getMetaTableRows(htd.getName());
- String regionToDeleteInFS = table
- .getRegionsInRange(Bytes.toBytes(""), Bytes.toBytes("")).get(0)
- .getRegionInfo().getEncodedName();
List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
// add custom ones
int count = 0;
@@ -1206,14 +1169,6 @@
Bytes.toStringBinary(row));
meta.delete(new Delete(row));
}
- if (cleanupFS) {
- // see HBASE-7417 - this confused TestReplication
- // remove the "old" region from FS
- Path tableDir = new Path(getDefaultRootDirPath().toString()
- + System.getProperty("file.separator") + htd.getNameAsString()
- + System.getProperty("file.separator") + regionToDeleteInFS);
- getDFSCluster().getFileSystem().delete(tableDir);
- }
// flush cache of regions
HConnection conn = table.getConnection();
conn.clearRegionCache();
diff --git a/src/test/java/org/apache/hadoop/hbase/IntegrationTestRebalanceAndKillServers.java b/src/test/java/org/apache/hadoop/hbase/IntegrationTestRebalanceAndKillServers.java
new file mode 100644
index 0000000..c6d761e
--- /dev/null
+++ b/src/test/java/org/apache/hadoop/hbase/IntegrationTestRebalanceAndKillServers.java
@@ -0,0 +1,100 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ChaosMonkey;
+import org.apache.hadoop.hbase.util.LoadTestTool;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.ChaosMonkey.Action;
+import org.apache.hadoop.hbase.util.ChaosMonkey.RestartActiveMaster;
+import org.apache.hadoop.hbase.util.ChaosMonkey.RestartRandomRs;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import com.google.common.collect.Lists;
+
+/**
+ * A system test which does large data ingestion and verify using {@link LoadTestTool},
+ * while killing the region servers and the master(s) randomly. You can configure how long
+ * should the load test run by using "hbase.IntegrationTestRebalanceAndKillServers s.runtime"
+ * configuration parameter.
+ */
+@Category(IntegrationTests.class)
+public class IntegrationTestRebalanceAndKillServers extends IngestIntegrationTestBase {
+ private static final int NUM_SLAVES_BASE = 4; // number of slaves for the smallest cluster
+ private static final long DEFAULT_RUN_TIME = 5 * 60 * 1000; // run for 5 min by default
+
+ private static final long KILL_SERVICE_EVERY_MS = 45 * 1000;
+ private static final int SERVER_PER_MASTER_KILL = 3;
+ private static final long KILL_SERVER_FOR_MS = 5 * 1000;
+ private static final long KILL_MASTER_FOR_MS = 100;
+
+ private static final long UNBALANCE_REGIONS_EVERY_MS = 30 * 1000;
+ /** @see ChaosMonkey.UnbalanceRegionsAction#UnbalanceRegionsAction(double, double) */
+ private static final double UNBALANCE_TO_FRC_OF_SERVERS = 0.5;
+ /** @see ChaosMonkey.UnbalanceRegionsAction#UnbalanceRegionsAction(double, double) */
+ private static final double UNBALANCE_FRC_OF_REGIONS = 0.5;
+
+ private static final long BALANCE_REGIONS_EVERY_MS = 10 * 1000;
+
+ private ChaosMonkey monkey;
+
+ @Before
+ @SuppressWarnings("unchecked")
+ public void setUp() throws Exception {
+ super.setUp(NUM_SLAVES_BASE);
+
+ ChaosMonkey.Policy killPolicy = new ChaosMonkey.PeriodicRandomActionPolicy(
+ KILL_SERVICE_EVERY_MS,
+ new Pair<Action,Integer>(new ChaosMonkey.RestartActiveMaster(KILL_MASTER_FOR_MS), 1),
+ new Pair<Action,Integer>(new ChaosMonkey.RestartRandomRs(KILL_SERVER_FOR_MS), SERVER_PER_MASTER_KILL));
+
+ ChaosMonkey.Policy unbalancePolicy = new ChaosMonkey.PeriodicRandomActionPolicy(
+ UNBALANCE_REGIONS_EVERY_MS,
+ new ChaosMonkey.UnbalanceRegionsAction(UNBALANCE_FRC_OF_REGIONS, UNBALANCE_TO_FRC_OF_SERVERS));
+
+ ChaosMonkey.Policy balancePolicy = new ChaosMonkey.PeriodicRandomActionPolicy(
+ BALANCE_REGIONS_EVERY_MS, new ChaosMonkey.ForceBalancerAction());
+
+ monkey = new ChaosMonkey(util, killPolicy, unbalancePolicy, balancePolicy);
+ monkey.start();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (monkey != null) {
+ monkey.stop("tearDown");
+ monkey.waitForStop();
+ }
+ super.tearDown();
+ }
+
+ @Test
+ public void testDataIngest() throws Exception {
+ runIngestTest(DEFAULT_RUN_TIME, 2500, 10, 100, 20);
+ }
+}
diff --git a/src/test/java/org/apache/hadoop/hbase/IntegrationTestsDriver.java b/src/test/java/org/apache/hadoop/hbase/IntegrationTestsDriver.java
index a6f3760..3b233ce 100644
--- a/src/test/java/org/apache/hadoop/hbase/IntegrationTestsDriver.java
+++ b/src/test/java/org/apache/hadoop/hbase/IntegrationTestsDriver.java
@@ -21,7 +21,6 @@
import java.io.IOException;
import java.util.List;
import java.util.Set;
-import java.util.regex.Pattern;
import org.apache.commons.cli.CommandLine;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
@@ -39,55 +38,30 @@
* already deployed distributed cluster.
*/
public class IntegrationTestsDriver extends AbstractHBaseTool {
- private static final String TESTS_ARG = "test";
private static final Log LOG = LogFactory.getLog(IntegrationTestsDriver.class);
- private IntegrationTestFilter intTestFilter = new IntegrationTestFilter();
public static void main(String[] args) throws Exception {
int ret = ToolRunner.run(new IntegrationTestsDriver(), args);
System.exit(ret);
}
- private class IntegrationTestFilter extends ClassTestFinder.TestClassFilter {
- private Pattern testFilterRe = Pattern.compile(".*");
- public IntegrationTestFilter() {
- super(IntegrationTests.class);
- }
-
- public void setPattern(String pattern) {
- testFilterRe = Pattern.compile(pattern);
- }
-
- @Override
- public boolean isCandidateClass(Class<?> c) {
- return super.isCandidateClass(c) && testFilterRe.matcher(c.getName()).find();
- }
- }
-
@Override
protected void addOptions() {
- addOptWithArg(TESTS_ARG, "a Java regular expression to filter tests on");
}
@Override
protected void processOptions(CommandLine cmd) {
- String testFilterString = cmd.getOptionValue(TESTS_ARG, null);
- if (testFilterString != null) {
- intTestFilter.setPattern(testFilterString);
- }
}
/**
- * Returns test classes annotated with @Category(IntegrationTests.class),
- * according to the filter specific on the command line (if any).
+ * Returns test classes annotated with @Category(IntegrationTests.class)
*/
private Class<?>[] findIntegrationTestClasses()
throws ClassNotFoundException, LinkageError, IOException {
- ClassTestFinder.TestFileNameFilter nameFilter = new ClassTestFinder.TestFileNameFilter();
- ClassFinder classFinder = new ClassFinder(nameFilter, intTestFilter);
- Set<Class<?>> classes = classFinder.findClasses(true);
- return classes.toArray(new Class<?>[classes.size()]);
- }
+ ClassTestFinder classFinder = new ClassTestFinder(IntegrationTests.class);
+ Set<Class<?>> classes = classFinder.findClasses(true);
+ return classes.toArray(new Class<?>[classes.size()]);
+ }
@Override
@@ -104,4 +78,4 @@
return result.wasSuccessful() ? 0 : 1;
}
-}
+}
\ No newline at end of file
diff --git a/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java b/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
index 4679cf2..a1992c3 100644
--- a/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
+++ b/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
@@ -54,14 +54,13 @@
private static final byte [] TABLENAME = Bytes.toBytes("t");
private static final byte [] FAMILY = Bytes.toBytes("f");
private static final int COUNT_OF_REGIONS = HBaseTestingUtility.KEYS.length;
- private static final int NB_SLAVES = 5;
/**
* Spin up a cluster with a bunch of regions on it.
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
- TEST_UTIL.startMiniCluster(NB_SLAVES);
+ TEST_UTIL.startMiniCluster(5);
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
HTableDescriptor htd = new HTableDescriptor(TABLENAME);
@@ -74,25 +73,14 @@
createTableDescriptor(fs, FSUtils.getRootDir(TEST_UTIL.getConfiguration()), htd);
// Assign out the regions we just created.
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
- MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
admin.disableTable(TABLENAME);
admin.enableTable(TABLENAME);
- boolean ready = false;
- while (!ready) {
- ZKAssign.blockUntilNoRIT(zkw);
- // Assert that every regionserver has some regions on it, else invoke the balancer.
- ready = true;
- for (int i = 0; i < NB_SLAVES; i++) {
- HRegionServer hrs = cluster.getRegionServer(i);
- if (hrs.getOnlineRegions().isEmpty()) {
- ready = false;
- break;
- }
- }
- if (!ready) {
- admin.balancer();
- Thread.sleep(100);
- }
+ ZKAssign.blockUntilNoRIT(zkw);
+ // Assert that every regionserver has some regions on it.
+ MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
+ for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
+ HRegionServer hrs = cluster.getRegionServer(i);
+ Assert.assertFalse(hrs.getOnlineRegions().isEmpty());
}
}
diff --git a/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
index ec04f4d..5574b7f 100644
--- a/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
+++ b/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
@@ -218,10 +218,9 @@
* Wait until all the regions are assigned.
*/
private void waitForAllRegionsAssigned() throws IOException {
- int totalRegions = HBaseTestingUtility.KEYS.length+2;
- while (getRegionCount() < totalRegions) {
+ while (getRegionCount() < 22) {
// while (!cluster.getMaster().allRegionsAssigned()) {
- LOG.debug("Waiting for there to be "+ totalRegions +" regions, but there are " + getRegionCount() + " right now.");
+ LOG.debug("Waiting for there to be 22 regions, but there are " + getRegionCount() + " right now.");
try {
Thread.sleep(200);
} catch (InterruptedException e) {}
diff --git a/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java b/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
index e1dd0c5..f6142f2 100644
--- a/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
+++ b/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
@@ -35,7 +35,7 @@
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -55,7 +55,7 @@
* Test that the {@link HFileArchiver} correctly removes all the parts of a region when cleaning up
* a region
*/
-@Category(MediumTests.class)
+@Category(LargeTests.class)
public class TestHFileArchiving {
private static final String STRING_TABLE_NAME = "test_table";
@@ -230,7 +230,15 @@
// then get the current store files
Path regionDir = region.getRegionDir();
- List<String> storeFiles = getRegionStoreFiles(fs, regionDir);
+ List<String> storeFiles = getAllFileNames(fs, regionDir);
+ // remove all the non-storefile named files for the region
+ for (int i = 0; i < storeFiles.size(); i++) {
+ String file = storeFiles.get(i);
+ if (file.contains(HRegion.REGIONINFO_FILE) || file.contains("hlog")) {
+ storeFiles.remove(i--);
+ }
+ }
+ storeFiles.remove(HRegion.REGIONINFO_FILE);
// then delete the table so the hfiles get archived
UTIL.deleteTable(TABLE_NAME);
@@ -254,66 +262,6 @@
archivedFiles.containsAll(storeFiles));
}
- /**
- * Test that the store files are archived when a column family is removed.
- * @throws Exception
- */
- @Test
- public void testArchiveOnTableFamilyDelete() throws Exception {
- List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(TABLE_NAME);
- // make sure we only have 1 region serving this table
- assertEquals(1, servingRegions.size());
- HRegion region = servingRegions.get(0);
-
- // get the parent RS and monitor
- HRegionServer hrs = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
- FileSystem fs = hrs.getFileSystem();
-
- // put some data on the region
- LOG.debug("-------Loading table");
- UTIL.loadRegion(region, TEST_FAM);
-
- // get the hfiles in the region
- List<HRegion> regions = hrs.getOnlineRegions(TABLE_NAME);
- assertEquals("More that 1 region for test table.", 1, regions.size());
-
- region = regions.get(0);
- // wait for all the compactions to complete
- region.waitForFlushesAndCompactions();
-
- // disable table to prevent new updates
- UTIL.getHBaseAdmin().disableTable(TABLE_NAME);
- LOG.debug("Disabled table");
-
- // remove all the files from the archive to get a fair comparison
- clearArchiveDirectory();
-
- // then get the current store files
- Path regionDir = region.getRegionDir();
- List<String> storeFiles = getRegionStoreFiles(fs, regionDir);
-
- // then delete the table so the hfiles get archived
- UTIL.getHBaseAdmin().deleteColumn(TABLE_NAME, TEST_FAM);
-
- // then get the files in the archive directory.
- Path archiveDir = HFileArchiveUtil.getArchivePath(UTIL.getConfiguration());
- List<String> archivedFiles = getAllFileNames(fs, archiveDir);
- Collections.sort(storeFiles);
- Collections.sort(archivedFiles);
-
- LOG.debug("Store files:");
- for (int i = 0; i < storeFiles.size(); i++) {
- LOG.debug(i + " - " + storeFiles.get(i));
- }
- LOG.debug("Archive files:");
- for (int i = 0; i < archivedFiles.size(); i++) {
- LOG.debug(i + " - " + archivedFiles.get(i));
- }
-
- assertTrue("Archived files are missing some of the store files!",
- archivedFiles.containsAll(storeFiles));
- }
-
private void clearArchiveDirectory() throws IOException {
UTIL.getTestFileSystem().delete(new Path(UTIL.getDefaultRootDirPath(), ".archive"), true);
}
@@ -342,18 +290,4 @@
}
return fileNames;
}
-
- private List<String> getRegionStoreFiles(final FileSystem fs, final Path regionDir)
- throws IOException {
- List<String> storeFiles = getAllFileNames(fs, regionDir);
- // remove all the non-storefile named files for the region
- for (int i = 0; i < storeFiles.size(); i++) {
- String file = storeFiles.get(i);
- if (file.contains(HRegion.REGIONINFO_FILE) || file.contains("hlog")) {
- storeFiles.remove(i--);
- }
- }
- storeFiles.remove(HRegion.REGIONINFO_FILE);
- return storeFiles;
- }
-}
+}
\ No newline at end of file
diff --git a/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java b/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
index b89ef18..bea1fe1 100644
--- a/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
+++ b/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
@@ -60,7 +60,6 @@
static final String cpName4 = "TestCP4";
static final String cpName5 = "TestCP5";
static final String cpName6 = "TestCP6";
- static final String cpNameInvalid = "TestCPInvalid";
private static Class<?> regionCoprocessor1 = ColumnAggregationEndpoint.class;
private static Class<?> regionCoprocessor2 = GenericEndpoint.class;
@@ -69,6 +68,14 @@
private static final String[] regionServerSystemCoprocessors =
new String[]{
+ regionCoprocessor1.getSimpleName(),
+ regionServerCoprocessor.getSimpleName()
+ };
+
+ private static final String[] regionServerSystemAndUserCoprocessors =
+ new String[] {
+ regionCoprocessor1.getSimpleName(),
+ regionCoprocessor2.getSimpleName(),
regionServerCoprocessor.getSimpleName()
};
@@ -200,18 +207,16 @@
new Path(fs.getUri().toString() + Path.SEPARATOR));
String jarFileOnHDFS1 = fs.getUri().toString() + Path.SEPARATOR +
jarFile1.getName();
- Path pathOnHDFS1 = new Path(jarFileOnHDFS1);
assertTrue("Copy jar file to HDFS failed.",
- fs.exists(pathOnHDFS1));
+ fs.exists(new Path(jarFileOnHDFS1)));
LOG.info("Copied jar file to HDFS: " + jarFileOnHDFS1);
fs.copyFromLocalFile(new Path(jarFile2.getPath()),
new Path(fs.getUri().toString() + Path.SEPARATOR));
String jarFileOnHDFS2 = fs.getUri().toString() + Path.SEPARATOR +
jarFile2.getName();
- Path pathOnHDFS2 = new Path(jarFileOnHDFS2);
assertTrue("Copy jar file to HDFS failed.",
- fs.exists(pathOnHDFS2));
+ fs.exists(new Path(jarFileOnHDFS2)));
LOG.info("Copied jar file to HDFS: " + jarFileOnHDFS2);
// create a table that references the coprocessors
@@ -223,78 +228,41 @@
// with configuration values
htd.setValue("COPROCESSOR$2", jarFileOnHDFS2.toString() + "|" + cpName2 +
"|" + Coprocessor.PRIORITY_USER + "|k1=v1,k2=v2,k3=v3");
- // same jar but invalid class name (should fail to load this class)
- htd.setValue("COPROCESSOR$3", jarFileOnHDFS2.toString() + "|" + cpNameInvalid +
- "|" + Coprocessor.PRIORITY_USER);
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
if (admin.tableExists(tableName)) {
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
- CoprocessorHost.classLoadersCache.clear();
- byte[] startKey = {10, 63};
- byte[] endKey = {12, 43};
- admin.createTable(htd, startKey, endKey, 4);
+ admin.createTable(htd);
waitForTable(htd.getName());
// verify that the coprocessors were loaded
- boolean foundTableRegion=false;
- boolean found_invalid = true, found1 = true, found2 = true, found2_k1 = true,
- found2_k2 = true, found2_k3 = true;
- Map<HRegion, Set<ClassLoader>> regionsActiveClassLoaders =
- new HashMap<HRegion, Set<ClassLoader>>();
+ boolean found1 = false, found2 = false, found2_k1 = false,
+ found2_k2 = false, found2_k3 = false;
MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster();
for (HRegion region:
hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionNameAsString().startsWith(tableName)) {
- foundTableRegion = true;
CoprocessorEnvironment env;
env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName1);
- found1 = found1 && (env != null);
- env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName2);
- found2 = found2 && (env != null);
if (env != null) {
- Configuration conf = env.getConfiguration();
- found2_k1 = found2_k1 && (conf.get("k1") != null);
- found2_k2 = found2_k2 && (conf.get("k2") != null);
- found2_k3 = found2_k3 && (conf.get("k3") != null);
- } else {
- found2_k1 = found2_k2 = found2_k3 = false;
+ found1 = true;
}
- env = region.getCoprocessorHost().findCoprocessorEnvironment(cpNameInvalid);
- found_invalid = found_invalid && (env != null);
-
- regionsActiveClassLoaders
- .put(region, ((CoprocessorHost) region.getCoprocessorHost()).getExternalClassLoaders());
+ env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName2);
+ if (env != null) {
+ found2 = true;
+ Configuration conf = env.getConfiguration();
+ found2_k1 = conf.get("k1") != null;
+ found2_k2 = conf.get("k2") != null;
+ found2_k3 = conf.get("k3") != null;
+ }
}
}
-
- assertTrue("No region was found for table " + tableName, foundTableRegion);
assertTrue("Class " + cpName1 + " was missing on a region", found1);
assertTrue("Class " + cpName2 + " was missing on a region", found2);
- //an invalid CP class name is defined for this table, validate that it is not loaded
- assertFalse("Class " + cpNameInvalid + " was found on a region", found_invalid);
assertTrue("Configuration key 'k1' was missing on a region", found2_k1);
assertTrue("Configuration key 'k2' was missing on a region", found2_k2);
assertTrue("Configuration key 'k3' was missing on a region", found2_k3);
- // check if CP classloaders are cached
- assertTrue(jarFileOnHDFS1 + " was not cached",
- CoprocessorHost.classLoadersCache.containsKey(pathOnHDFS1));
- assertTrue(jarFileOnHDFS2 + " was not cached",
- CoprocessorHost.classLoadersCache.containsKey(pathOnHDFS2));
- //two external jar used, should be one classloader per jar
- assertEquals("The number of cached classloaders should be equal to the number" +
- " of external jar files",
- 2, CoprocessorHost.classLoadersCache.size());
- //check if region active classloaders are shared across all RS regions
- Set<ClassLoader> externalClassLoaders = new HashSet<ClassLoader>(
- CoprocessorHost.classLoadersCache.values());
- for (Map.Entry<HRegion, Set<ClassLoader>> regionCP : regionsActiveClassLoaders.entrySet()) {
- assertTrue("Some CP classloaders for region " + regionCP.getKey() + " are not cached."
- + " ClassLoader Cache:" + externalClassLoaders
- + " Region ClassLoaders:" + regionCP.getValue(),
- externalClassLoaders.containsAll(regionCP.getValue()));
- }
}
@Test
@@ -456,8 +424,6 @@
File outerJarFile = new File(TEST_UTIL.getDataTestDir().toString(), "outer.jar");
byte buffer[] = new byte[BUFFER_SIZE];
- // TODO: code here and elsewhere in this file is duplicated w/TestClassFinder.
- // Some refactoring may be in order...
// Open archive file
FileOutputStream stream = new FileOutputStream(outerJarFile);
JarOutputStream out = new JarOutputStream(stream, new Manifest());
@@ -467,7 +433,7 @@
JarEntry jarAdd = new JarEntry("/lib/" + jarFile.getName());
jarAdd.setTime(jarFile.lastModified());
out.putNextEntry(jarAdd);
-
+
// Write file to archive
FileInputStream in = new FileInputStream(jarFile);
while (true) {
@@ -539,12 +505,82 @@
@Test
public void testRegionServerCoprocessorsReported() throws Exception {
- // This was a test for HBASE-4070.
- // We are removing coprocessors from region load in HBASE-5258.
- // Therefore, this test now only checks system coprocessors.
+ // HBASE 4070: Improve region server metrics to report loaded coprocessors
+ // to master: verify that each regionserver is reporting the correct set of
+ // loaded coprocessors.
+
+ // We rely on the fact that getCoprocessors() will return a sorted
+ // display of the coprocessors' names, so for example, regionCoprocessor1's
+ // name "ColumnAggregationEndpoint" will appear before regionCoprocessor2's
+ // name "GenericEndpoint" because "C" is before "G" lexicographically.
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+
+ // disable all user tables, if any are loaded.
+ for (HTableDescriptor htd: admin.listTables()) {
+ if (!htd.isMetaTable()) {
+ String tableName = htd.getNameAsString();
+ if (admin.isTableEnabled(tableName)) {
+ try {
+ admin.disableTable(htd.getNameAsString());
+ } catch (TableNotEnabledException e) {
+ // ignoring this exception for now : not sure why it's happening.
+ }
+ }
+ }
+ }
+
+ // should only be system coprocessors loaded at this point.
assertAllRegionServers(regionServerSystemCoprocessors,null);
+
+ // The next two tests enable and disable user tables to see if coprocessor
+ // load reporting changes as coprocessors are loaded and unloaded.
+ //
+
+ // Create a table.
+ // should cause regionCoprocessor2 to be loaded, since we've specified it
+ // for loading on any user table with USER_REGION_COPROCESSOR_CONF_KEY
+ // in setUpBeforeClass().
+ String userTable1 = "userTable1";
+ HTableDescriptor userTD1 = new HTableDescriptor(userTable1);
+ admin.createTable(userTD1);
+ waitForTable(userTD1.getName());
+
+ // table should be enabled now.
+ assertTrue(admin.isTableEnabled(userTable1));
+ assertAllRegionServers(regionServerSystemAndUserCoprocessors, userTable1);
+
+ // unload and make sure we're back to only system coprocessors again.
+ admin.disableTable(userTable1);
+ assertAllRegionServers(regionServerSystemCoprocessors,null);
+
+ // create another table, with its own specified coprocessor.
+ String userTable2 = "userTable2";
+ HTableDescriptor htd2 = new HTableDescriptor(userTable2);
+
+ String userTableCP = "userTableCP";
+ File jarFile1 = buildCoprocessorJar(userTableCP);
+ htd2.addFamily(new HColumnDescriptor("myfamily"));
+ htd2.setValue("COPROCESSOR$1", jarFile1.toString() + "|" + userTableCP +
+ "|" + Coprocessor.PRIORITY_USER);
+ admin.createTable(htd2);
+ waitForTable(htd2.getName());
+ // table should be enabled now.
+ assertTrue(admin.isTableEnabled(userTable2));
+
+ ArrayList<String> existingCPsPlusNew =
+ new ArrayList<String>(Arrays.asList(regionServerSystemAndUserCoprocessors));
+ existingCPsPlusNew.add(userTableCP);
+ String[] existingCPsPlusNewArray = new String[existingCPsPlusNew.size()];
+ assertAllRegionServers(existingCPsPlusNew.toArray(existingCPsPlusNewArray),
+ userTable2);
+
+ admin.disableTable(userTable2);
+ assertTrue(admin.isTableDisabled(userTable2));
+
+ // we should be back to only system coprocessors again.
+ assertAllRegionServers(regionServerSystemCoprocessors, null);
+
}
/**
@@ -591,7 +627,7 @@
}
boolean any_failed = false;
for(Map.Entry<ServerName,HServerLoad> server: servers.entrySet()) {
- actualCoprocessors = server.getValue().getRsCoprocessors();
+ actualCoprocessors = server.getValue().getCoprocessors();
if (!Arrays.equals(actualCoprocessors, expectedCoprocessors)) {
LOG.debug("failed comparison: actual: " +
Arrays.toString(actualCoprocessors) +
diff --git a/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
index 3dcd8f1..1833aab 100644
--- a/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
+++ b/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
@@ -85,18 +85,6 @@
}
@Override
- public boolean nextRaw(List<KeyValue> result, int limit, String metric)
- throws IOException {
- return delegate.nextRaw(result, limit, metric);
- }
-
- @Override
- public boolean nextRaw(List<KeyValue> result, String metric)
- throws IOException {
- return delegate.nextRaw(result, metric);
- }
-
- @Override
public void close() throws IOException {
delegate.close();
}
@@ -116,10 +104,6 @@
return false;
}
- @Override
- public long getMvccReadPoint() {
- return delegate.getMvccReadPoint();
- }
}
public static class CoprocessorImpl extends BaseRegionObserver {
diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
index bf6f064..6456ccb 100644
--- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
+++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
@@ -55,6 +55,7 @@
import org.apache.hadoop.hbase.util.ChecksumType;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.*;
@@ -718,8 +719,7 @@
}
BlockType bt = BlockType.values()[blockTypeOrdinal];
DataOutputStream dos = hbw.startWriting(bt);
- int size = rand.nextInt(500);
- for (int j = 0; j < size; ++j) {
+ for (int j = 0; j < rand.nextInt(500); ++j) {
// This might compress well.
dos.writeShort(i + 1);
dos.writeInt(j + 1);
diff --git a/src/test/java/org/apache/hadoop/hbase/master/Mocking.java b/src/test/java/org/apache/hadoop/hbase/master/Mocking.java
deleted file mode 100644
index 187e07d..0000000
--- a/src/test/java/org/apache/hadoop/hbase/master/Mocking.java
+++ /dev/null
@@ -1,46 +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.master;
-
-import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
-
-/**
- * Package scoped mocking utility.
- */
-public class Mocking {
-
- static void waitForRegionPendingOpenInRIT(AssignmentManager am, String encodedName)
- throws InterruptedException {
- // We used to do a check like this:
- //!Mocking.verifyRegionState(this.watcher, REGIONINFO, EventType.M_ZK_REGION_OFFLINE)) {
- // There is a race condition with this: because we may do the transition to
- // RS_ZK_REGION_OPENING before the RIT is internally updated. We need to wait for the
- // RIT to be as we need it to be instead. This cannot happen in a real cluster as we
- // update the RIT before sending the openRegion request.
-
- boolean wait = true;
- while (wait) {
- RegionState state = am.getRegionsInTransition().get(encodedName);
- if (state != null && state.isPendingOpen()){
- wait = false;
- } else {
- Thread.sleep(1);
- }
- }
- }
-}
diff --git a/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java b/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
index aee8537..6e92149 100644
--- a/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
+++ b/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
@@ -36,9 +36,9 @@
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.Get;
@@ -61,12 +61,12 @@
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKTable.TableState;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKTable.TableState;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@@ -74,7 +74,6 @@
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
-import org.mockito.internal.util.reflection.Whitebox;
import com.google.protobuf.ServiceException;
@@ -82,7 +81,7 @@
/**
* Test {@link AssignmentManager}
*/
-@Category(MediumTests.class)
+@Category(SmallTests.class)
public class TestAssignmentManager {
private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
private static final ServerName SERVERNAME_A =
@@ -92,10 +91,6 @@
private static final HRegionInfo REGIONINFO =
new HRegionInfo(Bytes.toBytes("t"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
- private static final HRegionInfo REGIONINFO_2 = new HRegionInfo(Bytes.toBytes("t"),
- Bytes.toBytes("a"),Bytes.toBytes( "b"));
- private static int assignmentCount;
- private static boolean enabling = false;
// Mocked objects or; get redone for each test.
private Server server;
@@ -162,7 +157,7 @@
/**
* Test a balance going on at same time as a master failover
- *
+ *
* @throws IOException
* @throws KeeperException
* @throws InterruptedException
@@ -184,8 +179,10 @@
int versionid =
ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
assertNotSame(versionid, -1);
- Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
+ while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO,
+ EventType.M_ZK_REGION_OFFLINE)) {
+ Threads.sleep(1);
+ }
// Get current versionid else will fail on transition from OFFLINE to
// OPENING below
versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
@@ -226,8 +223,10 @@
ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
assertNotSame(versionid, -1);
am.gate.set(false);
- Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
+ while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO,
+ EventType.M_ZK_REGION_OFFLINE)) {
+ Threads.sleep(1);
+ }
// Get current versionid else will fail on transition from OFFLINE to
// OPENING below
versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
@@ -267,8 +266,10 @@
int versionid =
ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
assertNotSame(versionid, -1);
- Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
+ while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO,
+ EventType.M_ZK_REGION_OFFLINE)) {
+ Threads.sleep(1);
+ }
am.gate.set(false);
// Get current versionid else will fail on transition from OFFLINE to
// OPENING below
@@ -307,11 +308,10 @@
* from one server to another mocking regionserver responding over zk.
* @throws IOException
* @throws KeeperException
- * @throws InterruptedException
*/
- @Test(timeout = 10000)
+ @Test
public void testBalance()
- throws IOException, KeeperException, InterruptedException {
+ throws IOException, KeeperException {
// Create and startup an executor. This is used by AssignmentManager
// handling zk callbacks.
ExecutorService executor = startupMasterExecutor("testBalanceExecutor");
@@ -345,9 +345,11 @@
// AM is going to notice above CLOSED and queue up a new assign. The
// assign will go to open the region in the new location set by the
// balancer. The zk node will be OFFLINE waiting for regionserver to
- // transition it through OPENING, OPENED. Wait till we see the RIT
- // before we proceed.
- Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
+ // transition it through OPENING, OPENED. Wait till we see the OFFLINE
+ // zk node before we proceed.
+ while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO, EventType.M_ZK_REGION_OFFLINE)) {
+ Threads.sleep(1);
+ }
// Get current versionid else will fail on transition from OFFLINE to OPENING below
versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
assertNotSame(-1, versionid);
@@ -400,7 +402,7 @@
/**
* To test closed region handler to remove rit and delete corresponding znode if region in pending
- * close or closing while processing shutdown of a region server.(HBASE-5927).
+ * close or closing while processing shutdown of a region server.(HBASE-5927).
* @throws KeeperException
* @throws IOException
*/
@@ -410,7 +412,7 @@
testCaseWithPartiallyDisabledState(TableState.DISABLING);
testCaseWithPartiallyDisabledState(TableState.DISABLED);
}
-
+
/**
* To test if the split region is removed from RIT if the region was in SPLITTING state
* but the RS has actually completed the splitting in META but went down. See HBASE-6070
@@ -444,7 +446,7 @@
am.regionsInTransition.put(REGIONINFO.getEncodedName(), new RegionState(REGIONINFO,
State.SPLITTING, System.currentTimeMillis(), SERVERNAME_A));
am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
-
+
RegionTransitionData data = new RegionTransitionData(EventType.RS_ZK_REGION_SPLITTING,
REGIONINFO.getRegionName(), SERVERNAME_A);
String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
@@ -452,11 +454,11 @@
ZKUtil.createAndWatch(this.watcher, node, data.getBytes());
try {
-
+
processServerShutdownHandler(ct, am, regionSplitDone);
// check znode deleted or not.
// In both cases the znode should be deleted.
-
+
if(regionSplitDone){
assertTrue("Region state of region in SPLITTING should be removed from rit.",
am.regionsInTransition.isEmpty());
@@ -499,7 +501,7 @@
} else {
am.getZKTable().setDisabledTable(REGIONINFO.getTableNameAsString());
}
-
+
RegionTransitionData data = new RegionTransitionData(EventType.M_ZK_REGION_CLOSING,
REGIONINFO.getRegionName(), SERVERNAME_A);
String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
@@ -574,7 +576,7 @@
* @param hri Region to serialize into HRegionInfo
* @return A mocked up Result that fakes a Get on a row in the
* <code>.META.</code> table.
- * @throws IOException
+ * @throws IOException
*/
private Result getMetaTableRowResult(final HRegionInfo hri,
final ServerName sn)
@@ -593,13 +595,13 @@
Bytes.toBytes(sn.getStartcode())));
return new Result(kvs);
}
-
+
/**
* @param sn ServerName to use making startcode and server in meta
* @param hri Region to serialize into HRegionInfo
* @return A mocked up Result that fakes a Get on a row in the
* <code>.META.</code> table.
- * @throws IOException
+ * @throws IOException
*/
private Result getMetaTableRowResultAsSplitRegion(final HRegionInfo hri, final ServerName sn)
throws IOException {
@@ -661,12 +663,12 @@
am.shutdown();
}
}
-
+
/**
* Tests the processDeadServersAndRegionsInTransition should not fail with NPE
* when it failed to get the children. Let's abort the system in this
* situation
- * @throws ServiceException
+ * @throws ServiceException
*/
@Test(timeout = 5000)
public void testProcessDeadServersAndRegionsInTransitionShouldNotFailWithNPE()
@@ -706,8 +708,8 @@
* @param region region to be created as offline
* @param serverName server event originates from
* @return Version of znode created.
- * @throws KeeperException
- * @throws IOException
+ * @throws KeeperException
+ * @throws IOException
*/
// Copied from SplitTransaction rather than open the method over there in
// the regionserver package.
@@ -766,27 +768,14 @@
// with an encoded name by doing a Get on .META.
HRegionInterface ri = Mockito.mock(HRegionInterface.class);
// Get a meta row result that has region up on SERVERNAME_A for REGIONINFO
- Result[] result = null;
- if (enabling) {
- result = new Result[2];
- result[0] = getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
- result[1] = getMetaTableRowResult(REGIONINFO_2, SERVERNAME_A);
- }
Result r = getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
Mockito.when(ri .openScanner((byte[]) Mockito.any(), (Scan) Mockito.any())).
thenReturn(System.currentTimeMillis());
- if (enabling) {
- Mockito.when(ri.next(Mockito.anyLong(), Mockito.anyInt())).thenReturn(result, result, result,
- (Result[]) null);
- // If a get, return the above result too for REGIONINFO_2
- Mockito.when(ri.get((byte[]) Mockito.any(), (Get) Mockito.any())).thenReturn(
- getMetaTableRowResult(REGIONINFO_2, SERVERNAME_A));
- } else {
- // Return good result 'r' first and then return null to indicate end of scan
- Mockito.when(ri.next(Mockito.anyLong(), Mockito.anyInt())).thenReturn(new Result[] { r });
- // If a get, return the above result too for REGIONINFO
- Mockito.when(ri.get((byte[]) Mockito.any(), (Get) Mockito.any())).thenReturn(r);
- }
+ // Return good result 'r' first and then return null to indicate end of scan
+ Mockito.when(ri.next(Mockito.anyLong(), Mockito.anyInt())).thenReturn(new Result[] { r });
+ // If a get, return the above result too for REGIONINFO
+ Mockito.when(ri.get((byte[]) Mockito.any(), (Get) Mockito.any())).
+ thenReturn(r);
// Get a connection w/ mocked up common methods.
HConnection connection = HConnectionTestingUtility.
getMockedConnectionAndDecorate(HTU.getConfiguration(), ri, SERVERNAME_B,
@@ -800,9 +789,9 @@
server, manager, ct, balancer, executor);
return am;
}
-
+
/**
- * TestCase verifies that the regionPlan is updated whenever a region fails to open
+ * TestCase verifies that the regionPlan is updated whenever a region fails to open
* and the master tries to process RS_ZK_FAILED_OPEN state.(HBASE-5546).
*/
@Test
@@ -850,18 +839,17 @@
assertNotSame("Same region plan should not come", regionPlan, newRegionPlan);
assertTrue("Destnation servers should be different.", !(regionPlan.getDestination().equals(
newRegionPlan.getDestination())));
- Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
} finally {
this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
DefaultLoadBalancer.class, LoadBalancer.class);
am.shutdown();
}
}
-
+
/**
* Test verifies whether assignment is skipped for regions of tables in DISABLING state during
* clean cluster startup. See HBASE-6281.
- *
+ *
* @throws KeeperException
* @throws IOException
* @throws Exception
@@ -904,53 +892,6 @@
}
/**
- * Test verifies whether all the enabling table regions assigned only once during master startup.
- *
- * @throws KeeperException
- * @throws IOException
- * @throws Exception
- */
- @Test
- public void testMasterRestartWhenTableInEnabling() throws KeeperException, IOException, Exception {
- enabling = true;
- this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
- DefaultLoadBalancer.class, LoadBalancer.class);
- Map<ServerName, HServerLoad> serverAndLoad = new HashMap<ServerName, HServerLoad>();
- serverAndLoad.put(SERVERNAME_A, null);
- Mockito.when(this.serverManager.getOnlineServers()).thenReturn(serverAndLoad);
- Mockito.when(this.serverManager.isServerOnline(SERVERNAME_B)).thenReturn(false);
- Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
- HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
- Server server = new HMaster(HTU.getConfiguration());
- Whitebox.setInternalState(server, "serverManager", this.serverManager);
- assignmentCount = 0;
- AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
- this.serverManager);
- am.regionOnline(new HRegionInfo("t1".getBytes(), HConstants.EMPTY_START_ROW,
- HConstants.EMPTY_END_ROW), SERVERNAME_A);
- am.gate.set(false);
- try {
- // set table in enabling state.
- am.getZKTable().setEnablingTable(REGIONINFO.getTableNameAsString());
- ZKAssign.createNodeOffline(this.watcher, REGIONINFO_2, SERVERNAME_B);
-
- am.joinCluster();
- while (!am.getZKTable().isEnabledTable(REGIONINFO.getTableNameAsString())) {
- Thread.sleep(10);
- }
- assertEquals("Number of assignments should be equal.", 2, assignmentCount);
- assertTrue("Table should be enabled.",
- am.getZKTable().isEnabledTable(REGIONINFO.getTableNameAsString()));
- } finally {
- enabling = false;
- am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
- am.shutdown();
- ZKAssign.deleteAllNodes(this.watcher);
- assignmentCount = 0;
- }
- }
-
- /**
* Mocked load balancer class used in the testcase to make sure that the testcase waits until
* random assignment is called and the gate variable is set to true.
*/
@@ -967,7 +908,7 @@
this.gate.set(true);
return randomServerName;
}
-
+
@Override
public Map<ServerName, List<HRegionInfo>> retainAssignment(
Map<HRegionInfo, ServerName> regions, List<ServerName> servers) {
@@ -1019,13 +960,8 @@
@Override
public void assign(HRegionInfo region, boolean setOfflineInZK, boolean forceNewPlan,
boolean hijack) {
- if (enabling) {
- assignmentCount++;
- this.regionOnline(region, SERVERNAME_A);
- } else {
- assignInvoked = true;
- super.assign(region, setOfflineInZK, forceNewPlan, hijack);
- }
+ assignInvoked = true;
+ super.assign(region, setOfflineInZK, forceNewPlan, hijack);
}
@Override
diff --git a/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 1f75842..c83d4ba 100644
--- a/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -241,11 +241,13 @@
return new TableDescriptors() {
@Override
public HTableDescriptor remove(String tablename) throws IOException {
+ // TODO Auto-generated method stub
return null;
}
@Override
public Map<String, HTableDescriptor> getAll() throws IOException {
+ // TODO Auto-generated method stub
return null;
}
@@ -263,6 +265,8 @@
@Override
public void add(HTableDescriptor htd) throws IOException {
+ // TODO Auto-generated method stub
+
}
};
}
@@ -281,34 +285,6 @@
public <T extends CoprocessorProtocol> boolean registerProtocol(Class<T> protocol, T handler) {
return false;
}
-
- @Override
- public void deleteTable(byte[] tableName) throws IOException {
- }
-
- @Override
- public void modifyTable(byte[] tableName, HTableDescriptor descriptor) throws IOException {
- }
-
- @Override
- public void enableTable(byte[] tableName) throws IOException {
- }
-
- @Override
- public void disableTable(byte[] tableName) throws IOException {
- }
-
- @Override
- public void addColumn(byte[] tableName, HColumnDescriptor column) throws IOException {
- }
-
- @Override
- public void modifyColumn(byte[] tableName, HColumnDescriptor descriptor) throws IOException {
- }
-
- @Override
- public void deleteColumn(byte[] tableName, byte[] columnName) throws IOException {
- }
}
@Test
diff --git a/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java b/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
deleted file mode 100644
index 0781d11..0000000
--- a/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
+++ /dev/null
@@ -1,66 +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.master;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test the master filesystem in a local cluster
- */
-@Category(MediumTests.class)
-public class TestMasterFileSystem {
-
- private static final Log LOG = LogFactory.getLog(TestMasterFileSystem.class);
- private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
- @BeforeClass
- public static void setupTest() throws Exception {
- UTIL.startMiniCluster();
- }
-
- @AfterClass
- public static void teardownTest() throws Exception {
- UTIL.shutdownMiniCluster();
- }
-
- @Test
- public void testFsUriSetProperly() throws Exception {
- HMaster master = UTIL.getMiniHBaseCluster().getMaster();
- MasterFileSystem fs = master.getMasterFileSystem();
- Path masterRoot = FSUtils.getRootDir(fs.conf);
- Path rootDir = FSUtils.getRootDir(fs.getFileSystem().getConf());
- // make sure the fs and the found root dir have the same scheme
- LOG.debug("from fs uri:" + FileSystem.getDefaultUri(fs.getFileSystem().getConf()));
- LOG.debug("from configuration uri:" + FileSystem.getDefaultUri(fs.conf));
- // make sure the set uri matches by forcing it.
- assertEquals(masterRoot, rootDir);
- }
-
-}
diff --git a/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 2392099..e3ad07c 100644
--- a/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -88,7 +88,7 @@
+ status.getAccessTime();
}
- @Test(timeout = 60 *1000)
+ @Test
public void testHFileCleaning() throws Exception {
final EnvironmentEdge originalEdge = EnvironmentEdgeManager.getDelegate();
String prefix = "someHFileThatWouldBeAUUID";
diff --git a/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java b/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java
deleted file mode 100644
index ad56bd4..0000000
--- a/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java
+++ /dev/null
@@ -1,159 +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.master.handler;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.LargeTests;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class TestTableDeleteFamilyHandler {
-
- private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
- private static final String TABLENAME = "column_family_handlers";
- private static final byte[][] FAMILIES = new byte[][] { Bytes.toBytes("cf1"),
- Bytes.toBytes("cf2"), Bytes.toBytes("cf3") };
-
- /**
- * Start up a mini cluster and put a small table of empty regions into it.
- *
- * @throws Exception
- */
- @BeforeClass
- public static void beforeAllTests() throws Exception {
-
- TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
- TEST_UTIL.startMiniCluster(2);
-
- // Create a table of three families. This will assign a region.
- TEST_UTIL.createTable(Bytes.toBytes(TABLENAME), FAMILIES);
- HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
-
- // Create multiple regions in all the three column families
- TEST_UTIL.createMultiRegions(t, FAMILIES[0]);
-
- // Load the table with data for all families
- TEST_UTIL.loadTable(t, FAMILIES);
-
- TEST_UTIL.flush();
-
- t.close();
- }
-
- @AfterClass
- public static void afterAllTests() throws Exception {
- TEST_UTIL.deleteTable(Bytes.toBytes(TABLENAME));
- TEST_UTIL.shutdownMiniCluster();
- }
-
- @Before
- public void setup() throws IOException, InterruptedException {
- TEST_UTIL.ensureSomeRegionServersAvailable(2);
- }
-
- @Test
- public void deleteColumnFamilyWithMultipleRegions() throws Exception {
-
- HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
- HTableDescriptor beforehtd = admin.getTableDescriptor(Bytes
- .toBytes(TABLENAME));
-
- FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
-
- // 1 - Check if table exists in descriptor
- assertTrue(admin.isTableAvailable(TABLENAME));
-
- // 2 - Check if all three families exist in descriptor
- assertEquals(3, beforehtd.getColumnFamilies().length);
- HColumnDescriptor[] families = beforehtd.getColumnFamilies();
- for (int i = 0; i < families.length; i++) {
-
- assertTrue(families[i].getNameAsString().equals("cf" + (i + 1)));
- }
-
- // 3 - Check if table exists in FS
- Path tableDir = new Path(TEST_UTIL.getDefaultRootDirPath().toString() + "/"
- + TABLENAME);
- assertTrue(fs.exists(tableDir));
-
- // 4 - Check if all the 3 column families exist in FS
- FileStatus[] fileStatus = fs.listStatus(tableDir);
- for (int i = 0; i < fileStatus.length; i++) {
- if (fileStatus[i].isDir() == true) {
- FileStatus[] cf = fs.listStatus(fileStatus[i].getPath());
- int k = 1;
- for (int j = 0; j < cf.length; j++) {
- if (cf[j].isDir() == true
- && cf[j].getPath().getName().startsWith(".") == false) {
- assertTrue(cf[j].getPath().getName().equals("cf" + k));
- k++;
- }
- }
- }
- }
-
- // TEST - Disable and delete the column family
- admin.disableTable(TABLENAME);
- admin.deleteColumn(TABLENAME, "cf2");
-
- // 5 - Check if only 2 column families exist in the descriptor
- HTableDescriptor afterhtd = admin.getTableDescriptor(Bytes
- .toBytes(TABLENAME));
- assertEquals(2, afterhtd.getColumnFamilies().length);
- HColumnDescriptor[] newFamilies = afterhtd.getColumnFamilies();
- assertTrue(newFamilies[0].getNameAsString().equals("cf1"));
- assertTrue(newFamilies[1].getNameAsString().equals("cf3"));
-
- // 6 - Check if the second column family is gone from the FS
- fileStatus = fs.listStatus(tableDir);
- for (int i = 0; i < fileStatus.length; i++) {
- if (fileStatus[i].isDir() == true) {
- FileStatus[] cf = fs.listStatus(fileStatus[i].getPath());
- for (int j = 0; j < cf.length; j++) {
- if (cf[j].isDir() == true) {
- assertFalse(cf[j].getPath().getName().equals("cf2"));
- }
- }
- }
- }
- }
-
- @org.junit.Rule
- public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
- new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
-}
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index 93908e7..8134f4a 100644
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -587,10 +587,8 @@
List<StoreFile> storeFiles = store.getStorefiles();
long maxId = StoreFile.getMaxSequenceIdInList(storeFiles);
- Compactor tool = new Compactor(this.conf);
- StoreFile.Writer compactedFile =
- tool.compact(store, storeFiles, false, maxId);
+ StoreFile.Writer compactedFile = store.compactStore(storeFiles, false, maxId);
// Now lets corrupt the compacted file.
FileSystem fs = FileSystem.get(conf);
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java
index a80ea5c..df6c293 100644
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java
+++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestHBase7051.java
@@ -16,9 +16,6 @@
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
-import org.apache.hadoop.hbase.MultithreadedTestUtil;
-import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
-import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
@@ -33,32 +30,19 @@
import com.google.common.collect.Lists;
-/**
- * Test of HBASE-7051; that checkAndPuts and puts behave atomically with respect to each other.
- * Rather than perform a bunch of trials to verify atomicity, this test recreates a race condition
- * that causes the test to fail if checkAndPut doesn't wait for outstanding put transactions
- * to complete. It does this by invasively overriding HRegion function to affect the timing of
- * the operations.
- */
@Category(SmallTests.class)
public class TestHBase7051 {
+ private static volatile boolean putCompleted = false;
private static CountDownLatch latch = new CountDownLatch(1);
- private enum TestStep {
- INIT, // initial put of 10 to set value of the cell
- PUT_STARTED, // began doing a put of 50 to cell
- PUT_COMPLETED, // put complete (released RowLock, but may not have advanced MVCC).
- CHECKANDPUT_STARTED, // began checkAndPut: if 10 -> 11
- CHECKANDPUT_COMPLETED // completed checkAndPut
- // NOTE: at the end of these steps, the value of the cell should be 50, not 11!
- }
- private static volatile TestStep testStep = TestStep.INIT;
- private final String family = "f1";
-
+ private boolean checkAndPutCompleted = false;
+ private static int count = 0;
+
@Test
public void testPutAndCheckAndPutInParallel() throws Exception {
final String tableName = "testPutAndCheckAndPut";
+ final String family = "f1";
Configuration conf = HBaseConfiguration.create();
conf.setClass(HConstants.REGION_IMPL, MockHRegion.class, HeapSize.class);
final MockHRegion region = (MockHRegion) TestHRegion.initHRegion(Bytes.toBytes(tableName),
@@ -73,16 +57,14 @@
putsAndLocks.add(pair);
+ count++;
region.batchMutate(putsAndLocks.toArray(new Pair[0]));
- MultithreadedTestUtil.TestContext ctx =
- new MultithreadedTestUtil.TestContext(conf);
- ctx.addThread(new PutThread(ctx, region));
- ctx.addThread(new CheckAndPutThread(ctx, region));
- ctx.startThreads();
- while (testStep != TestStep.CHECKANDPUT_COMPLETED) {
+ makeCheckAndPut(family, region);
+
+ makePut(family, region);
+ while (!checkAndPutCompleted) {
Thread.sleep(100);
}
- ctx.stop();
Scan s = new Scan();
RegionScanner scanner = region.getScanner(s);
List<KeyValue> results = new ArrayList<KeyValue>();
@@ -93,46 +75,54 @@
}
- private class PutThread extends TestThread {
- private MockHRegion region;
- PutThread(TestContext ctx, MockHRegion region) {
- super(ctx);
- this.region = region;
- }
-
- public void doWork() throws Exception {
- List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
- Put[] puts = new Put[1];
- Put put = new Put(Bytes.toBytes("r1"));
- put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
- puts[0] = put;
- Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
- putsAndLocks.add(pair);
- testStep = TestStep.PUT_STARTED;
- region.batchMutate(putsAndLocks.toArray(new Pair[0]));
- }
+ private void makePut(final String family, final MockHRegion region) {
+ new Thread() {
+ public void run() {
+ List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
+ Put[] puts = new Put[1];
+ Put put = new Put(Bytes.toBytes("r1"));
+ put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
+ puts[0] = put;
+ try {
+ Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
+ putsAndLocks.add(pair);
+ count++;
+ region.batchMutate(putsAndLocks.toArray(new Pair[0]));
+ } catch (IOException e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+ }.start();
}
- private class CheckAndPutThread extends TestThread {
- private MockHRegion region;
- CheckAndPutThread(TestContext ctx, MockHRegion region) {
- super(ctx);
- this.region = region;
- }
+ private void makeCheckAndPut(final String family, final MockHRegion region) {
+ new Thread() {
- public void doWork() throws Exception {
- Put[] puts = new Put[1];
- Put put = new Put(Bytes.toBytes("r1"));
- put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("11"));
- puts[0] = put;
- while (testStep != TestStep.PUT_COMPLETED) {
- Thread.sleep(100);
+ public void run() {
+ Put[] puts = new Put[1];
+ Put put = new Put(Bytes.toBytes("r1"));
+ put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("11"));
+ puts[0] = put;
+ try {
+ while (putCompleted == false) {
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+ count++;
+ region.checkAndMutate(Bytes.toBytes("r1"), Bytes.toBytes(family), Bytes.toBytes("q1"),
+ CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, null, true);
+ checkAndPutCompleted = true;
+ } catch (IOException e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
}
- testStep = TestStep.CHECKANDPUT_STARTED;
- region.checkAndMutate(Bytes.toBytes("r1"), Bytes.toBytes(family), Bytes.toBytes("q1"),
- CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, null, true);
- testStep = TestStep.CHECKANDPUT_COMPLETED;
- }
+ }.start();
}
public static class MockHRegion extends HRegion {
@@ -144,39 +134,36 @@
@Override
public void releaseRowLock(Integer lockId) {
- if (testStep == TestStep.INIT) {
+ if (count == 1) {
super.releaseRowLock(lockId);
return;
}
- if (testStep == TestStep.PUT_STARTED) {
+ if (count == 2) {
try {
- testStep = TestStep.PUT_COMPLETED;
+ putCompleted = true;
super.releaseRowLock(lockId);
- // put has been written to the memstore and the row lock has been released, but the
- // MVCC has not been advanced. Prior to fixing HBASE-7051, the following order of
- // operations would cause the non-atomicity to show up:
- // 1) Put releases row lock (where we are now)
- // 2) CheckAndPut grabs row lock and reads the value prior to the put (10)
- // because the MVCC has not advanced
- // 3) Put advances MVCC
- // So, in order to recreate this order, we wait for the checkAndPut to grab the rowLock
- // (see below), and then wait some more to give the checkAndPut time to read the old
- // value.
latch.await();
- Thread.sleep(1000);
} catch (InterruptedException e) {
- Thread.currentThread().interrupt();
+ // TODO Auto-generated catch block
+ e.printStackTrace();
}
}
- else if (testStep == TestStep.CHECKANDPUT_STARTED) {
+ if (count == 3) {
super.releaseRowLock(lockId);
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ latch.countDown();
}
}
@Override
public Integer getLock(Integer lockid, byte[] row, boolean waitForLock) throws IOException {
- if (testStep == TestStep.CHECKANDPUT_STARTED) {
+ if (count == 3) {
latch.countDown();
}
return super.getLock(lockid, row, waitForLock);
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 392ae58..6a91edf 100644
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -21,7 +21,6 @@
import java.io.IOException;
-import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
@@ -52,15 +51,17 @@
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.MultithreadedTestUtil;
-import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
+import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
@@ -70,6 +71,8 @@
import org.apache.hadoop.hbase.filter.NullComparator;
import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -102,7 +105,6 @@
* HRegions or in the HBaseMaster, so only basic testing is possible.
*/
@Category(MediumTests.class)
-@SuppressWarnings("deprecation")
public class TestHRegion extends HBaseTestCase {
// Do not spin up clusters in here. If you need to spin up a cluster, do it
// over in TestHRegionOnCluster.
@@ -155,6 +157,7 @@
String method = "testCompactionAffectedByScanners";
byte[] tableName = Bytes.toBytes(method);
byte[] family = Bytes.toBytes("family");
+ Configuration conf = HBaseConfiguration.create();
this.region = initHRegion(tableName, method, conf, family);
Put put = new Put(Bytes.toBytes("r1"));
@@ -206,6 +209,7 @@
String method = "testToShowNPEOnRegionScannerReseek";
byte[] tableName = Bytes.toBytes(method);
byte[] family = Bytes.toBytes("family");
+ Configuration conf = HBaseConfiguration.create();
this.region = initHRegion(tableName, method, conf, family);
Put put = new Put(Bytes.toBytes("r1"));
@@ -238,6 +242,7 @@
String method = "testSkipRecoveredEditsReplay";
byte[] tableName = Bytes.toBytes(method);
byte[] family = Bytes.toBytes("family");
+ Configuration conf = HBaseConfiguration.create();
this.region = initHRegion(tableName, method, conf, family);
try {
Path regiondir = region.getRegionDir();
@@ -283,7 +288,7 @@
String method = "testSkipRecoveredEditsReplaySomeIgnored";
byte[] tableName = Bytes.toBytes(method);
byte[] family = Bytes.toBytes("family");
- this.region = initHRegion(tableName, method, conf, family);
+ this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
try {
Path regiondir = region.getRegionDir();
FileSystem fs = region.getFilesystem();
@@ -333,7 +338,7 @@
String method = "testSkipRecoveredEditsReplayAllIgnored";
byte[] tableName = Bytes.toBytes(method);
byte[] family = Bytes.toBytes("family");
- this.region = initHRegion(tableName, method, conf, family);
+ this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
try {
Path regiondir = region.getRegionDir();
FileSystem fs = region.getFilesystem();
@@ -459,7 +464,7 @@
byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
- this.region = initHRegion(TABLE, getName(), conf, FAMILIES);
+ this.region = initHRegion(TABLE, getName(), FAMILIES);
try {
String value = "this is the value";
String value2 = "this is some other value";
@@ -580,7 +585,7 @@
public void testFamilyWithAndWithoutColon() throws Exception {
byte [] b = Bytes.toBytes(getName());
byte [] cf = Bytes.toBytes(COLUMN_FAMILY);
- this.region = initHRegion(b, getName(), conf, cf);
+ this.region = initHRegion(b, getName(), cf);
try {
Put p = new Put(b);
byte [] cfwithcolon = Bytes.toBytes(COLUMN_FAMILY + ":");
@@ -604,7 +609,7 @@
byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
byte[] qual = Bytes.toBytes("qual");
byte[] val = Bytes.toBytes("val");
- this.region = initHRegion(b, getName(), conf, cf);
+ this.region = initHRegion(b, getName(), cf);
try {
HLog.getSyncTime(); // clear counter from prior tests
assertEquals(0, HLog.getSyncTime().count);
@@ -638,7 +643,7 @@
Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
MultithreadedTestUtil.TestContext ctx =
- new MultithreadedTestUtil.TestContext(conf);
+ new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
final AtomicReference<OperationStatus[]> retFromThread =
new AtomicReference<OperationStatus[]>();
TestThread putter = new TestThread(ctx) {
@@ -705,7 +710,9 @@
byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
byte[] qual = Bytes.toBytes("qual");
byte[] val = Bytes.toBytes("val");
- Configuration conf = HBaseConfiguration.create(this.conf);
+
+ HBaseConfiguration conf = new HBaseConfiguration();
+
// add data with a timestamp that is too recent for range. Ensure assert
conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
@@ -752,7 +759,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1);
+ this.region = initHRegion(tableName, method, fam1);
try {
//Putting empty data in key
Put put = new Put(row1);
@@ -827,7 +834,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1);
+ this.region = initHRegion(tableName, method, fam1);
try {
//Putting data in key
Put put = new Put(row1);
@@ -861,7 +868,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1);
+ this.region = initHRegion(tableName, method, fam1);
try {
//Putting data in key
Put put = new Put(row1);
@@ -899,7 +906,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
//Putting data in the key to check
Put put = new Put(row1);
@@ -938,7 +945,7 @@
}
public void testCheckAndPut_wrongRowInPut() throws IOException {
- this.region = initHRegion(tableName, this.getName(), conf, COLUMNS);
+ this.region = initHRegion(tableName, this.getName(), COLUMNS);
try {
Put put = new Put(row2);
put.add(fam1, qual1, value1);
@@ -973,7 +980,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
//Put content
Put put = new Put(row1);
@@ -1048,7 +1055,7 @@
put.add(fam1, qual, 2, value);
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1);
+ this.region = initHRegion(tableName, method, fam1);
try {
region.put(put);
@@ -1078,7 +1085,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1, fam2, fam3);
+ this.region = initHRegion(tableName, method, fam1, fam2, fam3);
try {
List<KeyValue> kvs = new ArrayList<KeyValue>();
kvs.add(new KeyValue(row1, fam4, null, null));
@@ -1116,7 +1123,7 @@
byte [] fam = Bytes.toBytes("info");
byte [][] families = {fam};
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
@@ -1184,7 +1191,7 @@
byte [] fam = Bytes.toBytes("info");
byte [][] families = {fam};
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
byte [] row = Bytes.toBytes("table_name");
// column names
@@ -1227,7 +1234,7 @@
byte [] fam = Bytes.toBytes("info");
byte [][] families = {fam};
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
byte [] row = Bytes.toBytes("row1");
// column names
@@ -1280,7 +1287,7 @@
byte[] fam = Bytes.toBytes("info");
byte[][] families = { fam };
String method = this.getName();
- Configuration conf = HBaseConfiguration.create(this.conf);
+ HBaseConfiguration conf = new HBaseConfiguration();
// add data with a timestamp that is too recent for range. Ensure assert
conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
@@ -1311,7 +1318,7 @@
byte [] tableName = Bytes.toBytes("test_table");
byte [] fam1 = Bytes.toBytes("columnA");
byte [] fam2 = Bytes.toBytes("columnB");
- this.region = initHRegion(tableName, getName(), conf, fam1, fam2);
+ this.region = initHRegion(tableName, getName(), fam1, fam2);
try {
byte [] rowA = Bytes.toBytes("rowA");
byte [] rowB = Bytes.toBytes("rowB");
@@ -1364,7 +1371,7 @@
public void doTestDelete_AndPostInsert(Delete delete)
throws IOException, InterruptedException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
Put put = new Put(row);
@@ -1417,7 +1424,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1);
+ this.region = initHRegion(tableName, method, fam1);
try {
//Building checkerList
List<KeyValue> kvs = new ArrayList<KeyValue>();
@@ -1457,7 +1464,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1);
+ this.region = initHRegion(tableName, method, fam1);
try {
Get get = new Get(row1);
get.addColumn(fam2, col1);
@@ -1488,7 +1495,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1);
+ this.region = initHRegion(tableName, method, fam1);
try {
//Add to memstore
Put put = new Put(row1);
@@ -1538,7 +1545,7 @@
byte [] fam = Bytes.toBytes("fam");
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam);
+ this.region = initHRegion(tableName, method, fam);
try {
Get get = new Get(row);
get.addFamily(fam);
@@ -1558,8 +1565,7 @@
public void stestGet_Root() throws IOException {
//Setting up region
String method = this.getName();
- this.region = initHRegion(HConstants.ROOT_TABLE_NAME,
- method, conf, HConstants.CATALOG_FAMILY);
+ this.region = initHRegion(HConstants.ROOT_TABLE_NAME, method, HConstants.CATALOG_FAMILY);
try {
//Add to memstore
Put put = new Put(HConstants.EMPTY_START_ROW);
@@ -1791,7 +1797,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
Scan scan = new Scan();
scan.addFamily(fam1);
@@ -1816,7 +1822,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
Scan scan = new Scan();
scan.addFamily(fam2);
@@ -1845,7 +1851,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
//Putting data in Region
@@ -1893,7 +1899,7 @@
//Setting up region
String method = this.getName();
try {
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
} catch (IOException e) {
e.printStackTrace();
fail("Got IOException during initHRegion, " + e.getMessage());
@@ -1929,7 +1935,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
//Putting data in Region
Put put = null;
@@ -1996,7 +2002,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
//Putting data in Region
Put put = null;
@@ -2056,7 +2062,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
//Putting data in Region
Put put = null;
@@ -2121,7 +2127,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
//Putting data in Region
KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
@@ -2203,7 +2209,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
//Putting data in Region
Put put = null;
@@ -2264,7 +2270,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1);
+ this.region = initHRegion(tableName, method, fam1);
try {
//Putting data in Region
Put put = null;
@@ -2315,7 +2321,7 @@
public void testScanner_StopRow1542() throws IOException {
byte [] tableName = Bytes.toBytes("test_table");
byte [] family = Bytes.toBytes("testFamily");
- this.region = initHRegion(tableName, getName(), conf, family);
+ this.region = initHRegion(tableName, getName(), family);
try {
byte [] row1 = Bytes.toBytes("row111");
byte [] row2 = Bytes.toBytes("row222");
@@ -2362,7 +2368,7 @@
}
public void testIncrementColumnValue_UpdatingInPlace() throws IOException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
long value = 1L;
long amount = 3L;
@@ -2390,7 +2396,7 @@
public void testIncrementColumnValue_BumpSnapshot() throws IOException {
ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
EnvironmentEdgeManagerTestHelper.injectEdge(mee);
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
long value = 42L;
long incr = 44L;
@@ -2429,7 +2435,7 @@
}
public void testIncrementColumnValue_ConcurrentFlush() throws IOException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
long value = 1L;
long amount = 3L;
@@ -2463,7 +2469,7 @@
public void testIncrementColumnValue_heapSize() throws IOException {
EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
long byAmount = 1L;
long size;
@@ -2482,7 +2488,7 @@
public void testIncrementColumnValue_UpdatingInPlace_Negative()
throws IOException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
long value = 3L;
long amount = -1L;
@@ -2503,7 +2509,7 @@
public void testIncrementColumnValue_AddingNew()
throws IOException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
long value = 1L;
long amount = 3L;
@@ -2532,7 +2538,7 @@
}
public void testIncrementColumnValue_UpdatingFromSF() throws IOException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
long value = 1L;
long amount = 3L;
@@ -2560,7 +2566,7 @@
public void testIncrementColumnValue_AddingNewAfterSFCheck()
throws IOException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
long value = 1L;
long amount = 3L;
@@ -2599,7 +2605,7 @@
* @throws IOException
*/
public void testIncrementColumnValue_UpdatingInPlace_TimestampClobber() throws IOException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
long value = 1L;
long amount = 3L;
@@ -2647,7 +2653,7 @@
}
public void testIncrementColumnValue_WrongInitialSize() throws IOException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
int row1Field1 = 0;
@@ -2675,7 +2681,7 @@
}
public void testIncrement_WrongInitialSize() throws IOException {
- this.region = initHRegion(tableName, getName(), conf, fam1);
+ this.region = initHRegion(tableName, getName(), fam1);
try {
byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
long row1Field1 = 0;
@@ -2751,7 +2757,7 @@
//Setting up region
String method = this.getName();
- this.region = initHRegion(tableName, method, conf, fam1);
+ this.region = initHRegion(tableName, method, fam1);
try {
//Putting data in Region
KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
@@ -2966,7 +2972,7 @@
int compactInterval = 10 * flushAndScanInterval;
String method = "testFlushCacheWhileScanning";
- this.region = initHRegion(tableName,method, conf, family);
+ this.region = initHRegion(tableName,method, family);
try {
FlushThread flushThread = new FlushThread();
flushThread.start();
@@ -3097,7 +3103,7 @@
}
String method = "testWritesWhileScanning";
- this.region = initHRegion(tableName, method, conf, families);
+ this.region = initHRegion(tableName, method, families);
try {
PutThread putThread = new PutThread(numRows, families, qualifiers);
putThread.start();
@@ -3219,8 +3225,6 @@
}
numPutsFinished++;
}
- } catch (InterruptedIOException e) {
- // This is fine. It means we are done, or didn't get the lock on time
} catch (IOException e) {
LOG.error("error while putting records", e);
error = e;
@@ -3257,9 +3261,8 @@
qualifiers[i] = Bytes.toBytes("qual" + i);
}
- Configuration conf = HBaseConfiguration.create(this.conf);
-
String method = "testWritesWhileGetting";
+ Configuration conf = HBaseConfiguration.create();
// This test flushes constantly and can cause many files to be created, possibly
// extending over the ulimit. Make sure compactions are aggressive in reducing
// the number of HFiles created.
@@ -3268,7 +3271,7 @@
this.region = initHRegion(tableName, method, conf, families);
PutThread putThread = null;
MultithreadedTestUtil.TestContext ctx =
- new MultithreadedTestUtil.TestContext(conf);
+ new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
try {
putThread = new PutThread(numRows, families, qualifiers);
putThread.start();
@@ -3354,7 +3357,7 @@
byte[] tableName = Bytes.toBytes(method);
byte[] family = Bytes.toBytes("family");
this.region = initHRegion(tableName, Bytes.toBytes("x"), Bytes.toBytes("z"), method,
- conf, family);
+ HBaseConfiguration.create(), family);
try {
byte[] rowNotServed = Bytes.toBytes("a");
Get g = new Get(rowNotServed);
@@ -3418,7 +3421,7 @@
//Setting up region
String method = "testIndexesScanWithOneDeletedRow";
- this.region = initHRegion(tableName, method, conf, family);
+ this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
try {
Put put = new Put(Bytes.toBytes(1L));
put.add(family, qual1, 1L, Bytes.toBytes(1L));
@@ -3871,6 +3874,7 @@
*/
@Test
public void testParallelIncrementWithMemStoreFlush() throws Exception {
+ Configuration conf = HBaseConfiguration.create();
String method = "testParallelIncrementWithMemStoreFlush";
byte[] tableName = Bytes.toBytes(method);
byte[] family = Incrementer.family;
@@ -4003,8 +4007,7 @@
}
private Configuration initSplit() {
- Configuration conf = HBaseConfiguration.create(this.conf);
-
+ Configuration conf = HBaseConfiguration.create();
// Always compact if there is more than one store file.
conf.setInt("hbase.hstore.compactionThreshold", 2);
@@ -4025,6 +4028,19 @@
/**
* @param tableName
* @param callingMethod
+ * @param families
+ * @return A region on which you must call {@link HRegion#closeHRegion(HRegion)} when done.
+ * @throws IOException
+ */
+ private static HRegion initHRegion (byte [] tableName, String callingMethod,
+ byte[] ... families)
+ throws IOException {
+ return initHRegion(tableName, callingMethod, HBaseConfiguration.create(), families);
+ }
+
+ /**
+ * @param tableName
+ * @param callingMethod
* @param conf
* @param families
* @throws IOException
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionBusyWait.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionBusyWait.java
deleted file mode 100644
index 10a9370..0000000
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionBusyWait.java
+++ /dev/null
@@ -1,90 +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.regionserver;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.RegionTooBusyException;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * TestHRegion with hbase.busy.wait.duration set to 1000 (1 second).
- * We can't use parameterized test since TestHRegion is old fashion.
- */
-@Category(MediumTests.class)
-@SuppressWarnings("deprecation")
-public class TestHRegionBusyWait extends TestHRegion {
- public TestHRegionBusyWait() {
- conf.set("hbase.busy.wait.duration", "1000");
- }
-
- /**
- * Test RegionTooBusyException thrown when region is busy
- */
- @Test (timeout=2000)
- public void testRegionTooBusy() throws IOException {
- String method = "testRegionTooBusy";
- byte[] tableName = Bytes.toBytes(method);
- byte[] family = Bytes.toBytes("family");
- region = initHRegion(tableName, method, conf, family);
- final AtomicBoolean stopped = new AtomicBoolean(true);
- Thread t = new Thread(new Runnable() {
- @Override
- public void run() {
- try {
- region.lock.writeLock().lock();
- stopped.set(false);
- while (!stopped.get()) {
- Thread.sleep(100);
- }
- } catch (InterruptedException ie) {
- } finally {
- region.lock.writeLock().unlock();
- }
- }
- });
- t.start();
- Get get = new Get(row);
- try {
- while (stopped.get()) {
- Thread.sleep(100);
- }
- region.get(get, null);
- fail("Should throw RegionTooBusyException");
- } catch (InterruptedException ie) {
- fail("test interrupted");
- } catch (RegionTooBusyException e) {
- // Good, expected
- } finally {
- stopped.set(true);
- try {
- t.join();
- } catch (Throwable e) {
- }
-
- HRegion.closeHRegion(region);
- region = null;
- }
- }
-}
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
index 956572c..79db715 100644
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
+++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
@@ -113,8 +113,7 @@
memstore.add(new KeyValue(row2, fam1, col1, data));
List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
- KeyValue k = memstore.get(0);
- qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+ qm.setRow(memstore.get(0).getRow());
for (KeyValue kv : memstore){
actual.add(qm.match(kv));
@@ -159,8 +158,7 @@
List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
- KeyValue k = memstore.get(0);
- qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+ qm.setRow(memstore.get(0).getRow());
for(KeyValue kv : memstore) {
actual.add(qm.match(kv));
@@ -212,8 +210,7 @@
new KeyValue(row2, fam1, col1, now-10, data)
};
- KeyValue k = kvs[0];
- qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+ qm.setRow(kvs[0].getRow());
List<MatchCode> actual = new ArrayList<MatchCode>(kvs.length);
for (KeyValue kv : kvs) {
@@ -265,8 +262,7 @@
new KeyValue(row1, fam2, col5, now-10000, data),
new KeyValue(row2, fam1, col1, now-10, data)
};
- KeyValue k = kvs[0];
- qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
+ qm.setRow(kvs[0].getRow());
List<ScanQueryMatcher.MatchCode> actual =
new ArrayList<ScanQueryMatcher.MatchCode>(kvs.length);
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 1d3d1db..e777081 100644
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -36,7 +36,6 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@@ -728,44 +727,6 @@
}
}
- @Test(timeout = 20000)
- public void testTableExistsIfTheSpecifiedTableRegionIsSplitParent() throws Exception {
- final byte[] tableName =
- Bytes.toBytes("testTableExistsIfTheSpecifiedTableRegionIsSplitParent");
- HRegionServer regionServer = null;
- List<HRegion> regions = null;
- HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
- try {
- // Create table then get the single region for our new table.
- HTableDescriptor htd = new HTableDescriptor(tableName);
- htd.addFamily(new HColumnDescriptor("cf"));
- admin.createTable(htd);
- HTable t = new HTable(cluster.getConfiguration(), tableName);
- regions = cluster.getRegions(tableName);
- int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
- regionServer = cluster.getRegionServer(regionServerIndex);
- insertData(tableName, admin, t);
- // Turn off balancer so it doesn't cut in and mess up our placements.
- cluster.getMaster().setCatalogJanitorEnabled(false);
- boolean tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
- Bytes.toString(tableName));
- assertEquals("The specified table should present.", true, tableExists);
- SplitTransaction st = new SplitTransaction(regions.get(0), Bytes.toBytes("row2"));
- try {
- st.prepare();
- st.createDaughters(regionServer, regionServer);
- } catch (IOException e) {
-
- }
- tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
- Bytes.toString(tableName));
- assertEquals("The specified table should present.", true, tableExists);
- } finally {
- cluster.getMaster().setCatalogJanitorEnabled(true);
- admin.close();
- }
- }
-
private void insertData(final byte[] tableName, HBaseAdmin admin, HTable t) throws IOException,
InterruptedException {
Put p = new Put(Bytes.toBytes("row1"));
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java b/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java
index 45c4727..16db167 100644
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java
+++ b/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java
@@ -49,9 +49,6 @@
HLogKey key = HLog.newKey(conf);
WALEdit val = new WALEdit();
HLog.Entry e = new HLog.Entry(key, val);
- if (compressionContext != null) {
- e.setCompressionContext(compressionContext);
- }
b = this.reader.next(e.getKey(), e.getEdit());
nextQueue.offer(e);
numberOfFileEntries++;
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java b/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
index 8feb5d6..9fa4480 100644
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
+++ b/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
@@ -475,13 +475,15 @@
throw t.exception;
// Make sure you can read all the content
- HLog.Reader reader = HLog.getReader(this.fs, walPath, this.conf);
+ SequenceFile.Reader reader
+ = new SequenceFile.Reader(this.fs, walPath, this.conf);
int count = 0;
- HLog.Entry entry = new HLog.Entry();
- while (reader.next(entry) != null) {
+ HLogKey key = HLog.newKey(conf);
+ WALEdit val = new WALEdit();
+ while (reader.next(key, val)) {
count++;
assertTrue("Should be one KeyValue per WALEdit",
- entry.getEdit().getKeyValues().size() == 1);
+ val.getKeyValues().size() == 1);
}
assertEquals(total, count);
reader.close();
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java b/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
index 3736628..69a4e22 100644
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
+++ b/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
@@ -86,7 +86,7 @@
private Configuration conf;
private FileSystem fs;
- protected final static HBaseTestingUtility
+ private final static HBaseTestingUtility
TEST_UTIL = new HBaseTestingUtility();
diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplitCompressed.java b/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplitCompressed.java
deleted file mode 100644
index 101678a..0000000
--- a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplitCompressed.java
+++ /dev/null
@@ -1,35 +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.regionserver.wal;
-
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.LargeTests;
-import org.junit.BeforeClass;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class TestHLogSplitCompressed extends TestHLogSplit {
-
- @BeforeClass
- public static void setUpBeforeClass() throws Exception {
- TestHLogSplit.setUpBeforeClass();
- TEST_UTIL.getConfiguration().setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
- }
-}
diff --git a/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java b/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
index bc2c16f..96eb211 100644
--- a/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
+++ b/src/test/java/org/apache/hadoop/hbase/replication/TestReplication.java
@@ -61,7 +61,7 @@
private static final Log LOG = LogFactory.getLog(TestReplication.class);
- protected static Configuration conf1 = HBaseConfiguration.create();
+ private static Configuration conf1;
private static Configuration conf2;
private static Configuration CONF_WITH_LOCALFS;
@@ -78,8 +78,8 @@
private static final int NB_ROWS_IN_BATCH = 100;
private static final int NB_ROWS_IN_BIG_BATCH =
NB_ROWS_IN_BATCH * 10;
- private static final long SLEEP_TIME = 1500;
- private static final int NB_RETRIES = 15;
+ private static final long SLEEP_TIME = 500;
+ private static final int NB_RETRIES = 10;
private static final byte[] tableName = Bytes.toBytes("test");
private static final byte[] famName = Bytes.toBytes("f");
@@ -91,6 +91,7 @@
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
+ conf1 = HBaseConfiguration.create();
conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
// smaller block size and capacity to trigger more operations
// and test them
@@ -519,7 +520,7 @@
// disable and start the peer
admin.disablePeer("2");
- utility2.startMiniHBaseCluster(1, 2);
+ utility2.startMiniHBaseCluster(1, 1);
Get get = new Get(rowkey);
for (int i = 0; i < NB_RETRIES; i++) {
Result res = htable2.get(get);
@@ -716,7 +717,7 @@
*/
@Test(timeout=300000)
public void queueFailover() throws Exception {
- utility1.createMultiRegions(htable1, famName, false);
+ utility1.createMultiRegions(htable1, famName);
// killing the RS with .META. can result into failed puts until we solve
// IO fencing
@@ -759,8 +760,7 @@
int lastCount = 0;
final long start = System.currentTimeMillis();
- int i = 0;
- while (true) {
+ for (int i = 0; i < NB_RETRIES; i++) {
if (i==NB_RETRIES-1) {
fail("Waited too much time for queueFailover replication. " +
"Waited "+(System.currentTimeMillis() - start)+"ms.");
@@ -772,8 +772,6 @@
if (res2.length < initialCount) {
if (lastCount < res2.length) {
i--; // Don't increment timeout if we make progress
- } else {
- i++;
}
lastCount = res2.length;
LOG.info("Only got " + lastCount + " rows instead of " +
@@ -793,7 +791,7 @@
Thread.sleep(timeout);
utility.expireRegionServerSession(rs);
} catch (Exception e) {
- LOG.error("Couldn't kill a region server", e);
+ LOG.error(e);
}
}
};
diff --git a/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithCompression.java b/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithCompression.java
deleted file mode 100644
index 2c87a7b..0000000
--- a/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithCompression.java
+++ /dev/null
@@ -1,40 +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.replication;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.LargeTests;
-import org.junit.BeforeClass;
-import org.junit.experimental.categories.Category;
-
-/**
- * Run the same test as TestReplication but with HLog compression enabled
- */
-@Category(LargeTests.class)
-public class TestReplicationWithCompression extends TestReplication {
-
- /**
- * @throws java.lang.Exception
- */
- @BeforeClass
- public static void setUpBeforeClass() throws Exception {
- conf1.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
- TestReplication.setUpBeforeClass();
- }
-}
diff --git a/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java b/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
index 41893a4..23673c7 100644
--- a/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
+++ b/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
@@ -114,6 +114,7 @@
protected Map<String, CmdDescriptor> commands = new TreeMap<String, CmdDescriptor>();
protected static Cluster cluster = new Cluster();
+ protected static String accessToken = null;
volatile Configuration conf;
private boolean nomapred = false;
@@ -448,7 +449,8 @@
*/
private boolean checkTable() throws IOException {
HTableDescriptor tableDescriptor = getTableDescriptor();
- RemoteAdmin admin = new RemoteAdmin(new Client(cluster), conf);
+ RemoteAdmin admin =
+ new RemoteAdmin(new Client(cluster), conf, accessToken);
if (!admin.isTableAvailable(tableDescriptor.getName())) {
admin.createTable(tableDescriptor);
return true;
@@ -712,7 +714,8 @@
}
void testSetup() throws IOException {
- this.table = new RemoteHTable(new Client(cluster), conf, tableName);
+ this.table = new RemoteHTable(new Client(cluster), conf, tableName,
+ accessToken);
}
void testTakedown() throws IOException {
@@ -1130,6 +1133,7 @@
System.err.println();
System.err.println("Options:");
System.err.println(" host String. Specify Stargate endpoint.");
+ System.err.println(" token String. API access token.");
System.err.println(" rows Integer. Rows each client runs. Default: One million");
System.err.println(" rowsPerPut Integer. Rows each Stargate (multi)Put. Default: 100");
System.err.println(" nomapred (Flag) Run multiple clients using threads " +
@@ -1204,6 +1208,12 @@
continue;
}
+ final String token = "--token=";
+ if (cmd.startsWith(token)) {
+ accessToken = cmd.substring(token.length());
+ continue;
+ }
+
Class<? extends Test> cmdClass = determineCommandClass(cmd);
if (cmdClass != null) {
getArgs(i + 1, args);
diff --git a/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
index a3ab9ec..7b7a670 100644
--- a/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
+++ b/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
@@ -101,7 +101,7 @@
remoteTable = new RemoteHTable(
new Client(new Cluster().add("localhost",
REST_TEST_UTIL.getServletPort())),
- TEST_UTIL.getConfiguration(), TABLE);
+ TEST_UTIL.getConfiguration(), TABLE, null);
}
@AfterClass
@@ -222,45 +222,6 @@
}
@Test
- public void testMultiGet() throws Exception {
- ArrayList<Get> gets = new ArrayList<Get>();
- gets.add(new Get(ROW_1));
- gets.add(new Get(ROW_2));
- Result[] results = remoteTable.get(gets);
- assertNotNull(results);
- assertEquals(2, results.length);
- assertEquals(1, results[0].size());
- assertEquals(2, results[1].size());
-
- //Test Versions
- gets = new ArrayList<Get>();
- Get g = new Get(ROW_1);
- g.setMaxVersions(3);
- gets.add(g);
- gets.add(new Get(ROW_2));
- results = remoteTable.get(gets);
- assertNotNull(results);
- assertEquals(2, results.length);
- assertEquals(1, results[0].size());
- assertEquals(3, results[1].size());
-
- //404
- gets = new ArrayList<Get>();
- gets.add(new Get(Bytes.toBytes("RESALLYREALLYNOTTHERE")));
- results = remoteTable.get(gets);
- assertNotNull(results);
- assertEquals(0, results.length);
-
- gets = new ArrayList<Get>();
- gets.add(new Get(Bytes.toBytes("RESALLYREALLYNOTTHERE")));
- gets.add(new Get(ROW_1));
- gets.add(new Get(ROW_2));
- results = remoteTable.get(gets);
- assertNotNull(results);
- assertEquals(0, results.length);
- }
-
- @Test
public void testPut() throws IOException {
Put put = new Put(ROW_3);
put.add(COLUMN_1, QUALIFIER_1, VALUE_1);
diff --git a/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java b/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
index 21ac529..339a120 100644
--- a/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
+++ b/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
@@ -90,26 +90,20 @@
// given the default replication factor is 3, the same as the number of
// datanodes; the locality index for each host should be 100%,
// or getWeight for each host should be the same as getUniqueBlocksWeights
- final long maxTime = System.currentTimeMillis() + 2000;
- boolean ok;
- do {
- ok = true;
- FileStatus status = fs.getFileStatus(testFile);
- HDFSBlocksDistribution blocksDistribution =
- FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
- long uniqueBlocksTotalWeight =
- blocksDistribution.getUniqueBlocksTotalWeight();
- for (String host : hosts) {
- long weight = blocksDistribution.getWeight(host);
- ok = (ok && uniqueBlocksTotalWeight == weight);
- }
- } while (!ok && System.currentTimeMillis() < maxTime);
- assertTrue(ok);
- } finally {
+ FileStatus status = fs.getFileStatus(testFile);
+ HDFSBlocksDistribution blocksDistribution =
+ FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
+ long uniqueBlocksTotalWeight =
+ blocksDistribution.getUniqueBlocksTotalWeight();
+ for (String host : hosts) {
+ long weight = blocksDistribution.getWeight(host);
+ assertTrue(uniqueBlocksTotalWeight == weight);
+ }
+ } finally {
htu.shutdownMiniDFSCluster();
}
-
+
try {
// set up a cluster with 4 nodes
String hosts[] = new String[] { "host1", "host2", "host3", "host4" };
@@ -124,22 +118,16 @@
// given the default replication factor is 3, we will have total of 9
// replica of blocks; thus the host with the highest weight should have
// weight == 3 * DEFAULT_BLOCK_SIZE
- final long maxTime = System.currentTimeMillis() + 2000;
- long weight;
- long uniqueBlocksTotalWeight;
- do {
- FileStatus status = fs.getFileStatus(testFile);
- HDFSBlocksDistribution blocksDistribution =
- FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
- uniqueBlocksTotalWeight = blocksDistribution.getUniqueBlocksTotalWeight();
-
- String tophost = blocksDistribution.getTopHosts().get(0);
- weight = blocksDistribution.getWeight(tophost);
-
- // NameNode is informed asynchronously, so we may have a delay. See HBASE-6175
- } while (uniqueBlocksTotalWeight != weight && System.currentTimeMillis() < maxTime);
+ FileStatus status = fs.getFileStatus(testFile);
+ HDFSBlocksDistribution blocksDistribution =
+ FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
+ long uniqueBlocksTotalWeight =
+ blocksDistribution.getUniqueBlocksTotalWeight();
+
+ String tophost = blocksDistribution.getTopHosts().get(0);
+ long weight = blocksDistribution.getWeight(tophost);
assertTrue(uniqueBlocksTotalWeight == weight);
-
+
} finally {
htu.shutdownMiniDFSCluster();
}
@@ -158,16 +146,11 @@
// given the default replication factor is 3, we will have total of 3
// replica of blocks; thus there is one host without weight
- final long maxTime = System.currentTimeMillis() + 2000;
- HDFSBlocksDistribution blocksDistribution;
- do {
- FileStatus status = fs.getFileStatus(testFile);
- blocksDistribution = FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
- // NameNode is informed asynchronously, so we may have a delay. See HBASE-6175
- }
- while (blocksDistribution.getTopHosts().size() != 3 && System.currentTimeMillis() < maxTime);
+ FileStatus status = fs.getFileStatus(testFile);
+ HDFSBlocksDistribution blocksDistribution =
+ FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
assertEquals("Wrong number of hosts distributing blocks.", 3,
- blocksDistribution.getTopHosts().size());
+ blocksDistribution.getTopHosts().size());
} finally {
htu.shutdownMiniDFSCluster();
}
diff --git a/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
index 3771d9a..e0d3c19 100644
--- a/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
+++ b/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
@@ -1658,35 +1658,6 @@
}
/**
- * Test fixing lingering reference file.
- */
- @Test
- public void testLingeringReferenceFile() throws Exception {
- String table = "testLingeringReferenceFile";
- try {
- setupTable(table);
- assertEquals(ROWKEYS.length, countRows());
-
- // Mess it up by creating a fake reference file
- FileSystem fs = FileSystem.get(conf);
- Path tableDir= FSUtils.getTablePath(FSUtils.getRootDir(conf), table);
- Path regionDir = FSUtils.getRegionDirs(fs, tableDir).get(0);
- Path famDir = new Path(regionDir, FAM_STR);
- Path fakeReferenceFile = new Path(famDir, "fbce357483ceea.12144538");
- fs.create(fakeReferenceFile);
-
- HBaseFsck hbck = doFsck(conf, false);
- assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.LINGERING_REFERENCE_HFILE });
- // fix reference file
- doFsck(conf, true);
- // check that reference file fixed
- assertNoErrors(doFsck(conf, false));
- } finally {
- deleteTable(table);
- }
- }
-
- /**
* Test pluggable error reporter. It can be plugged in
* from system property or configuration.
*/
diff --git a/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java b/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
index 99f4f9b..299ae6e 100644
--- a/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
+++ b/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
@@ -30,7 +30,6 @@
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
public class HbckTestingUtil {
- private static ExecutorService exec = new ScheduledThreadPoolExecutor(10);
public static HBaseFsck doFsck(
Configuration conf, boolean fix) throws Exception {
return doFsck(conf, fix, null);
@@ -38,14 +37,14 @@
public static HBaseFsck doFsck(
Configuration conf, boolean fix, String table) throws Exception {
- return doFsck(conf, fix, fix, fix, fix,fix, fix, fix, fix, table);
+ return doFsck(conf, fix, fix, fix, fix,fix, fix, fix, table);
}
public static HBaseFsck doFsck(Configuration conf, boolean fixAssignments,
boolean fixMeta, boolean fixHdfsHoles, boolean fixHdfsOverlaps,
boolean fixHdfsOrphans, boolean fixTableOrphans, boolean fixVersionFile,
- boolean fixReferenceFiles, String table) throws Exception {
- HBaseFsck fsck = new HBaseFsck(conf, exec);
+ String table) throws Exception {
+ HBaseFsck fsck = new HBaseFsck(conf);
fsck.connect();
fsck.setDisplayFullReport(); // i.e. -details
fsck.setTimeLag(0);
@@ -56,7 +55,6 @@
fsck.setFixHdfsOrphans(fixHdfsOrphans);
fsck.setFixTableOrphans(fixTableOrphans);
fsck.setFixVersionFile(fixVersionFile);
- fsck.setFixReferenceFiles(fixReferenceFiles);
if (table != null) {
fsck.includeTable(table);
}
@@ -73,6 +71,7 @@
*/
public static HBaseFsck doHFileQuarantine(Configuration conf, String table) throws Exception {
String[] args = {"-sidelineCorruptHFiles", "-ignorePreCheckPermission", table};
+ ExecutorService exec = new ScheduledThreadPoolExecutor(10);
HBaseFsck hbck = new HBaseFsck(conf, exec);
hbck.exec(exec, args);
return hbck;
diff --git a/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java b/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
deleted file mode 100644
index dd00372..0000000
--- a/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
+++ /dev/null
@@ -1,291 +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.zookeeper;
-
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.LinkedList;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
-import org.apache.zookeeper.KeeperException;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test ZooKeeper multi-update functionality
- */
-@Category(MediumTests.class)
-public class TestZKMulti {
- private static final Log LOG = LogFactory.getLog(TestZKMulti.class);
- private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
- private static ZooKeeperWatcher zkw = null;
-
- @BeforeClass
- public static void setUpBeforeClass() throws Exception {
- TEST_UTIL.startMiniZKCluster();
- Configuration conf = TEST_UTIL.getConfiguration();
- conf.setBoolean("hbase.zookeeper.useMulti", true);
- Abortable abortable = new Abortable() {
- @Override
- public void abort(String why, Throwable e) {
- LOG.info(why, e);
- }
-
- @Override
- public boolean isAborted() {
- return false;
- }
- };
- zkw = new ZooKeeperWatcher(conf,
- "TestZKMulti", abortable, true);
- }
-
- @AfterClass
- public static void tearDownAfterClass() throws Exception {
- TEST_UTIL.shutdownMiniZKCluster();
- }
-
- @Test
- public void testSimpleMulti() throws Exception {
- // null multi
- ZKUtil.multiOrSequential(zkw, null, false);
-
- // empty multi
- ZKUtil.multiOrSequential(zkw, new LinkedList<ZKUtilOp>(), false);
-
- // single create
- String path = ZKUtil.joinZNode(zkw.baseZNode, "testSimpleMulti");
- LinkedList<ZKUtilOp> singleCreate = new LinkedList<ZKUtilOp>();
- singleCreate.add(ZKUtilOp.createAndFailSilent(path, new byte[0]));
- ZKUtil.multiOrSequential(zkw, singleCreate, false);
- assertTrue(ZKUtil.checkExists(zkw, path) != -1);
-
- // single setdata
- LinkedList<ZKUtilOp> singleSetData = new LinkedList<ZKUtilOp>();
- byte [] data = Bytes.toBytes("foobar");
- singleSetData.add(ZKUtilOp.setData(path, data));
- ZKUtil.multiOrSequential(zkw, singleSetData, false);
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path), data));
-
- // single delete
- LinkedList<ZKUtilOp> singleDelete = new LinkedList<ZKUtilOp>();
- singleDelete.add(ZKUtilOp.deleteNodeFailSilent(path));
- ZKUtil.multiOrSequential(zkw, singleDelete, false);
- assertTrue(ZKUtil.checkExists(zkw, path) == -1);
- }
-
- @Test
- public void testComplexMulti() throws Exception {
- String path1 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti1");
- String path2 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti2");
- String path3 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti3");
- String path4 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti4");
- String path5 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti5");
- String path6 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti6");
- // create 4 nodes that we'll setData on or delete later
- LinkedList<ZKUtilOp> create4Nodes = new LinkedList<ZKUtilOp>();
- create4Nodes.add(ZKUtilOp.createAndFailSilent(path1, Bytes.toBytes(path1)));
- create4Nodes.add(ZKUtilOp.createAndFailSilent(path2, Bytes.toBytes(path2)));
- create4Nodes.add(ZKUtilOp.createAndFailSilent(path3, Bytes.toBytes(path3)));
- create4Nodes.add(ZKUtilOp.createAndFailSilent(path4, Bytes.toBytes(path4)));
- ZKUtil.multiOrSequential(zkw, create4Nodes, false);
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path1), Bytes.toBytes(path1)));
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path2), Bytes.toBytes(path2)));
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path3), Bytes.toBytes(path3)));
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path4), Bytes.toBytes(path4)));
-
- // do multiple of each operation (setData, delete, create)
- LinkedList<ZKUtilOp> ops = new LinkedList<ZKUtilOp>();
- // setData
- ops.add(ZKUtilOp.setData(path1, Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1))));
- ops.add(ZKUtilOp.setData(path2, Bytes.add(Bytes.toBytes(path2), Bytes.toBytes(path2))));
- // delete
- ops.add(ZKUtilOp.deleteNodeFailSilent(path3));
- ops.add(ZKUtilOp.deleteNodeFailSilent(path4));
- // create
- ops.add(ZKUtilOp.createAndFailSilent(path5, Bytes.toBytes(path5)));
- ops.add(ZKUtilOp.createAndFailSilent(path6, Bytes.toBytes(path6)));
- ZKUtil.multiOrSequential(zkw, ops, false);
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path1),
- Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1))));
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path2),
- Bytes.add(Bytes.toBytes(path2), Bytes.toBytes(path2))));
- assertTrue(ZKUtil.checkExists(zkw, path3) == -1);
- assertTrue(ZKUtil.checkExists(zkw, path4) == -1);
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path5), Bytes.toBytes(path5)));
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path6), Bytes.toBytes(path6)));
- }
-
- @Test
- public void testSingleFailure() throws Exception {
- // try to delete a node that doesn't exist
- boolean caughtNoNode = false;
- String path = ZKUtil.joinZNode(zkw.baseZNode, "testSingleFailureZ");
- LinkedList<ZKUtilOp> ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.deleteNodeFailSilent(path));
- try {
- ZKUtil.multiOrSequential(zkw, ops, false);
- } catch (KeeperException.NoNodeException nne) {
- caughtNoNode = true;
- }
- assertTrue(caughtNoNode);
-
- // try to setData on a node that doesn't exist
- caughtNoNode = false;
- ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.setData(path, Bytes.toBytes(path)));
- try {
- ZKUtil.multiOrSequential(zkw, ops, false);
- } catch (KeeperException.NoNodeException nne) {
- caughtNoNode = true;
- }
- assertTrue(caughtNoNode);
-
- // try to create on a node that already exists
- boolean caughtNodeExists = false;
- ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.createAndFailSilent(path, Bytes.toBytes(path)));
- ZKUtil.multiOrSequential(zkw, ops, false);
- try {
- ZKUtil.multiOrSequential(zkw, ops, false);
- } catch (KeeperException.NodeExistsException nee) {
- caughtNodeExists = true;
- }
- assertTrue(caughtNodeExists);
- }
-
- @Test
- public void testSingleFailureInMulti() throws Exception {
- // try a multi where all but one operation succeeds
- String pathA = ZKUtil.joinZNode(zkw.baseZNode, "testSingleFailureInMultiA");
- String pathB = ZKUtil.joinZNode(zkw.baseZNode, "testSingleFailureInMultiB");
- String pathC = ZKUtil.joinZNode(zkw.baseZNode, "testSingleFailureInMultiC");
- LinkedList<ZKUtilOp> ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.createAndFailSilent(pathA, Bytes.toBytes(pathA)));
- ops.add(ZKUtilOp.createAndFailSilent(pathB, Bytes.toBytes(pathB)));
- ops.add(ZKUtilOp.deleteNodeFailSilent(pathC));
- boolean caughtNoNode = false;
- try {
- ZKUtil.multiOrSequential(zkw, ops, false);
- } catch (KeeperException.NoNodeException nne) {
- caughtNoNode = true;
- }
- assertTrue(caughtNoNode);
- // assert that none of the operations succeeded
- assertTrue(ZKUtil.checkExists(zkw, pathA) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathB) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathC) == -1);
- }
-
- @Test
- public void testMultiFailure() throws Exception {
- String pathX = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureX");
- String pathY = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureY");
- String pathZ = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureZ");
- // create X that we will use to fail create later
- LinkedList<ZKUtilOp> ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX)));
- ZKUtil.multiOrSequential(zkw, ops, false);
-
- // fail one of each create ,setData, delete
- String pathV = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureV");
- String pathW = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureW");
- ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX))); // fail -- already exists
- ops.add(ZKUtilOp.setData(pathY, Bytes.toBytes(pathY))); // fail -- doesn't exist
- ops.add(ZKUtilOp.deleteNodeFailSilent(pathZ)); // fail -- doesn't exist
- ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathV))); // pass
- ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathW))); // pass
- boolean caughtNodeExists = false;
- try {
- ZKUtil.multiOrSequential(zkw, ops, false);
- } catch (KeeperException.NodeExistsException nee) {
- // check first operation that fails throws exception
- caughtNodeExists = true;
- }
- assertTrue(caughtNodeExists);
- // check that no modifications were made
- assertFalse(ZKUtil.checkExists(zkw, pathX) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathY) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathZ) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathW) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathV) == -1);
-
- // test that with multiple failures, throws an exception corresponding to first failure in list
- ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.setData(pathY, Bytes.toBytes(pathY))); // fail -- doesn't exist
- ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX))); // fail -- exists
- boolean caughtNoNode = false;
- try {
- ZKUtil.multiOrSequential(zkw, ops, false);
- } catch (KeeperException.NoNodeException nne) {
- // check first operation that fails throws exception
- caughtNoNode = true;
- }
- assertTrue(caughtNoNode);
- // check that no modifications were made
- assertFalse(ZKUtil.checkExists(zkw, pathX) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathY) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathZ) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathW) == -1);
- assertTrue(ZKUtil.checkExists(zkw, pathV) == -1);
- }
-
- @Test
- public void testRunSequentialOnMultiFailure() throws Exception {
- String path1 = ZKUtil.joinZNode(zkw.baseZNode, "runSequential1");
- String path2 = ZKUtil.joinZNode(zkw.baseZNode, "runSequential2");
- String path3 = ZKUtil.joinZNode(zkw.baseZNode, "runSequential3");
- String path4 = ZKUtil.joinZNode(zkw.baseZNode, "runSequential4");
-
- // create some nodes that we will use later
- LinkedList<ZKUtilOp> ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.createAndFailSilent(path1, Bytes.toBytes(path1)));
- ops.add(ZKUtilOp.createAndFailSilent(path2, Bytes.toBytes(path2)));
- ZKUtil.multiOrSequential(zkw, ops, false);
-
- // test that, even with operations that fail, the ones that would pass will pass
- // with runSequentialOnMultiFailure
- ops = new LinkedList<ZKUtilOp>();
- ops.add(ZKUtilOp.setData(path1, Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1)))); // pass
- ops.add(ZKUtilOp.deleteNodeFailSilent(path2)); // pass
- ops.add(ZKUtilOp.deleteNodeFailSilent(path3)); // fail -- node doesn't exist
- ops.add(ZKUtilOp.createAndFailSilent(path4,
- Bytes.add(Bytes.toBytes(path4), Bytes.toBytes(path4)))); // pass
- ZKUtil.multiOrSequential(zkw, ops, true);
- assertTrue(Bytes.equals(ZKUtil.getData(zkw, path1),
- Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1))));
- assertTrue(ZKUtil.checkExists(zkw, path2) == -1);
- assertTrue(ZKUtil.checkExists(zkw, path3) == -1);
- assertFalse(ZKUtil.checkExists(zkw, path4) == -1);
- }
-}
diff --git a/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java b/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java
index 643c583..4748b95 100644
--- a/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java
+++ b/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java
@@ -27,7 +27,6 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.zookeeper.ZKTable.TableState;
import org.apache.zookeeper.KeeperException;
@@ -111,9 +110,14 @@
assertFalse(zkt.isTablePresent(name));
}
- private void runTest9294CompatibilityTest(String tableName, Configuration conf)
- throws Exception {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+ /**
+ * Test that ZK table writes table state in formats expected by 0.92 and 0.94 clients
+ */
+ @Test
+ public void test9294Compatibility() throws Exception {
+ final String tableName = "test9294Compatibility";
+
+ ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
tableName, abortable, true);
ZKTable zkt = new ZKTable(zkw);
zkt.setEnabledTable(tableName);
@@ -125,22 +129,6 @@
}
/**
- * Test that ZK table writes table state in formats expected by 0.92 and 0.94 clients
- */
- @Test
- public void test9294Compatibility() throws Exception {
- // without useMulti
- String tableName = "test9294Compatibility";
- runTest9294CompatibilityTest(tableName, TEST_UTIL.getConfiguration());
-
- // with useMulti
- tableName = "test9294CompatibilityWithMulti";
- Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
- conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
- runTest9294CompatibilityTest(tableName, conf);
- }
-
- /**
* RecoverableZookeeper that throws a KeeperException after throwExceptionInNumOperations
*/
class ThrowingRecoverableZookeeper extends RecoverableZooKeeper {
@@ -182,17 +170,14 @@
* Because two ZooKeeper nodes are written for each table state transition
* {@link ZooKeeperWatcher#masterTableZNode} and {@link ZooKeeperWatcher#masterTableZNode92}
* it is possible that we fail in between the two operations and are left with
- * inconsistent state (when hbase.zookeeper.useMulti is false).
- * Check that we can get back to a consistent state by retrying the operation.
+ * inconsistent state. Check that we can get back to a consistent state by
+ * retrying the operation.
*/
@Test
public void testDisableTableRetry() throws Exception {
final String tableName = "testDisableTableRetry";
- Configuration conf = TEST_UTIL.getConfiguration();
- // test only relevant if useMulti is false
- conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, false);
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+ ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
tableName, abortable, true);
ThrowingRecoverableZookeeper throwing = new ThrowingRecoverableZookeeper(zkw);
ZooKeeperWatcher spyZookeeperWatcher = Mockito.spy(zkw);
@@ -226,10 +211,7 @@
public void testEnableTableRetry() throws Exception {
final String tableName = "testEnableTableRetry";
- Configuration conf = TEST_UTIL.getConfiguration();
- // test only relevant if useMulti is false
- conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, false);
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+ ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
tableName, abortable, true);
ThrowingRecoverableZookeeper throwing = new ThrowingRecoverableZookeeper(zkw);
ZooKeeperWatcher spyZookeeperWatcher = Mockito.spy(zkw);
@@ -263,10 +245,7 @@
public void testDeleteTableRetry() throws Exception {
final String tableName = "testEnableTableRetry";
- Configuration conf = TEST_UTIL.getConfiguration();
- // test only relevant if useMulti is false
- conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, false);
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+ ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
tableName, abortable, true);
ThrowingRecoverableZookeeper throwing = new ThrowingRecoverableZookeeper(zkw);
ZooKeeperWatcher spyZookeeperWatcher = Mockito.spy(zkw);
diff --git a/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTableReadOnly.java b/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTableReadOnly.java
index 36baf6c..12b398a 100644
--- a/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTableReadOnly.java
+++ b/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTableReadOnly.java
@@ -68,53 +68,33 @@
return ZKTableReadOnly.isEnabledTable(zkw, tableName);
}
- private void runClientCompatiblityWith92ZNodeTest(String tableName, Configuration conf)
- throws Exception {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
- tableName, abortable, true);
- assertTrue(enableAndCheckEnabled(zkw, tableName));
- }
/**
- * Test that client ZK reader can handle the 0.92 table format znode.
+ * Test that client ZK reader can handle the 0.92 table znode format.
*/
@Test
public void testClientCompatibilityWith92ZNode() throws Exception {
- // test without useMulti
- String tableName = "testClientCompatibilityWith92ZNode";
+ final String tableName = "testClientCompatibilityWith92ZNode";
+
// Set the client to read from the 0.92 table znode format
Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
String znode92 = conf.get("zookeeper.znode.masterTableEnableDisable92", "table92");
conf.set("zookeeper.znode.clientTableEnableDisable", znode92);
- runClientCompatiblityWith92ZNodeTest(tableName, conf);
- // test with useMulti
- tableName = "testClientCompatibilityWith92ZNodeUseMulti";
- conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
- runClientCompatiblityWith92ZNodeTest(tableName, conf);
- }
-
- private void runClientCompatibilityWith94ZNodeTest(String tableName, Configuration conf)
- throws Exception {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+ ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
tableName, abortable, true);
assertTrue(enableAndCheckEnabled(zkw, tableName));
}
/**
- * Test that client ZK reader can handle the current (0.94) table format znode.
+ * Test that client ZK reader can handle the current (0.94) table format znode
*/
@Test
public void testClientCompatibilityWith94ZNode() throws Exception {
- String tableName = "testClientCompatibilityWith94ZNode";
+ final String tableName = "testClientCompatibilityWith94ZNode";
- // without useMulti
- runClientCompatibilityWith94ZNodeTest(tableName, TEST_UTIL.getConfiguration());
-
- // with useMulti
- tableName = "testClientCompatiblityWith94ZNodeUseMulti";
- Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
- conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
- runClientCompatibilityWith94ZNodeTest(tableName, conf);
+ ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+ tableName, abortable, true);
+ assertTrue(enableAndCheckEnabled(zkw, tableName));
}
@org.junit.Rule