HBASE-2979 Fix failing TestMultParrallel in hudson build
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@997437 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
787d807b37
commit
5d8b73585f
|
@ -521,6 +521,7 @@ Release 0.21.0 - Unreleased
|
|||
HBASE-2983 TestHLog unit test is mis-comparing an assertion
|
||||
(Alex Newman via Todd Lipcon)
|
||||
HBASE-2986 multi writable can npe causing client hang
|
||||
HBASE-2979 Fix failing TestMultParrallel in hudson build
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-1760 Cleanup TODOs in HTable
|
||||
|
|
|
@ -82,89 +82,3 @@ else
|
|||
--hosts "${HBASE_REGIONSERVERS}" restart regionserver
|
||||
|
||||
fi
|
||||
|
||||
#!/usr/bin/env bash
|
||||
#
|
||||
#/**
|
||||
# * Copyright 2010 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.
|
||||
# */
|
||||
#
|
||||
# Run a shell command on all regionserver hosts.
|
||||
#
|
||||
# Environment Variables
|
||||
#
|
||||
# HBASE_REGIONSERVERS File naming remote hosts.
|
||||
# Default is ${HADOOP_CONF_DIR}/regionservers
|
||||
# HADOOP_CONF_DIR Alternate conf dir. Default is ${HADOOP_HOME}/conf.
|
||||
# HBASE_CONF_DIR Alternate hbase conf dir. Default is ${HBASE_HOME}/conf.
|
||||
# HADOOP_SLAVE_SLEEP Seconds to sleep between spawning remote commands.
|
||||
# HADOOP_SLAVE_TIMEOUT Seconds to wait for timing out a remote command.
|
||||
# HADOOP_SSH_OPTS Options passed to ssh when running remote commands.
|
||||
#
|
||||
# Modelled after $HADOOP_HOME/bin/slaves.sh.
|
||||
|
||||
usage="Usage: $0 [--config <hbase-confdir>] commands..."
|
||||
|
||||
bin=`dirname "$0"`
|
||||
bin=`cd "$bin">/dev/null; pwd`
|
||||
|
||||
. "$bin"/hbase-config.sh
|
||||
|
||||
# start hbase daemons
|
||||
errCode=$?
|
||||
if [ $errCode -ne 0 ]
|
||||
then
|
||||
exit $errCode
|
||||
fi
|
||||
|
||||
# quick function to get a value from the HBase config file
|
||||
distMode=`$bin/hbase org.apache.hadoop.hbase.util.HBaseConfTool hbase.cluster.distributed`
|
||||
if [ "$distMode" == 'false' ]; then
|
||||
"$bin"/hbase-daemon.sh restart master
|
||||
else
|
||||
# stop all masters before re-start to avoid races for master znode
|
||||
"$bin"/hbase-daemon.sh --config "${HBASE_CONF_DIR}" stop master
|
||||
"$bin"/hbase-daemons.sh --config "${HBASE_CONF_DIR}" \
|
||||
--hosts "${HBASE_BACKUP_MASTERS}" stop master-backup
|
||||
|
||||
# make sure the master znode has been deleted before continuing
|
||||
zparent=`$bin/hbase org.apache.hadoop.hbase.util.HBaseConfTool zookeeper.znode.parent`
|
||||
if [ "$zparent" == "null" ]; then zparent="/hbase"; fi
|
||||
zmaster=`$bin/hbase org.apache.hadoop.hbase.util.HBaseConfTool zookeeper.znode.master`
|
||||
if [ "$zmaster" == "null" ]; then zmaster="master"; fi
|
||||
zmaster=$zparent/$zmaster
|
||||
echo -n "Waiting for Master ZNode to expire"
|
||||
while bin/hbase zkcli stat $zmaster >/dev/null 2>&1; do
|
||||
echo -n "."
|
||||
sleep 1
|
||||
done
|
||||
echo #force a newline
|
||||
|
||||
# all masters are down, now restart
|
||||
"$bin"/hbase-daemon.sh --config "${HBASE_CONF_DIR}" start master
|
||||
"$bin"/hbase-daemons.sh --config "${HBASE_CONF_DIR}" \
|
||||
--hosts "${HBASE_BACKUP_MASTERS}" start master-backup
|
||||
|
||||
# unlike the masters, roll all regionservers one-at-a-time
|
||||
export HBASE_SLAVE_PARALLEL=false
|
||||
"$bin"/hbase-daemons.sh --config "${HBASE_CONF_DIR}" \
|
||||
--hosts "${HBASE_REGIONSERVERS}" restart regionserver
|
||||
|
||||
fi
|
||||
|
||||
|
|
|
@ -30,6 +30,8 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
|
@ -166,7 +168,8 @@ public class LocalHBaseCluster {
|
|||
public List<JVMClusterUtil.RegionServerThread> getLiveRegionServers() {
|
||||
List<JVMClusterUtil.RegionServerThread> liveServers =
|
||||
new ArrayList<JVMClusterUtil.RegionServerThread>();
|
||||
for (JVMClusterUtil.RegionServerThread rst: getRegionServers()) {
|
||||
List<RegionServerThread> list = getRegionServers();
|
||||
for (JVMClusterUtil.RegionServerThread rst: list) {
|
||||
if (rst.isAlive()) liveServers.add(rst);
|
||||
}
|
||||
return liveServers;
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.hadoop.hbase.catalog;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.net.ConnectException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
|
@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.NotServingRegionException;
|
|||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaNodeTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
|
||||
|
@ -43,38 +45,34 @@ import org.apache.zookeeper.KeeperException;
|
|||
/**
|
||||
* Tracks the availability of the catalog tables <code>-ROOT-</code> and
|
||||
* <code>.META.</code>.
|
||||
* <p>
|
||||
*
|
||||
* This class is "read-only" in that the locations of the catalog tables cannot
|
||||
* be explicitly set. Instead, ZooKeeper is used to learn of the availability
|
||||
* and location of ROOT. ROOT is used to learn of the location of META. If not
|
||||
* available in ROOT, ZooKeeper is used to monitor for a new location of META.
|
||||
* and location of <code>-ROOT-</code>. <code>-ROOT-</code> is used to learn of
|
||||
* the location of <code>.META.</code> If not available in <code>-ROOT-</code>,
|
||||
* ZooKeeper is used to monitor for a new location of <code>.META.</code>.
|
||||
*
|
||||
* <p>Call {@link #start()} to start up operation.
|
||||
*/
|
||||
public class CatalogTracker {
|
||||
private static final Log LOG = LogFactory.getLog(CatalogTracker.class);
|
||||
|
||||
private final HConnection connection;
|
||||
|
||||
private final ZooKeeperWatcher zookeeper;
|
||||
|
||||
private final RootRegionTracker rootRegionTracker;
|
||||
|
||||
private final MetaNodeTracker metaNodeTracker;
|
||||
|
||||
private final AtomicBoolean metaAvailable = new AtomicBoolean(false);
|
||||
private HServerAddress metaLocation;
|
||||
|
||||
private final int defaultTimeout;
|
||||
|
||||
public static final byte [] ROOT_REGION =
|
||||
HRegionInfo.ROOT_REGIONINFO.getRegionName();
|
||||
|
||||
public static final byte [] META_REGION =
|
||||
HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
|
||||
|
||||
/**
|
||||
* Constructs the catalog tracker. Find current state of catalog tables and
|
||||
* begin active tracking by executing {@link #start()}.
|
||||
* begin active tracking by executing {@link #start()} post construction.
|
||||
* Does not timeout.
|
||||
* @param zk
|
||||
* @param connection server connection
|
||||
* @param abortable if fatal exception
|
||||
|
@ -88,11 +86,12 @@ public class CatalogTracker {
|
|||
|
||||
/**
|
||||
* Constructs the catalog tracker. Find current state of catalog tables and
|
||||
* begin active tracking by executing {@link #start()}.
|
||||
* begin active tracking by executing {@link #start()} post construction.
|
||||
* @param zk
|
||||
* @param connection server connection
|
||||
* @param abortable if fatal exception
|
||||
* @param defaultTimeout Timeout to use.
|
||||
* @param defaultTimeout Timeout to use. Pass zero for no timeout
|
||||
* ({@link Object#wait(long)} when passed a <code>0</code> waits for ever).
|
||||
* @throws IOException
|
||||
*/
|
||||
public CatalogTracker(final ZooKeeperWatcher zk, final HConnection connection,
|
||||
|
@ -101,26 +100,22 @@ public class CatalogTracker {
|
|||
this.zookeeper = zk;
|
||||
this.connection = connection;
|
||||
this.rootRegionTracker = new RootRegionTracker(zookeeper, abortable);
|
||||
this.metaNodeTracker = new MetaNodeTracker(zookeeper, this);
|
||||
this.metaNodeTracker = new MetaNodeTracker(zookeeper, this, abortable);
|
||||
this.defaultTimeout = defaultTimeout;
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts the catalog tracker.
|
||||
* <p>
|
||||
* Determines current availability of catalog tables and ensures all further
|
||||
* transitions of either region is tracked.
|
||||
* transitions of either region are tracked.
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public void start() throws IOException, InterruptedException {
|
||||
// Register listeners with zk
|
||||
zookeeper.registerListener(rootRegionTracker);
|
||||
zookeeper.registerListener(metaNodeTracker);
|
||||
// Start root tracking
|
||||
rootRegionTracker.start();
|
||||
this.rootRegionTracker.start();
|
||||
this.metaNodeTracker.start();
|
||||
// Determine meta assignment; may not work because root and meta not yet
|
||||
// deployed.
|
||||
// deployed. Calling the below will set {@link #metaLocation}.
|
||||
getMetaServerConnection(true);
|
||||
}
|
||||
|
||||
|
@ -148,7 +143,7 @@ public class CatalogTracker {
|
|||
*/
|
||||
public void waitForRoot()
|
||||
throws InterruptedException {
|
||||
rootRegionTracker.getRootRegionLocation();
|
||||
this.rootRegionTracker.blockUntilAvailable();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -161,7 +156,7 @@ public class CatalogTracker {
|
|||
* @throws NotAllMetaRegionsOnlineException if root not available before
|
||||
* timeout
|
||||
*/
|
||||
public HServerAddress waitForRoot(final long timeout)
|
||||
HServerAddress waitForRoot(final long timeout)
|
||||
throws InterruptedException, NotAllMetaRegionsOnlineException {
|
||||
HServerAddress address = rootRegionTracker.waitRootRegionLocation(timeout);
|
||||
if (address == null) {
|
||||
|
@ -235,27 +230,27 @@ public class CatalogTracker {
|
|||
*/
|
||||
private HRegionInterface getMetaServerConnection(boolean refresh)
|
||||
throws IOException, InterruptedException {
|
||||
synchronized(metaAvailable) {
|
||||
if(metaAvailable.get()) {
|
||||
synchronized (metaAvailable) {
|
||||
if (metaAvailable.get()) {
|
||||
HRegionInterface current = getCachedConnection(metaLocation);
|
||||
if(!refresh) {
|
||||
if (!refresh) {
|
||||
return current;
|
||||
}
|
||||
if(verifyRegionLocation(current, META_REGION)) {
|
||||
if (verifyRegionLocation(current, META_REGION)) {
|
||||
return current;
|
||||
}
|
||||
resetMetaLocation();
|
||||
}
|
||||
HRegionInterface rootConnection = getRootServerConnection();
|
||||
if(rootConnection == null) {
|
||||
if (rootConnection == null) {
|
||||
return null;
|
||||
}
|
||||
HServerAddress newLocation = MetaReader.readMetaLocation(rootConnection);
|
||||
if(newLocation == null) {
|
||||
if (newLocation == null) {
|
||||
return null;
|
||||
}
|
||||
HRegionInterface newConnection = getCachedConnection(newLocation);
|
||||
if(verifyRegionLocation(newConnection, META_REGION)) {
|
||||
if (verifyRegionLocation(newConnection, META_REGION)) {
|
||||
setMetaLocation(newLocation);
|
||||
return newConnection;
|
||||
}
|
||||
|
@ -270,7 +265,7 @@ public class CatalogTracker {
|
|||
*/
|
||||
public void waitForMeta() throws InterruptedException {
|
||||
synchronized(metaAvailable) {
|
||||
while(!metaAvailable.get()) {
|
||||
while (!metaAvailable.get()) {
|
||||
metaAvailable.wait();
|
||||
}
|
||||
}
|
||||
|
@ -339,8 +334,8 @@ public class CatalogTracker {
|
|||
|
||||
private void resetMetaLocation() {
|
||||
LOG.info("Current cached META location is not valid, resetting");
|
||||
metaAvailable.set(false);
|
||||
metaLocation = null;
|
||||
this.metaAvailable.set(false);
|
||||
this.metaLocation = null;
|
||||
}
|
||||
|
||||
private void setMetaLocation(HServerAddress metaLocation) {
|
||||
|
@ -368,11 +363,20 @@ public class CatalogTracker {
|
|||
|
||||
private boolean verifyRegionLocation(HRegionInterface metaServer,
|
||||
byte [] regionName) {
|
||||
Throwable t = null;
|
||||
try {
|
||||
return metaServer.getRegionInfo(regionName) != null;
|
||||
} catch (NotServingRegionException e) {
|
||||
return false;
|
||||
t = e;
|
||||
} catch (UndeclaredThrowableException e) {
|
||||
// We can get a ConnectException wrapped by a UTE if client fails connect
|
||||
// If not a ConnectException, rethrow.
|
||||
if (!(e.getCause() instanceof ConnectException)) throw e;
|
||||
t = e.getCause();
|
||||
}
|
||||
LOG.info("Failed verification of " + Bytes.toString(regionName) +
|
||||
": " + t.getMessage());
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -407,4 +411,8 @@ public class CatalogTracker {
|
|||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
MetaNodeTracker getMetaNodeTracker() {
|
||||
return this.metaNodeTracker;
|
||||
}
|
||||
}
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.hadoop.hbase.catalog;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.ConnectException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -115,11 +116,15 @@ public class MetaEditor {
|
|||
* @param regionInfo region to update location of
|
||||
* @param serverInfo server the region is located on
|
||||
* @throws IOException
|
||||
* @throws ConnectException Usually because the regionserver carrying .META.
|
||||
* is down.
|
||||
* @throws NullPointerException Because no -ROOT- server connection
|
||||
*/
|
||||
public static void updateMetaLocation(CatalogTracker catalogTracker,
|
||||
HRegionInfo regionInfo, HServerInfo serverInfo)
|
||||
throws IOException {
|
||||
throws IOException, ConnectException {
|
||||
HRegionInterface server = catalogTracker.waitForRootServerConnectionDefault();
|
||||
if (server == null) throw new NullPointerException("No server for -ROOT-");
|
||||
updateLocation(server, CatalogTracker.ROOT_REGION, regionInfo, serverInfo);
|
||||
}
|
||||
|
||||
|
@ -152,9 +157,10 @@ public class MetaEditor {
|
|||
* @param catalogRegionName name of catalog region being updated
|
||||
* @param regionInfo region to update location of
|
||||
* @param serverInfo server the region is located on
|
||||
* @throws IOException
|
||||
* @throws IOException In particular could throw {@link java.net.ConnectException}
|
||||
* if the server is down on other end.
|
||||
*/
|
||||
public static void updateLocation(HRegionInterface server,
|
||||
private static void updateLocation(HRegionInterface server,
|
||||
byte [] catalogRegionName, HRegionInfo regionInfo, HServerInfo serverInfo)
|
||||
throws IOException {
|
||||
Put put = new Put(regionInfo.getRegionName());
|
||||
|
|
|
@ -273,7 +273,9 @@ public class HBaseAdmin implements Abortable {
|
|||
byte [] lastKey = null;
|
||||
for(byte [] splitKey : splitKeys) {
|
||||
if(lastKey != null && Bytes.equals(splitKey, lastKey)) {
|
||||
throw new IllegalArgumentException("All split keys must be unique, found duplicate");
|
||||
throw new IllegalArgumentException("All split keys must be unique, " +
|
||||
"found duplicate: " + Bytes.toStringBinary(splitKey) +
|
||||
", " + Bytes.toStringBinary(lastKey));
|
||||
}
|
||||
lastKey = splitKey;
|
||||
}
|
||||
|
|
|
@ -1198,7 +1198,7 @@ public class HConnectionManager {
|
|||
ExecutorService pool) throws IOException {
|
||||
Result[] results = new Result[list.size()];
|
||||
processBatch((List) list, tableName, pool, results);
|
||||
|
||||
|
||||
// mutate list so that it is empty for complete success, or contains only failed records
|
||||
// results are returned in the same order as the requests in list
|
||||
// walk the list backwards, so we can remove from list without impacting the indexes of earlier members
|
||||
|
|
|
@ -20,14 +20,15 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.mortbay.log.Log;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -35,6 +36,7 @@ import java.io.IOException;
|
|||
* Used by {@link ResultScanner}s made by {@link HTable}.
|
||||
*/
|
||||
public class ScannerCallable extends ServerCallable<Result[]> {
|
||||
private static final Log LOG = LogFactory.getLog(ScannerCallable.class);
|
||||
private long scannerId = -1L;
|
||||
private boolean instantiated = false;
|
||||
private boolean closed = false;
|
||||
|
@ -103,7 +105,7 @@ public class ScannerCallable extends ServerCallable<Result[]> {
|
|||
try {
|
||||
this.server.close(this.scannerId);
|
||||
} catch (IOException e) {
|
||||
Log.warn("Ignore, probably already closed", e);
|
||||
LOG.warn("Ignore, probably already closed", e);
|
||||
}
|
||||
this.scannerId = -1L;
|
||||
}
|
||||
|
|
|
@ -31,9 +31,9 @@ import org.apache.hadoop.hbase.Stoppable;
|
|||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.MultiAction;
|
||||
import org.apache.hadoop.hbase.client.MultiResponse;
|
||||
import org.apache.hadoop.hbase.client.MultiPut;
|
||||
import org.apache.hadoop.hbase.client.MultiPutResponse;
|
||||
import org.apache.hadoop.hbase.client.MultiResponse;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -56,7 +56,6 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion, Stoppable, Ab
|
|||
public HRegionInfo getRegionInfo(final byte [] regionName)
|
||||
throws NotServingRegionException;
|
||||
|
||||
|
||||
/**
|
||||
* Return all the data for the row that matches <i>row</i> exactly,
|
||||
* or the one that immediately preceeds it.
|
||||
|
@ -288,8 +287,8 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion, Stoppable, Ab
|
|||
/**
|
||||
* Bulk load an HFile into an open region
|
||||
*/
|
||||
public void bulkLoadHFile(String hfilePath,
|
||||
byte[] regionName, byte[] familyName) throws IOException;
|
||||
public void bulkLoadHFile(String hfilePath, byte[] regionName, byte[] familyName)
|
||||
throws IOException;
|
||||
|
||||
// Master methods
|
||||
|
||||
|
@ -355,4 +354,4 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion, Stoppable, Ab
|
|||
* @throws IOException
|
||||
*/
|
||||
public void replicateLogEntries(HLog.Entry[] entries) throws IOException;
|
||||
}
|
||||
}
|
|
@ -88,7 +88,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
private final Map<String, RegionState> regionsInTransition =
|
||||
new TreeMap<String, RegionState>();
|
||||
|
||||
/** Plans for region movement. */
|
||||
/** Plans for region movement. Key is the encoded version of a region name*/
|
||||
// TODO: When do plans get cleaned out? Ever?
|
||||
// Its cleaned on server shutdown processing -- St.Ack
|
||||
private final Map<String, RegionPlan> regionPlans =
|
||||
|
@ -240,9 +240,10 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
private void handleRegion(RegionTransitionData data) {
|
||||
synchronized(regionsInTransition) {
|
||||
// Verify this is a known server
|
||||
if(!serverManager.isServerOnline(data.getServerName())) {
|
||||
LOG.warn("Attempted to handle region transition for server " +
|
||||
data.getServerName() + " but server is not online");
|
||||
if (!serverManager.isServerOnline(data.getServerName()) &&
|
||||
!this.master.getServerName().equals(data.getServerName())) {
|
||||
LOG.warn("Attempted to handle region transition for server but " +
|
||||
"server is not online: " + data);
|
||||
return;
|
||||
}
|
||||
String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
|
||||
|
@ -251,6 +252,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
", server=" + data.getServerName() + ", region=" + prettyPrintedRegionName);
|
||||
RegionState regionState = regionsInTransition.get(encodedName);
|
||||
switch(data.getEventType()) {
|
||||
case M2ZK_REGION_OFFLINE:
|
||||
LOG.warn("What to do with this event? " + data);
|
||||
break;
|
||||
|
||||
case RS2ZK_REGION_CLOSING:
|
||||
// Should see CLOSING after we have asked it to CLOSE or additional
|
||||
|
@ -494,7 +498,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// go of this lock. There must be a better construct that this -- St.Ack 20100811
|
||||
this.assignLock.lock();
|
||||
try {
|
||||
synchronized(regionsInTransition) {
|
||||
synchronized (regionsInTransition) {
|
||||
state = regionsInTransition.get(encodedName);
|
||||
if(state == null) {
|
||||
state = new RegionState(region, RegionState.State.OFFLINE);
|
||||
|
@ -546,6 +550,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
plan = new RegionPlan(state.getRegion(), null,
|
||||
LoadBalancer.randomAssignment(serverManager.getOnlineServersList()));
|
||||
regionPlans.put(encodedName, plan);
|
||||
} else {
|
||||
LOG.debug("Using preexisting plan=" + plan);
|
||||
}
|
||||
}
|
||||
try {
|
||||
|
@ -934,14 +940,26 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Clean out any exisiting assignment plans for this server
|
||||
synchronized (this.regionPlans) {
|
||||
for (Iterator <Map.Entry<String, RegionPlan>> i =
|
||||
this.regionPlans.entrySet().iterator(); i.hasNext();) {
|
||||
this.regionPlans.entrySet().iterator(); i.hasNext();) {
|
||||
Map.Entry<String, RegionPlan> e = i.next();
|
||||
if (e.getValue().getDestination().equals(hsi)) {
|
||||
// Use iterator's remove else we'll get CME.fail a
|
||||
// Use iterator's remove else we'll get CME
|
||||
i.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
// Remove assignment info related to the downed server. Remove the downed
|
||||
// server from list of servers else it looks like a server w/ no load.
|
||||
synchronized (this.regions) {
|
||||
Set<HRegionInfo> hris = new HashSet<HRegionInfo>();
|
||||
for (Map.Entry<HRegionInfo, HServerInfo> e: this.regions.entrySet()) {
|
||||
// Add to a Set -- don't call setOffline in here else we get a CME.
|
||||
if (e.getValue().equals(hsi)) hris.add(e.getKey());
|
||||
}
|
||||
for (HRegionInfo hri: hris) setOffline(hri);
|
||||
this.servers.remove(hsi);
|
||||
}
|
||||
// If anything in transition related to the server, clean it up.
|
||||
synchronized (regionsInTransition) {
|
||||
// Iterate all regions in transition checking if were on this server
|
||||
final String serverName = hsi.getServerName();
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HMsg;
|
||||
|
@ -55,8 +54,8 @@ import org.apache.hadoop.hbase.catalog.MetaReader;
|
|||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.client.MetaScanner;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
|
||||
import org.apache.hadoop.hbase.ipc.HBaseRPC;
|
||||
|
@ -87,7 +86,6 @@ import org.apache.hadoop.io.MapWritable;
|
|||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.net.DNS;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
|
||||
|
@ -258,7 +256,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
this.serverManager = new ServerManager(this, this);
|
||||
|
||||
this.catalogTracker = new CatalogTracker(this.zooKeeper, this.connection,
|
||||
this, conf.getInt("hbase.master.catalog.timeout", -1));
|
||||
this, conf.getInt("hbase.master.catalog.timeout", Integer.MAX_VALUE));
|
||||
this.catalogTracker.start();
|
||||
|
||||
this.assignmentManager = new AssignmentManager(this, serverManager,
|
||||
|
@ -563,6 +561,9 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
this.assignmentManager.getRegionsInTransition());
|
||||
return false;
|
||||
}
|
||||
if (!this.serverManager.getDeadServers().isEmpty()) {
|
||||
LOG.debug("Not running balancer because dead regionserver processing");
|
||||
}
|
||||
Map<HServerInfo, List<HRegionInfo>> assignments =
|
||||
this.assignmentManager.getAssignments();
|
||||
// Returned Map from AM does not include mention of servers w/o assignments.
|
||||
|
@ -576,6 +577,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
List<RegionPlan> plans = this.balancer.balanceCluster(assignments);
|
||||
if (plans != null && !plans.isEmpty()) {
|
||||
for (RegionPlan plan: plans) {
|
||||
LOG.info("balance=" + plan);
|
||||
this.assignmentManager.balance(plan);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -141,8 +141,8 @@ public class LoadBalancer {
|
|||
}
|
||||
int numRegions = 0;
|
||||
// Iterate so we can count regions as we build the map
|
||||
for(Map.Entry<HServerInfo, List<HRegionInfo>> server :
|
||||
clusterState.entrySet()) {
|
||||
for(Map.Entry<HServerInfo, List<HRegionInfo>> server:
|
||||
clusterState.entrySet()) {
|
||||
server.getKey().getLoad().setNumberOfRegions(server.getValue().size());
|
||||
numRegions += server.getKey().getLoad().getNumberOfRegions();
|
||||
serversByLoad.put(server.getKey(), server.getValue());
|
||||
|
@ -527,8 +527,6 @@ public class LoadBalancer {
|
|||
private final HServerInfo source;
|
||||
private HServerInfo dest;
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Instantiate a plan for a region move, moving the specified region from
|
||||
* the specified source server to the specified destination server.
|
||||
|
@ -589,5 +587,11 @@ public class LoadBalancer {
|
|||
public int compareTo(RegionPlan o) {
|
||||
return getRegionName().compareTo(o.getRegionName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "hri=" + this.hri.getRegionNameAsString() + ", src=" +
|
||||
this.source.getServerName() + ", dest=" + this.dest.getServerName();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -499,7 +499,7 @@ public class ServerManager {
|
|||
*/
|
||||
public void sendRegionOpen(HServerInfo server, HRegionInfo region) {
|
||||
HRegionInterface hri = getServerConnection(server);
|
||||
if(hri == null) {
|
||||
if (hri == null) {
|
||||
LOG.warn("Attempting to send OPEN RPC to server " + server.getServerName()
|
||||
+ " failed because no RPC connection found to this server");
|
||||
return;
|
||||
|
@ -533,7 +533,7 @@ public class ServerManager {
|
|||
HConnection connection =
|
||||
HConnectionManager.getConnection(this.master.getConfiguration());
|
||||
HRegionInterface hri = serverConnections.get(info.getServerName());
|
||||
if(hri == null) {
|
||||
if (hri == null) {
|
||||
LOG.info("new connection");
|
||||
hri = connection.getHRegionConnection(info.getServerAddress(), false);
|
||||
serverConnections.put(info.getServerName(), hri);
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.io.IOException;
|
|||
import java.lang.Thread.UncaughtExceptionHandler;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.lang.management.MemoryUsage;
|
||||
import java.lang.management.RuntimeMXBean;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.net.BindException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -64,7 +63,6 @@ import org.apache.hadoop.hbase.HServerAddress;
|
|||
import org.apache.hadoop.hbase.HServerInfo;
|
||||
import org.apache.hadoop.hbase.HServerLoad;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.LocalHBaseCluster;
|
||||
import org.apache.hadoop.hbase.MasterAddressTracker;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
||||
|
@ -352,7 +350,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
|
||||
// create the catalog tracker and start it
|
||||
this.catalogTracker = new CatalogTracker(this.zooKeeper, this.connection,
|
||||
this, this.conf.getInt("hbase.regionserver.catalog.timeout", -1));
|
||||
this, this.conf.getInt("hbase.regionserver.catalog.timeout", Integer.MAX_VALUE));
|
||||
catalogTracker.start();
|
||||
|
||||
this.clusterStatusTracker = new ClusterStatusTracker(this.zooKeeper, this);
|
||||
|
@ -1143,7 +1141,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
RootLocationEditor.setRootLocation(getZooKeeper(),
|
||||
getServerInfo().getServerAddress());
|
||||
} else if (r.getRegionInfo().isMetaRegion()) {
|
||||
// TODO: doh, this has weird naming between RootEditor/MetaEditor
|
||||
MetaEditor.updateMetaLocation(ct, r.getRegionInfo(), getServerInfo());
|
||||
} else {
|
||||
if (daughter) {
|
||||
|
@ -1361,8 +1358,9 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HRegionInfo getRegionInfo(final byte[] regionName)
|
||||
throws NotServingRegionException {
|
||||
throws NotServingRegionException {
|
||||
requestCount.incrementAndGet();
|
||||
return getRegion(regionName).getRegionInfo();
|
||||
}
|
||||
|
@ -2228,20 +2226,19 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
}
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
if (multi.size() == 1) {
|
||||
throw ioe;
|
||||
} else {
|
||||
LOG.error("Exception found while attempting " + action.toString()
|
||||
+ " " + StringUtils.stringifyException(ioe));
|
||||
response.add(regionName,null);
|
||||
// stop processing on this region, continue to the next.
|
||||
}
|
||||
if (multi.size() == 1) {
|
||||
throw ioe;
|
||||
} else {
|
||||
LOG.error("Exception found while attempting " + action.toString() +
|
||||
" " + StringUtils.stringifyException(ioe));
|
||||
response.add(regionName,null);
|
||||
// stop processing on this region, continue to the next.
|
||||
}
|
||||
}
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use HRegionServer.multi( MultiAction action) instead
|
||||
*/
|
||||
|
|
|
@ -155,7 +155,7 @@ public class OpenRegionHandler extends EventHandler {
|
|||
|
||||
// Finally, Transition ZK node to OPENED
|
||||
try {
|
||||
if(ZKAssign.transitionNodeOpened(server.getZooKeeper(), regionInfo,
|
||||
if (ZKAssign.transitionNodeOpened(server.getZooKeeper(), regionInfo,
|
||||
server.getServerName(), openingVersion) == -1) {
|
||||
LOG.warn("Completed the OPEN of a region but when transitioning from " +
|
||||
" OPENING to OPENED got a version mismatch, someone else clashed " +
|
||||
|
|
|
@ -23,24 +23,23 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
|
||||
/**
|
||||
* Tracks the unassigned zookeeper node used by the META table.
|
||||
* <p>
|
||||
* A callback is made into a {@link CatalogTracker} when META completes a new
|
||||
* assignment.
|
||||
*
|
||||
* A callback is made into the passed {@link CatalogTracker} when
|
||||
* <code>.META.</code> completes a new assignment.
|
||||
* <p>
|
||||
* If META is already assigned when instantiating this class, you will not
|
||||
* receive any notification for that assignment. You will receive a
|
||||
* notification after META has been successfully assigned to a new location.
|
||||
*/
|
||||
public class MetaNodeTracker extends ZooKeeperListener {
|
||||
public class MetaNodeTracker extends ZooKeeperNodeTracker {
|
||||
private static final Log LOG = LogFactory.getLog(MetaNodeTracker.class);
|
||||
|
||||
private final String node;
|
||||
|
||||
/** Catalog tracker to notify when META has a new assignment completed. */
|
||||
private final CatalogTracker catalogTracker;
|
||||
|
||||
|
@ -49,25 +48,22 @@ public class MetaNodeTracker extends ZooKeeperListener {
|
|||
* @param watcher
|
||||
* @param abortable
|
||||
*/
|
||||
public MetaNodeTracker(ZooKeeperWatcher watcher,
|
||||
CatalogTracker catalogTracker) {
|
||||
super(watcher);
|
||||
public MetaNodeTracker(final ZooKeeperWatcher watcher,
|
||||
final CatalogTracker catalogTracker, final Abortable abortable) {
|
||||
super(watcher, ZKUtil.joinZNode(watcher.assignmentZNode,
|
||||
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()), abortable);
|
||||
this.catalogTracker = catalogTracker;
|
||||
node = ZKUtil.joinZNode(watcher.assignmentZNode,
|
||||
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void nodeDeleted(String path) {
|
||||
if(path.equals(node)) {
|
||||
LOG.info("Detected completed assignment of META, notifying catalog " +
|
||||
"tracker");
|
||||
try {
|
||||
catalogTracker.waitForMetaServerConnectionDefault();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Tried to reset META server location after seeing the " +
|
||||
"completion of a new META assignment but got an IOE", e);
|
||||
}
|
||||
if (!path.equals(node)) return;
|
||||
LOG.info("Detected completed assignment of META, notifying catalog tracker");
|
||||
try {
|
||||
this.catalogTracker.waitForMetaServerConnectionDefault();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Tried to reset META server location after seeing the " +
|
||||
"completion of a new META assignment but got an IOE", e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -28,7 +28,8 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
/**
|
||||
* Tracks the root region server location node in zookeeper.
|
||||
* Root region location is set by {@link RootLocationEditor} usually called
|
||||
* out of {@link RegionServerServices#postOpenDeployTasks(org.apache.hadoop.hbase.regionserver.HRegion, org.apache.hadoop.hbase.catalog.CatalogTracker)}
|
||||
* out of {@link RegionServerServices#postOpenDeployTasks(org.apache.hadoop.hbase.regionserver.HRegion, org.apache.hadoop.hbase.catalog.CatalogTracker)}.
|
||||
* This class has a watcher on the root location and notices changes.
|
||||
*/
|
||||
public class RootRegionTracker extends ZooKeeperNodeTracker {
|
||||
/**
|
||||
|
|
|
@ -132,6 +132,10 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
|
|||
return data;
|
||||
}
|
||||
|
||||
public String getNode() {
|
||||
return this.node;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void nodeCreated(String path) {
|
||||
if(path.equals(node)) {
|
||||
|
|
|
@ -169,9 +169,6 @@ public class ZooKeeperWatcher implements Watcher {
|
|||
"state=" + event.getState() + ", " +
|
||||
"path=" + event.getPath());
|
||||
|
||||
// While we are still using both ZKWs, need to call parent process()
|
||||
// super.process(event);
|
||||
|
||||
switch(event.getType()) {
|
||||
|
||||
// If event type is NONE, this is a connection status change
|
||||
|
|
|
@ -609,6 +609,18 @@ public class HBaseTestingUtility {
|
|||
return createMultiRegions(getConfiguration(), table, columnFamily);
|
||||
}
|
||||
|
||||
public static final byte[][] KEYS = {
|
||||
HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("bbb"),
|
||||
Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
|
||||
Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
|
||||
Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
|
||||
Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
|
||||
Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
|
||||
Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
|
||||
Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
|
||||
Bytes.toBytes("xxx"), Bytes.toBytes("yyy")
|
||||
};
|
||||
|
||||
/**
|
||||
* Creates many regions names "aaa" to "zzz".
|
||||
* @param c Configuration to use.
|
||||
|
@ -620,17 +632,6 @@ public class HBaseTestingUtility {
|
|||
public int createMultiRegions(final Configuration c, final HTable table,
|
||||
final byte[] columnFamily)
|
||||
throws IOException {
|
||||
byte[][] KEYS = {
|
||||
HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("bbb"),
|
||||
Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
|
||||
Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
|
||||
Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
|
||||
Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
|
||||
Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
|
||||
Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
|
||||
Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
|
||||
Bytes.toBytes("xxx"), Bytes.toBytes("yyy")
|
||||
};
|
||||
return createMultiRegions(c, table, columnFamily, KEYS);
|
||||
}
|
||||
|
||||
|
@ -963,16 +964,18 @@ public class HBaseTestingUtility {
|
|||
* Make sure that at least the specified number of region servers
|
||||
* are running
|
||||
* @param num minimum number of region servers that should be running
|
||||
* @return True if we started some servers
|
||||
* @throws IOException
|
||||
*/
|
||||
public void ensureSomeRegionServersAvailable(final int num)
|
||||
public boolean ensureSomeRegionServersAvailable(final int num)
|
||||
throws IOException {
|
||||
if (this.getHBaseCluster().getLiveRegionServerThreads().size() < num) {
|
||||
// Need at least "num" servers.
|
||||
LOG.info("Started new server=" +
|
||||
this.getHBaseCluster().startRegionServer());
|
||||
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -223,6 +223,8 @@ public class MiniHBaseCluster {
|
|||
try {
|
||||
LOG.info("Hook closing fs=" + this.fs);
|
||||
this.fs.close();
|
||||
} catch (NullPointerException npe) {
|
||||
LOG.debug("Need to fix these: " + npe.toString());
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Running hook", e);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,247 @@
|
|||
/**
|
||||
* Copyright 2010 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.catalog;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HServerAddress;
|
||||
import org.apache.hadoop.hbase.HServerInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
/**
|
||||
* Test {@link CatalogTracker}
|
||||
*/
|
||||
public class TestCatalogTracker {
|
||||
private static final Log LOG = LogFactory.getLog(TestCatalogTracker.class);
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
private static final HServerAddress HSA =
|
||||
new HServerAddress("example.org:1234");
|
||||
private ZooKeeperWatcher watcher;
|
||||
private Abortable abortable;
|
||||
|
||||
@BeforeClass public static void beforeClass() throws Exception {
|
||||
UTIL.startMiniZKCluster();
|
||||
}
|
||||
|
||||
@AfterClass public static void afterClass() throws IOException {
|
||||
UTIL.getZkCluster().shutdown();
|
||||
}
|
||||
|
||||
@Before public void before() throws IOException {
|
||||
this.abortable = new Abortable() {
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
LOG.info(why, e);
|
||||
}
|
||||
};
|
||||
this.watcher = new ZooKeeperWatcher(UTIL.getConfiguration(),
|
||||
this.getClass().getSimpleName(), this.abortable);
|
||||
}
|
||||
|
||||
@After public void after() {
|
||||
this.watcher.close();
|
||||
}
|
||||
|
||||
private CatalogTracker constructAndStartCatalogTracker()
|
||||
throws IOException, InterruptedException {
|
||||
return constructAndStartCatalogTracker(null);
|
||||
}
|
||||
|
||||
private CatalogTracker constructAndStartCatalogTracker(final HConnection c)
|
||||
throws IOException, InterruptedException {
|
||||
CatalogTracker ct = new CatalogTracker(this.watcher, c, this.abortable);
|
||||
ct.start();
|
||||
return ct;
|
||||
}
|
||||
|
||||
@Test (expected = NotAllMetaRegionsOnlineException.class)
|
||||
public void testTimeoutWaitForRoot()
|
||||
throws IOException, InterruptedException {
|
||||
final CatalogTracker ct = constructAndStartCatalogTracker();
|
||||
ct.waitForRoot(100);
|
||||
}
|
||||
|
||||
@Test (expected = NotAllMetaRegionsOnlineException.class)
|
||||
public void testTimeoutWaitForMeta()
|
||||
throws IOException, InterruptedException {
|
||||
final CatalogTracker ct = constructAndStartCatalogTracker();
|
||||
ct.waitForMeta(100);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test waiting on root w/ no timeout specified.
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
* @throws KeeperException
|
||||
*/
|
||||
@Test public void testNoTimeoutWaitForRoot()
|
||||
throws IOException, InterruptedException, KeeperException {
|
||||
final CatalogTracker ct = constructAndStartCatalogTracker();
|
||||
HServerAddress hsa = ct.getRootLocation();
|
||||
Assert.assertNull(hsa);
|
||||
|
||||
// Now test waiting on root location getting set.
|
||||
Thread t = new WaitOnMetaThread(ct);
|
||||
startWaitAliveThenWaitItLives(t, 1000);
|
||||
// Set a root location.
|
||||
hsa = setRootLocation();
|
||||
// Join the thread... should exit shortly.
|
||||
t.join();
|
||||
// Now root is available.
|
||||
Assert.assertTrue(ct.getRootLocation().equals(hsa));
|
||||
}
|
||||
|
||||
private HServerAddress setRootLocation() throws KeeperException {
|
||||
RootLocationEditor.setRootLocation(this.watcher, HSA);
|
||||
return HSA;
|
||||
}
|
||||
|
||||
/**
|
||||
* Test waiting on meta w/ no timeout specified.
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
* @throws KeeperException
|
||||
*/
|
||||
@Test public void testNoTimeoutWaitForMeta()
|
||||
throws IOException, InterruptedException, KeeperException {
|
||||
// Mock an HConnection and a HRegionInterface implementation. Have the
|
||||
// HConnection return the HRI. Have the HRI return a few mocked up responses
|
||||
// to make our test work.
|
||||
HConnection connection = Mockito.mock(HConnection.class);
|
||||
HRegionInterface mockHRI = Mockito.mock(HRegionInterface.class);
|
||||
// Make the HRI return an answer no matter how Get is called. Same for
|
||||
// getHRegionInfo. Thats enough for this test.
|
||||
Mockito.when(connection.getHRegionConnection((HServerAddress)Mockito.any(), Mockito.anyBoolean())).
|
||||
thenReturn(mockHRI);
|
||||
|
||||
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
|
||||
HServerAddress hsa = ct.getMetaLocation();
|
||||
Assert.assertNull(hsa);
|
||||
|
||||
// Now test waiting on meta location getting set.
|
||||
Thread t = new WaitOnMetaThread(ct) {
|
||||
@Override
|
||||
void doWaiting() throws InterruptedException {
|
||||
this.ct.waitForMeta();
|
||||
}
|
||||
};
|
||||
startWaitAliveThenWaitItLives(t, 1000);
|
||||
|
||||
// Now the ct is up... set into the mocks some answers that make it look
|
||||
// like things have been getting assigned. Make it so we'll return a
|
||||
// location (no matter what the Get is). Same for getHRegionInfo -- always
|
||||
// just return the meta region.
|
||||
List<KeyValue> kvs = new ArrayList<KeyValue>();
|
||||
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
|
||||
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
|
||||
Bytes.toBytes(HSA.toString())));
|
||||
final Result result = new Result(kvs);
|
||||
Mockito.when(mockHRI.get((byte [])Mockito.any(), (Get)Mockito.any())).
|
||||
thenReturn(result);
|
||||
Mockito.when(mockHRI.getRegionInfo((byte [])Mockito.any())).
|
||||
thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
// This should trigger wake up of meta wait (Its the removal of the meta
|
||||
// region unassigned node that triggers catalogtrackers that a meta has
|
||||
// been assigned.
|
||||
String node = ct.getMetaNodeTracker().getNode();
|
||||
ZKUtil.createAndFailSilent(this.watcher, node);
|
||||
MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO,
|
||||
new HServerInfo(HSA, -1, "example.com"));
|
||||
ZKUtil.deleteNode(this.watcher, node);
|
||||
// Join the thread... should exit shortly.
|
||||
t.join();
|
||||
// Now meta is available.
|
||||
Assert.assertTrue(ct.getMetaLocation().equals(HSA));
|
||||
}
|
||||
|
||||
private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
|
||||
t.start();
|
||||
while(!t.isAlive()) {
|
||||
// Wait
|
||||
}
|
||||
// Wait one second.
|
||||
Threads.sleep(ms);
|
||||
Assert.assertTrue("Assert " + t.getName() + " still waiting", t.isAlive());
|
||||
}
|
||||
|
||||
class CountingProgressable implements Progressable {
|
||||
final AtomicInteger counter = new AtomicInteger(0);
|
||||
@Override
|
||||
public void progress() {
|
||||
this.counter.incrementAndGet();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait on META.
|
||||
* Default is wait on -ROOT-.
|
||||
*/
|
||||
class WaitOnMetaThread extends Thread {
|
||||
final CatalogTracker ct;
|
||||
|
||||
WaitOnMetaThread(final CatalogTracker ct) {
|
||||
super("WaitOnMeta");
|
||||
this.ct = ct;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
doWaiting();
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException("Failed wait on root", e);
|
||||
}
|
||||
LOG.info("Exiting " + getName());
|
||||
}
|
||||
|
||||
void doWaiting() throws InterruptedException {
|
||||
this.ct.waitForRoot();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,54 +17,67 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.client.Row;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
public class TestMultiParallel extends MultiRegionTable {
|
||||
// This test needs to be rewritten to use HBaseTestingUtility -- St.Ack 20100910
|
||||
public class TestMultiParallel {
|
||||
private static final Log LOG = LogFactory.getLog(TestMultiParallel.class);
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
private static final byte[] VALUE = Bytes.toBytes("value");
|
||||
private static final byte[] QUALIFIER = Bytes.toBytes("qual");
|
||||
private static final String FAMILY = "family";
|
||||
private static final String TEST_TABLE = "multi_test_table";
|
||||
private static final byte[] BYTES_FAMILY = Bytes.toBytes(FAMILY);
|
||||
private static final byte[] ONE_ROW = Bytes.toBytes("xxx");
|
||||
private static final byte [][] KEYS = makeKeys();
|
||||
|
||||
List<byte[]> keys = new ArrayList<byte[]>();
|
||||
|
||||
public TestMultiParallel() {
|
||||
super(2, FAMILY);
|
||||
desc = new HTableDescriptor(TEST_TABLE);
|
||||
desc.addFamily(new HColumnDescriptor(FAMILY));
|
||||
makeKeys();
|
||||
@BeforeClass public static void beforeClass() throws Exception {
|
||||
UTIL.startMiniCluster(2);
|
||||
HTable t = UTIL.createTable(Bytes.toBytes(TEST_TABLE), Bytes.toBytes(FAMILY));
|
||||
UTIL.createMultiRegions(t, Bytes.toBytes(FAMILY));
|
||||
}
|
||||
|
||||
private void makeKeys() {
|
||||
@AfterClass public static void afterClass() throws IOException {
|
||||
UTIL.getMiniHBaseCluster().shutdown();
|
||||
}
|
||||
|
||||
@Before public void before() throws IOException {
|
||||
LOG.info("before");
|
||||
if (UTIL.ensureSomeRegionServersAvailable(2)) {
|
||||
// Distribute regions
|
||||
UTIL.getMiniHBaseCluster().getMaster().balance();
|
||||
}
|
||||
LOG.info("before done");
|
||||
}
|
||||
|
||||
private static byte[][] makeKeys() {
|
||||
byte [][] starterKeys = HBaseTestingUtility.KEYS;
|
||||
// Create a "non-uniform" test set with the following characteristics:
|
||||
// a) Unequal number of keys per region
|
||||
|
||||
// Don't use integer as a multiple, so that we have a number of keys that is
|
||||
// not a multiple of the number of regions
|
||||
int numKeys = (int) ((float) KEYS.length * 10.33F);
|
||||
int numKeys = (int) ((float) starterKeys.length * 10.33F);
|
||||
|
||||
List<byte[]> keys = new ArrayList<byte[]>();
|
||||
for (int i = 0; i < numKeys; i++) {
|
||||
int kIdx = i % KEYS.length;
|
||||
byte[] k = KEYS[kIdx];
|
||||
int kIdx = i % starterKeys.length;
|
||||
byte[] k = starterKeys[kIdx];
|
||||
byte[] cp = new byte[k.length + 1];
|
||||
System.arraycopy(k, 0, cp, 0, k.length);
|
||||
cp[k.length] = new Integer(i % 256).byteValue();
|
||||
|
@ -76,18 +89,19 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
// c) keys are not in sorted order (within a region), to ensure that the
|
||||
// sorting code and index mapping doesn't break the functionality
|
||||
for (int i = 0; i < 100; i++) {
|
||||
int kIdx = i % KEYS.length;
|
||||
byte[] k = KEYS[kIdx];
|
||||
int kIdx = i % starterKeys.length;
|
||||
byte[] k = starterKeys[kIdx];
|
||||
byte[] cp = new byte[k.length + 1];
|
||||
System.arraycopy(k, 0, cp, 0, k.length);
|
||||
cp[k.length] = new Integer(i % 256).byteValue();
|
||||
keys.add(cp);
|
||||
}
|
||||
return keys.toArray(new byte [][] {new byte [] {}});
|
||||
}
|
||||
|
||||
public void testBatchWithGet() throws Exception {
|
||||
@Test public void testBatchWithGet() throws Exception {
|
||||
LOG.info("test=testBatchWithGet");
|
||||
HTable table = new HTable(conf, TEST_TABLE);
|
||||
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
|
||||
|
||||
// load test data
|
||||
List<Row> puts = constructPutRequests();
|
||||
|
@ -95,7 +109,7 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
|
||||
// create a list of gets and run it
|
||||
List<Row> gets = new ArrayList<Row>();
|
||||
for (byte[] k : keys) {
|
||||
for (byte[] k : KEYS) {
|
||||
Get get = new Get(k);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
gets.add(get);
|
||||
|
@ -110,14 +124,14 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
}
|
||||
|
||||
// Compare results
|
||||
assertEquals(singleRes.size(), multiRes.length);
|
||||
Assert.assertEquals(singleRes.size(), multiRes.length);
|
||||
for (int i = 0; i < singleRes.size(); i++) {
|
||||
assertTrue(singleRes.get(i).containsColumn(BYTES_FAMILY, QUALIFIER));
|
||||
Assert.assertTrue(singleRes.get(i).containsColumn(BYTES_FAMILY, QUALIFIER));
|
||||
KeyValue[] singleKvs = singleRes.get(i).raw();
|
||||
KeyValue[] multiKvs = multiRes[i].raw();
|
||||
for (int j = 0; j < singleKvs.length; j++) {
|
||||
assertEquals(singleKvs[j], multiKvs[j]);
|
||||
assertEquals(0, Bytes.compareTo(singleKvs[j].getValue(), multiKvs[j]
|
||||
Assert.assertEquals(singleKvs[j], multiKvs[j]);
|
||||
Assert.assertEquals(0, Bytes.compareTo(singleKvs[j].getValue(), multiKvs[j]
|
||||
.getValue()));
|
||||
}
|
||||
}
|
||||
|
@ -129,32 +143,32 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
*
|
||||
* @throws Exception
|
||||
*/
|
||||
public void testFlushCommitsWithAbort() throws Exception {
|
||||
@Test public void testFlushCommitsWithAbort() throws Exception {
|
||||
LOG.info("test=testFlushCommitsWithAbort");
|
||||
doTestFlushCommits(true);
|
||||
}
|
||||
|
||||
public void testFlushCommitsNoAbort() throws Exception {
|
||||
@Test public void testFlushCommitsNoAbort() throws Exception {
|
||||
LOG.info("test=testFlushCommitsNoAbort");
|
||||
doTestFlushCommits(false);
|
||||
}
|
||||
|
||||
public void doTestFlushCommits(boolean doAbort) throws Exception {
|
||||
LOG.info("test=doTestFlushCommits");
|
||||
private void doTestFlushCommits(boolean doAbort) throws Exception {
|
||||
// Load the data
|
||||
Configuration newconf = new Configuration(conf);
|
||||
newconf.setInt("hbase.client.retries.number", 10);
|
||||
HTable table = new HTable(newconf, TEST_TABLE);
|
||||
LOG.info("get new table");
|
||||
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
|
||||
table.setAutoFlush(false);
|
||||
table.setWriteBufferSize(10 * 1024 * 1024);
|
||||
|
||||
LOG.info("constructPutRequests");
|
||||
List<Row> puts = constructPutRequests();
|
||||
for (Row put : puts) {
|
||||
table.put((Put) put);
|
||||
}
|
||||
LOG.info("puts");
|
||||
table.flushCommits();
|
||||
|
||||
if (doAbort) {
|
||||
cluster.abortRegionServer(0);
|
||||
LOG.info("Aborted=" + UTIL.getMiniHBaseCluster().abortRegionServer(0));
|
||||
|
||||
// try putting more keys after the abort. same key/qual... just validating
|
||||
// no exceptions thrown
|
||||
|
@ -166,100 +180,107 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
table.flushCommits();
|
||||
}
|
||||
|
||||
LOG.info("validating loaded data");
|
||||
validateLoadedData(table);
|
||||
|
||||
// Validate server and region count
|
||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
||||
ClusterStatus cs = admin.getClusterStatus();
|
||||
assertEquals((doAbort ? 1 : 2), cs.getServers());
|
||||
for (HServerInfo info : cs.getServerInfo()) {
|
||||
System.out.println(info);
|
||||
assertTrue(info.getLoad().getNumberOfRegions() > 10);
|
||||
List<JVMClusterUtil.RegionServerThread> liveRSs =
|
||||
UTIL.getMiniHBaseCluster().getLiveRegionServerThreads();
|
||||
int count = 0;
|
||||
for (JVMClusterUtil.RegionServerThread t: liveRSs) {
|
||||
count++;
|
||||
LOG.info("Count=" + count + ", Alive=" + t.getRegionServer());
|
||||
}
|
||||
LOG.info("Count=" + count);
|
||||
Assert.assertEquals("Server count=" + count + ", abort=" + doAbort,
|
||||
(doAbort ? 1 : 2), count);
|
||||
for (JVMClusterUtil.RegionServerThread t: liveRSs) {
|
||||
int regions = t.getRegionServer().getOnlineRegions().size();
|
||||
Assert.assertTrue("Count of regions=" + regions, regions > 10);
|
||||
}
|
||||
LOG.info("done");
|
||||
}
|
||||
|
||||
public void testBatchWithPut() throws Exception {
|
||||
@Test public void testBatchWithPut() throws Exception {
|
||||
LOG.info("test=testBatchWithPut");
|
||||
Configuration newconf = new Configuration(conf);
|
||||
newconf.setInt("hbase.client.retries.number", 10);
|
||||
HTable table = new HTable(newconf, TEST_TABLE);
|
||||
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
|
||||
|
||||
// put multiple rows using a batch
|
||||
List<Row> puts = constructPutRequests();
|
||||
|
||||
Result[] results = table.batch(puts);
|
||||
validateSizeAndEmpty(results, keys.size());
|
||||
validateSizeAndEmpty(results, KEYS.length);
|
||||
|
||||
if (true) {
|
||||
cluster.abortRegionServer(0);
|
||||
UTIL.getMiniHBaseCluster().abortRegionServer(0);
|
||||
|
||||
puts = constructPutRequests();
|
||||
results = table.batch(puts);
|
||||
validateSizeAndEmpty(results, keys.size());
|
||||
validateSizeAndEmpty(results, KEYS.length);
|
||||
}
|
||||
|
||||
validateLoadedData(table);
|
||||
}
|
||||
|
||||
public void testBatchWithDelete() throws Exception {
|
||||
@Test public void testBatchWithDelete() throws Exception {
|
||||
LOG.info("test=testBatchWithDelete");
|
||||
HTable table = new HTable(conf, TEST_TABLE);
|
||||
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
|
||||
|
||||
// Load some data
|
||||
List<Row> puts = constructPutRequests();
|
||||
Result[] results = table.batch(puts);
|
||||
validateSizeAndEmpty(results, keys.size());
|
||||
validateSizeAndEmpty(results, KEYS.length);
|
||||
|
||||
// Deletes
|
||||
List<Row> deletes = new ArrayList<Row>();
|
||||
for (int i = 0; i < keys.size(); i++) {
|
||||
Delete delete = new Delete(keys.get(i));
|
||||
for (int i = 0; i < KEYS.length; i++) {
|
||||
Delete delete = new Delete(KEYS[i]);
|
||||
delete.deleteFamily(BYTES_FAMILY);
|
||||
deletes.add(delete);
|
||||
}
|
||||
results = table.batch(deletes);
|
||||
validateSizeAndEmpty(results, keys.size());
|
||||
validateSizeAndEmpty(results, KEYS.length);
|
||||
|
||||
// Get to make sure ...
|
||||
for (byte[] k : keys) {
|
||||
for (byte[] k : KEYS) {
|
||||
Get get = new Get(k);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
assertFalse(table.exists(get));
|
||||
Assert.assertFalse(table.exists(get));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testHTableDeleteWithList() throws Exception {
|
||||
@Test public void testHTableDeleteWithList() throws Exception {
|
||||
LOG.info("test=testHTableDeleteWithList");
|
||||
HTable table = new HTable(conf, TEST_TABLE);
|
||||
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
|
||||
|
||||
// Load some data
|
||||
List<Row> puts = constructPutRequests();
|
||||
Result[] results = table.batch(puts);
|
||||
validateSizeAndEmpty(results, keys.size());
|
||||
validateSizeAndEmpty(results, KEYS.length);
|
||||
|
||||
// Deletes
|
||||
ArrayList<Delete> deletes = new ArrayList<Delete>();
|
||||
for (int i = 0; i < keys.size(); i++) {
|
||||
Delete delete = new Delete(keys.get(i));
|
||||
for (int i = 0; i < KEYS.length; i++) {
|
||||
Delete delete = new Delete(KEYS[i]);
|
||||
delete.deleteFamily(BYTES_FAMILY);
|
||||
deletes.add(delete);
|
||||
}
|
||||
table.delete(deletes);
|
||||
assertTrue(deletes.isEmpty());
|
||||
Assert.assertTrue(deletes.isEmpty());
|
||||
|
||||
// Get to make sure ...
|
||||
for (byte[] k : keys) {
|
||||
for (byte[] k : KEYS) {
|
||||
Get get = new Get(k);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
assertFalse(table.exists(get));
|
||||
Assert.assertFalse(table.exists(get));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testBatchWithManyColsInOneRowGetAndPut() throws Exception {
|
||||
@Test public void testBatchWithManyColsInOneRowGetAndPut() throws Exception {
|
||||
LOG.info("test=testBatchWithManyColsInOneRowGetAndPut");
|
||||
HTable table = new HTable(conf, TEST_TABLE);
|
||||
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
|
||||
|
||||
List<Row> puts = new ArrayList<Row>();
|
||||
for (int i = 0; i < 100; i++) {
|
||||
|
@ -293,13 +314,13 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
|
||||
}
|
||||
|
||||
public void testBatchWithMixedActions() throws Exception {
|
||||
@Test public void testBatchWithMixedActions() throws Exception {
|
||||
LOG.info("test=testBatchWithMixedActions");
|
||||
HTable table = new HTable(conf, TEST_TABLE);
|
||||
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
|
||||
|
||||
// Load some data to start
|
||||
Result[] results = table.batch(constructPutRequests());
|
||||
validateSizeAndEmpty(results, keys.size());
|
||||
validateSizeAndEmpty(results, KEYS.length);
|
||||
|
||||
// Batch: get, get, put(new col), delete, get, get of put, get of deleted,
|
||||
// put
|
||||
|
@ -309,42 +330,42 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
byte[] val2 = Bytes.toBytes("putvalue2");
|
||||
|
||||
// 0 get
|
||||
Get get = new Get(keys.get(10));
|
||||
Get get = new Get(KEYS[10]);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
actions.add(get);
|
||||
|
||||
// 1 get
|
||||
get = new Get(keys.get(11));
|
||||
get = new Get(KEYS[11]);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
actions.add(get);
|
||||
|
||||
// 2 put of new column
|
||||
Put put = new Put(keys.get(10));
|
||||
Put put = new Put(KEYS[10]);
|
||||
put.add(BYTES_FAMILY, qual2, val2);
|
||||
actions.add(put);
|
||||
|
||||
// 3 delete
|
||||
Delete delete = new Delete(keys.get(20));
|
||||
Delete delete = new Delete(KEYS[20]);
|
||||
delete.deleteFamily(BYTES_FAMILY);
|
||||
actions.add(delete);
|
||||
|
||||
// 4 get
|
||||
get = new Get(keys.get(30));
|
||||
get = new Get(KEYS[30]);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
actions.add(get);
|
||||
|
||||
// 5 get of the put in #2 (entire family)
|
||||
get = new Get(keys.get(10));
|
||||
get = new Get(KEYS[10]);
|
||||
get.addFamily(BYTES_FAMILY);
|
||||
actions.add(get);
|
||||
|
||||
// 6 get of the delete from #3
|
||||
get = new Get(keys.get(20));
|
||||
get = new Get(KEYS[20]);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
actions.add(get);
|
||||
|
||||
// 7 put of new column
|
||||
put = new Put(keys.get(40));
|
||||
put = new Put(KEYS[40]);
|
||||
put.add(BYTES_FAMILY, qual2, val2);
|
||||
actions.add(put);
|
||||
|
||||
|
@ -363,7 +384,7 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
validateEmpty(results[7]);
|
||||
|
||||
// validate last put, externally from the batch
|
||||
get = new Get(keys.get(40));
|
||||
get = new Get(KEYS[40]);
|
||||
get.addColumn(BYTES_FAMILY, qual2);
|
||||
Result r = table.get(get);
|
||||
validateResult(r, qual2, val2);
|
||||
|
@ -376,13 +397,13 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
}
|
||||
|
||||
private void validateResult(Result r, byte[] qual, byte[] val) {
|
||||
assertTrue(r.containsColumn(BYTES_FAMILY, qual));
|
||||
assertEquals(0, Bytes.compareTo(val, r.getValue(BYTES_FAMILY, qual)));
|
||||
Assert.assertTrue(r.containsColumn(BYTES_FAMILY, qual));
|
||||
Assert.assertEquals(0, Bytes.compareTo(val, r.getValue(BYTES_FAMILY, qual)));
|
||||
}
|
||||
|
||||
private List<Row> constructPutRequests() {
|
||||
List<Row> puts = new ArrayList<Row>();
|
||||
for (byte[] k : keys) {
|
||||
for (byte[] k : KEYS) {
|
||||
Put put = new Put(k);
|
||||
put.add(BYTES_FAMILY, QUALIFIER, VALUE);
|
||||
puts.add(put);
|
||||
|
@ -392,28 +413,28 @@ public class TestMultiParallel extends MultiRegionTable {
|
|||
|
||||
private void validateLoadedData(HTable table) throws IOException {
|
||||
// get the data back and validate that it is correct
|
||||
for (byte[] k : keys) {
|
||||
for (byte[] k : KEYS) {
|
||||
LOG.info("Assert=" + Bytes.toString(k));
|
||||
Get get = new Get(k);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
Result r = table.get(get);
|
||||
assertTrue(r.containsColumn(BYTES_FAMILY, QUALIFIER));
|
||||
assertEquals(0, Bytes.compareTo(VALUE, r
|
||||
Assert.assertTrue(r.containsColumn(BYTES_FAMILY, QUALIFIER));
|
||||
Assert.assertEquals(0, Bytes.compareTo(VALUE, r
|
||||
.getValue(BYTES_FAMILY, QUALIFIER)));
|
||||
}
|
||||
}
|
||||
|
||||
private void validateEmpty(Result result) {
|
||||
assertTrue(result != null);
|
||||
assertTrue(result.getRow() == null);
|
||||
assertEquals(0, result.raw().length);
|
||||
Assert.assertTrue(result != null);
|
||||
Assert.assertTrue(result.getRow() == null);
|
||||
Assert.assertEquals(0, result.raw().length);
|
||||
}
|
||||
|
||||
private void validateSizeAndEmpty(Result[] results, int expectedSize) {
|
||||
// Validate got back the same number of Result objects, all empty
|
||||
assertEquals(expectedSize, results.length);
|
||||
Assert.assertEquals(expectedSize, results.length);
|
||||
for (Result result : results) {
|
||||
validateEmpty(result);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue