HADOOP-1282. Omnibus HBase patch. Improved tests and configuration. Contributed by Jim Kellerman.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@532083 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
114d67c614
commit
ec9a568764
|
@ -0,0 +1,48 @@
|
|||
<?xml version="1.0"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>hbase.master</name>
|
||||
<value>localhost:60000</value>
|
||||
<description>The host and port that the HBase master runs at.
|
||||
TODO: Support 'local' (All running in single context).
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.regionserver</name>
|
||||
<value>localhost:60010</value>
|
||||
<description>The host and port a HBase region server runs at.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.regiondir</name>
|
||||
<value>${hadoop.tmp.dir}/hbase</value>
|
||||
<description>The directory shared by region servers.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.client.timeout.length</name>
|
||||
<value>10000</value>
|
||||
<description>Client timeout in milliseconds</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.client.timeout.number</name>
|
||||
<value>5</value>
|
||||
<description>Try this many timeouts before giving up.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.client.retries.number</name>
|
||||
<value>2</value>
|
||||
<description>Count of maximum retries fetching the root region from root
|
||||
region server.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.master.meta.thread.rescanfrequency</name>
|
||||
<value>60000</value>
|
||||
<description>How long the HMaster sleeps (in milliseconds) between scans of
|
||||
the META table.
|
||||
</description>
|
||||
</property>
|
||||
</configuration>
|
|
@ -60,17 +60,17 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
String column = col.toString();
|
||||
try {
|
||||
int colpos = column.indexOf(":") + 1;
|
||||
if (colpos == 0) {
|
||||
if(colpos == 0) {
|
||||
throw new IllegalArgumentException("Column name has no family indicator.");
|
||||
}
|
||||
|
||||
String columnkey = column.substring(colpos);
|
||||
|
||||
if (columnkey == null || columnkey.length() == 0) {
|
||||
if(columnkey == null || columnkey.length() == 0) {
|
||||
this.matchType = MATCH_TYPE.FAMILY_ONLY;
|
||||
this.family = column.substring(0, colpos);
|
||||
|
||||
} else if (isRegexPattern.matcher(columnkey).matches()) {
|
||||
} else if(isRegexPattern.matcher(columnkey).matches()) {
|
||||
this.matchType = MATCH_TYPE.REGEX;
|
||||
this.columnMatcher = Pattern.compile(column);
|
||||
|
||||
|
@ -86,13 +86,13 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
// Matching method
|
||||
|
||||
boolean matches(Text col) throws IOException {
|
||||
if (this.matchType == MATCH_TYPE.SIMPLE) {
|
||||
if(this.matchType == MATCH_TYPE.SIMPLE) {
|
||||
return col.equals(this.col);
|
||||
|
||||
} else if (this.matchType == MATCH_TYPE.FAMILY_ONLY) {
|
||||
} else if(this.matchType == MATCH_TYPE.FAMILY_ONLY) {
|
||||
return col.toString().startsWith(this.family);
|
||||
|
||||
} else if (this.matchType == MATCH_TYPE.REGEX) {
|
||||
} else if(this.matchType == MATCH_TYPE.REGEX) {
|
||||
return this.columnMatcher.matcher(col.toString()).matches();
|
||||
|
||||
} else {
|
||||
|
@ -121,7 +121,7 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
for(int i = 0; i < targetCols.length; i++) {
|
||||
Text family = HStoreKey.extractFamily(targetCols[i]);
|
||||
Vector<ColumnMatcher> matchers = okCols.get(family);
|
||||
if (matchers == null) {
|
||||
if(matchers == null) {
|
||||
matchers = new Vector<ColumnMatcher>();
|
||||
}
|
||||
matchers.add(new ColumnMatcher(targetCols[i]));
|
||||
|
@ -144,11 +144,11 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
Text column = keys[i].getColumn();
|
||||
Text family = HStoreKey.extractFamily(column);
|
||||
Vector<ColumnMatcher> matchers = okCols.get(family);
|
||||
if (matchers == null) {
|
||||
if(matchers == null) {
|
||||
return false;
|
||||
}
|
||||
for(int m = 0; m < matchers.size(); m++) {
|
||||
if (matchers.get(m).matches(column)) {
|
||||
if(matchers.get(m).matches(column)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
@ -180,7 +180,7 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
* @see org.apache.hadoop.hbase.HScannerInterface#next(org.apache.hadoop.hbase.HStoreKey, java.util.TreeMap)
|
||||
*/
|
||||
public boolean next(HStoreKey key, TreeMap<Text, byte[]> results)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
// Find the next row label (and timestamp)
|
||||
|
||||
|
@ -188,12 +188,12 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
long chosenTimestamp = -1;
|
||||
for(int i = 0; i < keys.length; i++) {
|
||||
while((keys[i] != null)
|
||||
&& (columnMatch(i))
|
||||
&& (keys[i].getTimestamp() <= this.timestamp)
|
||||
&& ((chosenRow == null)
|
||||
|| (keys[i].getRow().compareTo(chosenRow) < 0)
|
||||
|| ((keys[i].getRow().compareTo(chosenRow) == 0)
|
||||
&& (keys[i].getTimestamp() > chosenTimestamp)))) {
|
||||
&& (columnMatch(i))
|
||||
&& (keys[i].getTimestamp() <= this.timestamp)
|
||||
&& ((chosenRow == null)
|
||||
|| (keys[i].getRow().compareTo(chosenRow) < 0)
|
||||
|| ((keys[i].getRow().compareTo(chosenRow) == 0)
|
||||
&& (keys[i].getTimestamp() > chosenTimestamp)))) {
|
||||
|
||||
chosenRow = new Text(keys[i].getRow());
|
||||
chosenTimestamp = keys[i].getTimestamp();
|
||||
|
@ -203,7 +203,7 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
// Grab all the values that match this row/timestamp
|
||||
|
||||
boolean insertedItem = false;
|
||||
if (chosenRow != null) {
|
||||
if(chosenRow != null) {
|
||||
key.setRow(chosenRow);
|
||||
key.setVersion(chosenTimestamp);
|
||||
key.setColumn(new Text(""));
|
||||
|
@ -212,10 +212,10 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
// Fetch the data
|
||||
|
||||
while((keys[i] != null)
|
||||
&& (keys[i].getRow().compareTo(chosenRow) == 0)
|
||||
&& (keys[i].getTimestamp() == chosenTimestamp)) {
|
||||
&& (keys[i].getRow().compareTo(chosenRow) == 0)
|
||||
&& (keys[i].getTimestamp() == chosenTimestamp)) {
|
||||
|
||||
if (columnMatch(i)) {
|
||||
if(columnMatch(i)) {
|
||||
outbuf.reset();
|
||||
vals[i].write(outbuf);
|
||||
byte byteresults[] = outbuf.getData();
|
||||
|
@ -226,7 +226,7 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
insertedItem = true;
|
||||
}
|
||||
|
||||
if (!getNext(i)) {
|
||||
if (! getNext(i)) {
|
||||
closeSubScanner(i);
|
||||
}
|
||||
}
|
||||
|
@ -235,9 +235,9 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
// a valid timestamp, so we're ready next time.
|
||||
|
||||
while((keys[i] != null)
|
||||
&& ((keys[i].getRow().compareTo(chosenRow) <= 0)
|
||||
|| (keys[i].getTimestamp() > this.timestamp)
|
||||
|| (!columnMatch(i)))) {
|
||||
&& ((keys[i].getRow().compareTo(chosenRow) <= 0)
|
||||
|| (keys[i].getTimestamp() > this.timestamp)
|
||||
|| (! columnMatch(i)))) {
|
||||
|
||||
getNext(i);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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 org.apache.hadoop.conf.Configuration;
|
||||
|
||||
public class HBaseConfiguration extends Configuration {
|
||||
public HBaseConfiguration() {
|
||||
super();
|
||||
addDefaultResource("hbase-default.xml");
|
||||
}
|
||||
}
|
|
@ -15,29 +15,39 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.io.*;
|
||||
import org.apache.hadoop.ipc.*;
|
||||
import org.apache.hadoop.conf.*;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.Random;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/*******************************************************************************
|
||||
* HClient manages a connection to a single HRegionServer.
|
||||
******************************************************************************/
|
||||
public class HClient extends HGlobals implements HConstants {
|
||||
public class HClient implements HConstants {
|
||||
private final Logger LOG =
|
||||
Logger.getLogger(this.getClass().getName());
|
||||
|
||||
private static final Text[] metaColumns = {
|
||||
META_COLUMN_FAMILY
|
||||
};
|
||||
private static final Text startRow = new Text();
|
||||
|
||||
private boolean closed;
|
||||
private Configuration conf;
|
||||
private HServerAddress masterLocation;
|
||||
private long clientTimeout;
|
||||
private int numTimeouts;
|
||||
private int numRetries;
|
||||
private HMasterInterface master;
|
||||
private final Configuration conf;
|
||||
|
||||
private class TableInfo {
|
||||
public HRegionInfo regionInfo;
|
||||
|
@ -72,16 +82,11 @@ public class HClient extends HGlobals implements HConstants {
|
|||
public HClient(Configuration conf) {
|
||||
this.closed = false;
|
||||
this.conf = conf;
|
||||
|
||||
// Load config settings
|
||||
|
||||
this.masterLocation = new HServerAddress(this.conf.get(MASTER_DEFAULT_NAME));
|
||||
this.clientTimeout = this.conf.getLong("hbase.client.timeout.length", 10 * 1000);
|
||||
this.numTimeouts = this.conf.getInt("hbase.client.timeout.number", 5);
|
||||
this.numRetries = this.conf.getInt("hbase.client.retries.number", 2);
|
||||
|
||||
// Finish initialization
|
||||
|
||||
this.clientTimeout = conf.getLong("hbase.client.timeout.length", 10 * 1000);
|
||||
this.numTimeouts = conf.getInt("hbase.client.timeout.number", 5);
|
||||
this.numRetries = conf.getInt("hbase.client.retries.number", 2);
|
||||
|
||||
this.master = null;
|
||||
this.tablesToServers = new TreeMap<Text, TreeMap<Text, TableInfo>>();
|
||||
this.tableServers = null;
|
||||
|
@ -94,13 +99,33 @@ public class HClient extends HGlobals implements HConstants {
|
|||
this.rand = new Random();
|
||||
}
|
||||
|
||||
public synchronized void createTable(HTableDescriptor desc) throws IOException {
|
||||
if(closed) {
|
||||
throw new IllegalStateException("client is not open");
|
||||
}
|
||||
if(master == null) {
|
||||
locateRootRegion();
|
||||
}
|
||||
master.createTable(desc);
|
||||
}
|
||||
|
||||
public synchronized void deleteTable(Text tableName) throws IOException {
|
||||
if(closed) {
|
||||
throw new IllegalStateException("client is not open");
|
||||
}
|
||||
if(master == null) {
|
||||
locateRootRegion();
|
||||
}
|
||||
master.deleteTable(tableName);
|
||||
}
|
||||
|
||||
public synchronized void openTable(Text tableName) throws IOException {
|
||||
if (closed) {
|
||||
if(closed) {
|
||||
throw new IllegalStateException("client is not open");
|
||||
}
|
||||
|
||||
tableServers = tablesToServers.get(tableName);
|
||||
if (tableServers == null) { // We don't know where the table is
|
||||
if(tableServers == null ) { // We don't know where the table is
|
||||
findTableInMeta(tableName); // Load the information from meta
|
||||
}
|
||||
}
|
||||
|
@ -108,9 +133,9 @@ public class HClient extends HGlobals implements HConstants {
|
|||
private void findTableInMeta(Text tableName) throws IOException {
|
||||
TreeMap<Text, TableInfo> metaServers = tablesToServers.get(META_TABLE_NAME);
|
||||
|
||||
if (metaServers == null) { // Don't know where the meta is
|
||||
if(metaServers == null) { // Don't know where the meta is
|
||||
loadMetaFromRoot(tableName);
|
||||
if (tableName.equals(META_TABLE_NAME) || tableName.equals(ROOT_TABLE_NAME)) {
|
||||
if(tableName.equals(META_TABLE_NAME) || tableName.equals(ROOT_TABLE_NAME)) {
|
||||
// All we really wanted was the meta or root table
|
||||
return;
|
||||
}
|
||||
|
@ -119,7 +144,7 @@ public class HClient extends HGlobals implements HConstants {
|
|||
|
||||
tableServers = new TreeMap<Text, TableInfo>();
|
||||
for(Iterator<TableInfo> i = metaServers.tailMap(tableName).values().iterator();
|
||||
i.hasNext();) {
|
||||
i.hasNext(); ) {
|
||||
|
||||
TableInfo t = i.next();
|
||||
|
||||
|
@ -133,7 +158,7 @@ public class HClient extends HGlobals implements HConstants {
|
|||
*/
|
||||
private void loadMetaFromRoot(Text tableName) throws IOException {
|
||||
locateRootRegion();
|
||||
if (tableName.equals(ROOT_TABLE_NAME)) { // All we really wanted was the root
|
||||
if(tableName.equals(ROOT_TABLE_NAME)) { // All we really wanted was the root
|
||||
return;
|
||||
}
|
||||
scanRoot();
|
||||
|
@ -144,10 +169,12 @@ public class HClient extends HGlobals implements HConstants {
|
|||
* could be.
|
||||
*/
|
||||
private void locateRootRegion() throws IOException {
|
||||
if (master == null) {
|
||||
if(master == null) {
|
||||
HServerAddress masterLocation =
|
||||
new HServerAddress(this.conf.get(MASTER_ADDRESS));
|
||||
master = (HMasterInterface)RPC.getProxy(HMasterInterface.class,
|
||||
HMasterInterface.versionID,
|
||||
masterLocation.getInetSocketAddress(), conf);
|
||||
HMasterInterface.versionID,
|
||||
masterLocation.getInetSocketAddress(), conf);
|
||||
}
|
||||
|
||||
int tries = 0;
|
||||
|
@ -157,16 +184,15 @@ public class HClient extends HGlobals implements HConstants {
|
|||
while(rootRegionLocation == null && localTimeouts < numTimeouts) {
|
||||
rootRegionLocation = master.findRootRegion();
|
||||
|
||||
if (rootRegionLocation == null) {
|
||||
if(rootRegionLocation == null) {
|
||||
try {
|
||||
Thread.sleep(clientTimeout);
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
localTimeouts++;
|
||||
}
|
||||
}
|
||||
if (rootRegionLocation == null) {
|
||||
if(rootRegionLocation == null) {
|
||||
throw new IOException("Timed out trying to locate root region");
|
||||
}
|
||||
|
||||
|
@ -174,9 +200,9 @@ public class HClient extends HGlobals implements HConstants {
|
|||
|
||||
HRegionInterface rootRegion = getHRegionConnection(rootRegionLocation);
|
||||
|
||||
if (rootRegion.getRegionInfo(rootRegionInfo.regionName) != null) {
|
||||
if(rootRegion.getRegionInfo(HGlobals.rootRegionInfo.regionName) != null) {
|
||||
tableServers = new TreeMap<Text, TableInfo>();
|
||||
tableServers.put(startRow, new TableInfo(rootRegionInfo, rootRegionLocation));
|
||||
tableServers.put(startRow, new TableInfo(HGlobals.rootRegionInfo, rootRegionLocation));
|
||||
tablesToServers.put(ROOT_TABLE_NAME, tableServers);
|
||||
break;
|
||||
}
|
||||
|
@ -184,7 +210,7 @@ public class HClient extends HGlobals implements HConstants {
|
|||
|
||||
} while(rootRegionLocation == null && tries++ < numRetries);
|
||||
|
||||
if (rootRegionLocation == null) {
|
||||
if(rootRegionLocation == null) {
|
||||
closed = true;
|
||||
throw new IOException("unable to locate root region server");
|
||||
}
|
||||
|
@ -202,53 +228,64 @@ public class HClient extends HGlobals implements HConstants {
|
|||
|
||||
/*
|
||||
* Scans a single meta region
|
||||
* @param t - the table we're going to scan
|
||||
* @param tableName - the name of the table we're looking for
|
||||
* @param t the table we're going to scan
|
||||
* @param tableName the name of the table we're looking for
|
||||
*/
|
||||
private void scanOneMetaRegion(TableInfo t, Text tableName) throws IOException {
|
||||
HRegionInterface server = getHRegionConnection(t.serverAddress);
|
||||
HScannerInterface scanner = null;
|
||||
long scannerId = -1L;
|
||||
try {
|
||||
scanner = server.openScanner(t.regionInfo.regionName, metaColumns, tableName);
|
||||
HStoreKey key = new HStoreKey();
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
DataInputBuffer inbuf = new DataInputBuffer();
|
||||
scannerId = server.openScanner(t.regionInfo.regionName, metaColumns, tableName);
|
||||
|
||||
while(scanner.next(key, results)) {
|
||||
byte hRegionInfoBytes[] = results.get(META_COL_REGIONINFO);
|
||||
inbuf.reset(hRegionInfoBytes, hRegionInfoBytes.length);
|
||||
DataInputBuffer inbuf = new DataInputBuffer();
|
||||
while(true) {
|
||||
HStoreKey key = new HStoreKey();
|
||||
|
||||
LabelledData[] values = server.next(scannerId, key);
|
||||
if(values.length == 0) {
|
||||
break;
|
||||
}
|
||||
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
for(int i = 0; i < values.length; i++) {
|
||||
results.put(values[i].getLabel(), values[i].getData().get());
|
||||
}
|
||||
HRegionInfo regionInfo = new HRegionInfo();
|
||||
byte[] bytes = results.get(META_COL_REGIONINFO);
|
||||
inbuf.reset(bytes, bytes.length);
|
||||
regionInfo.readFields(inbuf);
|
||||
|
||||
if (!regionInfo.tableDesc.getName().equals(tableName)) {
|
||||
|
||||
if(!regionInfo.tableDesc.getName().equals(tableName)) {
|
||||
// We're done
|
||||
break;
|
||||
}
|
||||
|
||||
byte serverBytes[] = results.get(META_COL_SERVER);
|
||||
String serverName = new String(serverBytes, UTF8_ENCODING);
|
||||
|
||||
bytes = results.get(META_COL_SERVER);
|
||||
String serverName = new String(bytes, UTF8_ENCODING);
|
||||
|
||||
tableServers.put(regionInfo.startKey,
|
||||
new TableInfo(regionInfo, new HServerAddress(serverName)));
|
||||
new TableInfo(regionInfo, new HServerAddress(serverName)));
|
||||
|
||||
results.clear();
|
||||
}
|
||||
|
||||
} finally {
|
||||
scanner.close();
|
||||
if(scannerId != -1L) {
|
||||
server.close(scannerId);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized HRegionInterface getHRegionConnection(HServerAddress regionServer)
|
||||
throws IOException {
|
||||
synchronized HRegionInterface getHRegionConnection(HServerAddress regionServer)
|
||||
throws IOException {
|
||||
|
||||
// See if we already have a connection
|
||||
// See if we already have a connection
|
||||
|
||||
HRegionInterface server = servers.get(regionServer.toString());
|
||||
|
||||
if (server == null) { // Get a connection
|
||||
if(server == null) { // Get a connection
|
||||
|
||||
server = (HRegionInterface)RPC.waitForProxy(HRegionInterface.class,
|
||||
HRegionInterface.versionID, regionServer.getInetSocketAddress(), conf);
|
||||
HRegionInterface.versionID, regionServer.getInetSocketAddress(), conf);
|
||||
|
||||
servers.put(regionServer.toString(), server);
|
||||
}
|
||||
|
@ -257,7 +294,7 @@ public class HClient extends HGlobals implements HConstants {
|
|||
|
||||
/** Close the connection to the HRegionServer */
|
||||
public synchronized void close() throws IOException {
|
||||
if (!closed) {
|
||||
if(! closed) {
|
||||
RPC.stopClient();
|
||||
closed = true;
|
||||
}
|
||||
|
@ -270,48 +307,58 @@ public class HClient extends HGlobals implements HConstants {
|
|||
* catalog table that just contains table names and their descriptors.
|
||||
* Right now, it only exists as part of the META table's region info.
|
||||
*/
|
||||
public HTableDescriptor[] listTables() throws IOException {
|
||||
public synchronized HTableDescriptor[] listTables() throws IOException {
|
||||
TreeSet<HTableDescriptor> uniqueTables = new TreeSet<HTableDescriptor>();
|
||||
|
||||
TreeMap<Text, TableInfo> metaTables = tablesToServers.get(META_TABLE_NAME);
|
||||
if (metaTables == null) {
|
||||
if(metaTables == null) {
|
||||
// Meta is not loaded yet so go do that
|
||||
loadMetaFromRoot(META_TABLE_NAME);
|
||||
metaTables = tablesToServers.get(META_TABLE_NAME);
|
||||
}
|
||||
|
||||
for(Iterator<TableInfo>i = metaTables.values().iterator(); i.hasNext();) {
|
||||
TableInfo t = i.next();
|
||||
for(Iterator<TableInfo>it = metaTables.values().iterator(); it.hasNext(); ) {
|
||||
TableInfo t = it.next();
|
||||
HRegionInterface server = getHRegionConnection(t.serverAddress);
|
||||
HScannerInterface scanner = null;
|
||||
long scannerId = -1L;
|
||||
try {
|
||||
scanner = server.openScanner(t.regionInfo.regionName, metaColumns, startRow);
|
||||
scannerId = server.openScanner(t.regionInfo.regionName, metaColumns, startRow);
|
||||
HStoreKey key = new HStoreKey();
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
|
||||
DataInputBuffer inbuf = new DataInputBuffer();
|
||||
while(scanner.next(key, results)) {
|
||||
byte infoBytes[] = (byte[]) results.get(ROOT_COL_REGIONINFO);
|
||||
inbuf.reset(infoBytes, infoBytes.length);
|
||||
HRegionInfo info = new HRegionInfo();
|
||||
info.readFields(inbuf);
|
||||
|
||||
// Only examine the rows where the startKey is zero length
|
||||
|
||||
if (info.startKey.getLength() == 0) {
|
||||
uniqueTables.add(info.tableDesc);
|
||||
while(true) {
|
||||
LabelledData[] values = server.next(scannerId, key);
|
||||
if(values.length == 0) {
|
||||
break;
|
||||
}
|
||||
|
||||
for(int i = 0; i < values.length; i++) {
|
||||
if(values[i].getLabel().equals(META_COL_REGIONINFO)) {
|
||||
byte[] bytes = values[i].getData().get();
|
||||
inbuf.reset(bytes, bytes.length);
|
||||
HRegionInfo info = new HRegionInfo();
|
||||
info.readFields(inbuf);
|
||||
|
||||
// Only examine the rows where the startKey is zero length
|
||||
|
||||
if(info.startKey.getLength() == 0) {
|
||||
uniqueTables.add(info.tableDesc);
|
||||
}
|
||||
}
|
||||
}
|
||||
results.clear();
|
||||
}
|
||||
|
||||
} finally {
|
||||
scanner.close();
|
||||
if(scannerId != -1L) {
|
||||
server.close(scannerId);
|
||||
}
|
||||
}
|
||||
}
|
||||
return (HTableDescriptor[]) uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
|
||||
}
|
||||
|
||||
private TableInfo getTableInfo(Text row) {
|
||||
if (tableServers == null) {
|
||||
private synchronized TableInfo getTableInfo(Text row) {
|
||||
if(tableServers == null) {
|
||||
throw new IllegalStateException("Must open table first");
|
||||
}
|
||||
|
||||
|
@ -325,17 +372,17 @@ public class HClient extends HGlobals implements HConstants {
|
|||
public byte[] get(Text row, Text column) throws IOException {
|
||||
TableInfo info = getTableInfo(row);
|
||||
return getHRegionConnection(info.serverAddress).get(
|
||||
info.regionInfo.regionName, row, column).get();
|
||||
info.regionInfo.regionName, row, column).get();
|
||||
}
|
||||
|
||||
/** Get the specified number of versions of the specified row and column */
|
||||
public byte[][] get(Text row, Text column, int numVersions) throws IOException {
|
||||
TableInfo info = getTableInfo(row);
|
||||
BytesWritable[] values = getHRegionConnection(info.serverAddress).get(
|
||||
info.regionInfo.regionName, row, column, numVersions);
|
||||
info.regionInfo.regionName, row, column, numVersions);
|
||||
|
||||
ArrayList<byte[]> bytes = new ArrayList<byte[]>();
|
||||
for(int i = 0; i < values.length; i++) {
|
||||
for(int i = 0 ; i < values.length; i++) {
|
||||
bytes.add(values[i].get());
|
||||
}
|
||||
return bytes.toArray(new byte[values.length][]);
|
||||
|
@ -348,10 +395,10 @@ public class HClient extends HGlobals implements HConstants {
|
|||
public byte[][] get(Text row, Text column, long timestamp, int numVersions) throws IOException {
|
||||
TableInfo info = getTableInfo(row);
|
||||
BytesWritable[] values = getHRegionConnection(info.serverAddress).get(
|
||||
info.regionInfo.regionName, row, column, timestamp, numVersions);
|
||||
info.regionInfo.regionName, row, column, timestamp, numVersions);
|
||||
|
||||
ArrayList<byte[]> bytes = new ArrayList<byte[]>();
|
||||
for(int i = 0; i < values.length; i++) {
|
||||
for(int i = 0 ; i < values.length; i++) {
|
||||
bytes.add(values[i].get());
|
||||
}
|
||||
return bytes.toArray(new byte[values.length][]);
|
||||
|
@ -361,15 +408,15 @@ public class HClient extends HGlobals implements HConstants {
|
|||
public LabelledData[] getRow(Text row) throws IOException {
|
||||
TableInfo info = getTableInfo(row);
|
||||
return getHRegionConnection(info.serverAddress).getRow(
|
||||
info.regionInfo.regionName, row);
|
||||
info.regionInfo.regionName, row);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a scanner on the current table starting at the specified row.
|
||||
* Return the specified columns.
|
||||
*/
|
||||
public HScannerInterface obtainScanner(Text[] columns, Text startRow) throws IOException {
|
||||
if (tableServers == null) {
|
||||
public synchronized HScannerInterface obtainScanner(Text[] columns, Text startRow) throws IOException {
|
||||
if(tableServers == null) {
|
||||
throw new IllegalStateException("Must open table first");
|
||||
}
|
||||
return new ClientScanner(columns, startRow);
|
||||
|
@ -462,7 +509,7 @@ public class HClient extends HGlobals implements HConstants {
|
|||
private TableInfo[] regions;
|
||||
private int currentRegion;
|
||||
private HRegionInterface server;
|
||||
private HScannerInterface scanner;
|
||||
private long scannerId;
|
||||
|
||||
public ClientScanner(Text[] columns, Text startRow) throws IOException {
|
||||
this.columns = columns;
|
||||
|
@ -472,7 +519,7 @@ public class HClient extends HGlobals implements HConstants {
|
|||
this.regions = info.toArray(new TableInfo[info.size()]);
|
||||
this.currentRegion = -1;
|
||||
this.server = null;
|
||||
this.scanner = null;
|
||||
this.scannerId = -1L;
|
||||
nextScanner();
|
||||
}
|
||||
|
||||
|
@ -481,18 +528,19 @@ public class HClient extends HGlobals implements HConstants {
|
|||
* Returns false if there are no more scanners.
|
||||
*/
|
||||
private boolean nextScanner() throws IOException {
|
||||
if (scanner != null) {
|
||||
scanner.close();
|
||||
if(scannerId != -1L) {
|
||||
server.close(scannerId);
|
||||
scannerId = -1L;
|
||||
}
|
||||
currentRegion += 1;
|
||||
if (currentRegion == regions.length) {
|
||||
if(currentRegion == regions.length) {
|
||||
close();
|
||||
return false;
|
||||
}
|
||||
try {
|
||||
server = getHRegionConnection(regions[currentRegion].serverAddress);
|
||||
scanner = server.openScanner(regions[currentRegion].regionInfo.regionName,
|
||||
columns, startRow);
|
||||
scannerId = server.openScanner(regions[currentRegion].regionInfo.regionName,
|
||||
columns, startRow);
|
||||
|
||||
} catch(IOException e) {
|
||||
close();
|
||||
|
@ -505,29 +553,66 @@ public class HClient extends HGlobals implements HConstants {
|
|||
* @see org.apache.hadoop.hbase.HScannerInterface#next(org.apache.hadoop.hbase.HStoreKey, java.util.TreeMap)
|
||||
*/
|
||||
public boolean next(HStoreKey key, TreeMap<Text, byte[]> results) throws IOException {
|
||||
if (closed) {
|
||||
if(closed) {
|
||||
return false;
|
||||
}
|
||||
boolean status = scanner.next(key, results);
|
||||
if (!status) {
|
||||
status = nextScanner();
|
||||
if (status) {
|
||||
status = scanner.next(key, results);
|
||||
}
|
||||
LabelledData[] values = null;
|
||||
do {
|
||||
values = server.next(scannerId, key);
|
||||
} while(values.length == 0 && nextScanner());
|
||||
|
||||
for(int i = 0; i < values.length; i++) {
|
||||
results.put(values[i].getLabel(), values[i].getData().get());
|
||||
}
|
||||
return status;
|
||||
return values.length != 0;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.hadoop.hbase.HScannerInterface#close()
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
if (scanner != null) {
|
||||
scanner.close();
|
||||
if(scannerId != -1L) {
|
||||
server.close(scannerId);
|
||||
}
|
||||
server = null;
|
||||
closed = true;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void printUsage() {
|
||||
System.err.println("Usage: java " + this.getClass().getName() +
|
||||
" [--master=hostname:port]");
|
||||
}
|
||||
|
||||
private int doCommandLine(final String args[]) {
|
||||
// Process command-line args. TODO: Better cmd-line processing
|
||||
// (but hopefully something not as painful as cli options).
|
||||
for (String cmd: args) {
|
||||
if (cmd.equals("-h") || cmd.startsWith("--h")) {
|
||||
printUsage();
|
||||
return 0;
|
||||
}
|
||||
|
||||
final String masterArgKey = "--master=";
|
||||
if (cmd.startsWith(masterArgKey)) {
|
||||
this.conf.set(MASTER_ADDRESS,
|
||||
cmd.substring(masterArgKey.length()));
|
||||
}
|
||||
}
|
||||
|
||||
int errCode = -1;
|
||||
try {
|
||||
locateRootRegion();
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
return errCode;
|
||||
}
|
||||
|
||||
public static void main(final String args[]) {
|
||||
Configuration c = new HBaseConfiguration();
|
||||
int errCode = (new HClient(c)).doCommandLine(args);
|
||||
System.exit(errCode);
|
||||
}
|
||||
}
|
|
@ -24,7 +24,18 @@ public interface HConstants {
|
|||
|
||||
// Configuration parameters
|
||||
|
||||
static final String MASTER_DEFAULT_NAME = "hbase.master.default.name";
|
||||
// TODO: URL for hbase master, like hdfs URLs with host and port.
|
||||
// Or, like jdbc URLs:
|
||||
// jdbc:mysql://[host][,failoverhost...][:port]/[database]
|
||||
// jdbc:mysql://[host][,failoverhost...][:port]/[database][?propertyName1][=propertyValue1][&propertyName2][=propertyValue2]...
|
||||
|
||||
static final String MASTER_ADDRESS = "hbase.master";
|
||||
// TODO: Support 'local': i.e. default of all running in single
|
||||
// process. Same for regionserver.
|
||||
static final String DEFAULT_MASTER_ADDRESS = "localhost:60000";
|
||||
static final String REGIONSERVER_ADDRESS = "hbase.regionserver";
|
||||
static final String DEFAULT_REGIONSERVER_ADDRESS =
|
||||
"localhost:60010";
|
||||
static final String HREGION_DIR = "hbase.regiondir";
|
||||
static final String DEFAULT_HREGION_DIR = "/hbase";
|
||||
static final String HREGIONDIR_PREFIX = "hregion_";
|
||||
|
@ -37,10 +48,10 @@ public interface HConstants {
|
|||
// Do we ever need to know all the information that we are storing?
|
||||
|
||||
static final Text ROOT_TABLE_NAME = new Text("--ROOT--");
|
||||
static final Text ROOT_COLUMN_FAMILY = new Text("info");
|
||||
static final Text ROOT_COL_REGIONINFO = new Text(ROOT_COLUMN_FAMILY + ":" + "regioninfo");
|
||||
static final Text ROOT_COL_SERVER = new Text(ROOT_COLUMN_FAMILY + ":" + "server");
|
||||
static final Text ROOT_COL_STARTCODE = new Text(ROOT_COLUMN_FAMILY + ":" + "serverstartcode");
|
||||
static final Text ROOT_COLUMN_FAMILY = new Text("info:");
|
||||
static final Text ROOT_COL_REGIONINFO = new Text(ROOT_COLUMN_FAMILY + "regioninfo");
|
||||
static final Text ROOT_COL_SERVER = new Text(ROOT_COLUMN_FAMILY + "server");
|
||||
static final Text ROOT_COL_STARTCODE = new Text(ROOT_COLUMN_FAMILY + "serverstartcode");
|
||||
|
||||
static final Text META_TABLE_NAME = new Text("--META--");
|
||||
static final Text META_COLUMN_FAMILY = new Text(ROOT_COLUMN_FAMILY);
|
||||
|
|
|
@ -1,90 +0,0 @@
|
|||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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;
|
||||
|
||||
/*******************************************************************************
|
||||
* HLocking is a set of lock primitives that are pretty helpful in a few places
|
||||
* around the HBase code. For each independent entity that needs locking, create
|
||||
* a new HLocking instance.
|
||||
******************************************************************************/
|
||||
public class HLocking {
|
||||
Integer readerLock = new Integer(0);
|
||||
Integer writerLock = new Integer(0);
|
||||
int numReaders = 0;
|
||||
int numWriters = 0;
|
||||
|
||||
public HLocking() {
|
||||
}
|
||||
|
||||
/** Caller needs the nonexclusive read-lock */
|
||||
public void obtainReadLock() {
|
||||
synchronized(readerLock) {
|
||||
synchronized(writerLock) {
|
||||
while(numWriters > 0) {
|
||||
try {
|
||||
writerLock.wait();
|
||||
} catch (InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
numReaders++;
|
||||
readerLock.notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Caller is finished with the nonexclusive read-lock */
|
||||
public void releaseReadLock() {
|
||||
synchronized(readerLock) {
|
||||
synchronized(writerLock) {
|
||||
numReaders--;
|
||||
readerLock.notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Caller needs the exclusive write-lock */
|
||||
public void obtainWriteLock() {
|
||||
synchronized(readerLock) {
|
||||
synchronized(writerLock) {
|
||||
while(numReaders > 0) {
|
||||
try {
|
||||
readerLock.wait();
|
||||
} catch (InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
while(numWriters > 0) {
|
||||
try {
|
||||
writerLock.wait();
|
||||
} catch (InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
numWriters++;
|
||||
writerLock.notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Caller is finished with the write lock */
|
||||
public void releaseWriteLock() {
|
||||
synchronized(readerLock) {
|
||||
synchronized(writerLock) {
|
||||
numWriters--;
|
||||
writerLock.notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -101,12 +101,12 @@ public class HLog {
|
|||
newlog.close();
|
||||
}
|
||||
|
||||
if (fs.exists(srcDir)) {
|
||||
if(fs.exists(srcDir)) {
|
||||
|
||||
if (!fs.delete(srcDir)) {
|
||||
if(! fs.delete(srcDir)) {
|
||||
LOG.error("Cannot delete: " + srcDir);
|
||||
|
||||
if (!FileUtil.fullyDelete(new File(srcDir.toString()))) {
|
||||
if(! FileUtil.fullyDelete(new File(srcDir.toString()))) {
|
||||
throw new IOException("Cannot delete: " + srcDir);
|
||||
}
|
||||
}
|
||||
|
@ -127,7 +127,7 @@ public class HLog {
|
|||
this.conf = conf;
|
||||
this.logSeqNum = 0;
|
||||
|
||||
if (fs.exists(dir)) {
|
||||
if(fs.exists(dir)) {
|
||||
throw new IOException("Target HLog directory already exists: " + dir);
|
||||
}
|
||||
fs.mkdirs(dir);
|
||||
|
@ -154,7 +154,7 @@ public class HLog {
|
|||
|
||||
Vector<Path> toDeleteList = new Vector<Path>();
|
||||
synchronized(this) {
|
||||
if (closed) {
|
||||
if(closed) {
|
||||
throw new IOException("Cannot roll log; log is closed");
|
||||
}
|
||||
|
||||
|
@ -174,10 +174,10 @@ public class HLog {
|
|||
|
||||
// Close the current writer (if any), and grab a new one.
|
||||
|
||||
if (writer != null) {
|
||||
if(writer != null) {
|
||||
writer.close();
|
||||
|
||||
if (filenum > 0) {
|
||||
if(filenum > 0) {
|
||||
outputfiles.put(logSeqNum-1, computeFilename(filenum-1));
|
||||
}
|
||||
}
|
||||
|
@ -192,10 +192,10 @@ public class HLog {
|
|||
// over all the regions.
|
||||
|
||||
long oldestOutstandingSeqNum = Long.MAX_VALUE;
|
||||
for(Iterator<Long> it = regionToLastFlush.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<Long> it = regionToLastFlush.values().iterator(); it.hasNext(); ) {
|
||||
long curSeqNum = it.next().longValue();
|
||||
|
||||
if (curSeqNum < oldestOutstandingSeqNum) {
|
||||
if(curSeqNum < oldestOutstandingSeqNum) {
|
||||
oldestOutstandingSeqNum = curSeqNum;
|
||||
}
|
||||
}
|
||||
|
@ -205,10 +205,10 @@ public class HLog {
|
|||
|
||||
LOG.debug("removing old log files");
|
||||
|
||||
for(Iterator<Long> it = outputfiles.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Long> it = outputfiles.keySet().iterator(); it.hasNext(); ) {
|
||||
long maxSeqNum = it.next().longValue();
|
||||
|
||||
if (maxSeqNum < oldestOutstandingSeqNum) {
|
||||
if(maxSeqNum < oldestOutstandingSeqNum) {
|
||||
Path p = outputfiles.get(maxSeqNum);
|
||||
it.remove();
|
||||
toDeleteList.add(p);
|
||||
|
@ -221,7 +221,7 @@ public class HLog {
|
|||
|
||||
// Actually delete them, if any!
|
||||
|
||||
for(Iterator<Path> it = toDeleteList.iterator(); it.hasNext();) {
|
||||
for(Iterator<Path> it = toDeleteList.iterator(); it.hasNext(); ) {
|
||||
Path p = it.next();
|
||||
fs.delete(p);
|
||||
}
|
||||
|
@ -262,7 +262,7 @@ public class HLog {
|
|||
* We need to seize a lock on the writer so that writes are atomic.
|
||||
*/
|
||||
public synchronized void append(Text regionName, Text tableName, Text row, TreeMap<Text, byte[]> columns, long timestamp) throws IOException {
|
||||
if (closed) {
|
||||
if(closed) {
|
||||
throw new IOException("Cannot append; log is closed");
|
||||
}
|
||||
|
||||
|
@ -273,12 +273,12 @@ public class HLog {
|
|||
// that don't have any flush yet, the relevant operation is the
|
||||
// first one that's been added.
|
||||
|
||||
if (regionToLastFlush.get(regionName) == null) {
|
||||
if(regionToLastFlush.get(regionName) == null) {
|
||||
regionToLastFlush.put(regionName, seqNum[0]);
|
||||
}
|
||||
|
||||
int counter = 0;
|
||||
for(Iterator<Text> it = columns.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = columns.keySet().iterator(); it.hasNext(); ) {
|
||||
Text column = it.next();
|
||||
byte[] val = columns.get(column);
|
||||
HLogKey logKey = new HLogKey(regionName, tableName, row, seqNum[counter++]);
|
||||
|
@ -333,16 +333,16 @@ public class HLog {
|
|||
|
||||
/** Complete the cache flush */
|
||||
public synchronized void completeCacheFlush(Text regionName, Text tableName, long logSeqId) throws IOException {
|
||||
if (closed) {
|
||||
if(closed) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (!insideCacheFlush) {
|
||||
if(! insideCacheFlush) {
|
||||
throw new IOException("Impossible situation: inside completeCacheFlush(), but 'insideCacheFlush' flag is false");
|
||||
}
|
||||
|
||||
writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId),
|
||||
new HLogEdit(HLog.METACOLUMN, HStoreKey.COMPLETE_CACHEFLUSH, System.currentTimeMillis()));
|
||||
new HLogEdit(HLog.METACOLUMN, HStoreKey.COMPLETE_CACHEFLUSH, System.currentTimeMillis()));
|
||||
numEntries++;
|
||||
|
||||
// Remember the most-recent flush for each region.
|
||||
|
@ -353,4 +353,4 @@ public class HLog {
|
|||
insideCacheFlush = false;
|
||||
notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -67,5 +67,4 @@ public class HLogEdit implements Writable {
|
|||
this.val.readFields(in);
|
||||
this.timestamp = in.readLong();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -80,10 +80,10 @@ public class HLogKey implements WritableComparable {
|
|||
HLogKey other = (HLogKey) o;
|
||||
int result = this.regionName.compareTo(other.regionName);
|
||||
|
||||
if (result == 0) {
|
||||
if(result == 0) {
|
||||
result = this.row.compareTo(other.row);
|
||||
|
||||
if (result == 0) {
|
||||
if(result == 0) {
|
||||
|
||||
if (this.logSeqNum < other.logSeqNum) {
|
||||
result = -1;
|
||||
|
@ -113,5 +113,4 @@ public class HLogKey implements WritableComparable {
|
|||
this.row.readFields(in);
|
||||
this.logSeqNum = in.readLong();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -15,7 +15,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.io.*;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ipc.VersionedProtocol;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -23,7 +24,7 @@ import java.io.IOException;
|
|||
* Clients interact with the HMasterInterface to gain access to meta-level HBase
|
||||
* functionality, like finding an HRegionServer and creating/destroying tables.
|
||||
******************************************************************************/
|
||||
public interface HMasterInterface {
|
||||
public interface HMasterInterface extends VersionedProtocol {
|
||||
public static final long versionID = 1L; // initial version
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
|
|
@ -15,13 +15,15 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.*;
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.ipc.VersionedProtocol;
|
||||
|
||||
/*******************************************************************************
|
||||
* HRegionServers interact with the HMasterRegionInterface to report on local
|
||||
* goings-on and to obtain data-handling instructions from the HMaster.
|
||||
*********************************************/
|
||||
public interface HMasterRegionInterface {
|
||||
public static final long versionId = 1L;
|
||||
public interface HMasterRegionInterface extends VersionedProtocol {
|
||||
public static final long versionID = 1L;
|
||||
public void regionServerStartup(HServerInfo info) throws IOException;
|
||||
public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[]) throws IOException;
|
||||
}
|
||||
|
|
|
@ -22,6 +22,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
/*******************************************************************************
|
||||
* The HMemcache holds in-memory modifications to the HRegion. This is really a
|
||||
|
@ -31,14 +33,14 @@ public class HMemcache {
|
|||
private static final Log LOG = LogFactory.getLog(HMemcache.class);
|
||||
|
||||
TreeMap<HStoreKey, BytesWritable> memcache
|
||||
= new TreeMap<HStoreKey, BytesWritable>();
|
||||
= new TreeMap<HStoreKey, BytesWritable>();
|
||||
|
||||
Vector<TreeMap<HStoreKey, BytesWritable>> history
|
||||
= new Vector<TreeMap<HStoreKey, BytesWritable>>();
|
||||
= new Vector<TreeMap<HStoreKey, BytesWritable>>();
|
||||
|
||||
TreeMap<HStoreKey, BytesWritable> snapshot = null;
|
||||
|
||||
HLocking locking = new HLocking();
|
||||
ReadWriteLock locker = new ReentrantReadWriteLock();
|
||||
|
||||
public HMemcache() {
|
||||
}
|
||||
|
@ -52,23 +54,27 @@ public class HMemcache {
|
|||
}
|
||||
|
||||
/**
|
||||
* We want to return a snapshot of the current HMemcache with a known HLog
|
||||
* Returns a snapshot of the current HMemcache with a known HLog
|
||||
* sequence number at the same time.
|
||||
*
|
||||
* Return both the frozen HMemcache TreeMap, as well as the HLog seq number.
|
||||
*
|
||||
* We need to prevent any writing to the cache during this time, so we obtain
|
||||
* a write lock for the duration of the operation.
|
||||
* We need to prevent any writing to the cache during this time,
|
||||
* so we obtain a write lock for the duration of the operation.
|
||||
*
|
||||
* <p>If this method returns non-null, client must call
|
||||
* {@link #deleteSnapshot()} to clear 'snapshot-in-progress'
|
||||
* state when finished with the returned {@link Snapshot}.
|
||||
*
|
||||
* @return frozen HMemcache TreeMap and HLog sequence number.
|
||||
*/
|
||||
public Snapshot snapshotMemcacheForLog(HLog log) throws IOException {
|
||||
Snapshot retval = new Snapshot();
|
||||
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
if (snapshot != null) {
|
||||
if(snapshot != null) {
|
||||
throw new IOException("Snapshot in progress!");
|
||||
}
|
||||
if (memcache.size() == 0) {
|
||||
if(memcache.size() == 0) {
|
||||
LOG.debug("memcache empty. Skipping snapshot");
|
||||
return retval;
|
||||
}
|
||||
|
@ -86,7 +92,7 @@ public class HMemcache {
|
|||
return retval;
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -96,19 +102,19 @@ public class HMemcache {
|
|||
* Modifying the structure means we need to obtain a writelock.
|
||||
*/
|
||||
public void deleteSnapshot() throws IOException {
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
|
||||
try {
|
||||
if (snapshot == null) {
|
||||
if(snapshot == null) {
|
||||
throw new IOException("Snapshot not present!");
|
||||
}
|
||||
LOG.debug("deleting snapshot");
|
||||
|
||||
for(Iterator<TreeMap<HStoreKey, BytesWritable>> it = history.iterator();
|
||||
it.hasNext();) {
|
||||
it.hasNext(); ) {
|
||||
|
||||
TreeMap<HStoreKey, BytesWritable> cur = it.next();
|
||||
if (snapshot == cur) {
|
||||
if(snapshot == cur) {
|
||||
it.remove();
|
||||
break;
|
||||
}
|
||||
|
@ -118,7 +124,7 @@ public class HMemcache {
|
|||
LOG.debug("snapshot deleted");
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -128,9 +134,9 @@ public class HMemcache {
|
|||
* Operation uses a write lock.
|
||||
*/
|
||||
public void add(Text row, TreeMap<Text, byte[]> columns, long timestamp) {
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
for(Iterator<Text> it = columns.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = columns.keySet().iterator(); it.hasNext(); ) {
|
||||
Text column = it.next();
|
||||
byte[] val = columns.get(column);
|
||||
|
||||
|
@ -139,7 +145,7 @@ public class HMemcache {
|
|||
}
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -150,13 +156,13 @@ public class HMemcache {
|
|||
*/
|
||||
public byte[][] get(HStoreKey key, int numVersions) {
|
||||
Vector<byte[]> results = new Vector<byte[]>();
|
||||
locking.obtainReadLock();
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
Vector<byte[]> result = get(memcache, key, numVersions-results.size());
|
||||
results.addAll(0, result);
|
||||
|
||||
for(int i = history.size()-1; i >= 0; i--) {
|
||||
if (numVersions > 0 && results.size() >= numVersions) {
|
||||
if(numVersions > 0 && results.size() >= numVersions) {
|
||||
break;
|
||||
}
|
||||
|
||||
|
@ -164,7 +170,7 @@ public class HMemcache {
|
|||
results.addAll(results.size(), result);
|
||||
}
|
||||
|
||||
if (results.size() == 0) {
|
||||
if(results.size() == 0) {
|
||||
return null;
|
||||
|
||||
} else {
|
||||
|
@ -172,7 +178,7 @@ public class HMemcache {
|
|||
}
|
||||
|
||||
} finally {
|
||||
locking.releaseReadLock();
|
||||
this.locker.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -184,7 +190,7 @@ public class HMemcache {
|
|||
*/
|
||||
public TreeMap<Text, byte[]> getFull(HStoreKey key) throws IOException {
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
locking.obtainReadLock();
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
internalGetFull(memcache, key, results);
|
||||
for(int i = history.size()-1; i >= 0; i--) {
|
||||
|
@ -194,25 +200,25 @@ public class HMemcache {
|
|||
return results;
|
||||
|
||||
} finally {
|
||||
locking.releaseReadLock();
|
||||
this.locker.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
void internalGetFull(TreeMap<HStoreKey, BytesWritable> map, HStoreKey key,
|
||||
TreeMap<Text, byte[]> results) {
|
||||
TreeMap<Text, byte[]> results) {
|
||||
|
||||
SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(key);
|
||||
|
||||
for(Iterator<HStoreKey> it = tailMap.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreKey> it = tailMap.keySet().iterator(); it.hasNext(); ) {
|
||||
HStoreKey itKey = it.next();
|
||||
Text itCol = itKey.getColumn();
|
||||
|
||||
if (results.get(itCol) == null
|
||||
if(results.get(itCol) == null
|
||||
&& key.matchesWithoutColumn(itKey)) {
|
||||
BytesWritable val = tailMap.get(itKey);
|
||||
results.put(itCol, val.get());
|
||||
|
||||
} else if (key.getRow().compareTo(itKey.getRow()) > 0) {
|
||||
} else if(key.getRow().compareTo(itKey.getRow()) > 0) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -232,15 +238,15 @@ public class HMemcache {
|
|||
HStoreKey curKey = new HStoreKey(key.getRow(), key.getColumn(), key.getTimestamp());
|
||||
SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(curKey);
|
||||
|
||||
for(Iterator<HStoreKey> it = tailMap.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreKey> it = tailMap.keySet().iterator(); it.hasNext(); ) {
|
||||
HStoreKey itKey = it.next();
|
||||
|
||||
if (itKey.matchesRowCol(curKey)) {
|
||||
if(itKey.matchesRowCol(curKey)) {
|
||||
result.add(tailMap.get(itKey).get());
|
||||
curKey.setVersion(itKey.getTimestamp() - 1);
|
||||
}
|
||||
|
||||
if (numVersions > 0 && result.size() >= numVersions) {
|
||||
if(numVersions > 0 && result.size() >= numVersions) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -251,7 +257,7 @@ public class HMemcache {
|
|||
* Return a scanner over the keys in the HMemcache
|
||||
*/
|
||||
public HScannerInterface getScanner(long timestamp, Text targetCols[], Text firstRow)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
return new HMemcacheScanner(timestamp, targetCols, firstRow);
|
||||
}
|
||||
|
@ -267,16 +273,16 @@ public class HMemcache {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public HMemcacheScanner(long timestamp, Text targetCols[], Text firstRow)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
super(timestamp, targetCols);
|
||||
|
||||
locking.obtainReadLock();
|
||||
locker.readLock().lock();
|
||||
try {
|
||||
this.backingMaps = new TreeMap[history.size() + 1];
|
||||
int i = 0;
|
||||
for(Iterator<TreeMap<HStoreKey, BytesWritable>> it = history.iterator();
|
||||
it.hasNext();) {
|
||||
it.hasNext(); ) {
|
||||
|
||||
backingMaps[i++] = it.next();
|
||||
}
|
||||
|
@ -290,7 +296,7 @@ public class HMemcache {
|
|||
|
||||
HStoreKey firstKey = new HStoreKey(firstRow);
|
||||
for(i = 0; i < backingMaps.length; i++) {
|
||||
if (firstRow.getLength() != 0) {
|
||||
if(firstRow.getLength() != 0) {
|
||||
keyIterators[i] = backingMaps[i].tailMap(firstKey).keySet().iterator();
|
||||
|
||||
} else {
|
||||
|
@ -298,10 +304,10 @@ public class HMemcache {
|
|||
}
|
||||
|
||||
while(getNext(i)) {
|
||||
if (!findFirstRow(i, firstRow)) {
|
||||
if(! findFirstRow(i, firstRow)) {
|
||||
continue;
|
||||
}
|
||||
if (columnMatch(i)) {
|
||||
if(columnMatch(i)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -331,7 +337,7 @@ public class HMemcache {
|
|||
* @return - true if there is more data available
|
||||
*/
|
||||
boolean getNext(int i) {
|
||||
if (!keyIterators[i].hasNext()) {
|
||||
if(! keyIterators[i].hasNext()) {
|
||||
closeSubScanner(i);
|
||||
return false;
|
||||
}
|
||||
|
@ -350,16 +356,16 @@ public class HMemcache {
|
|||
|
||||
/** Shut down map iterators, and release the lock */
|
||||
public void close() throws IOException {
|
||||
if (!scannerClosed) {
|
||||
if(! scannerClosed) {
|
||||
try {
|
||||
for(int i = 0; i < keys.length; i++) {
|
||||
if (keyIterators[i] != null) {
|
||||
if(keyIterators[i] != null) {
|
||||
closeSubScanner(i);
|
||||
}
|
||||
}
|
||||
|
||||
} finally {
|
||||
locking.releaseReadLock();
|
||||
locker.readLock().unlock();
|
||||
scannerClosed = true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -67,13 +67,13 @@ public class HMsg implements Writable {
|
|||
// Writable
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeByte(msg);
|
||||
info.write(out);
|
||||
}
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeByte(msg);
|
||||
info.write(out);
|
||||
}
|
||||
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
this.msg = in.readByte();
|
||||
this.info.readFields(in);
|
||||
}
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
this.msg = in.readByte();
|
||||
this.info.readFields(in);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -61,21 +61,21 @@ public class HRegion implements HConstants {
|
|||
// Make sure that srcA comes first; important for key-ordering during
|
||||
// write of the merged file.
|
||||
|
||||
if (srcA.getStartKey() == null) {
|
||||
if (srcB.getStartKey() == null) {
|
||||
if(srcA.getStartKey() == null) {
|
||||
if(srcB.getStartKey() == null) {
|
||||
throw new IOException("Cannot merge two regions with null start key");
|
||||
}
|
||||
// A's start key is null but B's isn't. Assume A comes before B
|
||||
|
||||
} else if ((srcB.getStartKey() == null) // A is not null but B is
|
||||
|| (srcA.getStartKey().compareTo(srcB.getStartKey()) > 0)) { // A > B
|
||||
} else if((srcB.getStartKey() == null) // A is not null but B is
|
||||
|| (srcA.getStartKey().compareTo(srcB.getStartKey()) > 0)) { // A > B
|
||||
|
||||
HRegion tmp = srcA;
|
||||
srcA = srcB;
|
||||
srcB = tmp;
|
||||
}
|
||||
|
||||
if (!srcA.getEndKey().equals(srcB.getStartKey())) {
|
||||
if (! srcA.getEndKey().equals(srcB.getStartKey())) {
|
||||
throw new IOException("Cannot merge non-adjacent regions");
|
||||
}
|
||||
|
||||
|
@ -89,7 +89,7 @@ public class HRegion implements HConstants {
|
|||
Text endKey = srcB.getEndKey();
|
||||
|
||||
Path merges = new Path(srcA.getRegionDir(), MERGEDIR);
|
||||
if (!fs.exists(merges)) {
|
||||
if(! fs.exists(merges)) {
|
||||
fs.mkdirs(merges);
|
||||
}
|
||||
|
||||
|
@ -98,14 +98,14 @@ public class HRegion implements HConstants {
|
|||
|
||||
Path newRegionDir = HStoreFile.getHRegionDir(merges, newRegionInfo.regionName);
|
||||
|
||||
if (fs.exists(newRegionDir)) {
|
||||
if(fs.exists(newRegionDir)) {
|
||||
throw new IOException("Cannot merge; target file collision at " + newRegionDir);
|
||||
}
|
||||
|
||||
LOG.info("starting merge of regions: " + srcA.getRegionName() + " and "
|
||||
+ srcB.getRegionName() + " new region start key is '"
|
||||
+ (startKey == null ? "" : startKey) + "', end key is '"
|
||||
+ (endKey == null ? "" : endKey) + "'");
|
||||
+ srcB.getRegionName() + " new region start key is '"
|
||||
+ (startKey == null ? "" : startKey) + "', end key is '"
|
||||
+ (endKey == null ? "" : endKey) + "'");
|
||||
|
||||
// Flush each of the sources, and merge their files into a single
|
||||
// target for each column family.
|
||||
|
@ -114,10 +114,10 @@ public class HRegion implements HConstants {
|
|||
|
||||
TreeSet<HStoreFile> alreadyMerged = new TreeSet<HStoreFile>();
|
||||
TreeMap<Text, Vector<HStoreFile>> filesToMerge = new TreeMap<Text, Vector<HStoreFile>>();
|
||||
for(Iterator<HStoreFile> it = srcA.flushcache(true).iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = srcA.flushcache(true).iterator(); it.hasNext(); ) {
|
||||
HStoreFile src = it.next();
|
||||
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
||||
if (v == null) {
|
||||
if(v == null) {
|
||||
v = new Vector<HStoreFile>();
|
||||
filesToMerge.put(src.getColFamily(), v);
|
||||
}
|
||||
|
@ -126,10 +126,10 @@ public class HRegion implements HConstants {
|
|||
|
||||
LOG.debug("flushing and getting file names for region " + srcB.getRegionName());
|
||||
|
||||
for(Iterator<HStoreFile> it = srcB.flushcache(true).iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = srcB.flushcache(true).iterator(); it.hasNext(); ) {
|
||||
HStoreFile src = it.next();
|
||||
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
||||
if (v == null) {
|
||||
if(v == null) {
|
||||
v = new Vector<HStoreFile>();
|
||||
filesToMerge.put(src.getColFamily(), v);
|
||||
}
|
||||
|
@ -138,11 +138,11 @@ public class HRegion implements HConstants {
|
|||
|
||||
LOG.debug("merging stores");
|
||||
|
||||
for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext(); ) {
|
||||
Text colFamily = it.next();
|
||||
Vector<HStoreFile> srcFiles = filesToMerge.get(colFamily);
|
||||
HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName,
|
||||
colFamily, Math.abs(rand.nextLong()));
|
||||
colFamily, Math.abs(rand.nextLong()));
|
||||
|
||||
dst.mergeStoreFiles(srcFiles, fs, conf);
|
||||
alreadyMerged.addAll(srcFiles);
|
||||
|
@ -153,15 +153,15 @@ public class HRegion implements HConstants {
|
|||
// of any last-minute inserts
|
||||
|
||||
LOG.debug("flushing changes since start of merge for region "
|
||||
+ srcA.getRegionName());
|
||||
+ srcA.getRegionName());
|
||||
|
||||
filesToMerge.clear();
|
||||
for(Iterator<HStoreFile> it = srcA.close().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = srcA.close().iterator(); it.hasNext(); ) {
|
||||
HStoreFile src = it.next();
|
||||
|
||||
if (!alreadyMerged.contains(src)) {
|
||||
if(! alreadyMerged.contains(src)) {
|
||||
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
||||
if (v == null) {
|
||||
if(v == null) {
|
||||
v = new Vector<HStoreFile>();
|
||||
filesToMerge.put(src.getColFamily(), v);
|
||||
}
|
||||
|
@ -170,14 +170,14 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
|
||||
LOG.debug("flushing changes since start of merge for region "
|
||||
+ srcB.getRegionName());
|
||||
+ srcB.getRegionName());
|
||||
|
||||
for(Iterator<HStoreFile> it = srcB.close().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = srcB.close().iterator(); it.hasNext(); ) {
|
||||
HStoreFile src = it.next();
|
||||
|
||||
if (!alreadyMerged.contains(src)) {
|
||||
if(! alreadyMerged.contains(src)) {
|
||||
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
||||
if (v == null) {
|
||||
if(v == null) {
|
||||
v = new Vector<HStoreFile>();
|
||||
filesToMerge.put(src.getColFamily(), v);
|
||||
}
|
||||
|
@ -187,11 +187,11 @@ public class HRegion implements HConstants {
|
|||
|
||||
LOG.debug("merging changes since start of merge");
|
||||
|
||||
for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext(); ) {
|
||||
Text colFamily = it.next();
|
||||
Vector<HStoreFile> srcFiles = filesToMerge.get(colFamily);
|
||||
HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName,
|
||||
colFamily, Math.abs(rand.nextLong()));
|
||||
colFamily, Math.abs(rand.nextLong()));
|
||||
|
||||
dst.mergeStoreFiles(srcFiles, fs, conf);
|
||||
}
|
||||
|
@ -199,7 +199,7 @@ public class HRegion implements HConstants {
|
|||
// Done
|
||||
|
||||
HRegion dstRegion = new HRegion(dir, log, fs, conf, newRegionInfo,
|
||||
newRegionDir, null);
|
||||
newRegionDir, null);
|
||||
|
||||
// Get rid of merges directory
|
||||
|
||||
|
@ -284,7 +284,7 @@ public class HRegion implements HConstants {
|
|||
* written-to before), then read it from the supplied path.
|
||||
*/
|
||||
public HRegion(Path dir, HLog log, FileSystem fs, Configuration conf,
|
||||
HRegionInfo regionInfo, Path initialFiles, Path oldLogFile) throws IOException {
|
||||
HRegionInfo regionInfo, Path initialFiles, Path oldLogFile) throws IOException {
|
||||
|
||||
this.dir = dir;
|
||||
this.log = log;
|
||||
|
@ -303,29 +303,29 @@ public class HRegion implements HConstants {
|
|||
|
||||
// Move prefab HStore files into place (if any)
|
||||
|
||||
if (initialFiles != null && fs.exists(initialFiles)) {
|
||||
if(initialFiles != null && fs.exists(initialFiles)) {
|
||||
fs.rename(initialFiles, regiondir);
|
||||
}
|
||||
|
||||
// Load in all the HStores.
|
||||
|
||||
for(Iterator<Text> it = this.regionInfo.tableDesc.families().iterator();
|
||||
it.hasNext();) {
|
||||
it.hasNext(); ) {
|
||||
|
||||
Text colFamily = it.next();
|
||||
Text colFamily = HStoreKey.extractFamily(it.next());
|
||||
stores.put(colFamily, new HStore(dir, this.regionInfo.regionName, colFamily,
|
||||
this.regionInfo.tableDesc.getMaxVersions(), fs, oldLogFile, conf));
|
||||
this.regionInfo.tableDesc.getMaxVersions(), fs, oldLogFile, conf));
|
||||
}
|
||||
|
||||
// Get rid of any splits or merges that were lost in-progress
|
||||
|
||||
Path splits = new Path(regiondir, SPLITDIR);
|
||||
if (fs.exists(splits)) {
|
||||
if(fs.exists(splits)) {
|
||||
fs.delete(splits);
|
||||
}
|
||||
|
||||
Path merges = new Path(regiondir, MERGEDIR);
|
||||
if (fs.exists(merges)) {
|
||||
if(fs.exists(merges)) {
|
||||
fs.delete(merges);
|
||||
}
|
||||
|
||||
|
@ -345,6 +345,7 @@ public class HRegion implements HConstants {
|
|||
|
||||
/** Closes and deletes this HRegion. Called when doing a table deletion, for example */
|
||||
public void closeAndDelete() throws IOException {
|
||||
LOG.info("deleting region: " + regionInfo.regionName);
|
||||
close();
|
||||
fs.delete(regiondir);
|
||||
}
|
||||
|
@ -362,7 +363,7 @@ public class HRegion implements HConstants {
|
|||
public Vector<HStoreFile> close() throws IOException {
|
||||
boolean shouldClose = false;
|
||||
synchronized(writestate) {
|
||||
if (writestate.closed) {
|
||||
if(writestate.closed) {
|
||||
LOG.info("region " + this.regionInfo.regionName + " closed");
|
||||
return new Vector<HStoreFile>();
|
||||
}
|
||||
|
@ -376,13 +377,13 @@ public class HRegion implements HConstants {
|
|||
shouldClose = true;
|
||||
}
|
||||
|
||||
if (!shouldClose) {
|
||||
if(! shouldClose) {
|
||||
return null;
|
||||
|
||||
} else {
|
||||
LOG.info("closing region " + this.regionInfo.regionName);
|
||||
Vector<HStoreFile> allHStoreFiles = internalFlushcache();
|
||||
for(Iterator<HStore> it = stores.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStore> it = stores.values().iterator(); it.hasNext(); ) {
|
||||
HStore store = it.next();
|
||||
store.close();
|
||||
}
|
||||
|
@ -406,8 +407,8 @@ public class HRegion implements HConstants {
|
|||
* Returns two brand-new (and open) HRegions
|
||||
*/
|
||||
public HRegion[] closeAndSplit(Text midKey) throws IOException {
|
||||
if (((regionInfo.startKey.getLength() != 0)
|
||||
&& (regionInfo.startKey.compareTo(midKey) > 0))
|
||||
if(((regionInfo.startKey.getLength() != 0)
|
||||
&& (regionInfo.startKey.compareTo(midKey) > 0))
|
||||
|| ((regionInfo.endKey.getLength() != 0)
|
||||
&& (regionInfo.endKey.compareTo(midKey) < 0))) {
|
||||
throw new IOException("Region splitkey must lie within region boundaries.");
|
||||
|
@ -419,13 +420,13 @@ public class HRegion implements HConstants {
|
|||
// or compactions until close() is called.
|
||||
|
||||
Path splits = new Path(regiondir, SPLITDIR);
|
||||
if (!fs.exists(splits)) {
|
||||
if(! fs.exists(splits)) {
|
||||
fs.mkdirs(splits);
|
||||
}
|
||||
|
||||
long regionAId = Math.abs(rand.nextLong());
|
||||
HRegionInfo regionAInfo = new HRegionInfo(regionAId, regionInfo.tableDesc,
|
||||
regionInfo.startKey, midKey);
|
||||
regionInfo.startKey, midKey);
|
||||
|
||||
long regionBId = Math.abs(rand.nextLong());
|
||||
HRegionInfo regionBInfo
|
||||
|
@ -434,24 +435,24 @@ public class HRegion implements HConstants {
|
|||
Path dirA = HStoreFile.getHRegionDir(splits, regionAInfo.regionName);
|
||||
Path dirB = HStoreFile.getHRegionDir(splits, regionBInfo.regionName);
|
||||
|
||||
if (fs.exists(dirA) || fs.exists(dirB)) {
|
||||
if(fs.exists(dirA) || fs.exists(dirB)) {
|
||||
throw new IOException("Cannot split; target file collision at " + dirA
|
||||
+ " or " + dirB);
|
||||
+ " or " + dirB);
|
||||
}
|
||||
|
||||
TreeSet<HStoreFile> alreadySplit = new TreeSet<HStoreFile>();
|
||||
Vector<HStoreFile> hstoreFilesToSplit = flushcache(true);
|
||||
for(Iterator<HStoreFile> it = hstoreFilesToSplit.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = hstoreFilesToSplit.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
|
||||
LOG.debug("splitting HStore " + hsf.getRegionName() + "/" + hsf.getColFamily()
|
||||
+ "/" + hsf.fileId());
|
||||
+ "/" + hsf.fileId());
|
||||
|
||||
HStoreFile dstA = new HStoreFile(conf, splits, regionAInfo.regionName,
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
|
||||
HStoreFile dstB = new HStoreFile(conf, splits, regionBInfo.regionName,
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
|
||||
hsf.splitStoreFile(midKey, dstA, dstB, fs, conf);
|
||||
alreadySplit.add(hsf);
|
||||
|
@ -461,18 +462,18 @@ public class HRegion implements HConstants {
|
|||
// and copy the small remainder
|
||||
|
||||
hstoreFilesToSplit = close();
|
||||
for(Iterator<HStoreFile> it = hstoreFilesToSplit.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = hstoreFilesToSplit.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
|
||||
if (!alreadySplit.contains(hsf)) {
|
||||
if(! alreadySplit.contains(hsf)) {
|
||||
LOG.debug("splitting HStore " + hsf.getRegionName() + "/" + hsf.getColFamily()
|
||||
+ "/" + hsf.fileId());
|
||||
+ "/" + hsf.fileId());
|
||||
|
||||
HStoreFile dstA = new HStoreFile(conf, splits, regionAInfo.regionName,
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
|
||||
HStoreFile dstB = new HStoreFile(conf, splits, regionBInfo.regionName,
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
|
||||
hsf.splitStoreFile(midKey, dstA, dstB, fs, conf);
|
||||
}
|
||||
|
@ -494,7 +495,7 @@ public class HRegion implements HConstants {
|
|||
regions[1] = regionB;
|
||||
|
||||
LOG.info("region split complete. new regions are: " + regions[0].getRegionName()
|
||||
+ ", " + regions[1].getRegionName());
|
||||
+ ", " + regions[1].getRegionName());
|
||||
|
||||
return regions;
|
||||
}
|
||||
|
@ -565,10 +566,10 @@ public class HRegion implements HConstants {
|
|||
Text key = new Text();
|
||||
long maxSize = 0;
|
||||
|
||||
for(Iterator<HStore> i = stores.values().iterator(); i.hasNext();) {
|
||||
for(Iterator<HStore> i = stores.values().iterator(); i.hasNext(); ) {
|
||||
long size = i.next().getLargestFileSize(key);
|
||||
|
||||
if (size > maxSize) { // Largest so far
|
||||
if(size > maxSize) { // Largest so far
|
||||
maxSize = size;
|
||||
midKey.set(key);
|
||||
}
|
||||
|
@ -593,9 +594,9 @@ public class HRegion implements HConstants {
|
|||
public boolean compactStores() throws IOException {
|
||||
boolean shouldCompact = false;
|
||||
synchronized(writestate) {
|
||||
if ((!writestate.writesOngoing)
|
||||
if((! writestate.writesOngoing)
|
||||
&& writestate.writesEnabled
|
||||
&& (!writestate.closed)
|
||||
&& (! writestate.closed)
|
||||
&& recentCommits > MIN_COMMITS_FOR_COMPACTION) {
|
||||
|
||||
writestate.writesOngoing = true;
|
||||
|
@ -603,14 +604,14 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
}
|
||||
|
||||
if (!shouldCompact) {
|
||||
if(! shouldCompact) {
|
||||
LOG.info("not compacting region " + this.regionInfo.regionName);
|
||||
return false;
|
||||
|
||||
} else {
|
||||
try {
|
||||
LOG.info("starting compaction on region " + this.regionInfo.regionName);
|
||||
for(Iterator<HStore> it = stores.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStore> it = stores.values().iterator(); it.hasNext(); ) {
|
||||
HStore store = it.next();
|
||||
store.compact();
|
||||
}
|
||||
|
@ -632,7 +633,7 @@ public class HRegion implements HConstants {
|
|||
* only take if there have been a lot of uncommitted writes.
|
||||
*/
|
||||
public void optionallyFlush() throws IOException {
|
||||
if (commitsSinceFlush > maxUnflushedEntries) {
|
||||
if(commitsSinceFlush > maxUnflushedEntries) {
|
||||
flushcache(false);
|
||||
}
|
||||
}
|
||||
|
@ -657,20 +658,20 @@ public class HRegion implements HConstants {
|
|||
public Vector<HStoreFile> flushcache(boolean disableFutureWrites) throws IOException {
|
||||
boolean shouldFlush = false;
|
||||
synchronized(writestate) {
|
||||
if ((!writestate.writesOngoing)
|
||||
if((! writestate.writesOngoing)
|
||||
&& writestate.writesEnabled
|
||||
&& (!writestate.closed)) {
|
||||
&& (! writestate.closed)) {
|
||||
|
||||
writestate.writesOngoing = true;
|
||||
shouldFlush = true;
|
||||
|
||||
if (disableFutureWrites) {
|
||||
if(disableFutureWrites) {
|
||||
writestate.writesEnabled = false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!shouldFlush) {
|
||||
if(! shouldFlush) {
|
||||
LOG.debug("not flushing cache for region " + this.regionInfo.regionName);
|
||||
return null;
|
||||
|
||||
|
@ -731,8 +732,8 @@ public class HRegion implements HConstants {
|
|||
|
||||
HMemcache.Snapshot retval = memcache.snapshotMemcacheForLog(log);
|
||||
TreeMap<HStoreKey, BytesWritable> memcacheSnapshot = retval.memcacheSnapshot;
|
||||
if (memcacheSnapshot == null) {
|
||||
for(Iterator<HStore> it = stores.values().iterator(); it.hasNext();) {
|
||||
if(memcacheSnapshot == null) {
|
||||
for(Iterator<HStore> it = stores.values().iterator(); it.hasNext(); ) {
|
||||
HStore hstore = it.next();
|
||||
Vector<HStoreFile> hstoreFiles = hstore.getAllMapFiles();
|
||||
allHStoreFiles.addAll(0, hstoreFiles);
|
||||
|
@ -746,7 +747,7 @@ public class HRegion implements HConstants {
|
|||
|
||||
LOG.debug("flushing memcache to HStores");
|
||||
|
||||
for(Iterator<HStore> it = stores.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStore> it = stores.values().iterator(); it.hasNext(); ) {
|
||||
HStore hstore = it.next();
|
||||
Vector<HStoreFile> hstoreFiles
|
||||
= hstore.flushCache(memcacheSnapshot, logCacheFlushId);
|
||||
|
@ -762,7 +763,7 @@ public class HRegion implements HConstants {
|
|||
LOG.debug("writing flush cache complete to log");
|
||||
|
||||
log.completeCacheFlush(this.regionInfo.regionName,
|
||||
regionInfo.tableDesc.getName(), logCacheFlushId);
|
||||
regionInfo.tableDesc.getName(), logCacheFlushId);
|
||||
|
||||
// C. Delete the now-irrelevant memcache snapshot; its contents have been
|
||||
// dumped to disk-based HStores.
|
||||
|
@ -784,7 +785,7 @@ public class HRegion implements HConstants {
|
|||
/** Fetch a single data item. */
|
||||
public byte[] get(Text row, Text column) throws IOException {
|
||||
byte results[][] = get(row, column, Long.MAX_VALUE, 1);
|
||||
if (results == null) {
|
||||
if(results == null) {
|
||||
return null;
|
||||
|
||||
} else {
|
||||
|
@ -799,17 +800,16 @@ public class HRegion implements HConstants {
|
|||
|
||||
/** Fetch multiple versions of a single data item, with timestamp. */
|
||||
public byte[][] get(Text row, Text column, long timestamp, int numVersions)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
if (writestate.closed) {
|
||||
if(writestate.closed) {
|
||||
throw new IOException("HRegion is closed.");
|
||||
}
|
||||
|
||||
// Make sure this is a valid row and valid column
|
||||
|
||||
checkRow(row);
|
||||
Text colFamily = HStoreKey.extractFamily(column);
|
||||
checkFamily(colFamily);
|
||||
checkColumn(column);
|
||||
|
||||
// Obtain the row-lock
|
||||
|
||||
|
@ -830,7 +830,7 @@ public class HRegion implements HConstants {
|
|||
// Check the memcache
|
||||
|
||||
byte[][] result = memcache.get(key, numVersions);
|
||||
if (result != null) {
|
||||
if(result != null) {
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -838,7 +838,7 @@ public class HRegion implements HConstants {
|
|||
|
||||
Text colFamily = HStoreKey.extractFamily(key.getColumn());
|
||||
HStore targetStore = stores.get(colFamily);
|
||||
if (targetStore == null) {
|
||||
if(targetStore == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -859,7 +859,7 @@ public class HRegion implements HConstants {
|
|||
HStoreKey key = new HStoreKey(row, System.currentTimeMillis());
|
||||
|
||||
TreeMap<Text, byte[]> memResult = memcache.getFull(key);
|
||||
for(Iterator<Text> it = stores.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = stores.keySet().iterator(); it.hasNext(); ) {
|
||||
Text colFamily = it.next();
|
||||
HStore targetStore = stores.get(colFamily);
|
||||
targetStore.getFull(key, memResult);
|
||||
|
@ -879,7 +879,7 @@ public class HRegion implements HConstants {
|
|||
|
||||
HStore storelist[] = new HStore[families.size()];
|
||||
int i = 0;
|
||||
for(Iterator<Text> it = families.iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = families.iterator(); it.hasNext(); ) {
|
||||
Text family = it.next();
|
||||
storelist[i++] = stores.get(family);
|
||||
}
|
||||
|
@ -911,23 +911,23 @@ public class HRegion implements HConstants {
|
|||
/**
|
||||
* Put a cell value into the locked row. The user indicates the row-lock, the
|
||||
* target column, and the desired value. This stuff is set into a temporary
|
||||
* memory area until the user commits the change, at which pointit's logged
|
||||
* memory area until the user commits the change, at which point it's logged
|
||||
* and placed into the memcache.
|
||||
*
|
||||
* This method really just tests the input, then calls an internal localput()
|
||||
* method.
|
||||
*/
|
||||
public void put(long lockid, Text targetCol, byte[] val) throws IOException {
|
||||
if (val.length == HStoreKey.DELETE_BYTES.length) {
|
||||
if(val.length == HStoreKey.DELETE_BYTES.length) {
|
||||
boolean matches = true;
|
||||
for(int i = 0; i < val.length; i++) {
|
||||
if (val[i] != HStoreKey.DELETE_BYTES[i]) {
|
||||
if(val[i] != HStoreKey.DELETE_BYTES[i]) {
|
||||
matches = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (matches) {
|
||||
if(matches) {
|
||||
throw new IOException("Cannot insert value: " + val);
|
||||
}
|
||||
}
|
||||
|
@ -950,9 +950,11 @@ public class HRegion implements HConstants {
|
|||
* (Or until the user's write-lock expires.)
|
||||
*/
|
||||
void localput(long lockid, Text targetCol, byte[] val) throws IOException {
|
||||
checkColumn(targetCol);
|
||||
|
||||
Text row = getRowFromLock(lockid);
|
||||
if (row == null) {
|
||||
throw new IOException("No write lock for lockid " + lockid);
|
||||
if(row == null) {
|
||||
throw new LockException("No write lock for lockid " + lockid);
|
||||
}
|
||||
|
||||
// This sync block makes localput() thread-safe when multiple
|
||||
|
@ -964,13 +966,13 @@ public class HRegion implements HConstants {
|
|||
// This check makes sure that another thread from the client
|
||||
// hasn't aborted/committed the write-operation.
|
||||
|
||||
if (row != getRowFromLock(lockid)) {
|
||||
throw new IOException("Locking error: put operation on lock " + lockid
|
||||
+ " unexpected aborted by another thread");
|
||||
if(row != getRowFromLock(lockid)) {
|
||||
throw new LockException("Locking error: put operation on lock " + lockid
|
||||
+ " unexpected aborted by another thread");
|
||||
}
|
||||
|
||||
TreeMap<Text, byte[]> targets = targetColumns.get(lockid);
|
||||
if (targets == null) {
|
||||
if(targets == null) {
|
||||
targets = new TreeMap<Text, byte[]>();
|
||||
targetColumns.put(lockid, targets);
|
||||
}
|
||||
|
@ -985,8 +987,8 @@ public class HRegion implements HConstants {
|
|||
*/
|
||||
public void abort(long lockid) throws IOException {
|
||||
Text row = getRowFromLock(lockid);
|
||||
if (row == null) {
|
||||
throw new IOException("No write lock for lockid " + lockid);
|
||||
if(row == null) {
|
||||
throw new LockException("No write lock for lockid " + lockid);
|
||||
}
|
||||
|
||||
// This sync block makes abort() thread-safe when multiple
|
||||
|
@ -998,9 +1000,9 @@ public class HRegion implements HConstants {
|
|||
// This check makes sure another thread from the client
|
||||
// hasn't aborted/committed the write-operation.
|
||||
|
||||
if (row != getRowFromLock(lockid)) {
|
||||
throw new IOException("Locking error: abort() operation on lock "
|
||||
+ lockid + " unexpected aborted by another thread");
|
||||
if(row != getRowFromLock(lockid)) {
|
||||
throw new LockException("Locking error: abort() operation on lock "
|
||||
+ lockid + " unexpected aborted by another thread");
|
||||
}
|
||||
|
||||
targetColumns.remove(lockid);
|
||||
|
@ -1021,8 +1023,8 @@ public class HRegion implements HConstants {
|
|||
// that repeated executions won't screw this up.
|
||||
|
||||
Text row = getRowFromLock(lockid);
|
||||
if (row == null) {
|
||||
throw new IOException("No write lock for lockid " + lockid);
|
||||
if(row == null) {
|
||||
throw new LockException("No write lock for lockid " + lockid);
|
||||
}
|
||||
|
||||
// This check makes sure that another thread from the client
|
||||
|
@ -1035,7 +1037,7 @@ public class HRegion implements HConstants {
|
|||
|
||||
long commitTimestamp = System.currentTimeMillis();
|
||||
log.append(regionInfo.regionName, regionInfo.tableDesc.getName(), row,
|
||||
targetColumns.get(lockid), commitTimestamp);
|
||||
targetColumns.get(lockid), commitTimestamp);
|
||||
|
||||
memcache.add(row, targetColumns.get(lockid), commitTimestamp);
|
||||
|
||||
|
@ -1054,25 +1056,26 @@ public class HRegion implements HConstants {
|
|||
|
||||
/** Make sure this is a valid row for the HRegion */
|
||||
void checkRow(Text row) throws IOException {
|
||||
if (((regionInfo.startKey.getLength() == 0)
|
||||
|| (regionInfo.startKey.compareTo(row) <= 0))
|
||||
if(((regionInfo.startKey.getLength() == 0)
|
||||
|| (regionInfo.startKey.compareTo(row) <= 0))
|
||||
&& ((regionInfo.endKey.getLength() == 0)
|
||||
|| (regionInfo.endKey.compareTo(row) > 0))) {
|
||||
// all's well
|
||||
|
||||
} else {
|
||||
throw new IOException("Requested row out of range for HRegion "
|
||||
+ regionInfo.regionName + ", startKey='" + regionInfo.startKey
|
||||
+ "', endKey='" + regionInfo.endKey + "', row='" + row + "'");
|
||||
+ regionInfo.regionName + ", startKey='" + regionInfo.startKey
|
||||
+ "', endKey='" + regionInfo.endKey + "', row='" + row + "'");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Make sure this is a valid column for the current table */
|
||||
void checkFamily(Text family) throws IOException {
|
||||
if (!regionInfo.tableDesc.hasFamily(family)) {
|
||||
void checkColumn(Text columnName) throws IOException {
|
||||
Text family = new Text(HStoreKey.extractFamily(columnName) + ":");
|
||||
if(! regionInfo.tableDesc.hasFamily(family)) {
|
||||
throw new IOException("Requested column family " + family
|
||||
+ " does not exist in HRegion " + regionInfo.regionName
|
||||
+ " for table " + regionInfo.tableDesc.getName());
|
||||
+ " does not exist in HRegion " + regionInfo.regionName
|
||||
+ " for table " + regionInfo.tableDesc.getName());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1092,6 +1095,8 @@ public class HRegion implements HConstants {
|
|||
* which maybe we'll do in the future.
|
||||
*/
|
||||
long obtainLock(Text row) throws IOException {
|
||||
checkRow(row);
|
||||
|
||||
synchronized(rowsToLocks) {
|
||||
while(rowsToLocks.get(row) != null) {
|
||||
try {
|
||||
|
@ -1109,6 +1114,8 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
|
||||
Text getRowFromLock(long lockid) throws IOException {
|
||||
// Pattern is that all access to rowsToLocks and/or to
|
||||
// locksToRows is via a lock on rowsToLocks.
|
||||
synchronized(rowsToLocks) {
|
||||
return locksToRows.get(lockid);
|
||||
}
|
||||
|
@ -1150,7 +1157,7 @@ public class HRegion implements HConstants {
|
|||
keys[i] = new HStoreKey();
|
||||
resultSets[i] = new TreeMap<Text, byte[]>();
|
||||
|
||||
if (!scanners[i].next(keys[i], resultSets[i])) {
|
||||
if(! scanners[i].next(keys[i], resultSets[i])) {
|
||||
closeScanner(i);
|
||||
}
|
||||
}
|
||||
|
@ -1167,7 +1174,7 @@ public class HRegion implements HConstants {
|
|||
Text chosenRow = null;
|
||||
long chosenTimestamp = -1;
|
||||
for(int i = 0; i < keys.length; i++) {
|
||||
if (scanners[i] != null
|
||||
if(scanners[i] != null
|
||||
&& (chosenRow == null
|
||||
|| (keys[i].getRow().compareTo(chosenRow) < 0)
|
||||
|| ((keys[i].getRow().compareTo(chosenRow) == 0)
|
||||
|
@ -1181,21 +1188,21 @@ public class HRegion implements HConstants {
|
|||
// Store the key and results for each sub-scanner. Merge them as appropriate.
|
||||
|
||||
boolean insertedItem = false;
|
||||
if (chosenTimestamp > 0) {
|
||||
if(chosenTimestamp > 0) {
|
||||
key.setRow(chosenRow);
|
||||
key.setVersion(chosenTimestamp);
|
||||
key.setColumn(new Text(""));
|
||||
|
||||
for(int i = 0; i < scanners.length; i++) {
|
||||
while((scanners[i] != null)
|
||||
&& (keys[i].getRow().compareTo(chosenRow) == 0)
|
||||
&& (keys[i].getTimestamp() == chosenTimestamp)) {
|
||||
&& (keys[i].getRow().compareTo(chosenRow) == 0)
|
||||
&& (keys[i].getTimestamp() == chosenTimestamp)) {
|
||||
|
||||
results.putAll(resultSets[i]);
|
||||
insertedItem = true;
|
||||
|
||||
resultSets[i].clear();
|
||||
if (!scanners[i].next(keys[i], resultSets[i])) {
|
||||
if(! scanners[i].next(keys[i], resultSets[i])) {
|
||||
closeScanner(i);
|
||||
}
|
||||
}
|
||||
|
@ -1204,10 +1211,10 @@ public class HRegion implements HConstants {
|
|||
// row label, then its timestamp is bad. We need to advance it.
|
||||
|
||||
while((scanners[i] != null)
|
||||
&& (keys[i].getRow().compareTo(chosenRow) <= 0)) {
|
||||
&& (keys[i].getRow().compareTo(chosenRow) <= 0)) {
|
||||
|
||||
resultSets[i].clear();
|
||||
if (!scanners[i].next(keys[i], resultSets[i])) {
|
||||
if(! scanners[i].next(keys[i], resultSets[i])) {
|
||||
closeScanner(i);
|
||||
}
|
||||
}
|
||||
|
@ -1231,7 +1238,7 @@ public class HRegion implements HConstants {
|
|||
/** All done with the scanner. */
|
||||
public void close() throws IOException {
|
||||
for(int i = 0; i < scanners.length; i++) {
|
||||
if (scanners[i] != null) {
|
||||
if(scanners[i] != null) {
|
||||
closeScanner(i);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,28 +38,28 @@ public class HRegionInfo implements Writable {
|
|||
}
|
||||
|
||||
public HRegionInfo(long regionId, HTableDescriptor tableDesc, Text startKey,
|
||||
Text endKey) throws IllegalArgumentException {
|
||||
Text endKey) throws IllegalArgumentException {
|
||||
|
||||
this.regionId = regionId;
|
||||
|
||||
if (tableDesc == null) {
|
||||
if(tableDesc == null) {
|
||||
throw new IllegalArgumentException("tableDesc cannot be null");
|
||||
}
|
||||
|
||||
this.tableDesc = tableDesc;
|
||||
|
||||
this.startKey = new Text();
|
||||
if (startKey != null) {
|
||||
if(startKey != null) {
|
||||
this.startKey.set(startKey);
|
||||
}
|
||||
|
||||
this.endKey = new Text();
|
||||
if (endKey != null) {
|
||||
if(endKey != null) {
|
||||
this.endKey.set(endKey);
|
||||
}
|
||||
|
||||
this.regionName = new Text(tableDesc.getName() + "_"
|
||||
+ (startKey == null ? "" : startKey.toString()) + "_" + regionId);
|
||||
+ (startKey == null ? "" : startKey.toString()) + "_" + regionId);
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
|
|
@ -15,7 +15,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.io.*;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ipc.VersionedProtocol;
|
||||
|
||||
import java.io.*;
|
||||
|
||||
|
@ -23,17 +25,13 @@ import java.io.*;
|
|||
* Clients interact with HRegionServers using
|
||||
* a handle to the HRegionInterface.
|
||||
******************************************************************************/
|
||||
public interface HRegionInterface {
|
||||
public interface HRegionInterface extends VersionedProtocol {
|
||||
public static final long versionID = 1L; // initial version
|
||||
|
||||
// Get metainfo about an HRegion
|
||||
|
||||
public HRegionInfo getRegionInfo(Text regionName);
|
||||
|
||||
// Start a scanner for a given HRegion.
|
||||
|
||||
public HScannerInterface openScanner(Text regionName, Text[] columns, Text startRow) throws IOException;
|
||||
|
||||
// GET methods for an HRegion.
|
||||
|
||||
public BytesWritable get(Text regionName, Text row, Text column) throws IOException;
|
||||
|
@ -58,4 +56,41 @@ public interface HRegionInterface {
|
|||
public void abort(Text regionName, long clientid, long lockid) throws IOException;
|
||||
public void commit(Text regionName, long clientid, long lockid) throws IOException;
|
||||
public void renewLease(long lockid, long clientid) throws IOException;
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// remote scanner interface
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
/**
|
||||
* Opens a remote scanner.
|
||||
*
|
||||
* @param clientId - client identifier (so we can associate a scanner with a client)
|
||||
* @param regionName - name of region to scan
|
||||
* @param columns - columns to scan
|
||||
* @param startRow - starting row to scan
|
||||
*
|
||||
* @param scannerId - scanner identifier used in other calls
|
||||
* @throws IOException
|
||||
*/
|
||||
public long openScanner(Text regionName, Text[] columns, Text startRow) throws IOException;
|
||||
|
||||
/**
|
||||
* Get the next set of values
|
||||
*
|
||||
* @param scannerId - clientId passed to openScanner
|
||||
* @param key - the next HStoreKey
|
||||
* @param columns - an array of column names
|
||||
* @param values - an array of byte[] values (corresponds 1-1 with columns)
|
||||
* @return - true if a value was retrieved
|
||||
* @throws IOException
|
||||
*/
|
||||
public LabelledData[] next(long scannerId, HStoreKey key) throws IOException;
|
||||
|
||||
/**
|
||||
* Close a scanner
|
||||
*
|
||||
* @param scannerId - the scanner id returned by openScanner
|
||||
* @throws IOException
|
||||
*/
|
||||
public void close(long scannerId) throws IOException;
|
||||
}
|
||||
|
|
|
@ -15,6 +15,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.io.*;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.apache.hadoop.ipc.*;
|
||||
|
@ -22,19 +24,34 @@ import org.apache.hadoop.conf.*;
|
|||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
/*******************************************************************************
|
||||
* HRegionServer makes a set of HRegions available to clients. It checks in with
|
||||
* the HMaster. There are many HRegionServers in a single HBase deployment.
|
||||
******************************************************************************/
|
||||
public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||
public class HRegionServer
|
||||
implements HConstants, HRegionInterface, Runnable {
|
||||
|
||||
public long getProtocolVersion(String protocol,
|
||||
long clientVersion) throws IOException {
|
||||
if (protocol.equals(HRegionInterface.class.getName())) {
|
||||
return HRegionInterface.versionID;
|
||||
} else {
|
||||
throw new IOException("Unknown protocol to name node: " + protocol);
|
||||
}
|
||||
}
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(HRegionServer.class);
|
||||
|
||||
private boolean stopRequested;
|
||||
private Path regionDir;
|
||||
private HServerAddress address;
|
||||
private Configuration conf;
|
||||
private Random rand;
|
||||
private TreeMap<Text, HRegion> regions; // region name -> HRegion
|
||||
private HLocking locking;
|
||||
private ReadWriteLock locker;
|
||||
private Vector<HMsg> outboundMsgs;
|
||||
|
||||
private long threadWakeFrequency;
|
||||
|
@ -61,29 +78,29 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
}
|
||||
|
||||
public void run() {
|
||||
while(!stopRequested) {
|
||||
while(! stopRequested) {
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
// Grab a list of regions to check
|
||||
|
||||
Vector<HRegion> checkSplit = new Vector<HRegion>();
|
||||
locking.obtainReadLock();
|
||||
locker.readLock().lock();
|
||||
try {
|
||||
checkSplit.addAll(regions.values());
|
||||
|
||||
} finally {
|
||||
locking.releaseReadLock();
|
||||
locker.readLock().unlock();
|
||||
}
|
||||
|
||||
// Check to see if they need splitting
|
||||
|
||||
Vector<SplitRegion> toSplit = new Vector<SplitRegion>();
|
||||
for(Iterator<HRegion> it = checkSplit.iterator(); it.hasNext();) {
|
||||
for(Iterator<HRegion> it = checkSplit.iterator(); it.hasNext(); ) {
|
||||
HRegion cur = it.next();
|
||||
Text midKey = new Text();
|
||||
|
||||
try {
|
||||
if (cur.needsSplit(midKey)) {
|
||||
if(cur.needsSplit(midKey)) {
|
||||
toSplit.add(new SplitRegion(cur, midKey));
|
||||
}
|
||||
|
||||
|
@ -92,17 +109,19 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
}
|
||||
}
|
||||
|
||||
for(Iterator<SplitRegion> it = toSplit.iterator(); it.hasNext();) {
|
||||
for(Iterator<SplitRegion> it = toSplit.iterator(); it.hasNext(); ) {
|
||||
SplitRegion r = it.next();
|
||||
|
||||
locking.obtainWriteLock();
|
||||
locker.writeLock().lock();
|
||||
regions.remove(r.region.getRegionName());
|
||||
locking.releaseWriteLock();
|
||||
locker.writeLock().unlock();
|
||||
|
||||
HRegion[] newRegions = null;
|
||||
try {
|
||||
Text oldRegion = r.region.getRegionName();
|
||||
|
||||
LOG.info("splitting region: " + oldRegion);
|
||||
|
||||
newRegions = r.region.closeAndSplit(r.midKey);
|
||||
|
||||
// When a region is split, the META table needs to updated if we're
|
||||
|
@ -111,8 +130,10 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
Text tableToUpdate
|
||||
= (oldRegion.find(META_TABLE_NAME.toString()) == 0)
|
||||
? ROOT_TABLE_NAME : META_TABLE_NAME;
|
||||
? ROOT_TABLE_NAME : META_TABLE_NAME;
|
||||
|
||||
LOG.debug("region split complete. updating meta");
|
||||
|
||||
client.openTable(tableToUpdate);
|
||||
long lockid = client.startUpdate(oldRegion);
|
||||
client.delete(lockid, META_COL_REGIONINFO);
|
||||
|
@ -132,7 +153,14 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
// Now tell the master about the new regions
|
||||
|
||||
LOG.debug("reporting region split to master");
|
||||
|
||||
reportSplit(newRegions[0].getRegionInfo(), newRegions[1].getRegionInfo());
|
||||
|
||||
LOG.info("region split successful. old region=" + oldRegion
|
||||
+ ", new regions: " + newRegions[0].getRegionName() + ", "
|
||||
+ newRegions[1].getRegionName());
|
||||
|
||||
newRegions[0].close();
|
||||
newRegions[1].close();
|
||||
|
||||
|
@ -145,11 +173,15 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
// Sleep
|
||||
|
||||
long endTime = System.currentTimeMillis();
|
||||
try {
|
||||
Thread.sleep(splitCheckFrequency - (endTime - startTime));
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
long waitTime =
|
||||
splitCheckFrequency - (System.currentTimeMillis() - startTime);
|
||||
|
||||
if(waitTime > 0) {
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -161,23 +193,23 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
private Thread cacheFlusherThread;
|
||||
private class Flusher implements Runnable {
|
||||
public void run() {
|
||||
while(!stopRequested) {
|
||||
while(! stopRequested) {
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
// Grab a list of items to flush
|
||||
|
||||
Vector<HRegion> toFlush = new Vector<HRegion>();
|
||||
locking.obtainReadLock();
|
||||
locker.readLock().lock();
|
||||
try {
|
||||
toFlush.addAll(regions.values());
|
||||
|
||||
} finally {
|
||||
locking.releaseReadLock();
|
||||
locker.readLock().unlock();
|
||||
}
|
||||
|
||||
// Flush them, if necessary
|
||||
|
||||
for(Iterator<HRegion> it = toFlush.iterator(); it.hasNext();) {
|
||||
for(Iterator<HRegion> it = toFlush.iterator(); it.hasNext(); ) {
|
||||
HRegion cur = it.next();
|
||||
|
||||
try {
|
||||
|
@ -190,11 +222,15 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
// Sleep
|
||||
|
||||
long endTime = System.currentTimeMillis();
|
||||
try {
|
||||
Thread.sleep(threadWakeFrequency - (endTime - startTime));
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
long waitTime =
|
||||
threadWakeFrequency - (System.currentTimeMillis() - startTime);
|
||||
|
||||
if(waitTime > 0) {
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -212,12 +248,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
private Thread logRollerThread;
|
||||
private class LogRoller implements Runnable {
|
||||
public void run() {
|
||||
while(!stopRequested) {
|
||||
while(! stopRequested) {
|
||||
|
||||
// If the number of log entries is high enough, roll the log. This is a
|
||||
// very fast operation, but should not be done too frequently.
|
||||
|
||||
if (log.getNumEntries() > maxLogEntries) {
|
||||
if(log.getNumEntries() > maxLogEntries) {
|
||||
try {
|
||||
log.rollWriter();
|
||||
|
||||
|
@ -249,24 +285,24 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
/** Start a HRegionServer at the default location */
|
||||
public HRegionServer(Configuration conf) throws IOException {
|
||||
this(new Path(conf.get(HREGION_DIR, DEFAULT_HREGION_DIR)),
|
||||
new HServerAddress(conf.get("hbase.regionserver.default.name")),
|
||||
conf);
|
||||
new HServerAddress(conf.get(REGIONSERVER_ADDRESS, "localhost:0")),
|
||||
conf);
|
||||
}
|
||||
|
||||
/** Start a HRegionServer at an indicated location */
|
||||
public HRegionServer(Path regionDir, HServerAddress address, Configuration conf)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
// Basic setup
|
||||
|
||||
this.stopRequested = false;
|
||||
this.regionDir = regionDir;
|
||||
this.address = address;
|
||||
this.conf = conf;
|
||||
this.rand = new Random();
|
||||
this.regions = new TreeMap<Text, HRegion>();
|
||||
this.locking = new HLocking();
|
||||
this.locker = new ReentrantReadWriteLock();
|
||||
this.outboundMsgs = new Vector<HMsg>();
|
||||
this.scanners = Collections.synchronizedMap(new TreeMap<Text, HScannerInterface>());
|
||||
|
||||
// Config'ed params
|
||||
|
||||
|
@ -278,53 +314,69 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
// Cache flushing
|
||||
|
||||
this.cacheFlusher = new Flusher();
|
||||
this.cacheFlusherThread = new Thread(cacheFlusher);
|
||||
this.cacheFlusherThread = new Thread(cacheFlusher, "HRegionServer.cacheFlusher");
|
||||
|
||||
// Check regions to see if they need to be split
|
||||
|
||||
this.splitChecker = new SplitChecker();
|
||||
this.splitCheckerThread = new Thread(splitChecker);
|
||||
this.splitCheckerThread = new Thread(splitChecker, "HRegionServer.splitChecker");
|
||||
|
||||
// Process requests from Master
|
||||
|
||||
this.toDo = new Vector<HMsg>();
|
||||
this.worker = new Worker();
|
||||
this.workerThread = new Thread(worker, "HRegionServer.worker");
|
||||
|
||||
try {
|
||||
|
||||
// Server to handle client requests
|
||||
|
||||
this.server = RPC.getServer(this, address.getBindAddress().toString(),
|
||||
address.getPort(), conf.getInt("hbase.hregionserver.handler.count", 10), false, conf);
|
||||
|
||||
this.address = new HServerAddress(server.getListenerAddress());
|
||||
|
||||
// Local file paths
|
||||
|
||||
this.fs = FileSystem.get(conf);
|
||||
Path newlogdir = new Path(regionDir, "log" + "_" + address.toString());
|
||||
this.oldlogfile = new Path(regionDir, "oldlogfile" + "_" + address.toString());
|
||||
String serverName = this.address.getBindAddress() + "_" + this.address.getPort();
|
||||
Path newlogdir = new Path(regionDir, "log" + "_" + serverName);
|
||||
this.oldlogfile = new Path(regionDir, "oldlogfile" + "_" + serverName);
|
||||
|
||||
// Logging
|
||||
|
||||
this.fs = FileSystem.get(conf);
|
||||
HLog.consolidateOldLog(newlogdir, oldlogfile, fs, conf);
|
||||
this.log = new HLog(fs, newlogdir, conf);
|
||||
this.logRoller = new LogRoller();
|
||||
this.logRollerThread = new Thread(logRoller);
|
||||
this.logRollerThread = new Thread(logRoller, "HRegionServer.logRoller");
|
||||
|
||||
// Remote HMaster
|
||||
|
||||
this.hbaseMaster = (HMasterRegionInterface)
|
||||
RPC.waitForProxy(HMasterRegionInterface.class,
|
||||
HMasterRegionInterface.versionId,
|
||||
new HServerAddress(conf.get(MASTER_DEFAULT_NAME)).getInetSocketAddress(),
|
||||
conf);
|
||||
RPC.waitForProxy(HMasterRegionInterface.class,
|
||||
HMasterRegionInterface.versionID,
|
||||
new HServerAddress(conf.get(MASTER_ADDRESS)).getInetSocketAddress(),
|
||||
conf);
|
||||
|
||||
// Threads
|
||||
|
||||
this.workerThread.start();
|
||||
this.cacheFlusherThread.start();
|
||||
this.splitCheckerThread.start();
|
||||
this.logRollerThread.start();
|
||||
this.leases = new Leases(conf.getLong("hbase.hregionserver.lease.period",
|
||||
3 * 60 * 1000), threadWakeFrequency);
|
||||
3 * 60 * 1000), threadWakeFrequency);
|
||||
|
||||
// Server
|
||||
|
||||
this.server = RPC.getServer(this, address.getBindAddress().toString(),
|
||||
address.getPort(), conf.getInt("hbase.hregionserver.handler.count", 10), false, conf);
|
||||
this.server.start();
|
||||
|
||||
} catch(IOException e) {
|
||||
this.stopRequested = true;
|
||||
throw e;
|
||||
}
|
||||
|
||||
LOG.info("HRegionServer started at: " + address.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -334,7 +386,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
* processing to cease.
|
||||
*/
|
||||
public void stop() throws IOException {
|
||||
if (!stopRequested) {
|
||||
if(! stopRequested) {
|
||||
stopRequested = true;
|
||||
|
||||
closeAllRegions();
|
||||
|
@ -342,11 +394,17 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
fs.close();
|
||||
server.stop();
|
||||
}
|
||||
|
||||
LOG.info("stopping server at: " + address.toString());
|
||||
}
|
||||
|
||||
/** Call join to wait for all the threads to finish */
|
||||
public void join() {
|
||||
try {
|
||||
this.workerThread.join();
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
|
||||
try {
|
||||
this.logRollerThread.join();
|
||||
|
||||
|
@ -366,7 +424,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
|
||||
LOG.info("server stopped at: " + address.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -375,7 +433,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
* load/unload instructions.
|
||||
*/
|
||||
public void run() {
|
||||
while(!stopRequested) {
|
||||
while(! stopRequested) {
|
||||
HServerInfo info = new HServerInfo(address, rand.nextLong());
|
||||
long lastMsg = 0;
|
||||
long waitTime;
|
||||
|
@ -388,18 +446,20 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
} catch(IOException e) {
|
||||
waitTime = msgInterval - (System.currentTimeMillis() - lastMsg);
|
||||
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
if(waitTime > 0) {
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
}
|
||||
continue;
|
||||
}
|
||||
|
||||
// Now ask the master what it wants us to do and tell it what we have done.
|
||||
|
||||
while(!stopRequested) {
|
||||
if ((System.currentTimeMillis() - lastMsg) >= msgInterval) {
|
||||
while(! stopRequested) {
|
||||
if((System.currentTimeMillis() - lastMsg) >= msgInterval) {
|
||||
|
||||
HMsg outboundArray[] = null;
|
||||
synchronized(outboundMsgs) {
|
||||
|
@ -411,10 +471,33 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
HMsg msgs[] = hbaseMaster.regionServerReport(info, outboundArray);
|
||||
lastMsg = System.currentTimeMillis();
|
||||
|
||||
// Process the HMaster's instruction stream
|
||||
// Queue up the HMaster's instruction stream for processing
|
||||
|
||||
if (!processMessages(msgs)) {
|
||||
break;
|
||||
synchronized(toDo) {
|
||||
boolean restartOrStop = false;
|
||||
for(int i = 0; i < msgs.length; i++) {
|
||||
switch(msgs[i].getMsg()) {
|
||||
|
||||
case HMsg.MSG_CALL_SERVER_STARTUP:
|
||||
closeAllRegions();
|
||||
restartOrStop = true;
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGIONSERVER_ALREADY_RUNNING:
|
||||
stop();
|
||||
restartOrStop = true;
|
||||
break;
|
||||
|
||||
default:
|
||||
toDo.add(msgs[i]);
|
||||
}
|
||||
}
|
||||
if(toDo.size() > 0) {
|
||||
toDo.notifyAll();
|
||||
}
|
||||
if(restartOrStop) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
} catch(IOException e) {
|
||||
|
@ -424,55 +507,16 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
waitTime = msgInterval - (System.currentTimeMillis() - lastMsg);
|
||||
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
} catch(InterruptedException iex) {
|
||||
if(waitTime > 0) {
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean processMessages(HMsg[] msgs) throws IOException {
|
||||
for(int i = 0; i < msgs.length; i++) {
|
||||
switch(msgs[i].getMsg()) {
|
||||
|
||||
case HMsg.MSG_REGION_OPEN: // Open a region
|
||||
openRegion(msgs[i].getRegionInfo());
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGION_CLOSE: // Close a region
|
||||
closeRegion(msgs[i].getRegionInfo(), true);
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGION_MERGE: // Merge two regions
|
||||
//TODO ???
|
||||
throw new IOException("TODO: need to figure out merge");
|
||||
//break;
|
||||
|
||||
case HMsg.MSG_CALL_SERVER_STARTUP: // Close regions, restart
|
||||
closeAllRegions();
|
||||
return false;
|
||||
|
||||
case HMsg.MSG_REGIONSERVER_ALREADY_RUNNING: // Go away
|
||||
stop();
|
||||
return false;
|
||||
|
||||
case HMsg.MSG_REGION_CLOSE_WITHOUT_REPORT: // Close a region, don't reply
|
||||
closeRegion(msgs[i].getRegionInfo(), false);
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGION_CLOSE_AND_DELETE:
|
||||
closeAndDeleteRegion(msgs[i].getRegionInfo());
|
||||
break;
|
||||
|
||||
default:
|
||||
throw new IOException("Impossible state during msg processing. Instruction: " + msgs[i]);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Add to the outbound message buffer */
|
||||
private void reportOpen(HRegion region) {
|
||||
synchronized(outboundMsgs) {
|
||||
|
@ -508,9 +552,68 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
// HMaster-given operations
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
private Vector<HMsg> toDo;
|
||||
private Worker worker;
|
||||
private Thread workerThread;
|
||||
private class Worker implements Runnable {
|
||||
public void run() {
|
||||
while(!stopRequested) {
|
||||
HMsg msg = null;
|
||||
synchronized(toDo) {
|
||||
while(toDo.size() == 0) {
|
||||
try {
|
||||
toDo.wait();
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
}
|
||||
msg = toDo.remove(0);
|
||||
}
|
||||
try {
|
||||
switch(msg.getMsg()) {
|
||||
|
||||
case HMsg.MSG_REGION_OPEN: // Open a region
|
||||
openRegion(msg.getRegionInfo());
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGION_CLOSE: // Close a region
|
||||
closeRegion(msg.getRegionInfo(), true);
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGION_MERGE: // Merge two regions
|
||||
//TODO ???
|
||||
throw new IOException("TODO: need to figure out merge");
|
||||
//break;
|
||||
|
||||
case HMsg.MSG_CALL_SERVER_STARTUP: // Close regions, restart
|
||||
closeAllRegions();
|
||||
continue;
|
||||
|
||||
case HMsg.MSG_REGIONSERVER_ALREADY_RUNNING: // Go away
|
||||
stop();
|
||||
continue;
|
||||
|
||||
case HMsg.MSG_REGION_CLOSE_WITHOUT_REPORT: // Close a region, don't reply
|
||||
closeRegion(msg.getRegionInfo(), false);
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGION_CLOSE_AND_DELETE:
|
||||
closeAndDeleteRegion(msg.getRegionInfo());
|
||||
break;
|
||||
|
||||
default:
|
||||
throw new IOException("Impossible state during msg processing. Instruction: " + msg);
|
||||
}
|
||||
} catch(IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void openRegion(HRegionInfo regionInfo) throws IOException {
|
||||
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
HRegion region = new HRegion(regionDir, log, fs, conf, regionInfo, null, oldlogfile);
|
||||
|
||||
|
@ -518,57 +621,57 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
reportOpen(region);
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
private void closeRegion(HRegionInfo info, boolean reportWhenCompleted)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
HRegion region = regions.remove(info.regionName);
|
||||
|
||||
if (region != null) {
|
||||
if(region != null) {
|
||||
region.close();
|
||||
|
||||
if (reportWhenCompleted) {
|
||||
if(reportWhenCompleted) {
|
||||
reportClose(region);
|
||||
}
|
||||
}
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
private void closeAndDeleteRegion(HRegionInfo info) throws IOException {
|
||||
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
HRegion region = regions.remove(info.regionName);
|
||||
|
||||
if (region != null) {
|
||||
if(region != null) {
|
||||
region.closeAndDelete();
|
||||
}
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/** Called either when the master tells us to restart or from stop() */
|
||||
private void closeAllRegions() throws IOException {
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
for(Iterator<HRegion> it = regions.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<HRegion> it = regions.values().iterator(); it.hasNext(); ) {
|
||||
HRegion region = it.next();
|
||||
region.close();
|
||||
}
|
||||
regions.clear();
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -580,24 +683,24 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*
|
||||
* For now, we do not do merging. Splits are driven by the HRegionServer.
|
||||
****************************************************************************/
|
||||
/*
|
||||
private void mergeRegions(Text regionNameA, Text regionNameB) throws IOException {
|
||||
/*
|
||||
private void mergeRegions(Text regionNameA, Text regionNameB) throws IOException {
|
||||
locking.obtainWriteLock();
|
||||
try {
|
||||
HRegion srcA = regions.remove(regionNameA);
|
||||
HRegion srcB = regions.remove(regionNameB);
|
||||
HRegion newRegion = HRegion.closeAndMerge(srcA, srcB);
|
||||
regions.put(newRegion.getRegionName(), newRegion);
|
||||
HRegion srcA = regions.remove(regionNameA);
|
||||
HRegion srcB = regions.remove(regionNameB);
|
||||
HRegion newRegion = HRegion.closeAndMerge(srcA, srcB);
|
||||
regions.put(newRegion.getRegionName(), newRegion);
|
||||
|
||||
reportClose(srcA);
|
||||
reportClose(srcB);
|
||||
reportOpen(newRegion);
|
||||
reportClose(srcA);
|
||||
reportClose(srcB);
|
||||
reportOpen(newRegion);
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
locking.releaseWriteLock();
|
||||
}
|
||||
}
|
||||
*/
|
||||
}
|
||||
*/
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// HRegionInterface
|
||||
|
@ -606,32 +709,21 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
/** Obtain a table descriptor for the given region */
|
||||
public HRegionInfo getRegionInfo(Text regionName) {
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
return null;
|
||||
}
|
||||
return region.getRegionInfo();
|
||||
}
|
||||
|
||||
/** Start a scanner for a given HRegion. */
|
||||
public HScannerInterface openScanner(Text regionName, Text[] cols,
|
||||
Text firstRow) throws IOException {
|
||||
|
||||
HRegion r = getRegion(regionName);
|
||||
if (r == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
return r.getScanner(cols, firstRow);
|
||||
}
|
||||
|
||||
/** Get the indicated row/column */
|
||||
public BytesWritable get(Text regionName, Text row, Text column) throws IOException {
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
byte results[] = region.get(row, column);
|
||||
if (results != null) {
|
||||
if(results != null) {
|
||||
return new BytesWritable(results);
|
||||
}
|
||||
return null;
|
||||
|
@ -639,18 +731,18 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
/** Get multiple versions of the indicated row/col */
|
||||
public BytesWritable[] get(Text regionName, Text row, Text column,
|
||||
int numVersions) throws IOException {
|
||||
int numVersions) throws IOException {
|
||||
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
byte results[][] = region.get(row, column, numVersions);
|
||||
if (results != null) {
|
||||
if(results != null) {
|
||||
BytesWritable realResults[] = new BytesWritable[results.length];
|
||||
for(int i = 0; i < realResults.length; i++) {
|
||||
if (results[i] != null) {
|
||||
if(results[i] != null) {
|
||||
realResults[i] = new BytesWritable(results[i]);
|
||||
}
|
||||
}
|
||||
|
@ -661,18 +753,18 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
/** Get multiple timestamped versions of the indicated row/col */
|
||||
public BytesWritable[] get(Text regionName, Text row, Text column,
|
||||
long timestamp, int numVersions) throws IOException {
|
||||
long timestamp, int numVersions) throws IOException {
|
||||
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
byte results[][] = region.get(row, column, timestamp, numVersions);
|
||||
if (results != null) {
|
||||
if(results != null) {
|
||||
BytesWritable realResults[] = new BytesWritable[results.length];
|
||||
for(int i = 0; i < realResults.length; i++) {
|
||||
if (results[i] != null) {
|
||||
if(results[i] != null) {
|
||||
realResults[i] = new BytesWritable(results[i]);
|
||||
}
|
||||
}
|
||||
|
@ -684,14 +776,14 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
/** Get all the columns (along with their names) for a given row. */
|
||||
public LabelledData[] getRow(Text regionName, Text row) throws IOException {
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
TreeMap<Text, byte[]> map = region.getFull(row);
|
||||
LabelledData result[] = new LabelledData[map.size()];
|
||||
int counter = 0;
|
||||
for(Iterator<Text> it = map.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = map.keySet().iterator(); it.hasNext(); ) {
|
||||
Text colname = it.next();
|
||||
byte val[] = map.get(colname);
|
||||
result[counter++] = new LabelledData(colname, val);
|
||||
|
@ -723,77 +815,77 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
}
|
||||
|
||||
public long startUpdate(Text regionName, long clientid, Text row)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
long lockid = region.startUpdate(row);
|
||||
leases.createLease(new Text(String.valueOf(clientid)),
|
||||
new Text(String.valueOf(lockid)),
|
||||
new RegionListener(region, lockid));
|
||||
new Text(String.valueOf(lockid)),
|
||||
new RegionListener(region, lockid));
|
||||
|
||||
return lockid;
|
||||
}
|
||||
|
||||
/** Add something to the HBase. */
|
||||
public void put(Text regionName, long clientid, long lockid, Text column,
|
||||
BytesWritable val) throws IOException {
|
||||
BytesWritable val) throws IOException {
|
||||
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
leases.renewLease(new Text(String.valueOf(clientid)),
|
||||
new Text(String.valueOf(lockid)));
|
||||
new Text(String.valueOf(lockid)));
|
||||
|
||||
region.put(lockid, column, val.get());
|
||||
}
|
||||
|
||||
/** Remove a cell from the HBase. */
|
||||
public void delete(Text regionName, long clientid, long lockid, Text column)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
leases.renewLease(new Text(String.valueOf(clientid)),
|
||||
new Text(String.valueOf(lockid)));
|
||||
new Text(String.valueOf(lockid)));
|
||||
|
||||
region.delete(lockid, column);
|
||||
}
|
||||
|
||||
/** Abandon the transaction */
|
||||
public void abort(Text regionName, long clientid, long lockid)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
leases.cancelLease(new Text(String.valueOf(clientid)),
|
||||
new Text(String.valueOf(lockid)));
|
||||
new Text(String.valueOf(lockid)));
|
||||
|
||||
region.abort(lockid);
|
||||
}
|
||||
|
||||
/** Confirm the transaction */
|
||||
public void commit(Text regionName, long clientid, long lockid)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
HRegion region = getRegion(regionName);
|
||||
if (region == null) {
|
||||
if(region == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
leases.cancelLease(new Text(String.valueOf(clientid)),
|
||||
new Text(String.valueOf(lockid)));
|
||||
new Text(String.valueOf(lockid)));
|
||||
|
||||
region.commit(lockid);
|
||||
}
|
||||
|
@ -801,18 +893,131 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
/** Don't let the client's lease expire just yet... */
|
||||
public void renewLease(long lockid, long clientid) throws IOException {
|
||||
leases.renewLease(new Text(String.valueOf(clientid)),
|
||||
new Text(String.valueOf(lockid)));
|
||||
new Text(String.valueOf(lockid)));
|
||||
}
|
||||
|
||||
/** Private utility method for safely obtaining an HRegion handle. */
|
||||
private HRegion getRegion(Text regionName) {
|
||||
locking.obtainReadLock();
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
return regions.get(regionName);
|
||||
|
||||
} finally {
|
||||
locking.releaseReadLock();
|
||||
this.locker.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// remote scanner interface
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
private Map<Text, HScannerInterface> scanners;
|
||||
private class ScannerListener extends LeaseListener {
|
||||
private Text scannerName;
|
||||
|
||||
public ScannerListener(Text scannerName) {
|
||||
this.scannerName = scannerName;
|
||||
}
|
||||
|
||||
public void leaseExpired() {
|
||||
HScannerInterface s = scanners.remove(scannerName);
|
||||
if(s != null) {
|
||||
try {
|
||||
s.close();
|
||||
|
||||
} catch(IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Start a scanner for a given HRegion. */
|
||||
public long openScanner(Text regionName, Text[] cols, Text firstRow)
|
||||
throws IOException {
|
||||
|
||||
HRegion r = getRegion(regionName);
|
||||
if(r == null) {
|
||||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
long scannerId = -1L;
|
||||
try {
|
||||
HScannerInterface s = r.getScanner(cols, firstRow);
|
||||
scannerId = rand.nextLong();
|
||||
Text scannerName = new Text(String.valueOf(scannerId));
|
||||
scanners.put(scannerName, s);
|
||||
leases.createLease(scannerName, scannerName, new ScannerListener(scannerName));
|
||||
|
||||
} catch(IOException e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
}
|
||||
return scannerId;
|
||||
}
|
||||
|
||||
public LabelledData[] next(long scannerId, HStoreKey key) throws IOException {
|
||||
|
||||
Text scannerName = new Text(String.valueOf(scannerId));
|
||||
HScannerInterface s = scanners.get(scannerName);
|
||||
if(s == null) {
|
||||
throw new IOException("unknown scanner");
|
||||
}
|
||||
leases.renewLease(scannerName, scannerName);
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
ArrayList<LabelledData> values = new ArrayList<LabelledData>();
|
||||
if(s.next(key, results)) {
|
||||
for(Iterator<Map.Entry<Text, byte[]>> it = results.entrySet().iterator();
|
||||
it.hasNext(); ) {
|
||||
Map.Entry<Text, byte[]> e = it.next();
|
||||
values.add(new LabelledData(e.getKey(), e.getValue()));
|
||||
}
|
||||
}
|
||||
return values.toArray(new LabelledData[values.size()]);
|
||||
}
|
||||
|
||||
public void close(long scannerId) throws IOException {
|
||||
Text scannerName = new Text(String.valueOf(scannerId));
|
||||
HScannerInterface s = scanners.remove(scannerName);
|
||||
if(s == null) {
|
||||
throw new IOException("unknown scanner");
|
||||
}
|
||||
try {
|
||||
s.close();
|
||||
|
||||
} catch(IOException ex) {
|
||||
ex.printStackTrace();
|
||||
}
|
||||
leases.cancelLease(scannerName, scannerName);
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// Main program
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
private static void printUsage() {
|
||||
System.err.println("Usage: java " +
|
||||
"org.apache.hbase.HRegionServer [--bind=hostname:port]");
|
||||
}
|
||||
|
||||
public static void main(String [] args) throws IOException {
|
||||
Configuration conf = new HBaseConfiguration();
|
||||
|
||||
// Process command-line args. TODO: Better cmd-line processing
|
||||
// (but hopefully something not as painful as cli options).
|
||||
for (String cmd: args) {
|
||||
if (cmd.equals("-h") || cmd.startsWith("--h")) {
|
||||
printUsage();
|
||||
return;
|
||||
}
|
||||
|
||||
final String addressArgKey = "--bind=";
|
||||
if (cmd.startsWith(addressArgKey)) {
|
||||
conf.set(REGIONSERVER_ADDRESS,
|
||||
cmd.substring(addressArgKey.length()));
|
||||
}
|
||||
}
|
||||
|
||||
new HRegionServer(conf);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,9 +33,15 @@ public class HServerAddress implements Writable {
|
|||
this.stringValue = null;
|
||||
}
|
||||
|
||||
public HServerAddress(InetSocketAddress address) {
|
||||
this.address = address;
|
||||
this.stringValue = new String(address.getAddress().getHostAddress()
|
||||
+ ":" + address.getPort());
|
||||
}
|
||||
|
||||
public HServerAddress(String hostAndPort) {
|
||||
int colonIndex = hostAndPort.indexOf(':');
|
||||
if (colonIndex < 0) {
|
||||
if(colonIndex < 0) {
|
||||
throw new IllegalArgumentException("Not a host:port pair: " + hostAndPort);
|
||||
}
|
||||
String host = hostAndPort.substring(0, colonIndex);
|
||||
|
@ -80,7 +86,7 @@ public class HServerAddress implements Writable {
|
|||
String bindAddress = in.readUTF();
|
||||
int port = in.readInt();
|
||||
|
||||
if (bindAddress == null || bindAddress.length() == 0) {
|
||||
if(bindAddress == null || bindAddress.length() == 0) {
|
||||
address = null;
|
||||
stringValue = null;
|
||||
|
||||
|
@ -91,7 +97,7 @@ public class HServerAddress implements Writable {
|
|||
}
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
if (address == null) {
|
||||
if(address == null) {
|
||||
out.writeUTF("");
|
||||
out.writeInt(0);
|
||||
|
||||
|
|
|
@ -15,14 +15,26 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.TreeMap;
|
||||
import java.util.Vector;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.*;
|
||||
import org.apache.hadoop.io.*;
|
||||
import org.apache.hadoop.fs.*;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.MapFile;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
/*******************************************************************************
|
||||
* HStore maintains a bunch of data files. It is responsible for maintaining
|
||||
|
@ -53,7 +65,7 @@ public class HStore {
|
|||
Integer compactLock = new Integer(0);
|
||||
Integer flushLock = new Integer(0);
|
||||
|
||||
HLocking locking = new HLocking();
|
||||
ReadWriteLock locker = new ReentrantReadWriteLock();
|
||||
|
||||
TreeMap<Long, MapFile.Reader> maps = new TreeMap<Long, MapFile.Reader>();
|
||||
TreeMap<Long, HStoreFile> mapFiles = new TreeMap<Long, HStoreFile>();
|
||||
|
@ -88,7 +100,7 @@ public class HStore {
|
|||
* will be deleted (by whoever has instantiated the HStore).
|
||||
*/
|
||||
public HStore(Path dir, Text regionName, Text colFamily, int maxVersions,
|
||||
FileSystem fs, Path reconstructionLog, Configuration conf) throws IOException {
|
||||
FileSystem fs, Path reconstructionLog, Configuration conf) throws IOException {
|
||||
|
||||
this.dir = dir;
|
||||
this.regionName = regionName;
|
||||
|
@ -110,7 +122,7 @@ public class HStore {
|
|||
|
||||
this.compactdir = new Path(dir, COMPACTION_DIR);
|
||||
Path curCompactStore = HStoreFile.getHStoreDir(compactdir, regionName, colFamily);
|
||||
if (fs.exists(curCompactStore)) {
|
||||
if(fs.exists(curCompactStore)) {
|
||||
processReadyCompaction();
|
||||
fs.delete(curCompactStore);
|
||||
}
|
||||
|
@ -123,7 +135,7 @@ public class HStore {
|
|||
Vector<HStoreFile> hstoreFiles
|
||||
= HStoreFile.loadHStoreFiles(conf, dir, regionName, colFamily, fs);
|
||||
|
||||
for(Iterator<HStoreFile> it = hstoreFiles.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = hstoreFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
mapFiles.put(hsf.loadInfo(fs), hsf);
|
||||
}
|
||||
|
@ -138,11 +150,11 @@ public class HStore {
|
|||
// contain any updates also contained in the log.
|
||||
|
||||
long maxSeqID = -1;
|
||||
for(Iterator<HStoreFile> it = hstoreFiles.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = hstoreFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
long seqid = hsf.loadInfo(fs);
|
||||
if (seqid > 0) {
|
||||
if (seqid > maxSeqID) {
|
||||
if(seqid > 0) {
|
||||
if(seqid > maxSeqID) {
|
||||
maxSeqID = seqid;
|
||||
}
|
||||
}
|
||||
|
@ -157,7 +169,7 @@ public class HStore {
|
|||
|
||||
LOG.debug("reading reconstructionLog");
|
||||
|
||||
if (reconstructionLog != null && fs.exists(reconstructionLog)) {
|
||||
if(reconstructionLog != null && fs.exists(reconstructionLog)) {
|
||||
long maxSeqIdInLog = -1;
|
||||
TreeMap<HStoreKey, BytesWritable> reconstructedCache
|
||||
= new TreeMap<HStoreKey, BytesWritable>();
|
||||
|
@ -170,18 +182,18 @@ public class HStore {
|
|||
HLogEdit val = new HLogEdit();
|
||||
while(login.next(key, val)) {
|
||||
maxSeqIdInLog = Math.max(maxSeqIdInLog, key.getLogSeqNum());
|
||||
if (key.getLogSeqNum() <= maxSeqID) {
|
||||
if(key.getLogSeqNum() <= maxSeqID) {
|
||||
continue;
|
||||
}
|
||||
reconstructedCache.put(new HStoreKey(key.getRow(), val.getColumn(),
|
||||
val.getTimestamp()), val.getVal());
|
||||
val.getTimestamp()), val.getVal());
|
||||
}
|
||||
|
||||
} finally {
|
||||
login.close();
|
||||
}
|
||||
|
||||
if (reconstructedCache.size() > 0) {
|
||||
if(reconstructedCache.size() > 0) {
|
||||
|
||||
// We create a "virtual flush" at maxSeqIdInLog+1.
|
||||
|
||||
|
@ -195,7 +207,7 @@ public class HStore {
|
|||
// should be "timeless"; that is, it should not have an associated seq-ID,
|
||||
// because all log messages have been reflected in the TreeMaps at this point.
|
||||
|
||||
if (mapFiles.size() >= 1) {
|
||||
if(mapFiles.size() >= 1) {
|
||||
compactHelper(true);
|
||||
}
|
||||
|
||||
|
@ -204,7 +216,7 @@ public class HStore {
|
|||
|
||||
LOG.debug("starting map readers");
|
||||
|
||||
for(Iterator<Long> it = mapFiles.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Long> it = mapFiles.keySet().iterator(); it.hasNext(); ) {
|
||||
Long key = it.next().longValue();
|
||||
HStoreFile hsf = mapFiles.get(key);
|
||||
|
||||
|
@ -218,11 +230,11 @@ public class HStore {
|
|||
|
||||
/** Turn off all the MapFile readers */
|
||||
public void close() throws IOException {
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
LOG.info("closing HStore for " + this.regionName + "/" + this.colFamily);
|
||||
|
||||
try {
|
||||
for(Iterator<MapFile.Reader> it = maps.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<MapFile.Reader> it = maps.values().iterator(); it.hasNext(); ) {
|
||||
MapFile.Reader map = it.next();
|
||||
map.close();
|
||||
}
|
||||
|
@ -232,7 +244,7 @@ public class HStore {
|
|||
LOG.info("HStore closed for " + this.regionName + "/" + this.colFamily);
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -252,13 +264,13 @@ public class HStore {
|
|||
* Return the entire list of HStoreFiles currently used by the HStore.
|
||||
*/
|
||||
public Vector<HStoreFile> flushCache(TreeMap<HStoreKey, BytesWritable> inputCache,
|
||||
long logCacheFlushId) throws IOException {
|
||||
long logCacheFlushId) throws IOException {
|
||||
|
||||
return flushCacheHelper(inputCache, logCacheFlushId, true);
|
||||
}
|
||||
|
||||
Vector<HStoreFile> flushCacheHelper(TreeMap<HStoreKey, BytesWritable> inputCache,
|
||||
long logCacheFlushId, boolean addToAvailableMaps) throws IOException {
|
||||
long logCacheFlushId, boolean addToAvailableMaps) throws IOException {
|
||||
|
||||
synchronized(flushLock) {
|
||||
LOG.debug("flushing HStore " + this.regionName + "/" + this.colFamily);
|
||||
|
@ -270,12 +282,12 @@ public class HStore {
|
|||
|
||||
Path mapfile = flushedFile.getMapFilePath();
|
||||
MapFile.Writer out = new MapFile.Writer(conf, fs, mapfile.toString(),
|
||||
HStoreKey.class, BytesWritable.class);
|
||||
HStoreKey.class, BytesWritable.class);
|
||||
|
||||
try {
|
||||
for(Iterator<HStoreKey> it = inputCache.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreKey> it = inputCache.keySet().iterator(); it.hasNext(); ) {
|
||||
HStoreKey curkey = it.next();
|
||||
if (this.colFamily.equals(HStoreKey.extractFamily(curkey.getColumn()))) {
|
||||
if(this.colFamily.equals(HStoreKey.extractFamily(curkey.getColumn()))) {
|
||||
BytesWritable val = inputCache.get(curkey);
|
||||
out.append(curkey, val);
|
||||
}
|
||||
|
@ -294,8 +306,8 @@ public class HStore {
|
|||
|
||||
// C. Finally, make the new MapFile available.
|
||||
|
||||
if (addToAvailableMaps) {
|
||||
locking.obtainWriteLock();
|
||||
if(addToAvailableMaps) {
|
||||
this.locker.writeLock().lock();
|
||||
|
||||
try {
|
||||
maps.put(logCacheFlushId, new MapFile.Reader(fs, mapfile.toString(), conf));
|
||||
|
@ -303,7 +315,7 @@ public class HStore {
|
|||
LOG.debug("HStore available for " + this.regionName + "/" + this.colFamily);
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
return getAllMapFiles();
|
||||
|
@ -312,7 +324,7 @@ public class HStore {
|
|||
|
||||
public Vector<HStoreFile> getAllMapFiles() {
|
||||
Vector<HStoreFile> flushedFiles = new Vector<HStoreFile>();
|
||||
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
flushedFiles.add(hsf);
|
||||
}
|
||||
|
@ -355,22 +367,22 @@ public class HStore {
|
|||
// Grab a list of files to compact.
|
||||
|
||||
Vector<HStoreFile> toCompactFiles = null;
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
toCompactFiles = new Vector<HStoreFile>(mapFiles.values());
|
||||
|
||||
} finally {
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
|
||||
// Compute the max-sequenceID seen in any of the to-be-compacted TreeMaps
|
||||
|
||||
long maxSeenSeqID = -1;
|
||||
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
long seqid = hsf.loadInfo(fs);
|
||||
if (seqid > 0) {
|
||||
if (seqid > maxSeenSeqID) {
|
||||
if(seqid > 0) {
|
||||
if(seqid > maxSeenSeqID) {
|
||||
maxSeenSeqID = seqid;
|
||||
}
|
||||
}
|
||||
|
@ -380,11 +392,11 @@ public class HStore {
|
|||
HStoreFile compactedOutputFile
|
||||
= new HStoreFile(conf, compactdir, regionName, colFamily, -1);
|
||||
|
||||
if (toCompactFiles.size() == 1) {
|
||||
if(toCompactFiles.size() == 1) {
|
||||
LOG.debug("nothing to compact for " + this.regionName + "/" + this.colFamily);
|
||||
|
||||
HStoreFile hsf = toCompactFiles.elementAt(0);
|
||||
if (hsf.loadInfo(fs) == -1) {
|
||||
if(hsf.loadInfo(fs) == -1) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
@ -392,8 +404,8 @@ public class HStore {
|
|||
// Step through them, writing to the brand-new TreeMap
|
||||
|
||||
MapFile.Writer compactedOut = new MapFile.Writer(conf, fs,
|
||||
compactedOutputFile.getMapFilePath().toString(), HStoreKey.class,
|
||||
BytesWritable.class);
|
||||
compactedOutputFile.getMapFilePath().toString(), HStoreKey.class,
|
||||
BytesWritable.class);
|
||||
|
||||
try {
|
||||
|
||||
|
@ -414,7 +426,7 @@ public class HStore {
|
|||
BytesWritable[] vals = new BytesWritable[toCompactFiles.size()];
|
||||
boolean[] done = new boolean[toCompactFiles.size()];
|
||||
int pos = 0;
|
||||
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
readers[pos] = new MapFile.Reader(fs, hsf.getMapFilePath().toString(), conf);
|
||||
keys[pos] = new HStoreKey();
|
||||
|
@ -431,8 +443,8 @@ public class HStore {
|
|||
int numDone = 0;
|
||||
for(int i = 0; i < readers.length; i++) {
|
||||
readers[i].reset();
|
||||
done[i] = !readers[i].next(keys[i], vals[i]);
|
||||
if (done[i]) {
|
||||
done[i] = ! readers[i].next(keys[i], vals[i]);
|
||||
if(done[i]) {
|
||||
numDone++;
|
||||
}
|
||||
}
|
||||
|
@ -446,15 +458,15 @@ public class HStore {
|
|||
|
||||
int smallestKey = -1;
|
||||
for(int i = 0; i < readers.length; i++) {
|
||||
if (done[i]) {
|
||||
if(done[i]) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (smallestKey < 0) {
|
||||
if(smallestKey < 0) {
|
||||
smallestKey = i;
|
||||
|
||||
} else {
|
||||
if (keys[i].compareTo(keys[smallestKey]) < 0) {
|
||||
if(keys[i].compareTo(keys[smallestKey]) < 0) {
|
||||
smallestKey = i;
|
||||
}
|
||||
}
|
||||
|
@ -463,7 +475,7 @@ public class HStore {
|
|||
// Reflect the current key/val in the output
|
||||
|
||||
HStoreKey sk = keys[smallestKey];
|
||||
if (lastRow.equals(sk.getRow())
|
||||
if(lastRow.equals(sk.getRow())
|
||||
&& lastColumn.equals(sk.getColumn())) {
|
||||
|
||||
timesSeen++;
|
||||
|
@ -472,12 +484,12 @@ public class HStore {
|
|||
timesSeen = 1;
|
||||
}
|
||||
|
||||
if (timesSeen <= maxVersions) {
|
||||
if(timesSeen <= maxVersions) {
|
||||
|
||||
// Keep old versions until we have maxVersions worth.
|
||||
// Then just skip them.
|
||||
|
||||
if (sk.getRow().getLength() != 0
|
||||
if(sk.getRow().getLength() != 0
|
||||
&& sk.getColumn().getLength() != 0) {
|
||||
|
||||
// Only write out objects which have a non-zero length key and value
|
||||
|
@ -499,7 +511,7 @@ public class HStore {
|
|||
// Advance the smallest key. If that reader's all finished, then
|
||||
// mark it as done.
|
||||
|
||||
if (!readers[smallestKey].next(keys[smallestKey], vals[smallestKey])) {
|
||||
if(! readers[smallestKey].next(keys[smallestKey], vals[smallestKey])) {
|
||||
done[smallestKey] = true;
|
||||
readers[smallestKey].close();
|
||||
numDone++;
|
||||
|
@ -516,7 +528,7 @@ public class HStore {
|
|||
|
||||
// Now, write out an HSTORE_LOGINFOFILE for the brand-new TreeMap.
|
||||
|
||||
if ((!deleteSequenceInfo) && maxSeenSeqID >= 0) {
|
||||
if((! deleteSequenceInfo) && maxSeenSeqID >= 0) {
|
||||
compactedOutputFile.writeInfo(fs, maxSeenSeqID);
|
||||
|
||||
} else {
|
||||
|
@ -529,7 +541,7 @@ public class HStore {
|
|||
DataOutputStream out = new DataOutputStream(fs.create(filesToReplace));
|
||||
try {
|
||||
out.writeInt(toCompactFiles.size());
|
||||
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
hsf.write(out);
|
||||
}
|
||||
|
@ -583,11 +595,11 @@ public class HStore {
|
|||
|
||||
// 1. Acquiring the write-lock
|
||||
|
||||
locking.obtainWriteLock();
|
||||
this.locker.writeLock().lock();
|
||||
Path curCompactStore = HStoreFile.getHStoreDir(compactdir, regionName, colFamily);
|
||||
try {
|
||||
Path doneFile = new Path(curCompactStore, COMPACTION_DONE);
|
||||
if (!fs.exists(doneFile)) {
|
||||
if(! fs.exists(doneFile)) {
|
||||
|
||||
// The last execution didn't finish the compaction, so there's nothing
|
||||
// we can do. We'll just have to redo it. Abandon it and return.
|
||||
|
@ -622,18 +634,18 @@ public class HStore {
|
|||
// 3. Unload all the replaced MapFiles.
|
||||
|
||||
Iterator<HStoreFile> it2 = mapFiles.values().iterator();
|
||||
for(Iterator<MapFile.Reader> it = maps.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<MapFile.Reader> it = maps.values().iterator(); it.hasNext(); ) {
|
||||
MapFile.Reader curReader = it.next();
|
||||
HStoreFile curMapFile = it2.next();
|
||||
if (toCompactFiles.contains(curMapFile)) {
|
||||
if(toCompactFiles.contains(curMapFile)) {
|
||||
curReader.close();
|
||||
it.remove();
|
||||
}
|
||||
}
|
||||
|
||||
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext(); ) {
|
||||
HStoreFile curMapFile = it.next();
|
||||
if (toCompactFiles.contains(curMapFile)) {
|
||||
if(toCompactFiles.contains(curMapFile)) {
|
||||
it.remove();
|
||||
}
|
||||
}
|
||||
|
@ -645,7 +657,7 @@ public class HStore {
|
|||
|
||||
// 4. Delete all the old files, no longer needed
|
||||
|
||||
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
fs.delete(hsf.getMapFilePath());
|
||||
fs.delete(hsf.getInfoFilePath());
|
||||
|
@ -683,13 +695,13 @@ public class HStore {
|
|||
|
||||
mapFiles.put(orderVal, finalCompactedFile);
|
||||
maps.put(orderVal, new MapFile.Reader(fs,
|
||||
finalCompactedFile.getMapFilePath().toString(), conf));
|
||||
finalCompactedFile.getMapFilePath().toString(), conf));
|
||||
|
||||
} finally {
|
||||
|
||||
// 7. Releasing the write-lock
|
||||
|
||||
locking.releaseWriteLock();
|
||||
this.locker.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -705,7 +717,7 @@ public class HStore {
|
|||
* The returned object should map column names to byte arrays (byte[]).
|
||||
*/
|
||||
public void getFull(HStoreKey key, TreeMap<Text, byte[]> results) throws IOException {
|
||||
locking.obtainReadLock();
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
MapFile.Reader[] maparray
|
||||
= maps.values().toArray(new MapFile.Reader[maps.size()]);
|
||||
|
@ -720,12 +732,12 @@ public class HStore {
|
|||
|
||||
do {
|
||||
Text readcol = readkey.getColumn();
|
||||
if (results.get(readcol) == null
|
||||
if(results.get(readcol) == null
|
||||
&& key.matchesWithoutColumn(readkey)) {
|
||||
results.put(new Text(readcol), readval.get());
|
||||
readval = new BytesWritable();
|
||||
|
||||
} else if (key.getRow().compareTo(readkey.getRow()) > 0) {
|
||||
} else if(key.getRow().compareTo(readkey.getRow()) > 0) {
|
||||
break;
|
||||
}
|
||||
|
||||
|
@ -734,7 +746,7 @@ public class HStore {
|
|||
}
|
||||
|
||||
} finally {
|
||||
locking.releaseReadLock();
|
||||
this.locker.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -745,12 +757,12 @@ public class HStore {
|
|||
* If 'numVersions' is negative, the method returns all available versions.
|
||||
*/
|
||||
public byte[][] get(HStoreKey key, int numVersions) throws IOException {
|
||||
if (numVersions == 0) {
|
||||
if(numVersions == 0) {
|
||||
throw new IllegalArgumentException("Must request at least one value.");
|
||||
}
|
||||
|
||||
Vector<byte[]> results = new Vector<byte[]>();
|
||||
locking.obtainReadLock();
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
MapFile.Reader[] maparray
|
||||
= maps.values().toArray(new MapFile.Reader[maps.size()]);
|
||||
|
@ -763,12 +775,12 @@ public class HStore {
|
|||
map.reset();
|
||||
HStoreKey readkey = (HStoreKey)map.getClosest(key, readval);
|
||||
|
||||
if (readkey.matchesRowCol(key)) {
|
||||
if(readkey.matchesRowCol(key)) {
|
||||
results.add(readval.get());
|
||||
readval = new BytesWritable();
|
||||
|
||||
while(map.next(readkey, readval) && readkey.matchesRowCol(key)) {
|
||||
if (numVersions > 0 && (results.size() >= numVersions)) {
|
||||
if(numVersions > 0 && (results.size() >= numVersions)) {
|
||||
break;
|
||||
|
||||
} else {
|
||||
|
@ -778,12 +790,12 @@ public class HStore {
|
|||
}
|
||||
}
|
||||
}
|
||||
if (results.size() >= numVersions) {
|
||||
if(results.size() >= numVersions) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (results.size() == 0) {
|
||||
if(results.size() == 0) {
|
||||
return null;
|
||||
|
||||
} else {
|
||||
|
@ -791,7 +803,7 @@ public class HStore {
|
|||
}
|
||||
|
||||
} finally {
|
||||
locking.releaseReadLock();
|
||||
this.locker.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -804,18 +816,23 @@ public class HStore {
|
|||
*/
|
||||
public long getLargestFileSize(Text midKey) throws IOException {
|
||||
long maxSize = 0L;
|
||||
if (this.mapFiles.size() <= 0) {
|
||||
return maxSize;
|
||||
}
|
||||
|
||||
|
||||
long mapIndex = 0L;
|
||||
|
||||
// Iterate through all the MapFiles
|
||||
|
||||
for(Iterator<Map.Entry<Long, HStoreFile>> it = mapFiles.entrySet().iterator();
|
||||
it.hasNext();) {
|
||||
it.hasNext(); ) {
|
||||
|
||||
Map.Entry<Long, HStoreFile> e = it.next();
|
||||
HStoreFile curHSF = e.getValue();
|
||||
long size = fs.getLength(new Path(curHSF.getMapFilePath(), MapFile.DATA_FILE_NAME));
|
||||
|
||||
if (size > maxSize) { // This is the largest one so far
|
||||
if(size > maxSize) { // This is the largest one so far
|
||||
maxSize = size;
|
||||
mapIndex = e.getKey();
|
||||
}
|
||||
|
@ -850,7 +867,7 @@ public class HStore {
|
|||
* These should be closed after the user is done with them.
|
||||
*/
|
||||
public HScannerInterface getScanner(long timestamp, Text targetCols[],
|
||||
Text firstRow) throws IOException {
|
||||
Text firstRow) throws IOException {
|
||||
|
||||
return new HStoreScanner(timestamp, targetCols, firstRow);
|
||||
}
|
||||
|
@ -867,11 +884,11 @@ public class HStore {
|
|||
public HStoreScanner(long timestamp, Text targetCols[], Text firstRow) throws IOException {
|
||||
super(timestamp, targetCols);
|
||||
|
||||
locking.obtainReadLock();
|
||||
locker.readLock().lock();
|
||||
try {
|
||||
this.readers = new MapFile.Reader[mapFiles.size()];
|
||||
int i = 0;
|
||||
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext(); ) {
|
||||
HStoreFile curHSF = it.next();
|
||||
readers[i++] = new MapFile.Reader(fs, curHSF.getMapFilePath().toString(), conf);
|
||||
}
|
||||
|
@ -885,14 +902,14 @@ public class HStore {
|
|||
keys[i] = new HStoreKey();
|
||||
vals[i] = new BytesWritable();
|
||||
|
||||
if (firstRow.getLength() != 0) {
|
||||
if (findFirstRow(i, firstRow)) {
|
||||
if(firstRow.getLength() != 0) {
|
||||
if(findFirstRow(i, firstRow)) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
while(getNext(i)) {
|
||||
if (columnMatch(i)) {
|
||||
if(columnMatch(i)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -915,7 +932,7 @@ public class HStore {
|
|||
HStoreKey firstKey
|
||||
= (HStoreKey)readers[i].getClosest(new HStoreKey(firstRow), vals[i]);
|
||||
|
||||
if (firstKey == null) {
|
||||
if(firstKey == null) {
|
||||
|
||||
// Didn't find it. Close the scanner and return TRUE
|
||||
|
||||
|
@ -935,7 +952,7 @@ public class HStore {
|
|||
* @return - true if there is more data available
|
||||
*/
|
||||
boolean getNext(int i) throws IOException {
|
||||
if (!readers[i].next(keys[i], vals[i])) {
|
||||
if(! readers[i].next(keys[i], vals[i])) {
|
||||
closeSubScanner(i);
|
||||
return false;
|
||||
}
|
||||
|
@ -945,7 +962,7 @@ public class HStore {
|
|||
/** Close down the indicated reader. */
|
||||
void closeSubScanner(int i) throws IOException {
|
||||
try {
|
||||
if (readers[i] != null) {
|
||||
if(readers[i] != null) {
|
||||
readers[i].close();
|
||||
}
|
||||
|
||||
|
@ -958,16 +975,16 @@ public class HStore {
|
|||
|
||||
/** Shut it down! */
|
||||
public void close() throws IOException {
|
||||
if (!scannerClosed) {
|
||||
if(! scannerClosed) {
|
||||
try {
|
||||
for(int i = 0; i < readers.length; i++) {
|
||||
if (readers[i] != null) {
|
||||
if(readers[i] != null) {
|
||||
readers[i].close();
|
||||
}
|
||||
}
|
||||
|
||||
} finally {
|
||||
locking.releaseReadLock();
|
||||
locker.readLock().unlock();
|
||||
scannerClosed = true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -61,7 +61,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
}
|
||||
|
||||
public HStoreFile(Configuration conf, Path dir, Text regionName,
|
||||
Text colFamily, long fileId) {
|
||||
Text colFamily, long fileId) {
|
||||
|
||||
this.conf = conf;
|
||||
this.dir = dir;
|
||||
|
@ -92,12 +92,12 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
|
||||
public Path getMapFilePath() {
|
||||
return new Path(HStoreFile.getMapDir(dir, regionName, colFamily),
|
||||
HSTORE_DATFILE_PREFIX + fileId);
|
||||
HSTORE_DATFILE_PREFIX + fileId);
|
||||
}
|
||||
|
||||
public Path getInfoFilePath() {
|
||||
return new Path(HStoreFile.getInfoDir(dir, regionName, colFamily),
|
||||
HSTORE_INFOFILE_PREFIX + fileId);
|
||||
HSTORE_INFOFILE_PREFIX + fileId);
|
||||
}
|
||||
|
||||
// Static methods to build partial paths to internal directories. Useful for
|
||||
|
@ -105,17 +105,17 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
|
||||
public static Path getMapDir(Path dir, Text regionName, Text colFamily) {
|
||||
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
|
||||
new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
|
||||
new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
|
||||
}
|
||||
|
||||
public static Path getInfoDir(Path dir, Text regionName, Text colFamily) {
|
||||
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
|
||||
new Path(colFamily.toString(), HSTORE_INFO_DIR)));
|
||||
new Path(colFamily.toString(), HSTORE_INFO_DIR)));
|
||||
}
|
||||
|
||||
public static Path getHStoreDir(Path dir, Text regionName, Text colFamily) {
|
||||
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
|
||||
colFamily.toString()));
|
||||
colFamily.toString()));
|
||||
}
|
||||
|
||||
public static Path getHRegionDir(Path dir, Text regionName) {
|
||||
|
@ -127,7 +127,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
* filesystem if the file already exists.
|
||||
*/
|
||||
static HStoreFile obtainNewHStoreFile(Configuration conf, Path dir,
|
||||
Text regionName, Text colFamily, FileSystem fs) throws IOException {
|
||||
Text regionName, Text colFamily, FileSystem fs) throws IOException {
|
||||
|
||||
Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily);
|
||||
long fileId = Math.abs(rand.nextLong());
|
||||
|
@ -149,7 +149,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
* If only one exists, we'll delete it.
|
||||
*/
|
||||
static Vector<HStoreFile> loadHStoreFiles(Configuration conf, Path dir,
|
||||
Text regionName, Text colFamily, FileSystem fs) throws IOException {
|
||||
Text regionName, Text colFamily, FileSystem fs) throws IOException {
|
||||
|
||||
Vector<HStoreFile> results = new Vector<HStoreFile>();
|
||||
Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily);
|
||||
|
@ -158,13 +158,13 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
for(int i = 0; i < datfiles.length; i++) {
|
||||
String name = datfiles[i].getName();
|
||||
|
||||
if (name.startsWith(HSTORE_DATFILE_PREFIX)) {
|
||||
if(name.startsWith(HSTORE_DATFILE_PREFIX)) {
|
||||
Long fileId = Long.parseLong(name.substring(HSTORE_DATFILE_PREFIX.length()));
|
||||
HStoreFile curfile = new HStoreFile(conf, dir, regionName, colFamily, fileId);
|
||||
Path mapfile = curfile.getMapFilePath();
|
||||
Path infofile = curfile.getInfoFilePath();
|
||||
|
||||
if (fs.exists(infofile)) {
|
||||
if(fs.exists(infofile)) {
|
||||
results.add(curfile);
|
||||
|
||||
} else {
|
||||
|
@ -178,12 +178,12 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
for(int i = 0; i < infofiles.length; i++) {
|
||||
String name = infofiles[i].getName();
|
||||
|
||||
if (name.startsWith(HSTORE_INFOFILE_PREFIX)) {
|
||||
if(name.startsWith(HSTORE_INFOFILE_PREFIX)) {
|
||||
long fileId = Long.parseLong(name.substring(HSTORE_INFOFILE_PREFIX.length()));
|
||||
HStoreFile curfile = new HStoreFile(conf, dir, regionName, colFamily, fileId);
|
||||
Path mapfile = curfile.getMapFilePath();
|
||||
|
||||
if (!fs.exists(mapfile)) {
|
||||
if(! fs.exists(mapfile)) {
|
||||
fs.delete(curfile.getInfoFilePath());
|
||||
}
|
||||
}
|
||||
|
@ -200,18 +200,18 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
* brand-new HRegions.
|
||||
*/
|
||||
public void splitStoreFile(Text midKey, HStoreFile dstA, HStoreFile dstB,
|
||||
FileSystem fs, Configuration conf) throws IOException {
|
||||
FileSystem fs, Configuration conf) throws IOException {
|
||||
|
||||
// Copy the appropriate tuples to one MapFile or the other.
|
||||
|
||||
MapFile.Reader in = new MapFile.Reader(fs, getMapFilePath().toString(), conf);
|
||||
try {
|
||||
MapFile.Writer outA = new MapFile.Writer(conf, fs,
|
||||
dstA.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class);
|
||||
dstA.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class);
|
||||
|
||||
try {
|
||||
MapFile.Writer outB = new MapFile.Writer(conf, fs,
|
||||
dstB.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class);
|
||||
dstB.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class);
|
||||
|
||||
try {
|
||||
HStoreKey readkey = new HStoreKey();
|
||||
|
@ -220,7 +220,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
while(in.next(readkey, readval)) {
|
||||
Text key = readkey.getRow();
|
||||
|
||||
if (key.compareTo(midKey) < 0) {
|
||||
if(key.compareTo(midKey) < 0) {
|
||||
outA.append(readkey, readval);
|
||||
|
||||
} else {
|
||||
|
@ -252,15 +252,15 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
* We are merging multiple regions into a single new one.
|
||||
*/
|
||||
public void mergeStoreFiles(Vector<HStoreFile> srcFiles, FileSystem fs,
|
||||
Configuration conf) throws IOException {
|
||||
Configuration conf) throws IOException {
|
||||
|
||||
// Copy all the source MapFile tuples into this HSF's MapFile
|
||||
|
||||
MapFile.Writer out = new MapFile.Writer(conf, fs, getMapFilePath().toString(),
|
||||
HStoreKey.class, BytesWritable.class);
|
||||
HStoreKey.class, BytesWritable.class);
|
||||
|
||||
try {
|
||||
for(Iterator<HStoreFile> it = srcFiles.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = srcFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile src = it.next();
|
||||
MapFile.Reader in = new MapFile.Reader(fs, src.getMapFilePath().toString(), conf);
|
||||
|
||||
|
@ -283,11 +283,11 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
// Build a unified InfoFile from the source InfoFiles.
|
||||
|
||||
long unifiedSeqId = -1;
|
||||
for(Iterator<HStoreFile> it = srcFiles.iterator(); it.hasNext();) {
|
||||
for(Iterator<HStoreFile> it = srcFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
long curSeqId = hsf.loadInfo(fs);
|
||||
|
||||
if (curSeqId > unifiedSeqId) {
|
||||
if(curSeqId > unifiedSeqId) {
|
||||
unifiedSeqId = curSeqId;
|
||||
}
|
||||
}
|
||||
|
@ -301,7 +301,7 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
|
||||
try {
|
||||
byte flag = in.readByte();
|
||||
if (flag == INFO_SEQ_NUM) {
|
||||
if(flag == INFO_SEQ_NUM) {
|
||||
return in.readLong();
|
||||
|
||||
} else {
|
||||
|
@ -352,17 +352,17 @@ public class HStoreFile implements HConstants, WritableComparable {
|
|||
public int compareTo(Object o) {
|
||||
HStoreFile other = (HStoreFile) o;
|
||||
int result = this.dir.compareTo(other.dir);
|
||||
if (result == 0) {
|
||||
if(result == 0) {
|
||||
this.regionName.compareTo(other.regionName);
|
||||
}
|
||||
if (result == 0) {
|
||||
if(result == 0) {
|
||||
result = this.colFamily.compareTo(other.colFamily);
|
||||
}
|
||||
if (result == 0) {
|
||||
if (this.fileId < other.fileId) {
|
||||
if(result == 0) {
|
||||
if(this.fileId < other.fileId) {
|
||||
result = -1;
|
||||
|
||||
} else if (this.fileId > other.fileId) {
|
||||
} else if(this.fileId > other.fileId) {
|
||||
result = 1;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,7 +29,7 @@ public class HStoreKey implements WritableComparable {
|
|||
public static Text extractFamily(Text col) throws IOException {
|
||||
String column = col.toString();
|
||||
int colpos = column.indexOf(":");
|
||||
if (colpos < 0) {
|
||||
if(colpos < 0) {
|
||||
throw new IllegalArgumentException("Illegal column name has no family indicator: " + column);
|
||||
}
|
||||
return new Text(column.substring(0, colpos));
|
||||
|
@ -93,8 +93,13 @@ public class HStoreKey implements WritableComparable {
|
|||
return timestamp;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param other Key to compare against. Compares row and column.
|
||||
* @return True if same row and column.
|
||||
* @see {@link #matchesWithoutColumn(HStoreKey)}
|
||||
*/
|
||||
public boolean matchesRowCol(HStoreKey other) {
|
||||
if (this.row.compareTo(other.row) == 0 &&
|
||||
if(this.row.compareTo(other.row) == 0 &&
|
||||
this.column.compareTo(other.column) == 0) {
|
||||
return true;
|
||||
|
||||
|
@ -103,8 +108,15 @@ public class HStoreKey implements WritableComparable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param other Key to copmare against. Compares row and
|
||||
* timestamp.
|
||||
* @return True if same row and timestamp is greater than
|
||||
* <code>other</code>
|
||||
* @see {@link #matchesRowCol(HStoreKey)}
|
||||
*/
|
||||
public boolean matchesWithoutColumn(HStoreKey other) {
|
||||
if ((this.row.compareTo(other.row) == 0) &&
|
||||
if((this.row.compareTo(other.row) == 0) &&
|
||||
(this.timestamp >= other.getTimestamp())) {
|
||||
return true;
|
||||
|
||||
|
@ -124,14 +136,14 @@ public class HStoreKey implements WritableComparable {
|
|||
public int compareTo(Object o) {
|
||||
HStoreKey other = (HStoreKey) o;
|
||||
int result = this.row.compareTo(other.row);
|
||||
if (result == 0) {
|
||||
if(result == 0) {
|
||||
result = this.column.compareTo(other.column);
|
||||
|
||||
if (result == 0) {
|
||||
if (this.timestamp < other.timestamp) {
|
||||
if(result == 0) {
|
||||
if(this.timestamp < other.timestamp) {
|
||||
result = 1;
|
||||
|
||||
} else if (this.timestamp > other.timestamp) {
|
||||
} else if(this.timestamp > other.timestamp) {
|
||||
result = -1;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,7 +54,7 @@ public class HTableDescriptor implements WritableComparable {
|
|||
|
||||
/** Do we contain a given column? */
|
||||
public boolean hasFamily(Text family) {
|
||||
if (families.contains(family)) {
|
||||
if(families.contains(family)) {
|
||||
return true;
|
||||
|
||||
} else {
|
||||
|
@ -75,7 +75,7 @@ public class HTableDescriptor implements WritableComparable {
|
|||
name.write(out);
|
||||
out.writeInt(maxVersions);
|
||||
out.writeInt(families.size());
|
||||
for(Iterator<Text> it = families.iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = families.iterator(); it.hasNext(); ) {
|
||||
it.next().write(out);
|
||||
}
|
||||
}
|
||||
|
@ -99,21 +99,21 @@ public class HTableDescriptor implements WritableComparable {
|
|||
public int compareTo(Object o) {
|
||||
HTableDescriptor htd = (HTableDescriptor) o;
|
||||
int result = name.compareTo(htd.name);
|
||||
if (result == 0) {
|
||||
if(result == 0) {
|
||||
result = maxVersions - htd.maxVersions;
|
||||
}
|
||||
|
||||
if (result == 0) {
|
||||
if(result == 0) {
|
||||
result = families.size() - htd.families.size();
|
||||
}
|
||||
|
||||
if (result == 0) {
|
||||
if(result == 0) {
|
||||
Iterator<Text> it2 = htd.families.iterator();
|
||||
for(Iterator<Text> it = families.iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = families.iterator(); it.hasNext(); ) {
|
||||
Text family1 = it.next();
|
||||
Text family2 = it2.next();
|
||||
result = family1.compareTo(family2);
|
||||
if (result != 0) {
|
||||
if(result != 0) {
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,7 @@ public class LabelledData implements Writable {
|
|||
}
|
||||
|
||||
public LabelledData(Text label, byte[] data) {
|
||||
this.label.set(label);
|
||||
this.label = new Text(label);
|
||||
this.data = new BytesWritable(data);
|
||||
}
|
||||
|
||||
|
@ -40,7 +40,7 @@ public class LabelledData implements Writable {
|
|||
return label;
|
||||
}
|
||||
|
||||
public BytesWritable getDat() {
|
||||
public BytesWritable getData() {
|
||||
return data;
|
||||
}
|
||||
|
||||
|
|
|
@ -77,7 +77,7 @@ public class Leases {
|
|||
synchronized(sortedLeases) {
|
||||
Lease lease = new Lease(holderId, resourceId, listener);
|
||||
Text leaseId = lease.getLeaseId();
|
||||
if (leases.get(leaseId) != null) {
|
||||
if(leases.get(leaseId) != null) {
|
||||
throw new IOException("Impossible state for createLease(): Lease for holderId " + holderId + " and resourceId " + resourceId + " is still held.");
|
||||
}
|
||||
leases.put(leaseId, lease);
|
||||
|
@ -92,7 +92,7 @@ public class Leases {
|
|||
synchronized(sortedLeases) {
|
||||
Text leaseId = createLeaseId(holderId, resourceId);
|
||||
Lease lease = leases.get(leaseId);
|
||||
if (lease == null) {
|
||||
if(lease == null) {
|
||||
|
||||
// It's possible that someone tries to renew the lease, but
|
||||
// it just expired a moment ago. So fail.
|
||||
|
@ -113,7 +113,7 @@ public class Leases {
|
|||
synchronized(sortedLeases) {
|
||||
Text leaseId = createLeaseId(holderId, resourceId);
|
||||
Lease lease = leases.get(leaseId);
|
||||
if (lease == null) {
|
||||
if(lease == null) {
|
||||
|
||||
// It's possible that someone tries to renew the lease, but
|
||||
// it just expired a moment ago. So fail.
|
||||
|
@ -137,9 +137,9 @@ public class Leases {
|
|||
synchronized(sortedLeases) {
|
||||
Lease top;
|
||||
while((sortedLeases.size() > 0)
|
||||
&& ((top = sortedLeases.first()) != null)) {
|
||||
&& ((top = sortedLeases.first()) != null)) {
|
||||
|
||||
if (top.shouldExpire()) {
|
||||
if(top.shouldExpire()) {
|
||||
leases.remove(top.getLeaseId());
|
||||
sortedLeases.remove(top);
|
||||
|
||||
|
@ -205,10 +205,10 @@ public class Leases {
|
|||
|
||||
public int compareTo(Object o) {
|
||||
Lease other = (Lease) o;
|
||||
if (this.lastUpdate < other.lastUpdate) {
|
||||
if(this.lastUpdate < other.lastUpdate) {
|
||||
return -1;
|
||||
|
||||
} else if (this.lastUpdate > other.lastUpdate) {
|
||||
} else if(this.lastUpdate > other.lastUpdate) {
|
||||
return 1;
|
||||
|
||||
} else {
|
||||
|
|
|
@ -0,0 +1,28 @@
|
|||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class LockException extends IOException {
|
||||
public LockException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public LockException(String s) {
|
||||
super(s);
|
||||
}
|
||||
}
|
|
@ -29,27 +29,27 @@ public class Environment {
|
|||
String value = null;
|
||||
|
||||
value = System.getenv("DEBUGGING");
|
||||
if (value != null && value.equalsIgnoreCase("TRUE")) {
|
||||
if(value != null && value.equalsIgnoreCase("TRUE")) {
|
||||
debugging = true;
|
||||
}
|
||||
|
||||
value = System.getenv("LOGGING_LEVEL");
|
||||
if (value != null && value.length() != 0) {
|
||||
if (value.equalsIgnoreCase("ALL")) {
|
||||
if(value != null && value.length() != 0) {
|
||||
if(value.equalsIgnoreCase("ALL")) {
|
||||
logLevel = Level.ALL;
|
||||
} else if (value.equalsIgnoreCase("DEBUG")) {
|
||||
} else if(value.equalsIgnoreCase("DEBUG")) {
|
||||
logLevel = Level.DEBUG;
|
||||
} else if (value.equalsIgnoreCase("ERROR")) {
|
||||
} else if(value.equalsIgnoreCase("ERROR")) {
|
||||
logLevel = Level.ERROR;
|
||||
} else if (value.equalsIgnoreCase("FATAL")) {
|
||||
} else if(value.equalsIgnoreCase("FATAL")) {
|
||||
logLevel = Level.FATAL;
|
||||
} else if (value.equalsIgnoreCase("INFO")) {
|
||||
} else if(value.equalsIgnoreCase("INFO")) {
|
||||
logLevel = Level.INFO;
|
||||
} else if (value.equalsIgnoreCase("OFF")) {
|
||||
} else if(value.equalsIgnoreCase("OFF")) {
|
||||
logLevel = Level.OFF;
|
||||
} else if (value.equalsIgnoreCase("TRACE")) {
|
||||
} else if(value.equalsIgnoreCase("TRACE")) {
|
||||
logLevel = Level.TRACE;
|
||||
} else if (value.equalsIgnoreCase("WARN")) {
|
||||
} else if(value.equalsIgnoreCase("WARN")) {
|
||||
logLevel = Level.WARN;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,298 @@
|
|||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.File;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.dfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
* This class creates a single process HBase cluster for junit testing.
|
||||
* One thread is created for each server.
|
||||
*/
|
||||
public class MiniHBaseCluster implements HConstants {
|
||||
private Configuration conf;
|
||||
private MiniDFSCluster cluster;
|
||||
private FileSystem fs;
|
||||
private Path parentdir;
|
||||
private HMasterRunner master;
|
||||
private Thread masterThread;
|
||||
private HRegionServerRunner[] regionServers;
|
||||
private Thread[] regionThreads;
|
||||
|
||||
public MiniHBaseCluster(Configuration conf, int nRegionNodes) {
|
||||
this.conf = conf;
|
||||
|
||||
try {
|
||||
try {
|
||||
if(System.getProperty("test.build.data") == null) {
|
||||
File testDir = new File(new File("").getAbsolutePath(),
|
||||
"build/contrib/hbase/test");
|
||||
|
||||
String dir = testDir.getAbsolutePath();
|
||||
System.out.println(dir);
|
||||
System.setProperty("test.build.data", dir);
|
||||
}
|
||||
|
||||
// To run using configured filesystem, comment out this
|
||||
// line below that starts up the MiniDFSCluster.
|
||||
this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
|
||||
this.fs = FileSystem.get(conf);
|
||||
this.parentdir =
|
||||
new Path(conf.get(HREGION_DIR, DEFAULT_HREGION_DIR));
|
||||
fs.mkdirs(parentdir);
|
||||
|
||||
} catch(Throwable e) {
|
||||
System.err.println("Mini DFS cluster failed to start");
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
}
|
||||
|
||||
if(this.conf.get(MASTER_ADDRESS) == null) {
|
||||
this.conf.set(MASTER_ADDRESS, "localhost:0");
|
||||
}
|
||||
|
||||
// Create the master
|
||||
|
||||
this.master = new HMasterRunner();
|
||||
this.masterThread = new Thread(master, "HMaster");
|
||||
|
||||
// Start up the master
|
||||
|
||||
masterThread.start();
|
||||
while(! master.isCrashed() && ! master.isInitialized()) {
|
||||
try {
|
||||
System.err.println("Waiting for HMaster to initialize...");
|
||||
Thread.sleep(1000);
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
if(master.isCrashed()) {
|
||||
throw new RuntimeException("HMaster crashed");
|
||||
}
|
||||
}
|
||||
|
||||
// Set the master's port for the HRegionServers
|
||||
|
||||
this.conf.set(MASTER_ADDRESS, master.getHMasterAddress().toString());
|
||||
|
||||
// Start the HRegionServers
|
||||
|
||||
if(this.conf.get(REGIONSERVER_ADDRESS) == null) {
|
||||
this.conf.set(REGIONSERVER_ADDRESS, "localhost:0");
|
||||
}
|
||||
|
||||
startRegionServers(this.conf, nRegionNodes);
|
||||
|
||||
// Wait for things to get started
|
||||
|
||||
while(! master.isCrashed() && ! master.isUp()) {
|
||||
try {
|
||||
System.err.println("Waiting for Mini HBase cluster to start...");
|
||||
Thread.sleep(1000);
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
if(master.isCrashed()) {
|
||||
throw new RuntimeException("HMaster crashed");
|
||||
}
|
||||
}
|
||||
|
||||
} catch(Throwable e) {
|
||||
|
||||
// Delete all DFS files
|
||||
|
||||
deleteFile(new File(System.getProperty("test.build.data"), "dfs"));
|
||||
|
||||
throw new RuntimeException("Mini HBase cluster did not start");
|
||||
}
|
||||
}
|
||||
|
||||
private void startRegionServers(Configuration conf, int nRegionNodes) {
|
||||
this.regionServers = new HRegionServerRunner[nRegionNodes];
|
||||
this.regionThreads = new Thread[nRegionNodes];
|
||||
|
||||
for(int i = 0; i < nRegionNodes; i++) {
|
||||
regionServers[i] = new HRegionServerRunner(conf);
|
||||
regionThreads[i] = new Thread(regionServers[i], "HRegionServer-" + i);
|
||||
regionThreads[i].start();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the rpc address actually used by the master server, because the
|
||||
* supplied port is not necessarily the actual port used.
|
||||
*/
|
||||
public HServerAddress getHMasterAddress() {
|
||||
return master.getHMasterAddress();
|
||||
}
|
||||
|
||||
/** Shut down the HBase cluster */
|
||||
public void shutdown() {
|
||||
System.out.println("Shutting down the HBase Cluster");
|
||||
for(int i = 0; i < regionServers.length; i++) {
|
||||
regionServers[i].shutdown();
|
||||
}
|
||||
master.shutdown();
|
||||
|
||||
for(int i = 0; i < regionServers.length; i++) {
|
||||
try {
|
||||
regionThreads[i].join();
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
}
|
||||
try {
|
||||
masterThread.join();
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
|
||||
System.out.println("Shutting down Mini DFS cluster");
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
// Delete all DFS files
|
||||
|
||||
deleteFile(new File(System.getProperty("test.build.data"), "dfs"));
|
||||
|
||||
}
|
||||
|
||||
private void deleteFile(File f) {
|
||||
if(f.isDirectory()) {
|
||||
File[] children = f.listFiles();
|
||||
for(int i = 0; i < children.length; i++) {
|
||||
deleteFile(children[i]);
|
||||
}
|
||||
}
|
||||
f.delete();
|
||||
}
|
||||
|
||||
private class HMasterRunner implements Runnable {
|
||||
private HMaster master = null;
|
||||
private volatile boolean isInitialized = false;
|
||||
private boolean isCrashed = false;
|
||||
private boolean isRunning = true;
|
||||
|
||||
public HServerAddress getHMasterAddress() {
|
||||
return master.getMasterAddress();
|
||||
}
|
||||
|
||||
public synchronized boolean isInitialized() {
|
||||
return isInitialized;
|
||||
}
|
||||
|
||||
public synchronized boolean isCrashed() {
|
||||
return isCrashed;
|
||||
}
|
||||
|
||||
public boolean isUp() {
|
||||
if(master == null) {
|
||||
return false;
|
||||
}
|
||||
synchronized(this) {
|
||||
return isInitialized;
|
||||
}
|
||||
}
|
||||
|
||||
/** Create the HMaster and run it */
|
||||
public void run() {
|
||||
try {
|
||||
synchronized(this) {
|
||||
if(isRunning) {
|
||||
master = new HMaster(conf);
|
||||
}
|
||||
isInitialized = true;
|
||||
}
|
||||
} catch(Throwable e) {
|
||||
shutdown();
|
||||
System.err.println("HMaster crashed:");
|
||||
e.printStackTrace();
|
||||
synchronized(this) {
|
||||
isCrashed = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Shut down the HMaster and wait for it to finish */
|
||||
public synchronized void shutdown() {
|
||||
isRunning = false;
|
||||
if(master != null) {
|
||||
try {
|
||||
master.stop();
|
||||
|
||||
} catch(IOException e) {
|
||||
System.err.println("Master crashed during stop");
|
||||
e.printStackTrace();
|
||||
|
||||
} finally {
|
||||
master.join();
|
||||
master = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private class HRegionServerRunner implements Runnable {
|
||||
private HRegionServer server = null;
|
||||
private boolean isRunning = true;
|
||||
private Configuration conf;
|
||||
|
||||
public HRegionServerRunner(Configuration conf) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
/** Start up the HRegionServer */
|
||||
public void run() {
|
||||
try {
|
||||
synchronized(this) {
|
||||
if(isRunning) {
|
||||
server = new HRegionServer(conf);
|
||||
}
|
||||
}
|
||||
server.run();
|
||||
|
||||
} catch(Throwable e) {
|
||||
shutdown();
|
||||
System.err.println("HRegionServer crashed:");
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
/** Shut down the HRegionServer */
|
||||
public synchronized void shutdown() {
|
||||
isRunning = false;
|
||||
if(server != null) {
|
||||
try {
|
||||
server.stop();
|
||||
|
||||
} catch(IOException e) {
|
||||
System.err.println("HRegionServer crashed during stop");
|
||||
e.printStackTrace();
|
||||
|
||||
} finally {
|
||||
server.join();
|
||||
server = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,191 @@
|
|||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HMemcache.Snapshot;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
public class TestHMemcache extends TestCase {
|
||||
private final Logger LOG =
|
||||
Logger.getLogger(this.getClass().getName());
|
||||
|
||||
private HMemcache hmemcache;
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
private static final int ROW_COUNT = 3;
|
||||
|
||||
private static final int COLUMNS_COUNT = 3;
|
||||
|
||||
private static final String COLUMN_FAMILY = "column";
|
||||
|
||||
protected void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
this.hmemcache = new HMemcache();
|
||||
|
||||
// Set up a configuration that has configuration for a file
|
||||
// filesystem implementation.
|
||||
this.conf = new HBaseConfiguration();
|
||||
// The test hadoop-site.xml doesn't have a default file fs
|
||||
// implementation. Remove below when gets added.
|
||||
this.conf.set("fs.file.impl",
|
||||
"org.apache.hadoop.fs.LocalFileSystem");
|
||||
}
|
||||
|
||||
protected void tearDown() throws Exception {
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
private Text getRowName(final int index) {
|
||||
return new Text("row" + Integer.toString(index));
|
||||
}
|
||||
|
||||
private Text getColumnName(final int rowIndex,
|
||||
final int colIndex) {
|
||||
return new Text(COLUMN_FAMILY + ":" +
|
||||
Integer.toString(rowIndex) + ";" +
|
||||
Integer.toString(colIndex));
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds {@link #ROW_COUNT} rows and {@link #COLUMNS_COUNT}
|
||||
* @param hmc Instance to add rows to.
|
||||
*/
|
||||
private void addRows(final HMemcache hmc) {
|
||||
for (int i = 0; i < ROW_COUNT; i++) {
|
||||
TreeMap<Text, byte[]> columns = new TreeMap<Text, byte[]>();
|
||||
for (int ii = 0; ii < COLUMNS_COUNT; ii++) {
|
||||
Text k = getColumnName(i, ii);
|
||||
columns.put(k, k.toString().getBytes());
|
||||
}
|
||||
hmc.add(getRowName(i), columns, System.currentTimeMillis());
|
||||
}
|
||||
}
|
||||
|
||||
private HLog getLogfile() throws IOException {
|
||||
// Create a log file.
|
||||
Path testDir = new Path(conf.get("hadoop.tmp.dir", System
|
||||
.getProperty("java.tmp.dir")), "hbase");
|
||||
Path logFile = new Path(testDir, this.getName());
|
||||
FileSystem fs = testDir.getFileSystem(conf);
|
||||
// Cleanup any old log file.
|
||||
if (fs.exists(logFile)) {
|
||||
fs.delete(logFile);
|
||||
}
|
||||
return new HLog(fs, logFile, this.conf);
|
||||
}
|
||||
|
||||
private Snapshot runSnapshot(final HMemcache hmc, final HLog log)
|
||||
throws IOException {
|
||||
// Save off old state.
|
||||
int oldHistorySize = hmc.history.size();
|
||||
TreeMap<HStoreKey, BytesWritable> oldMemcache = hmc.memcache;
|
||||
// Run snapshot.
|
||||
Snapshot s = hmc.snapshotMemcacheForLog(log);
|
||||
// Make some assertions about what just happened.
|
||||
assertEquals("Snapshot equals old memcache", hmc.snapshot,
|
||||
oldMemcache);
|
||||
assertEquals("Returned snapshot holds old memcache",
|
||||
s.memcacheSnapshot, oldMemcache);
|
||||
assertEquals("History has been incremented",
|
||||
oldHistorySize + 1, hmc.history.size());
|
||||
assertEquals("History holds old snapshot",
|
||||
hmc.history.get(oldHistorySize), oldMemcache);
|
||||
return s;
|
||||
}
|
||||
|
||||
public void testSnapshotting() throws IOException {
|
||||
final int snapshotCount = 5;
|
||||
final Text tableName = new Text(getName());
|
||||
HLog log = getLogfile();
|
||||
try {
|
||||
// Add some rows, run a snapshot. Do it a few times.
|
||||
for (int i = 0; i < snapshotCount; i++) {
|
||||
addRows(this.hmemcache);
|
||||
Snapshot s = runSnapshot(this.hmemcache, log);
|
||||
log.completeCacheFlush(new Text(Integer.toString(i)),
|
||||
tableName, s.sequenceId);
|
||||
// Clean up snapshot now we are done with it.
|
||||
this.hmemcache.deleteSnapshot();
|
||||
}
|
||||
log.close();
|
||||
} finally {
|
||||
log.dir.getFileSystem(this.conf).delete(log.dir);
|
||||
}
|
||||
}
|
||||
|
||||
private void isExpectedRow(final int rowIndex,
|
||||
TreeMap<Text, byte[]> row) {
|
||||
int i = 0;
|
||||
for (Text colname: row.keySet()) {
|
||||
String expectedColname =
|
||||
getColumnName(rowIndex, i++).toString();
|
||||
String colnameStr = colname.toString();
|
||||
assertEquals("Column name", colnameStr, expectedColname);
|
||||
// Value is column name as bytes. Usually result is
|
||||
// 100 bytes in size at least. This is the default size
|
||||
// for BytesWriteable. For comparison, comvert bytes to
|
||||
// String and trim to remove trailing null bytes.
|
||||
String colvalueStr =
|
||||
new String(row.get(colname)).trim();
|
||||
assertEquals("Content", colnameStr, colvalueStr);
|
||||
}
|
||||
}
|
||||
|
||||
public void testGetFull() throws IOException {
|
||||
addRows(this.hmemcache);
|
||||
for (int i = 0; i < ROW_COUNT; i++) {
|
||||
HStoreKey hsk = new HStoreKey(getRowName(i));
|
||||
TreeMap<Text, byte[]> all = this.hmemcache.getFull(hsk);
|
||||
isExpectedRow(i, all);
|
||||
}
|
||||
}
|
||||
|
||||
public void testScanner() throws IOException {
|
||||
addRows(this.hmemcache);
|
||||
long timestamp = System.currentTimeMillis();
|
||||
Text [] cols = new Text[COLUMNS_COUNT * ROW_COUNT];
|
||||
for (int i = 0; i < ROW_COUNT; i++) {
|
||||
for (int ii = 0; ii < COLUMNS_COUNT; ii++) {
|
||||
cols[(ii + (i * COLUMNS_COUNT))] = getColumnName(i, ii);
|
||||
}
|
||||
}
|
||||
HScannerInterface scanner =
|
||||
this.hmemcache.getScanner(timestamp, cols, new Text());
|
||||
HStoreKey key = new HStoreKey();
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
for (int i = 0; scanner.next(key, results); i++) {
|
||||
assertTrue("Row name",
|
||||
key.toString().startsWith(getRowName(i).toString()));
|
||||
assertEquals("Count of columns", COLUMNS_COUNT,
|
||||
results.size());
|
||||
isExpectedRow(i, results);
|
||||
// Clear out set. Otherwise row results accumulate.
|
||||
results.clear();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,7 +21,10 @@ import junit.framework.TestSuite;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Enumeration;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -30,6 +33,9 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
import org.apache.log4j.Appender;
|
||||
import org.apache.log4j.ConsoleAppender;
|
||||
import org.apache.log4j.Layout;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.PatternLayout;
|
||||
|
@ -41,6 +47,7 @@ import org.apache.log4j.PatternLayout;
|
|||
* HRegions or in the HBaseMaster, so only basic testing is possible.
|
||||
*/
|
||||
public class TestHRegion extends TestCase {
|
||||
private Logger LOG = Logger.getLogger(this.getClass().getName());
|
||||
|
||||
/** Constructor */
|
||||
public TestHRegion(String name) {
|
||||
|
@ -51,6 +58,8 @@ public class TestHRegion extends TestCase {
|
|||
public static Test suite() {
|
||||
TestSuite suite = new TestSuite();
|
||||
suite.addTest(new TestHRegion("testSetup"));
|
||||
suite.addTest(new TestHRegion("testLocks"));
|
||||
suite.addTest(new TestHRegion("testBadPuts"));
|
||||
suite.addTest(new TestHRegion("testBasic"));
|
||||
suite.addTest(new TestHRegion("testScan"));
|
||||
suite.addTest(new TestHRegion("testBatchWrite"));
|
||||
|
@ -61,7 +70,7 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
|
||||
|
||||
private static final int FIRST_ROW = 0;
|
||||
private static final int FIRST_ROW = 1;
|
||||
private static final int N_ROWS = 1000000;
|
||||
private static final int NUM_VALS = 1000;
|
||||
private static final Text CONTENTS_BASIC = new Text("contents:basic");
|
||||
|
@ -88,28 +97,42 @@ public class TestHRegion extends TestCase {
|
|||
|
||||
// Set up environment, start mini cluster, etc.
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testSetup() throws IOException {
|
||||
try {
|
||||
if (System.getProperty("test.build.data") == null) {
|
||||
if(System.getProperty("test.build.data") == null) {
|
||||
String dir = new File(new File("").getAbsolutePath(), "build/contrib/hbase/test").getAbsolutePath();
|
||||
System.out.println(dir);
|
||||
System.setProperty("test.build.data", dir);
|
||||
}
|
||||
conf = new Configuration();
|
||||
conf = new HBaseConfiguration();
|
||||
|
||||
Environment.getenv();
|
||||
if (Environment.debugging) {
|
||||
if(Environment.debugging) {
|
||||
Logger rootLogger = Logger.getRootLogger();
|
||||
rootLogger.setLevel(Level.WARN);
|
||||
|
||||
ConsoleAppender consoleAppender = null;
|
||||
for(Enumeration<Appender> e = (Enumeration<Appender>)rootLogger.getAllAppenders();
|
||||
e.hasMoreElements();) {
|
||||
|
||||
PatternLayout consoleLayout
|
||||
= (PatternLayout)rootLogger.getAppender("console").getLayout();
|
||||
consoleLayout.setConversionPattern("%d %-5p [%t] %l: %m%n");
|
||||
|
||||
Appender a = e.nextElement();
|
||||
if(a instanceof ConsoleAppender) {
|
||||
consoleAppender = (ConsoleAppender)a;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if(consoleAppender != null) {
|
||||
Layout layout = consoleAppender.getLayout();
|
||||
if(layout instanceof PatternLayout) {
|
||||
PatternLayout consoleLayout = (PatternLayout)layout;
|
||||
consoleLayout.setConversionPattern("%d %-5p [%t] %l: %m%n");
|
||||
}
|
||||
}
|
||||
Logger.getLogger("org.apache.hadoop.hbase").setLevel(Environment.logLevel);
|
||||
}
|
||||
|
||||
cluster = new MiniDFSCluster(conf, 2, true, null);
|
||||
cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
|
||||
fs = cluster.getFileSystem();
|
||||
parentdir = new Path("/hbase");
|
||||
fs.mkdirs(parentdir);
|
||||
|
@ -118,10 +141,10 @@ public class TestHRegion extends TestCase {
|
|||
|
||||
log = new HLog(fs, newlogdir, conf);
|
||||
desc = new HTableDescriptor("test", 3);
|
||||
desc.addFamily(new Text("contents"));
|
||||
desc.addFamily(new Text("anchor"));
|
||||
desc.addFamily(new Text("contents:"));
|
||||
desc.addFamily(new Text("anchor:"));
|
||||
region = new HRegion(parentdir, log, fs, conf,
|
||||
new HRegionInfo(1, desc, null, null), null, oldlogfile);
|
||||
new HRegionInfo(1, desc, null, null), null, oldlogfile);
|
||||
|
||||
} catch(IOException e) {
|
||||
failures = true;
|
||||
|
@ -133,26 +156,39 @@ public class TestHRegion extends TestCase {
|
|||
// Test basic functionality. Writes to contents:basic and anchor:anchornum-*
|
||||
|
||||
public void testBasic() throws IOException {
|
||||
if (!initialized) {
|
||||
if(!initialized) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
try {
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
// Write out a bunch of values
|
||||
|
||||
for (int k = 0; k < NUM_VALS; k++) {
|
||||
for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
|
||||
long writeid = region.startUpdate(new Text("row_" + k));
|
||||
region.put(writeid, CONTENTS_BASIC, (CONTENTSTR + k).getBytes());
|
||||
region.put(writeid, new Text(ANCHORNUM + k), (ANCHORSTR + k).getBytes());
|
||||
region.commit(writeid);
|
||||
}
|
||||
System.out.println("Write " + NUM_VALS + " rows. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// Flush cache
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
region.flushcache(false);
|
||||
|
||||
System.out.println("Cache flush elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// Read them back in
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
Text collabel = null;
|
||||
for (int k = 0; k < NUM_VALS; k++) {
|
||||
for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
|
||||
Text rowlabel = new Text("row_" + k);
|
||||
|
||||
byte bodydata[] = region.get(rowlabel, CONTENTS_BASIC);
|
||||
|
@ -160,44 +196,127 @@ public class TestHRegion extends TestCase {
|
|||
String bodystr = new String(bodydata).toString().trim();
|
||||
String teststr = CONTENTSTR + k;
|
||||
assertEquals("Incorrect value for key: (" + rowlabel + "," + CONTENTS_BASIC
|
||||
+ "), expected: '" + teststr + "' got: '" + bodystr + "'",
|
||||
bodystr, teststr);
|
||||
+ "), expected: '" + teststr + "' got: '" + bodystr + "'",
|
||||
bodystr, teststr);
|
||||
collabel = new Text(ANCHORNUM + k);
|
||||
bodydata = region.get(rowlabel, collabel);
|
||||
bodystr = new String(bodydata).toString().trim();
|
||||
teststr = ANCHORSTR + k;
|
||||
assertEquals("Incorrect value for key: (" + rowlabel + "," + collabel
|
||||
+ "), expected: '" + teststr + "' got: '" + bodystr + "'",
|
||||
bodystr, teststr);
|
||||
/*
|
||||
// Check to make sure that null values are actually null
|
||||
for (int j = 0; j < Math.min(15, NUM_VALS); j++) {
|
||||
if (k != j) {
|
||||
collabel = new Text(ANCHORNUM + j);
|
||||
byte results[] = region.get(rowlabel, collabel);
|
||||
if (results != null) {
|
||||
throw new IOException("Found incorrect value at [" + rowlabel + ", " + collabel + "] == " + new String(results).toString().trim());
|
||||
}
|
||||
}
|
||||
}
|
||||
*/
|
||||
+ "), expected: '" + teststr + "' got: '" + bodystr + "'",
|
||||
bodystr, teststr);
|
||||
}
|
||||
|
||||
System.out.println("Read " + NUM_VALS + " rows. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
} catch(IOException e) {
|
||||
failures = true;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
public void testBadPuts() throws IOException {
|
||||
if(!initialized) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
// Try put with bad lockid.
|
||||
boolean exceptionThrown = false;
|
||||
try {
|
||||
region.put(-1, CONTENTS_BASIC, "bad input".getBytes());
|
||||
} catch (LockException e) {
|
||||
exceptionThrown = true;
|
||||
}
|
||||
assertTrue("Bad lock id", exceptionThrown);
|
||||
|
||||
// Try column name not registered in the table.
|
||||
exceptionThrown = false;
|
||||
long lockid = -1;
|
||||
try {
|
||||
lockid = region.startUpdate(new Text("Some old key"));
|
||||
String unregisteredColName = "FamilyGroup:FamilyLabel";
|
||||
region.put(lockid, new Text(unregisteredColName),
|
||||
unregisteredColName.getBytes());
|
||||
} catch (IOException e) {
|
||||
exceptionThrown = true;
|
||||
} finally {
|
||||
if (lockid != -1) {
|
||||
region.abort(lockid);
|
||||
}
|
||||
}
|
||||
assertTrue("Bad family", exceptionThrown);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test getting and releasing locks.
|
||||
*/
|
||||
public void testLocks() {
|
||||
final int threadCount = 10;
|
||||
final int lockCount = 10;
|
||||
|
||||
List<Thread>threads = new ArrayList<Thread>(threadCount);
|
||||
for (int i = 0; i < threadCount; i++) {
|
||||
threads.add(new Thread(Integer.toString(i)) {
|
||||
public void run() {
|
||||
long [] lockids = new long[lockCount];
|
||||
// Get locks.
|
||||
for (int i = 0; i < lockCount; i++) {
|
||||
try {
|
||||
Text rowid = new Text(Integer.toString(i));
|
||||
lockids[i] = region.obtainLock(rowid);
|
||||
rowid.equals(region.getRowFromLock(lockids[i]));
|
||||
LOG.debug(getName() + " locked " + rowid.toString());
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
LOG.debug(getName() + " set " +
|
||||
Integer.toString(lockCount) + " locks");
|
||||
|
||||
// Abort outstanding locks.
|
||||
for (int i = lockCount - 1; i >= 0; i--) {
|
||||
try {
|
||||
region.abort(lockids[i]);
|
||||
LOG.debug(getName() + " unlocked " +
|
||||
Integer.toString(i));
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
LOG.debug(getName() + " released " +
|
||||
Integer.toString(lockCount) + " locks");
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
// Startup all our threads.
|
||||
for (Thread t : threads) {
|
||||
t.start();
|
||||
}
|
||||
|
||||
// Now wait around till all are done.
|
||||
for (Thread t: threads) {
|
||||
while (t.isAlive()) {
|
||||
try {
|
||||
Thread.sleep(1);
|
||||
} catch (InterruptedException e) {
|
||||
// Go around again.
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Test scanners. Writes contents:firstcol and anchor:secondcol
|
||||
|
||||
public void testScan() throws IOException {
|
||||
if (!initialized) {
|
||||
if(!initialized) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
Text cols[] = new Text[] {
|
||||
CONTENTS_FIRSTCOL,
|
||||
ANCHOR_SECONDCOL
|
||||
CONTENTS_FIRSTCOL,
|
||||
ANCHOR_SECONDCOL
|
||||
};
|
||||
|
||||
// Test the Scanner!!!
|
||||
|
@ -207,6 +326,9 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
|
||||
// 1. Insert a bunch of values
|
||||
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
for(int k = 0; k < vals1.length / 2; k++) {
|
||||
String kLabel = String.format("%1$03d", k);
|
||||
|
||||
|
@ -217,7 +339,13 @@ public class TestHRegion extends TestCase {
|
|||
numInserted += 2;
|
||||
}
|
||||
|
||||
// 2. Scan
|
||||
System.out.println("Write " + (vals1.length / 2) + " elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// 2. Scan from cache
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
HScannerInterface s = region.getScanner(cols, new Text());
|
||||
int numFetched = 0;
|
||||
try {
|
||||
|
@ -225,16 +353,16 @@ public class TestHRegion extends TestCase {
|
|||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
|
||||
for(int j = 0; j < cols.length; j++) {
|
||||
if (col.compareTo(cols[j]) == 0) {
|
||||
if(col.compareTo(cols[j]) == 0) {
|
||||
assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
|
||||
+ ", Value for " + col + " should be: " + k
|
||||
+ ", but was fetched as: " + curval, k, curval);
|
||||
+ ", Value for " + col + " should be: " + k
|
||||
+ ", but was fetched as: " + curval, k, curval);
|
||||
numFetched++;
|
||||
}
|
||||
}
|
||||
|
@ -247,10 +375,23 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||
|
||||
System.out.println("Scanned " + (vals1.length / 2)
|
||||
+ " rows from cache. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// 3. Flush to disk
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
region.flushcache(false);
|
||||
|
||||
// 4. Scan
|
||||
System.out.println("Cache flush elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// 4. Scan from disk
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
s = region.getScanner(cols, new Text());
|
||||
numFetched = 0;
|
||||
try {
|
||||
|
@ -258,16 +399,16 @@ public class TestHRegion extends TestCase {
|
|||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
|
||||
for(int j = 0; j < cols.length; j++) {
|
||||
if (col.compareTo(cols[j]) == 0) {
|
||||
if(col.compareTo(cols[j]) == 0) {
|
||||
assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
|
||||
+ ", Value for " + col + " should be: " + k
|
||||
+ ", but was fetched as: " + curval, k, curval);
|
||||
+ ", Value for " + col + " should be: " + k
|
||||
+ ", but was fetched as: " + curval, k, curval);
|
||||
numFetched++;
|
||||
}
|
||||
}
|
||||
|
@ -280,7 +421,14 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||
|
||||
System.out.println("Scanned " + (vals1.length / 2)
|
||||
+ " rows from disk. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// 5. Insert more values
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
for(int k = vals1.length/2; k < vals1.length; k++) {
|
||||
String kLabel = String.format("%1$03d", k);
|
||||
|
||||
|
@ -291,7 +439,13 @@ public class TestHRegion extends TestCase {
|
|||
numInserted += 2;
|
||||
}
|
||||
|
||||
// 6. Scan
|
||||
System.out.println("Write " + (vals1.length / 2) + " rows. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// 6. Scan from cache and disk
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
s = region.getScanner(cols, new Text());
|
||||
numFetched = 0;
|
||||
try {
|
||||
|
@ -299,16 +453,16 @@ public class TestHRegion extends TestCase {
|
|||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
|
||||
for(int j = 0; j < cols.length; j++) {
|
||||
if (col.compareTo(cols[j]) == 0) {
|
||||
if(col.compareTo(cols[j]) == 0) {
|
||||
assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
|
||||
+ ", Value for " + col + " should be: " + k
|
||||
+ ", but was fetched as: " + curval, k, curval);
|
||||
+ ", Value for " + col + " should be: " + k
|
||||
+ ", but was fetched as: " + curval, k, curval);
|
||||
numFetched++;
|
||||
}
|
||||
}
|
||||
|
@ -321,10 +475,23 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||
|
||||
System.out.println("Scanned " + vals1.length
|
||||
+ " rows from cache and disk. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// 7. Flush to disk
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
region.flushcache(false);
|
||||
|
||||
// 8. Scan
|
||||
System.out.println("Cache flush elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// 8. Scan from disk
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
s = region.getScanner(cols, new Text());
|
||||
numFetched = 0;
|
||||
try {
|
||||
|
@ -332,7 +499,7 @@ public class TestHRegion extends TestCase {
|
|||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
|
@ -340,7 +507,7 @@ public class TestHRegion extends TestCase {
|
|||
for (int j = 0; j < cols.length; j++) {
|
||||
if (col.compareTo(cols[j]) == 0) {
|
||||
assertEquals("Value for " + col + " should be: " + k
|
||||
+ ", but was fetched as: " + curval, curval, k);
|
||||
+ ", but was fetched as: " + curval, curval, k);
|
||||
numFetched++;
|
||||
}
|
||||
}
|
||||
|
@ -353,8 +520,14 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||
|
||||
System.out.println("Scanned " + vals1.length
|
||||
+ " rows from disk. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// 9. Scan with a starting point
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
s = region.getScanner(cols, new Text("row_vals1_500"));
|
||||
numFetched = 0;
|
||||
try {
|
||||
|
@ -362,7 +535,7 @@ public class TestHRegion extends TestCase {
|
|||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
int k = 500;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
|
@ -370,7 +543,7 @@ public class TestHRegion extends TestCase {
|
|||
for (int j = 0; j < cols.length; j++) {
|
||||
if (col.compareTo(cols[j]) == 0) {
|
||||
assertEquals("Value for " + col + " should be: " + k
|
||||
+ ", but was fetched as: " + curval, curval, k);
|
||||
+ ", but was fetched as: " + curval, curval, k);
|
||||
numFetched++;
|
||||
}
|
||||
}
|
||||
|
@ -383,6 +556,9 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
assertEquals("Should have fetched " + (numInserted / 2) + " values, but fetched " + numFetched, (numInserted / 2), numFetched);
|
||||
|
||||
System.out.println("Scanned " + (numFetched / 2)
|
||||
+ " rows from disk with specified start point. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
}
|
||||
|
||||
// Do a large number of writes. Disabled if not debugging because it takes a
|
||||
|
@ -390,10 +566,10 @@ public class TestHRegion extends TestCase {
|
|||
// Creates contents:body
|
||||
|
||||
public void testBatchWrite() throws IOException {
|
||||
if (!initialized || failures) {
|
||||
if(!initialized || failures) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
if (!Environment.debugging) {
|
||||
if(! Environment.debugging) {
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -406,7 +582,7 @@ public class TestHRegion extends TestCase {
|
|||
// 1M writes
|
||||
|
||||
int valsize = 1000;
|
||||
for (int k = FIRST_ROW; k < N_ROWS; k++) {
|
||||
for (int k = FIRST_ROW; k <= N_ROWS; k++) {
|
||||
// Come up with a random 1000-byte string
|
||||
String randstr1 = "" + System.currentTimeMillis();
|
||||
StringBuffer buf1 = new StringBuffer("val_" + k + "__");
|
||||
|
@ -437,7 +613,7 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
}
|
||||
long startCompact = System.currentTimeMillis();
|
||||
if (region.compactStores()) {
|
||||
if(region.compactStores()) {
|
||||
totalCompact = System.currentTimeMillis() - startCompact;
|
||||
System.out.println("Region compacted - elapsedTime: " + (totalCompact / 1000.0));
|
||||
|
||||
|
@ -457,7 +633,8 @@ public class TestHRegion extends TestCase {
|
|||
System.out.println("Total time, rows/second: " + (N_ROWS / (totalElapsed / 1000.0)));
|
||||
System.out.println("Adjusted time (not including flush, compact, or log): " + ((totalElapsed - totalFlush - totalCompact - totalLog) / 1000.0));
|
||||
System.out.println("Adjusted time, rows/second: " + (N_ROWS / ((totalElapsed - totalFlush - totalCompact - totalLog) / 1000.0)));
|
||||
|
||||
System.out.println();
|
||||
|
||||
} catch(IOException e) {
|
||||
failures = true;
|
||||
throw e;
|
||||
|
@ -467,14 +644,14 @@ public class TestHRegion extends TestCase {
|
|||
// NOTE: This test depends on testBatchWrite succeeding
|
||||
|
||||
public void testSplitAndMerge() throws IOException {
|
||||
if (!initialized || failures) {
|
||||
if(!initialized || failures) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
try {
|
||||
Text midKey = new Text();
|
||||
|
||||
if (region.needsSplit(midKey)) {
|
||||
if(region.needsSplit(midKey)) {
|
||||
System.out.println("Needs split");
|
||||
}
|
||||
|
||||
|
@ -482,15 +659,28 @@ public class TestHRegion extends TestCase {
|
|||
|
||||
Text midkey = new Text("row_" + (Environment.debugging ? (N_ROWS / 2) : (NUM_VALS/2)));
|
||||
Path oldRegionPath = region.getRegionDir();
|
||||
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
HRegion subregions[] = region.closeAndSplit(midkey);
|
||||
|
||||
System.out.println("Split region elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
assertEquals("Number of subregions", subregions.length, 2);
|
||||
|
||||
// Now merge it back together
|
||||
|
||||
Path oldRegion1 = subregions[0].getRegionDir();
|
||||
Path oldRegion2 = subregions[1].getRegionDir();
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
region = HRegion.closeAndMerge(subregions[0], subregions[1]);
|
||||
|
||||
System.out.println("Merge regions elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
fs.delete(oldRegionPath);
|
||||
fs.delete(oldRegion1);
|
||||
fs.delete(oldRegion2);
|
||||
|
@ -504,17 +694,19 @@ public class TestHRegion extends TestCase {
|
|||
// This test verifies that everything is still there after splitting and merging
|
||||
|
||||
public void testRead() throws IOException {
|
||||
if (!initialized || failures) {
|
||||
if(!initialized || failures) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
// First verify the data written by testBasic()
|
||||
|
||||
Text[] cols = new Text[] {
|
||||
new Text(ANCHORNUM + "[0-9]+"),
|
||||
new Text(CONTENTS_BASIC)
|
||||
new Text(ANCHORNUM + "[0-9]+"),
|
||||
new Text(CONTENTS_BASIC)
|
||||
};
|
||||
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
HScannerInterface s = region.getScanner(cols, new Text());
|
||||
|
||||
try {
|
||||
|
@ -525,23 +717,23 @@ public class TestHRegion extends TestCase {
|
|||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
String curval = new String(val).trim();
|
||||
|
||||
if (col.compareTo(CONTENTS_BASIC) == 0) {
|
||||
if(col.compareTo(CONTENTS_BASIC) == 0) {
|
||||
assertTrue("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
|
||||
+ ", Value for " + col + " should start with: " + CONTENTSTR
|
||||
+ ", but was fetched as: " + curval,
|
||||
curval.startsWith(CONTENTSTR));
|
||||
+ ", Value for " + col + " should start with: " + CONTENTSTR
|
||||
+ ", but was fetched as: " + curval,
|
||||
curval.startsWith(CONTENTSTR));
|
||||
contentsFetched++;
|
||||
|
||||
} else if (col.toString().startsWith(ANCHORNUM)) {
|
||||
} else if(col.toString().startsWith(ANCHORNUM)) {
|
||||
assertTrue("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
|
||||
+ ", Value for " + col + " should start with: " + ANCHORSTR
|
||||
+ ", but was fetched as: " + curval,
|
||||
curval.startsWith(ANCHORSTR));
|
||||
+ ", Value for " + col + " should start with: " + ANCHORSTR
|
||||
+ ", but was fetched as: " + curval,
|
||||
curval.startsWith(ANCHORSTR));
|
||||
anchorFetched++;
|
||||
|
||||
} else {
|
||||
|
@ -554,6 +746,10 @@ public class TestHRegion extends TestCase {
|
|||
assertEquals("Expected " + NUM_VALS + " " + CONTENTS_BASIC + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched);
|
||||
assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched);
|
||||
|
||||
System.out.println("Scanned " + NUM_VALS
|
||||
+ " rows from disk. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
} finally {
|
||||
s.close();
|
||||
}
|
||||
|
@ -561,9 +757,11 @@ public class TestHRegion extends TestCase {
|
|||
// Verify testScan data
|
||||
|
||||
cols = new Text[] {
|
||||
CONTENTS_FIRSTCOL,
|
||||
ANCHOR_SECONDCOL
|
||||
CONTENTS_FIRSTCOL,
|
||||
ANCHOR_SECONDCOL
|
||||
};
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
s = region.getScanner(cols, new Text());
|
||||
try {
|
||||
|
@ -572,7 +770,7 @@ public class TestHRegion extends TestCase {
|
|||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
|
@ -580,7 +778,7 @@ public class TestHRegion extends TestCase {
|
|||
for (int j = 0; j < cols.length; j++) {
|
||||
if (col.compareTo(cols[j]) == 0) {
|
||||
assertEquals("Value for " + col + " should be: " + k
|
||||
+ ", but was fetched as: " + curval, curval, k);
|
||||
+ ", but was fetched as: " + curval, curval, k);
|
||||
numFetched++;
|
||||
}
|
||||
}
|
||||
|
@ -590,13 +788,19 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||
|
||||
System.out.println("Scanned " + (numFetched / 2)
|
||||
+ " rows from disk. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
} finally {
|
||||
s.close();
|
||||
}
|
||||
|
||||
// Verify testBatchWrite data
|
||||
|
||||
if (Environment.debugging) {
|
||||
if(Environment.debugging) {
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
s = region.getScanner(new Text[] { CONTENTS_BODY }, new Text());
|
||||
try {
|
||||
int numFetched = 0;
|
||||
|
@ -604,7 +808,7 @@ public class TestHRegion extends TestCase {
|
|||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
|
||||
|
@ -617,6 +821,10 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
assertEquals("Inserted " + N_ROWS + " values, but fetched " + numFetched, N_ROWS, numFetched);
|
||||
|
||||
System.out.println("Scanned " + N_ROWS
|
||||
+ " rows from disk. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
} finally {
|
||||
s.close();
|
||||
}
|
||||
|
@ -625,9 +833,11 @@ public class TestHRegion extends TestCase {
|
|||
// Test a scanner which only specifies the column family name
|
||||
|
||||
cols = new Text[] {
|
||||
new Text("anchor:")
|
||||
new Text("anchor:")
|
||||
};
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
s = region.getScanner(cols, new Text());
|
||||
|
||||
try {
|
||||
|
@ -635,7 +845,7 @@ public class TestHRegion extends TestCase {
|
|||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext();) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
it.next();
|
||||
fetched++;
|
||||
}
|
||||
|
@ -643,6 +853,10 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
assertEquals("Inserted " + (NUM_VALS + numInserted/2) + " values, but fetched " + fetched, (NUM_VALS + numInserted/2), fetched);
|
||||
|
||||
System.out.println("Scanned " + fetched
|
||||
+ " rows from disk. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
} finally {
|
||||
s.close();
|
||||
}
|
||||
|
@ -650,7 +864,7 @@ public class TestHRegion extends TestCase {
|
|||
|
||||
|
||||
private static void deleteFile(File f) {
|
||||
if (f.isDirectory()) {
|
||||
if(f.isDirectory()) {
|
||||
File[] children = f.listFiles();
|
||||
for(int i = 0; i < children.length; i++) {
|
||||
deleteFile(children[i]);
|
||||
|
@ -660,7 +874,7 @@ public class TestHRegion extends TestCase {
|
|||
}
|
||||
|
||||
public void testCleanup() throws IOException {
|
||||
if (!initialized) {
|
||||
if(!initialized) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
|
@ -672,5 +886,5 @@ public class TestHRegion extends TestCase {
|
|||
|
||||
deleteFile(new File(System.getProperty("test.build.data"), "dfs"));
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue