HADOOP-1678 On region split, master should designate which host should serve daughter splits.
Phase 1: Master balances load for new regions and when a region server fails. git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@564780 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
790e3d767e
commit
931d452cb2
|
@ -86,3 +86,6 @@ Trunk (unreleased changes)
|
|||
53. HADOOP-1528 HClient for multiple tables - expose close table function
|
||||
54. HADOOP-1466 Clean up warnings, visibility and javadoc issues in HBase.
|
||||
55. HADOOP-1662 Make region splits faster
|
||||
56. HADOOP-1678 On region split, master should designate which host should
|
||||
serve daughter splits. Phase 1: Master balances load for new regions and
|
||||
when a region server fails.
|
||||
|
|
|
@ -488,8 +488,9 @@ public class HBaseAdmin implements HConstants {
|
|||
* @throws IllegalArgumentException - if the table name is reserved
|
||||
*/
|
||||
protected void checkReservedTableName(Text tableName) {
|
||||
if(tableName.equals(ROOT_TABLE_NAME)
|
||||
|| tableName.equals(META_TABLE_NAME)) {
|
||||
if(tableName.charAt(0) == '-' ||
|
||||
tableName.charAt(0) == '.' ||
|
||||
tableName.find(",") != -1) {
|
||||
|
||||
throw new IllegalArgumentException(tableName + " is a reserved table name");
|
||||
}
|
||||
|
|
|
@ -92,15 +92,17 @@ public interface HConstants {
|
|||
// Do we ever need to know all the information that we are storing?
|
||||
|
||||
/** The root table's name. */
|
||||
static final Text ROOT_TABLE_NAME = new Text("--ROOT--");
|
||||
static final Text ROOT_TABLE_NAME = new Text("-ROOT-");
|
||||
|
||||
/** The META table's name. */
|
||||
static final Text META_TABLE_NAME = new Text("--META--");
|
||||
static final Text META_TABLE_NAME = new Text(".META.");
|
||||
|
||||
// Defines for the column names used in both ROOT and META HBase 'meta' tables.
|
||||
|
||||
/** The ROOT and META column family */
|
||||
/** The ROOT and META column family (string) */
|
||||
static final String COLUMN_FAMILY_STR = "info:";
|
||||
|
||||
/** The ROOT and META column family (Text) */
|
||||
static final Text COLUMN_FAMILY = new Text(COLUMN_FAMILY_STR);
|
||||
|
||||
/** Array of meta column names */
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.conf.*;
|
|||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/**
|
||||
* HLog stores all the edits to the HStore.
|
||||
|
@ -73,14 +74,14 @@ public class HLog implements HConstants {
|
|||
|
||||
SequenceFile.Writer writer;
|
||||
TreeMap<Long, Path> outputfiles = new TreeMap<Long, Path>();
|
||||
boolean insideCacheFlush = false;
|
||||
volatile boolean insideCacheFlush = false;
|
||||
|
||||
TreeMap<Text, Long> regionToLastFlush = new TreeMap<Text, Long>();
|
||||
|
||||
boolean closed = false;
|
||||
transient long logSeqNum = 0;
|
||||
volatile boolean closed = false;
|
||||
volatile long logSeqNum = 0;
|
||||
long filenum = 0;
|
||||
transient int numEntries = 0;
|
||||
AtomicInteger numEntries = new AtomicInteger(0);
|
||||
|
||||
Integer rollLock = new Integer(0);
|
||||
|
||||
|
@ -125,7 +126,7 @@ public class HLog implements HConstants {
|
|||
logWriters.put(regionName, w);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Edit " + key.toString());
|
||||
LOG.debug("Edit " + key.toString() + "=" + val.toString());
|
||||
}
|
||||
w.append(key, val);
|
||||
}
|
||||
|
@ -173,6 +174,16 @@ public class HLog implements HConstants {
|
|||
fs.mkdirs(dir);
|
||||
rollWriter();
|
||||
}
|
||||
|
||||
synchronized void setSequenceNumber(long newvalue) {
|
||||
if (newvalue > logSeqNum) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("changing sequence number from " + logSeqNum + " to " +
|
||||
newvalue);
|
||||
}
|
||||
logSeqNum = newvalue;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Roll the log writer. That is, start writing log messages to a new file.
|
||||
|
@ -266,7 +277,7 @@ public class HLog implements HConstants {
|
|||
}
|
||||
fs.delete(p);
|
||||
}
|
||||
this.numEntries = 0;
|
||||
this.numEntries.set(0);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -343,13 +354,13 @@ public class HLog implements HConstants {
|
|||
new HLogKey(regionName, tableName, row, seqNum[counter++]);
|
||||
HLogEdit logEdit = new HLogEdit(es.getKey(), es.getValue(), timestamp);
|
||||
writer.append(logKey, logEdit);
|
||||
numEntries++;
|
||||
numEntries.getAndIncrement();
|
||||
}
|
||||
}
|
||||
|
||||
/** @return How many items have been added to the log */
|
||||
int getNumEntries() {
|
||||
return numEntries;
|
||||
return numEntries.get();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -418,7 +429,7 @@ public class HLog implements HConstants {
|
|||
writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId),
|
||||
new HLogEdit(HLog.METACOLUMN, COMPLETE_CACHEFLUSH.get(),
|
||||
System.currentTimeMillis()));
|
||||
numEntries++;
|
||||
numEntries.getAndIncrement();
|
||||
|
||||
// Remember the most-recent flush for each region.
|
||||
// This is used to delete obsolete log files.
|
||||
|
|
|
@ -72,8 +72,14 @@ public class HLogEdit implements Writable {
|
|||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public String toString() {
|
||||
return getColumn().toString() + " " + this.getTimestamp() + " " +
|
||||
new String(getVal()).trim();
|
||||
String value = "";
|
||||
try {
|
||||
value = new String(getVal(), HConstants.UTF8_ENCODING);
|
||||
|
||||
} catch (UnsupportedEncodingException e) {
|
||||
throw new RuntimeException("UTF8 encoding not present?", e);
|
||||
}
|
||||
return "(" + getColumn().toString() + "/" + getTimestamp() + "/" + value + ")";
|
||||
}
|
||||
|
||||
// Writable
|
||||
|
|
|
@ -84,7 +84,7 @@ public class HLogKey implements WritableComparable {
|
|||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
return tablename + "," + regionName + "," + row + "," + logSeqNum;
|
||||
return tablename + "/" + regionName + "/" + row + "/" + logSeqNum;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -275,6 +275,7 @@ public class HRegion implements HConstants {
|
|||
int compactionThreshold = 0;
|
||||
private final HLocking lock = new HLocking();
|
||||
private long desiredMaxFileSize;
|
||||
private final long maxSequenceId;
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// Constructor
|
||||
|
@ -324,12 +325,26 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
|
||||
// Load in all the HStores.
|
||||
|
||||
long maxSeqId = -1;
|
||||
for(Map.Entry<Text, HColumnDescriptor> e :
|
||||
this.regionInfo.tableDesc.families().entrySet()) {
|
||||
Text colFamily = HStoreKey.extractFamily(e.getKey());
|
||||
stores.put(colFamily,
|
||||
new HStore(rootDir, this.regionInfo.regionName, e.getValue(), fs,
|
||||
oldLogFile, conf));
|
||||
|
||||
HStore store = new HStore(rootDir, this.regionInfo.regionName,
|
||||
e.getValue(), fs, oldLogFile, conf);
|
||||
|
||||
stores.put(colFamily, store);
|
||||
|
||||
long storeSeqId = store.getMaxSequenceId();
|
||||
if (storeSeqId > maxSeqId) {
|
||||
maxSeqId = storeSeqId;
|
||||
}
|
||||
}
|
||||
this.maxSequenceId = maxSeqId;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("maximum sequence id for region " + regionInfo.getRegionName() +
|
||||
" is " + this.maxSequenceId);
|
||||
}
|
||||
|
||||
// Get rid of any splits or merges that were lost in-progress
|
||||
|
@ -361,6 +376,10 @@ public class HRegion implements HConstants {
|
|||
this.writestate.writesOngoing = false;
|
||||
LOG.info("region " + this.regionInfo.regionName + " available");
|
||||
}
|
||||
|
||||
long getMaxSequenceId() {
|
||||
return this.maxSequenceId;
|
||||
}
|
||||
|
||||
/** Returns a HRegionInfo object for this region */
|
||||
HRegionInfo getRegionInfo() {
|
||||
|
@ -464,8 +483,8 @@ public class HRegion implements HConstants {
|
|||
* @throws IOException
|
||||
*/
|
||||
HRegion[] closeAndSplit(final Text midKey,
|
||||
final RegionUnavailableListener listener)
|
||||
throws IOException {
|
||||
final RegionUnavailableListener listener) throws IOException {
|
||||
|
||||
checkMidKey(midKey);
|
||||
long startTime = System.currentTimeMillis();
|
||||
Path splits = getSplitsDir();
|
||||
|
@ -496,6 +515,7 @@ public class HRegion implements HConstants {
|
|||
Vector<HStoreFile> hstoreFilesToSplit = close();
|
||||
if (hstoreFilesToSplit == null) {
|
||||
LOG.warn("Close came back null (Implement abort of close?)");
|
||||
throw new RuntimeException("close returned empty vector of HStoreFiles");
|
||||
}
|
||||
|
||||
// Tell listener that region is now closed and that they can therefore
|
||||
|
@ -690,8 +710,11 @@ public class HRegion implements HConstants {
|
|||
biggest = size;
|
||||
}
|
||||
}
|
||||
biggest.setSplitable(splitable);
|
||||
if (biggest != null) {
|
||||
biggest.setSplitable(splitable);
|
||||
}
|
||||
return biggest;
|
||||
|
||||
} finally {
|
||||
lock.releaseReadLock();
|
||||
}
|
||||
|
@ -1405,6 +1428,7 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public String toString() {
|
||||
return getRegionName().toString();
|
||||
|
@ -1842,9 +1866,7 @@ public class HRegion implements HConstants {
|
|||
if (bytes == null || bytes.length == 0) {
|
||||
return null;
|
||||
}
|
||||
return (HRegionInfo)((bytes == null || bytes.length == 0)?
|
||||
null:
|
||||
Writables.getWritable(bytes, new HRegionInfo()));
|
||||
return (HRegionInfo) Writables.getWritable(bytes, new HRegionInfo());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1905,6 +1927,13 @@ public class HRegion implements HConstants {
|
|||
return startCode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Computes the Path of the HRegion
|
||||
*
|
||||
* @param dir parent directory
|
||||
* @param regionName name of the region
|
||||
* @return Path of HRegion directory
|
||||
*/
|
||||
public static Path getRegionDir(final Path dir, final Text regionName) {
|
||||
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName));
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import java.util.Vector;
|
|||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -391,6 +392,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
// Leases
|
||||
private Leases leases;
|
||||
|
||||
// Request counter
|
||||
private AtomicInteger requestCount;
|
||||
|
||||
/**
|
||||
* Starts a HRegionServer at the default location
|
||||
|
@ -424,6 +428,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
Collections.synchronizedSortedMap(new TreeMap<Text, HRegion>());
|
||||
|
||||
this.outboundMsgs = new Vector<HMsg>();
|
||||
this.requestCount = new AtomicInteger();
|
||||
|
||||
// Config'ed params
|
||||
this.numRetries = conf.getInt("hbase.client.retries.number", 2);
|
||||
|
@ -597,6 +602,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Telling master we are up");
|
||||
}
|
||||
requestCount.set(0);
|
||||
serverInfo.setLoad(new HServerLoad(0, onlineRegions.size()));
|
||||
hbaseMaster.regionServerStartup(serverInfo);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Done telling master we are up");
|
||||
|
@ -626,6 +633,10 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
}
|
||||
|
||||
try {
|
||||
serverInfo.setLoad(new HServerLoad(requestCount.get(),
|
||||
onlineRegions.size()));
|
||||
requestCount.set(0);
|
||||
|
||||
HMsg msgs[] =
|
||||
hbaseMaster.regionServerReport(serverInfo, outboundArray);
|
||||
lastMsg = System.currentTimeMillis();
|
||||
|
@ -897,6 +908,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
this.lock.writeLock().lock();
|
||||
try {
|
||||
this.log.setSequenceNumber(region.getMaxSequenceId());
|
||||
this.onlineRegions.put(region.getRegionName(), region);
|
||||
} finally {
|
||||
this.lock.writeLock().unlock();
|
||||
|
@ -963,6 +975,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
public HRegionInfo getRegionInfo(final Text regionName)
|
||||
throws NotServingRegionException {
|
||||
requestCount.incrementAndGet();
|
||||
return getRegion(regionName).getRegionInfo();
|
||||
}
|
||||
|
||||
|
@ -971,6 +984,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
long clientid = rand.nextLong();
|
||||
long lockid = startUpdate(regionName, clientid, b.getRow());
|
||||
for(BatchOperation op: b) {
|
||||
|
@ -993,6 +1007,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
public byte [] get(final Text regionName, final Text row,
|
||||
final Text column)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
return getRegion(regionName).get(row, column);
|
||||
}
|
||||
|
||||
|
@ -1002,6 +1017,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
public byte [][] get(final Text regionName, final Text row,
|
||||
final Text column, final int numVersions)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
return getRegion(regionName).get(row, column, numVersions);
|
||||
}
|
||||
|
||||
|
@ -1010,6 +1026,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
public byte [][] get(final Text regionName, final Text row, final Text column,
|
||||
final long timestamp, final int numVersions) throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
return getRegion(regionName).get(row, column, timestamp, numVersions);
|
||||
}
|
||||
|
||||
|
@ -1018,6 +1035,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
public KeyedData[] getRow(final Text regionName, final Text row)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
HRegion region = getRegion(regionName);
|
||||
TreeMap<Text, byte[]> map = region.getFull(row);
|
||||
KeyedData result[] = new KeyedData[map.size()];
|
||||
|
@ -1034,6 +1052,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
public KeyedData[] next(final long scannerId)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
String scannerName = String.valueOf(scannerId);
|
||||
HInternalScannerInterface s = scanners.get(scannerName);
|
||||
if (s == null) {
|
||||
|
@ -1077,6 +1096,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
public long startUpdate(Text regionName, long clientid, Text row)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
HRegion region = getRegion(regionName);
|
||||
long lockid = region.startUpdate(row);
|
||||
this.leases.createLease(clientid, lockid,
|
||||
|
@ -1120,6 +1140,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
public void put(final Text regionName, final long clientid,
|
||||
final long lockid, final Text column, final byte [] val)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
HRegion region = getRegion(regionName, true);
|
||||
leases.renewLease(clientid, lockid);
|
||||
region.put(lockid, column, val);
|
||||
|
@ -1130,6 +1151,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
public void delete(Text regionName, long clientid, long lockid, Text column)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
HRegion region = getRegion(regionName);
|
||||
leases.renewLease(clientid, lockid);
|
||||
region.delete(lockid, column);
|
||||
|
@ -1140,6 +1162,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
public void abort(Text regionName, long clientid, long lockid)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
HRegion region = getRegion(regionName, true);
|
||||
leases.cancelLease(clientid, lockid);
|
||||
region.abort(lockid);
|
||||
|
@ -1150,6 +1173,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
public void commit(Text regionName, final long clientid, final long lockid,
|
||||
final long timestamp) throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
HRegion region = getRegion(regionName, true);
|
||||
leases.cancelLease(clientid, lockid);
|
||||
region.commit(lockid, timestamp);
|
||||
|
@ -1159,6 +1183,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
public void renewLease(long lockid, long clientid) throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
leases.renewLease(clientid, lockid);
|
||||
}
|
||||
|
||||
|
@ -1247,6 +1272,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
public long openScanner(Text regionName, Text[] cols, Text firstRow,
|
||||
final long timestamp, final RowFilterInterface filter)
|
||||
throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
HRegion r = getRegion(regionName);
|
||||
long scannerId = -1L;
|
||||
try {
|
||||
|
@ -1277,6 +1303,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
public void close(final long scannerId) throws IOException {
|
||||
requestCount.incrementAndGet();
|
||||
String scannerName = String.valueOf(scannerId);
|
||||
HInternalScannerInterface s = null;
|
||||
synchronized(scanners) {
|
||||
|
|
|
@ -33,21 +33,24 @@ import java.io.*;
|
|||
public class HServerInfo implements Writable {
|
||||
private HServerAddress serverAddress;
|
||||
private long startCode;
|
||||
private HServerLoad load;
|
||||
|
||||
/** default constructor - used by Writable */
|
||||
public HServerInfo() {
|
||||
this.serverAddress = new HServerAddress();
|
||||
this.startCode = 0;
|
||||
this.load = new HServerLoad();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a fully initialized object
|
||||
* Constructor
|
||||
* @param serverAddress
|
||||
* @param startCode
|
||||
*/
|
||||
public HServerInfo(HServerAddress serverAddress, long startCode) {
|
||||
this.serverAddress = new HServerAddress(serverAddress);
|
||||
this.startCode = startCode;
|
||||
this.load = new HServerLoad();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -57,6 +60,21 @@ public class HServerInfo implements Writable {
|
|||
public HServerInfo(HServerInfo other) {
|
||||
this.serverAddress = new HServerAddress(other.getServerAddress());
|
||||
this.startCode = other.getStartCode();
|
||||
this.load = other.getLoad();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the load
|
||||
*/
|
||||
public HServerLoad getLoad() {
|
||||
return load;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param load the load to set
|
||||
*/
|
||||
public void setLoad(HServerLoad load) {
|
||||
this.load = load;
|
||||
}
|
||||
|
||||
/** @return the server address */
|
||||
|
@ -72,7 +90,8 @@ public class HServerInfo implements Writable {
|
|||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public String toString() {
|
||||
return "address: " + this.serverAddress + ", startcode: " + this.startCode;
|
||||
return "address: " + this.serverAddress + ", startcode: " + this.startCode
|
||||
+ ", load: (" + this.load.toString() + ")";
|
||||
}
|
||||
|
||||
// Writable
|
||||
|
@ -81,11 +100,13 @@ public class HServerInfo implements Writable {
|
|||
public void readFields(DataInput in) throws IOException {
|
||||
this.serverAddress.readFields(in);
|
||||
this.startCode = in.readLong();
|
||||
this.load.readFields(in);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public void write(DataOutput out) throws IOException {
|
||||
this.serverAddress.write(out);
|
||||
out.writeLong(this.startCode);
|
||||
this.load.write(out);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,136 @@
|
|||
/**
|
||||
* Copyright 2007 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;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
||||
/**
|
||||
* This class encapsulates metrics for determining the load on a HRegionServer
|
||||
*/
|
||||
public class HServerLoad implements WritableComparable {
|
||||
private int numberOfRequests; // number of requests since last report
|
||||
private int numberOfRegions; // number of regions being served
|
||||
|
||||
/*
|
||||
* TODO: Other metrics that might be considered when the master is actually
|
||||
* doing load balancing instead of merely trying to decide where to assign
|
||||
* a region:
|
||||
* <ul>
|
||||
* <li># of CPUs, heap size (to determine the "class" of machine). For
|
||||
* now, we consider them to be homogeneous.</li>
|
||||
* <li>#requests per region (Map<{String|HRegionInfo}, Integer>)</li>
|
||||
* <li>#compactions and/or #splits (churn)</li>
|
||||
* <li>server death rate (maybe there is something wrong with this server)</li>
|
||||
* </ul>
|
||||
*/
|
||||
|
||||
/** default constructior (used by Writable) */
|
||||
public HServerLoad() {}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param numberOfRequests
|
||||
* @param numberOfRegions
|
||||
*/
|
||||
public HServerLoad(int numberOfRequests, int numberOfRegions) {
|
||||
this.numberOfRequests = numberOfRequests;
|
||||
this.numberOfRegions = numberOfRegions;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return load factor for this server
|
||||
*/
|
||||
public int getLoad() {
|
||||
int load = numberOfRequests == 0 ? 1 : numberOfRequests;
|
||||
load *= numberOfRegions == 0 ? 1 : numberOfRegions;
|
||||
return load;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public String toString() {
|
||||
return "requests: " + numberOfRequests + " regions: " + numberOfRegions;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
return compareTo(o) == 0;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = Integer.valueOf(numberOfRequests).hashCode();
|
||||
result ^= Integer.valueOf(numberOfRegions).hashCode();
|
||||
return result;
|
||||
}
|
||||
|
||||
// Getters
|
||||
|
||||
/**
|
||||
* @return the numberOfRegions
|
||||
*/
|
||||
public int getNumberOfRegions() {
|
||||
return numberOfRegions;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the numberOfRequests
|
||||
*/
|
||||
public int getNumberOfRequests() {
|
||||
return numberOfRequests;
|
||||
}
|
||||
|
||||
// Setters
|
||||
|
||||
/**
|
||||
* @param numberOfRegions the numberOfRegions to set
|
||||
*/
|
||||
public void setNumberOfRegions(int numberOfRegions) {
|
||||
this.numberOfRegions = numberOfRegions;
|
||||
}
|
||||
|
||||
// Writable
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
numberOfRequests = in.readInt();
|
||||
numberOfRegions = in.readInt();
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeInt(numberOfRequests);
|
||||
out.writeInt(numberOfRegions);
|
||||
}
|
||||
|
||||
// Comparable
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public int compareTo(Object o) {
|
||||
HServerLoad other = (HServerLoad) o;
|
||||
return this.getLoad() - other.getLoad();
|
||||
}
|
||||
}
|
|
@ -90,6 +90,8 @@ class HStore implements HConstants {
|
|||
TreeMap<Long, MapFile.Reader> readers = new TreeMap<Long, MapFile.Reader>();
|
||||
|
||||
Random rand = new Random();
|
||||
|
||||
private long maxSeqId;
|
||||
|
||||
/**
|
||||
* An HStore is a set of zero or more MapFiles, which stretch backwards over
|
||||
|
@ -196,6 +198,7 @@ class HStore implements HConstants {
|
|||
// If the HSTORE_LOGINFOFILE doesn't contain a number, just ignore it. That
|
||||
// means it was built prior to the previous run of HStore, and so it cannot
|
||||
// contain any updates also contained in the log.
|
||||
|
||||
long maxSeqID = -1;
|
||||
for (HStoreFile hsf: hstoreFiles) {
|
||||
long seqid = hsf.loadInfo(fs);
|
||||
|
@ -205,8 +208,14 @@ class HStore implements HConstants {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
doReconstructionLog(reconstructionLog, maxSeqID);
|
||||
this.maxSeqId = maxSeqID;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("maximum sequence id for hstore " + storeName + " is " +
|
||||
this.maxSeqId);
|
||||
}
|
||||
|
||||
doReconstructionLog(reconstructionLog, maxSeqId);
|
||||
this.maxSeqId += 1;
|
||||
|
||||
// Compact all the MapFiles into a single file. The resulting MapFile
|
||||
// should be "timeless"; that is, it should not have an associated seq-ID,
|
||||
|
@ -228,6 +237,10 @@ class HStore implements HConstants {
|
|||
}
|
||||
}
|
||||
|
||||
long getMaxSequenceId() {
|
||||
return this.maxSeqId;
|
||||
}
|
||||
|
||||
/*
|
||||
* Read the reconstructionLog to see whether we need to build a brand-new
|
||||
* MapFile out of non-flushed log entries.
|
||||
|
@ -258,6 +271,11 @@ class HStore implements HConstants {
|
|||
while (login.next(key, val)) {
|
||||
maxSeqIdInLog = Math.max(maxSeqIdInLog, key.getLogSeqNum());
|
||||
if (key.getLogSeqNum() <= maxSeqID) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Skipping edit <" + key.toString() + "=" +
|
||||
val.toString() + "> key sequence: " + key.getLogSeqNum() +
|
||||
" max sequence: " + maxSeqID);
|
||||
}
|
||||
continue;
|
||||
}
|
||||
// Check this edit is for me. Also, guard against writing
|
||||
|
@ -277,7 +295,8 @@ class HStore implements HConstants {
|
|||
}
|
||||
HStoreKey k = new HStoreKey(key.getRow(), column, val.getTimestamp());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Applying edit " + k.toString());
|
||||
LOG.debug("Applying edit <" + k.toString() + "=" + val.toString() +
|
||||
">");
|
||||
}
|
||||
reconstructedCache.put(k, val.getVal());
|
||||
}
|
||||
|
@ -428,16 +447,12 @@ class HStore implements HConstants {
|
|||
String name = flushedFile.toString();
|
||||
MapFile.Writer out = flushedFile.getWriter(this.fs, this.compression,
|
||||
this.bloomFilter);
|
||||
int count = 0;
|
||||
int total = 0;
|
||||
try {
|
||||
for (Map.Entry<HStoreKey, byte []> es: inputCache.entrySet()) {
|
||||
HStoreKey curkey = es.getKey();
|
||||
total++;
|
||||
if (this.familyName.
|
||||
equals(HStoreKey.extractFamily(curkey.getColumn()))) {
|
||||
out.append(curkey, new ImmutableBytesWritable(es.getValue()));
|
||||
count++;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
|
@ -1030,6 +1045,7 @@ class HStore implements HConstants {
|
|||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
class HStoreScanner extends HAbstractScanner {
|
||||
@SuppressWarnings("hiding")
|
||||
private MapFile.Reader[] readers;
|
||||
|
||||
HStoreScanner(long timestamp, Text[] targetCols, Text firstRow)
|
||||
|
|
|
@ -94,7 +94,17 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
static final String HSTORE_DATFILE_DIR = "mapfiles";
|
||||
static final String HSTORE_INFO_DIR = "info";
|
||||
static final String HSTORE_FILTER_DIR = "filter";
|
||||
public static enum Range {top, bottom}
|
||||
|
||||
/**
|
||||
* For split HStoreFiles, specifies if the file covers the lower half or
|
||||
* the upper half of the key range
|
||||
*/
|
||||
public static enum Range {
|
||||
/** HStoreFile contains upper half of key range */
|
||||
top,
|
||||
/** HStoreFile contains lower half of key range */
|
||||
bottom
|
||||
}
|
||||
|
||||
/*
|
||||
* Regex that will work for straight filenames and for reference names.
|
||||
|
@ -156,7 +166,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
/*
|
||||
* Data structure to hold reference to a store file over in another region.
|
||||
*/
|
||||
static class Reference {
|
||||
static class Reference implements Writable {
|
||||
Text regionName;
|
||||
long fileid;
|
||||
Range region;
|
||||
|
@ -190,11 +200,15 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
return this.regionName;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.regionName + "/" + this.fileid + "/" + this.region;
|
||||
}
|
||||
|
||||
// Make it serializable.
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public void write(DataOutput out) throws IOException {
|
||||
this.regionName.write(out);
|
||||
out.writeLong(this.fileid);
|
||||
|
@ -203,6 +217,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
this.midkey.write(out);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
this.regionName = new Text();
|
||||
this.regionName.readFields(in);
|
||||
|
@ -417,6 +432,8 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
private static boolean isReference(final Path p, final Matcher m) {
|
||||
if (m == null || !m.matches()) {
|
||||
LOG.warn("Failed match of store file name " + p.toString());
|
||||
throw new RuntimeException("Failed match of store file name " +
|
||||
p.toString());
|
||||
}
|
||||
return m.groupCount() > 1 && m.group(2) != null;
|
||||
}
|
||||
|
@ -662,6 +679,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
}
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@SuppressWarnings({ "unused"})
|
||||
@Override
|
||||
public synchronized void finalKey(WritableComparable key)
|
||||
|
@ -669,6 +687,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
throw new UnsupportedOperationException("Unsupported");
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public synchronized Writable get(WritableComparable key, Writable val)
|
||||
throws IOException {
|
||||
|
@ -676,6 +695,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
return super.get(key, val);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public synchronized WritableComparable getClosest(WritableComparable key,
|
||||
|
@ -692,6 +712,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
return super.getClosest(key, val);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@SuppressWarnings("unused")
|
||||
@Override
|
||||
public synchronized WritableComparable midKey() throws IOException {
|
||||
|
@ -699,6 +720,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
return null;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public synchronized boolean next(WritableComparable key, Writable val)
|
||||
|
@ -727,6 +749,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
return false;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public synchronized void reset() throws IOException {
|
||||
if (top) {
|
||||
|
@ -737,6 +760,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
super.reset();
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public synchronized boolean seek(WritableComparable key)
|
||||
throws IOException {
|
||||
|
@ -758,6 +782,15 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
static class Reader extends MapFile.Reader {
|
||||
private final Filter bloomFilter;
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param fs
|
||||
* @param dirName
|
||||
* @param conf
|
||||
* @param filter
|
||||
* @throws IOException
|
||||
*/
|
||||
public Reader(FileSystem fs, String dirName, Configuration conf,
|
||||
final Filter filter)
|
||||
throws IOException {
|
||||
|
@ -810,6 +843,18 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
private final Filter bloomFilter;
|
||||
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param conf
|
||||
* @param fs
|
||||
* @param dirName
|
||||
* @param keyClass
|
||||
* @param valClass
|
||||
* @param compression
|
||||
* @param filter
|
||||
* @throws IOException
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public Writer(Configuration conf, FileSystem fs, String dirName,
|
||||
Class keyClass, Class valClass,
|
||||
|
@ -905,6 +950,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
return (isReference())? l / 2: l;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.regionName.toString() + "/" + this.colFamily.toString() +
|
||||
|
@ -912,6 +958,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
(isReference()? "/" + this.reference.toString(): "");
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
return this.compareTo(o) == 0;
|
||||
|
|
|
@ -133,6 +133,7 @@ public class HTable implements HConstants {
|
|||
}
|
||||
|
||||
|
||||
/** @return the table name */
|
||||
public Text getTableName() {
|
||||
return this.tableName;
|
||||
}
|
||||
|
|
|
@ -41,13 +41,13 @@ public class HTableDescriptor implements WritableComparable {
|
|||
|
||||
/*
|
||||
* Legal table names can only contain 'word characters':
|
||||
* i.e. <code>[a-zA-Z_0-9]</code>.
|
||||
* i.e. <code>[a-zA-Z_0-9-.]</code>.
|
||||
* Lets be restrictive until a reason to be otherwise. One reason to limit
|
||||
* characters in table name is to ensure table regions as entries in META
|
||||
* regions can be found (See HADOOP-1581 'HBASE: Un-openable tablename bug').
|
||||
*/
|
||||
private static final Pattern LEGAL_TABLE_NAME =
|
||||
Pattern.compile("[\\w-]+");
|
||||
Pattern.compile("^[\\w-.]+$");
|
||||
|
||||
/** Constructs an empty object */
|
||||
public HTableDescriptor() {
|
||||
|
|
|
@ -143,33 +143,43 @@ public class MiniHBaseCluster implements HConstants {
|
|||
}
|
||||
}
|
||||
|
||||
/** runs the master server */
|
||||
public static class MasterThread extends Thread {
|
||||
private final HMaster master;
|
||||
MasterThread(final HMaster m) {
|
||||
super(m, "Master:" + m.getMasterAddress().toString());
|
||||
this.master = m;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public void run() {
|
||||
LOG.info("Starting " + getName());
|
||||
super.run();
|
||||
}
|
||||
|
||||
/** @return master server */
|
||||
public HMaster getMaster() {
|
||||
return this.master;
|
||||
}
|
||||
}
|
||||
|
||||
/** runs region servers */
|
||||
public static class RegionServerThread extends Thread {
|
||||
private final HRegionServer regionServer;
|
||||
RegionServerThread(final HRegionServer r, final int index) {
|
||||
super(r, "RegionServer:" + index);
|
||||
this.regionServer = r;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public void run() {
|
||||
LOG.info("Starting " + getName());
|
||||
super.run();
|
||||
}
|
||||
|
||||
/** @return the region server */
|
||||
public HRegionServer getRegionServer() {
|
||||
return this.regionServer;
|
||||
}
|
||||
|
@ -227,6 +237,11 @@ public class MiniHBaseCluster implements HConstants {
|
|||
return threads;
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts a region server thread running
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
public void startRegionServer() throws IOException {
|
||||
RegionServerThread t =
|
||||
startRegionServer(this.conf, this.regionThreads.size());
|
||||
|
@ -275,6 +290,7 @@ public class MiniHBaseCluster implements HConstants {
|
|||
* Shut down the specified region server cleanly
|
||||
*
|
||||
* @param serverNumber
|
||||
* @return the region server that was stopped
|
||||
*/
|
||||
public HRegionServer stopRegionServer(int serverNumber) {
|
||||
HRegionServer server =
|
||||
|
|
|
@ -34,10 +34,10 @@ public class TestCleanRegionServerExit extends HBaseClusterTestCase {
|
|||
|
||||
/** constructor */
|
||||
public TestCleanRegionServerExit() {
|
||||
super();
|
||||
super(2);
|
||||
conf.setInt("ipc.client.timeout", 5000); // reduce ipc client timeout
|
||||
conf.setInt("ipc.client.connect.max.retries", 5); // and number of retries
|
||||
conf.setInt("hbase.client.retries.number", 2); // reduce HBase retries
|
||||
conf.setInt("hbase.client.retries.number", 3); // reduce HBase retries
|
||||
Logger.getRootLogger().setLevel(Level.WARN);
|
||||
Logger.getLogger(this.getClass().getPackage().getName()).setLevel(Level.DEBUG);
|
||||
}
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.log4j.Logger;
|
|||
* HRegions or in the HBaseMaster, so only basic testing is possible.
|
||||
*/
|
||||
public class TestHRegion extends HBaseTestCase implements RegionUnavailableListener {
|
||||
private static final Logger LOG =
|
||||
static final Logger LOG =
|
||||
Logger.getLogger(TestHRegion.class.getName());
|
||||
|
||||
/** Constructor */
|
||||
|
|
|
@ -34,6 +34,9 @@ import org.apache.hadoop.io.SequenceFile;
|
|||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
||||
/**
|
||||
* Test HStoreFile
|
||||
*/
|
||||
public class TestHStoreFile extends TestCase {
|
||||
static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
|
||||
private static String DIR = System.getProperty("test.build.data", ".");
|
||||
|
@ -226,7 +229,9 @@ public class TestHStoreFile extends TestCase {
|
|||
}
|
||||
assertTrue(key.compareTo(midkey) < 0);
|
||||
}
|
||||
LOG.info("Last in bottom: " + previous.toString());
|
||||
if (previous != null) {
|
||||
LOG.info("Last in bottom: " + previous.toString());
|
||||
}
|
||||
// Now test reading from the top.
|
||||
top = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf,
|
||||
HStoreFile.Range.top, midkey);
|
||||
|
@ -249,17 +254,17 @@ public class TestHStoreFile extends TestCase {
|
|||
// Next test using a midkey that does not exist in the file.
|
||||
// First, do a key that is < than first key. Ensure splits behave
|
||||
// properly.
|
||||
midkey = new HStoreKey(new Text(" "));
|
||||
WritableComparable badkey = new HStoreKey(new Text(" "));
|
||||
bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(),
|
||||
this.conf, HStoreFile.Range.bottom, midkey);
|
||||
// When midkey is < than the bottom, should return no values.
|
||||
this.conf, HStoreFile.Range.bottom, badkey);
|
||||
// When badkey is < than the bottom, should return no values.
|
||||
assertFalse(bottom.next(key, value));
|
||||
// Now read from the top.
|
||||
top = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf,
|
||||
HStoreFile.Range.top, midkey);
|
||||
HStoreFile.Range.top, badkey);
|
||||
first = true;
|
||||
while (top.next(key, value)) {
|
||||
assertTrue(key.compareTo(midkey) >= 0);
|
||||
assertTrue(key.compareTo(badkey) >= 0);
|
||||
if (first) {
|
||||
first = false;
|
||||
LOG.info("First top when key < bottom: " + key.toString());
|
||||
|
@ -275,10 +280,10 @@ public class TestHStoreFile extends TestCase {
|
|||
assertTrue(tmp.charAt(i) == 'z');
|
||||
}
|
||||
|
||||
// Test when midkey is > than last key in file ('||' > 'zz').
|
||||
midkey = new HStoreKey(new Text("|||"));
|
||||
// Test when badkey is > than last key in file ('||' > 'zz').
|
||||
badkey = new HStoreKey(new Text("|||"));
|
||||
bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(),
|
||||
this.conf, HStoreFile.Range.bottom, midkey);
|
||||
this.conf, HStoreFile.Range.bottom, badkey);
|
||||
first = true;
|
||||
while (bottom.next(key, value)) {
|
||||
if (first) {
|
||||
|
@ -297,7 +302,7 @@ public class TestHStoreFile extends TestCase {
|
|||
}
|
||||
// Now look at top. Should not return any values.
|
||||
top = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf,
|
||||
HStoreFile.Range.top, midkey);
|
||||
HStoreFile.Range.top, badkey);
|
||||
assertFalse(top.next(key, value));
|
||||
|
||||
} finally {
|
||||
|
|
|
@ -34,10 +34,10 @@ public class TestRegionServerAbort extends HBaseClusterTestCase {
|
|||
|
||||
/** constructor */
|
||||
public TestRegionServerAbort() {
|
||||
super();
|
||||
super(2);
|
||||
conf.setInt("ipc.client.timeout", 5000); // reduce client timeout
|
||||
conf.setInt("ipc.client.connect.max.retries", 5); // and number of retries
|
||||
conf.setInt("hbase.client.retries.number", 2); // reduce HBase retries
|
||||
conf.setInt("hbase.client.retries.number", 3); // reduce HBase retries
|
||||
Logger.getRootLogger().setLevel(Level.WARN);
|
||||
Logger.getLogger(this.getClass().getPackage().getName()).setLevel(Level.DEBUG);
|
||||
}
|
||||
|
|
|
@ -206,7 +206,8 @@ public class TestScanner2 extends HBaseClusterTestCase {
|
|||
HRegionLocation rl = t.getRegionLocation(table);
|
||||
regionServer = t.getConnection().getHRegionConnection(rl.getServerAddress());
|
||||
scannerId = regionServer.openScanner(rl.getRegionInfo().getRegionName(),
|
||||
HMaster.METACOLUMNS, new Text(), System.currentTimeMillis(), null);
|
||||
HConstants.COLUMN_FAMILY_ARRAY, new Text(),
|
||||
System.currentTimeMillis(), null);
|
||||
while (true) {
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
KeyedData[] values = regionServer.next(scannerId);
|
||||
|
|
|
@ -46,6 +46,7 @@ public class TestSplit extends HBaseTestCase {
|
|||
private static final char FIRST_CHAR = 'a';
|
||||
private static final char LAST_CHAR = 'z';
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
@ -59,6 +60,7 @@ public class TestSplit extends HBaseTestCase {
|
|||
conf.setLong("hbase.hregion.max.filesize", 1024 * 128);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
try {
|
||||
|
|
|
@ -35,8 +35,8 @@ public class TestToString extends TestCase {
|
|||
HServerAddress address = new HServerAddress(hostport);
|
||||
assertEquals("HServerAddress toString", address.toString(), hostport);
|
||||
HServerInfo info = new HServerInfo(address, -1);
|
||||
assertEquals("HServerInfo", info.toString(),
|
||||
"address: " + hostport + ", startcode: " + -1);
|
||||
assertEquals("HServerInfo", "address: " + hostport + ", startcode: -1" +
|
||||
", load: (requests: 0 regions: 0)", info.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue