HBASE-1304 - New client server implementation of how gets and puts are handled. -- Thanks to jgray,holstad,stack,rawson

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@782178 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Ryan Rawson 2009-06-06 01:26:21 +00:00
parent 039ec8fb82
commit 6af4292630
208 changed files with 19828 additions and 14377 deletions

View File

@ -311,7 +311,6 @@ module HBase
arg[IN_MEMORY]? JBoolean.valueOf(arg[IN_MEMORY]): HColumnDescriptor::DEFAULT_IN_MEMORY,
arg[HColumnDescriptor::BLOCKCACHE]? JBoolean.valueOf(arg[HColumnDescriptor::BLOCKCACHE]): HColumnDescriptor::DEFAULT_BLOCKCACHE,
arg[HColumnDescriptor::BLOCKSIZE]? JInteger.valueOf(arg[HColumnDescriptor::BLOCKSIZE]): HColumnDescriptor::DEFAULT_BLOCKSIZE,
arg[HColumnDescriptor::LENGTH]? JInteger.new(arg[HColumnDescriptor::LENGTH]): HColumnDescriptor::DEFAULT_LENGTH,
arg[HColumnDescriptor::TTL]? JInteger.new(arg[HColumnDescriptor::TTL]): HColumnDescriptor::DEFAULT_TTL,
arg[HColumnDescriptor::BLOOMFILTER]? JBoolean.valueOf(arg[HColumnDescriptor::BLOOMFILTER]): HColumnDescriptor::DEFAULT_BLOOMFILTER)
end

View File

@ -26,6 +26,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
@ -93,20 +94,6 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
*/
public static final int DEFAULT_VERSIONS = 3;
/**
* Default maximum cell length.
*/
public static final int DEFAULT_LENGTH = Integer.MAX_VALUE;
/** Default maximum cell length as an Integer. */
public static final Integer DEFAULT_LENGTH_INTEGER =
Integer.valueOf(DEFAULT_LENGTH);
/*
* Cache here the HCD value.
* Question: its OK to cache since when we're reenable, we create a new HCD?
*/
private volatile Integer maxValueLength = null;
/*
* Cache here the HCD value.
* Question: its OK to cache since when we're reenable, we create a new HCD?
@ -180,7 +167,7 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
this (familyName == null || familyName.length <= 0?
HConstants.EMPTY_BYTE_ARRAY: familyName, DEFAULT_VERSIONS,
DEFAULT_COMPRESSION, DEFAULT_IN_MEMORY, DEFAULT_BLOCKCACHE,
Integer.MAX_VALUE, DEFAULT_TTL, false);
DEFAULT_TTL, false);
}
/**
@ -219,12 +206,44 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
*/
public HColumnDescriptor(final byte [] familyName, final int maxVersions,
final String compression, final boolean inMemory,
final boolean blockCacheEnabled, final int maxValueLength,
final boolean blockCacheEnabled,
final int timeToLive, final boolean bloomFilter) {
this(familyName, maxVersions, compression, inMemory, blockCacheEnabled,
DEFAULT_BLOCKSIZE, maxValueLength, timeToLive, bloomFilter);
DEFAULT_BLOCKSIZE, timeToLive, bloomFilter);
}
/**
* Backwards compatible Constructor. Maximum value length is no longer
* configurable.
*
* @param familyName Column family name. Must be 'printable' -- digit or
* letter -- and end in a <code>:<code>
* @param maxVersions Maximum number of versions to keep
* @param compression Compression type
* @param inMemory If true, column data should be kept in an HRegionServer's
* cache
* @param blockCacheEnabled If true, MapFile blocks should be cached
* @param blocksize
* @param maxValueLength Restrict values to &lt;= this value (UNSUPPORTED)
* @param timeToLive Time-to-live of cell contents, in seconds
* (use HConstants.FOREVER for unlimited TTL)
* @param bloomFilter Enable the specified bloom filter for this column
*
* @throws IllegalArgumentException if passed a family name that is made of
* other than 'word' characters: i.e. <code>[a-zA-Z_0-9]</code> and does not
* end in a <code>:</code>
* @throws IllegalArgumentException if the number of versions is &lt;= 0
* @deprecated As of hbase 0.20.0, max value length no longer supported
*/
// public HColumnDescriptor(final byte [] familyName, final int maxVersions,
// final String compression, final boolean inMemory,
// final boolean blockCacheEnabled, final int blocksize,
// final int maxValueLength,
// final int timeToLive, final boolean bloomFilter) {
// this(familyName, maxVersions, compression, inMemory, blockCacheEnabled,
// blocksize, timeToLive, bloomFilter);
// }
/**
* Constructor
* @param familyName Column family name. Must be 'printable' -- digit or
@ -235,7 +254,6 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
* cache
* @param blockCacheEnabled If true, MapFile blocks should be cached
* @param blocksize
* @param maxValueLength Restrict values to &lt;= this value
* @param timeToLive Time-to-live of cell contents, in seconds
* (use HConstants.FOREVER for unlimited TTL)
* @param bloomFilter Enable the specified bloom filter for this column
@ -248,10 +266,10 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
public HColumnDescriptor(final byte [] familyName, final int maxVersions,
final String compression, final boolean inMemory,
final boolean blockCacheEnabled, final int blocksize,
final int maxValueLength,
final int timeToLive, final boolean bloomFilter) {
isLegalFamilyName(familyName);
this.name = stripColon(familyName);
isLegalFamilyName(this.name);
if (maxVersions <= 0) {
// TODO: Allow maxVersion of 0 to be the way you say "Keep all versions".
// Until there is support, consider 0 or < 0 -- a configuration error.
@ -260,7 +278,6 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
setMaxVersions(maxVersions);
setInMemory(inMemory);
setBlockCacheEnabled(blockCacheEnabled);
setMaxValueLength(maxValueLength);
setTimeToLive(timeToLive);
setCompressionType(Compression.Algorithm.
valueOf(compression.toUpperCase()));
@ -269,10 +286,14 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
}
private static byte [] stripColon(final byte [] n) {
byte [] result = new byte [n.length - 1];
// Have the stored family name be absent the colon delimiter
System.arraycopy(n, 0, result, 0, n.length - 1);
return result;
byte col = n[n.length-1];
if (col == ':') {
// strip.
byte [] res = new byte[n.length-1];
System.arraycopy(n, 0, res, 0, n.length-1);
return res;
}
return n;
}
/**
@ -287,18 +308,14 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
if (b == null) {
return b;
}
if (b[b.length - 1] != ':') {
throw new IllegalArgumentException("Family names must end in a colon: " +
Bytes.toString(b));
}
if (b[0] == '.') {
throw new IllegalArgumentException("Family names cannot start with a " +
"period: " + Bytes.toString(b));
}
for (int i = 0; i < (b.length - 1); i++) {
if (Character.isISOControl(b[i])) {
if (Character.isISOControl(b[i]) || b[i] == ':') {
throw new IllegalArgumentException("Illegal character <" + b[i] +
">. Family names cannot contain control characters: " +
">. Family names cannot contain control characters or colons: " +
Bytes.toString(b));
}
}
@ -317,7 +334,7 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
*/
@TOJSON(fieldName = "name", base64=true)
public byte [] getNameWithColon() {
return HStoreKey.addDelimiter(this.name);
return Bytes.add(this.name, new byte[]{':'});
}
/**
@ -462,27 +479,6 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
setValue(HConstants.IN_MEMORY, Boolean.toString(inMemory));
}
/**
* @return Maximum value length.
*/
@TOJSON
public synchronized int getMaxValueLength() {
if (this.maxValueLength == null) {
String value = getValue(LENGTH);
this.maxValueLength = (value != null)?
Integer.decode(value): DEFAULT_LENGTH_INTEGER;
}
return this.maxValueLength.intValue();
}
/**
* @param maxLength Maximum value length.
*/
public void setMaxValueLength(int maxLength) {
setValue(LENGTH, Integer.toString(maxLength));
this.maxValueLength = null;
}
/**
* @return Time-to-live of cell contents, in seconds.
*/
@ -609,9 +605,10 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
Text t = new Text();
t.readFields(in);
this.name = t.getBytes();
if (HStoreKey.getFamilyDelimiterIndex(this.name) > 0) {
this.name = stripColon(this.name);
}
// if(KeyValue.getFamilyDelimiterIndex(this.name, 0, this.name.length)
// > 0) {
// this.name = stripColon(this.name);
// }
} else {
this.name = Bytes.readByteArray(in);
}
@ -620,7 +617,6 @@ public class HColumnDescriptor implements ISerializable, WritableComparable<HCol
int ordinal = in.readInt();
setCompressionType(Compression.Algorithm.values()[ordinal]);
setInMemory(in.readBoolean());
setMaxValueLength(in.readInt());
setBloomfilter(in.readBoolean());
if (isBloomfilter() && version < 5) {
// If a bloomFilter is enabled and the column descriptor is less than

View File

@ -136,6 +136,9 @@ public interface HConstants {
* when log splitting. More means faster but bigger mem consumption */
static final int DEFAULT_NUMBER_CONCURRENT_LOG_READS = 10;
/** Maximum value length, enforced on KeyValue construction */
static final int MAXIMUM_VALUE_LENGTH = Integer.MAX_VALUE;
// Always store the location of the root table's HRegion.
// This HRegion is never split.
@ -156,6 +159,11 @@ public interface HConstants {
// be the first to be reassigned if the server(s) they are being served by
// should go down.
//
// New stuff. Making a slow transition.
//
/** The root table's name.*/
static final byte [] ROOT_TABLE_NAME = Bytes.toBytes("-ROOT-");
@ -165,48 +173,30 @@ public interface HConstants {
/** delimiter used between portions of a region name */
public static final int META_ROW_DELIMITER = ',';
// Defines for the column names used in both ROOT and META HBase 'meta' tables.
/** The catalog family as a string*/
static final String CATALOG_FAMILY_STR = "info";
/** The ROOT and META column family (string) */
static final String COLUMN_FAMILY_STR = "info:";
/** The catalog family */
static final byte [] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR);
/** The META historian column family (string) */
static final String COLUMN_FAMILY_HISTORIAN_STR = "historian:";
/** The ROOT and META column family */
static final byte [] COLUMN_FAMILY = Bytes.toBytes(COLUMN_FAMILY_STR);
/** The catalog historian family */
static final byte [] CATALOG_HISTORIAN_FAMILY = Bytes.toBytes("historian");
/** The META historian column family */
static final byte [] COLUMN_FAMILY_HISTORIAN = Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR);
/** Array of meta column names */
static final byte[][] COLUMN_FAMILY_ARRAY = new byte[][] {COLUMN_FAMILY};
/** The regioninfo column qualifier */
static final byte [] REGIONINFO_QUALIFIER = Bytes.toBytes("regioninfo");
/** The server column qualifier */
static final byte [] SERVER_QUALIFIER = Bytes.toBytes("server");
/** ROOT/META column family member - contains HRegionInfo */
static final byte [] COL_REGIONINFO =
Bytes.toBytes(COLUMN_FAMILY_STR + "regioninfo");
/** Array of column - contains HRegionInfo */
static final byte[][] COL_REGIONINFO_ARRAY = new byte[][] {COL_REGIONINFO};
/** The startcode column qualifier */
static final byte [] STARTCODE_QUALIFIER = Bytes.toBytes("serverstartcode");
/** ROOT/META column family member - contains HServerAddress.toString() */
static final byte[] COL_SERVER = Bytes.toBytes(COLUMN_FAMILY_STR + "server");
/** The lower-half split region column qualifier */
static final byte [] SPLITA_QUALIFIER = Bytes.toBytes("splitA");
/** ROOT/META column family member - contains server start code (a long) */
static final byte [] COL_STARTCODE =
Bytes.toBytes(COLUMN_FAMILY_STR + "serverstartcode");
/** the lower half of a split region */
static final byte [] COL_SPLITA = Bytes.toBytes(COLUMN_FAMILY_STR + "splitA");
/** The upper-half split region column qualifier */
static final byte [] SPLITB_QUALIFIER = Bytes.toBytes("splitB");
/** the upper half of a split region */
static final byte [] COL_SPLITB = Bytes.toBytes(COLUMN_FAMILY_STR + "splitB");
/** All the columns in the catalog -ROOT- and .META. tables.
*/
static final byte[][] ALL_META_COLUMNS = {COL_REGIONINFO, COL_SERVER,
COL_STARTCODE, COL_SPLITA, COL_SPLITB};
// Other constants
/**
@ -245,6 +235,11 @@ public interface HConstants {
*/
static final long LATEST_TIMESTAMP = Long.MAX_VALUE;
/**
* LATEST_TIMESTAMP in bytes form
*/
static final byte [] LATEST_TIMESTAMP_BYTES = Bytes.toBytes(LATEST_TIMESTAMP);
/**
* Define for 'return-all-versions'.
*/
@ -253,8 +248,12 @@ public interface HConstants {
/**
* Unlimited time-to-live.
*/
static final int FOREVER = -1;
// static final int FOREVER = -1;
static final int FOREVER = Integer.MAX_VALUE;
/**
* Seconds in a week
*/
public static final int WEEK_IN_SECONDS = 7 * 24 * 3600;
//TODO: HBASE_CLIENT_RETRIES_NUMBER_KEY is only used by TestMigrate. Move it
@ -277,15 +276,12 @@ public interface HConstants {
public static int RETRY_BACKOFF[] = { 1, 1, 1, 2, 2, 4, 4, 8, 16, 32 };
/** modifyTable op for replacing the table descriptor */
public static final int MODIFY_TABLE_SET_HTD = 1;
/** modifyTable op for forcing a split */
public static final int MODIFY_TABLE_SPLIT = 2;
/** modifyTable op for forcing a compaction */
public static final int MODIFY_TABLE_COMPACT = 3;
// Messages client can send master.
public static final int MODIFY_CLOSE_REGION = MODIFY_TABLE_COMPACT + 1;
public static final int MODIFY_TABLE_FLUSH = MODIFY_CLOSE_REGION + 1;
public static final int MODIFY_TABLE_MAJOR_COMPACT = MODIFY_TABLE_FLUSH + 1;
public static enum Modify {
CLOSE_REGION,
TABLE_COMPACT,
TABLE_FLUSH,
TABLE_MAJOR_COMPACT,
TABLE_SET_HTD,
TABLE_SPLIT
}
}

View File

@ -1,5 +1,5 @@
/**
* Copyright 2007 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -29,13 +29,14 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Scanner;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
@ -193,7 +194,7 @@ class HMerge implements HConstants {
private static class OnlineMerger extends Merger {
private final byte [] tableName;
private final HTable table;
private final Scanner metaScanner;
private final ResultScanner metaScanner;
private HRegionInfo latestRegion;
OnlineMerger(HBaseConfiguration conf, FileSystem fs,
@ -202,22 +203,23 @@ class HMerge implements HConstants {
super(conf, fs, tableName);
this.tableName = tableName;
this.table = new HTable(conf, META_TABLE_NAME);
this.metaScanner = table.getScanner(COL_REGIONINFO_ARRAY, tableName);
this.metaScanner = table.getScanner(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
this.latestRegion = null;
}
private HRegionInfo nextRegion() throws IOException {
try {
RowResult results = getMetaRow();
Result results = getMetaRow();
if (results == null) {
return null;
}
Cell regionInfo = results.get(COL_REGIONINFO);
if (regionInfo == null || regionInfo.getValue().length == 0) {
byte [] regionInfoValue = results.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
if (regionInfoValue == null || regionInfoValue.length == 0) {
throw new NoSuchElementException("meta region entry missing " +
Bytes.toString(COL_REGIONINFO));
Bytes.toString(CATALOG_FAMILY) + ":" +
Bytes.toString(REGIONINFO_QUALIFIER));
}
HRegionInfo region = Writables.getHRegionInfo(regionInfo.getValue());
HRegionInfo region = Writables.getHRegionInfo(regionInfoValue);
if (!Bytes.equals(region.getTableDesc().getName(), this.tableName)) {
return null;
}
@ -244,13 +246,13 @@ class HMerge implements HConstants {
* @return A Map of the row content else null if we are off the end.
* @throws IOException
*/
private RowResult getMetaRow() throws IOException {
RowResult currentRow = metaScanner.next();
private Result getMetaRow() throws IOException {
Result currentRow = metaScanner.next();
boolean foundResult = false;
while (currentRow != null) {
LOG.info("Row: <" + Bytes.toString(currentRow.getRow()) + ">");
Cell regionInfo = currentRow.get(COL_REGIONINFO);
if (regionInfo == null || regionInfo.getValue().length == 0) {
byte [] regionInfoValue = currentRow.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
if (regionInfoValue == null || regionInfoValue.length == 0) {
currentRow = metaScanner.next();
continue;
}
@ -286,17 +288,18 @@ class HMerge implements HConstants {
if(Bytes.equals(regionsToDelete[r], latestRegion.getRegionName())) {
latestRegion = null;
}
table.deleteAll(regionsToDelete[r]);
Delete delete = new Delete(regionsToDelete[r]);
table.delete(delete);
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + Bytes.toString(regionsToDelete[r]));
}
}
newRegion.getRegionInfo().setOffline(true);
BatchUpdate update = new BatchUpdate(newRegion.getRegionName());
update.put(COL_REGIONINFO,
Put put = new Put(newRegion.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER,
Writables.getBytes(newRegion.getRegionInfo()));
table.commit(update);
table.put(put);
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: "
@ -325,9 +328,10 @@ class HMerge implements HConstants {
HRegionInfo.ROOT_REGIONINFO, null);
root.initialize(null, null);
Scan scan = new Scan();
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
InternalScanner rootScanner =
root.getScanner(COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW,
HConstants.LATEST_TIMESTAMP, null);
root.getScanner(scan);
try {
List<KeyValue> results = new ArrayList<KeyValue>();
@ -366,23 +370,29 @@ class HMerge implements HConstants {
throws IOException {
byte[][] regionsToDelete = {oldRegion1, oldRegion2};
for(int r = 0; r < regionsToDelete.length; r++) {
BatchUpdate b = new BatchUpdate(regionsToDelete[r]);
b.delete(COL_REGIONINFO);
b.delete(COL_SERVER);
b.delete(COL_STARTCODE);
b.delete(COL_SPLITA);
b.delete(COL_SPLITB);
root.batchUpdate(b,null);
Delete delete = new Delete(regionsToDelete[r]);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER);
root.delete(delete, null, true);
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + Bytes.toString(regionsToDelete[r]));
}
}
HRegionInfo newInfo = newRegion.getRegionInfo();
newInfo.setOffline(true);
BatchUpdate b = new BatchUpdate(newRegion.getRegionName());
b.put(COL_REGIONINFO, Writables.getBytes(newInfo));
root.batchUpdate(b,null);
Put put = new Put(newRegion.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(newInfo));
root.put(put);
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + Bytes.toString(newRegion.getRegionName()));
}

View File

@ -27,10 +27,11 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
//import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
@ -45,7 +46,8 @@ import agilejson.TOJSON;
* HTableDescriptor contains the name of an HTable, and its
* column families.
*/
public class HTableDescriptor implements WritableComparable<HTableDescriptor>, ISerializable {
public class HTableDescriptor implements WritableComparable<HTableDescriptor>,
ISerializable {
// Changes prior to version 3 were not recorded here.
// Version 3 adds metadata as a map where keys and values are byte[].
@ -100,12 +102,14 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor>, I
private volatile Boolean root = null;
// Key is hash of the family name.
private final Map<byte [], HColumnDescriptor> families =
public final Map<byte [], HColumnDescriptor> families =
new TreeMap<byte [], HColumnDescriptor>(KeyValue.FAMILY_COMPARATOR);
// private final Map<byte [], HColumnDescriptor> families =
// new TreeMap<byte [], HColumnDescriptor>(KeyValue.FAMILY_COMPARATOR);
// Key is indexId
private final Map<String, IndexSpecification> indexes =
new HashMap<String, IndexSpecification>();
// private final Map<String, IndexSpecification> indexes =
// new HashMap<String, IndexSpecification>();
/**
* Private constructor used internally creating table descriptors for
@ -125,24 +129,38 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor>, I
* Private constructor used internally creating table descriptors for
* catalog tables: e.g. .META. and -ROOT-.
*/
// protected HTableDescriptor(final byte [] name, HColumnDescriptor[] families,
// Collection<IndexSpecification> indexes,
// Map<ImmutableBytesWritable,ImmutableBytesWritable> values) {
// this.name = name.clone();
// this.nameAsString = Bytes.toString(this.name);
// setMetaFlags(name);
// for(HColumnDescriptor descriptor : families) {
// this.families.put(descriptor.getName(), descriptor);
// }
// for(IndexSpecification index : indexes) {
// this.indexes.put(index.getIndexId(), index);
// }
// for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> entry:
// values.entrySet()) {
// this.values.put(entry.getKey(), entry.getValue());
// }
// }
protected HTableDescriptor(final byte [] name, HColumnDescriptor[] families,
Collection<IndexSpecification> indexes,
Map<ImmutableBytesWritable,ImmutableBytesWritable> values) {
Map<ImmutableBytesWritable,ImmutableBytesWritable> values) {
this.name = name.clone();
this.nameAsString = Bytes.toString(this.name);
setMetaFlags(name);
for(HColumnDescriptor descriptor : families) {
this.families.put(descriptor.getName(), descriptor);
}
for(IndexSpecification index : indexes) {
this.indexes.put(index.getIndexId(), index);
}
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> entry:
values.entrySet()) {
this.values.put(entry.getKey(), entry.getValue());
}
}
/**
* Constructs an empty object.
* For deserializing an HTableDescriptor instance only.
@ -198,7 +216,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor>, I
desc.values.entrySet()) {
this.values.put(e.getKey(), e.getValue());
}
this.indexes.putAll(desc.indexes);
// this.indexes.putAll(desc.indexes);
}
/*
@ -437,21 +455,17 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor>, I
Bytes.toBytes(Integer.toString(memcacheFlushSize)));
}
public Collection<IndexSpecification> getIndexes() {
return indexes.values();
}
public IndexSpecification getIndex(String indexId) {
return indexes.get(indexId);
}
public void addIndex(IndexSpecification index) {
indexes.put(index.getIndexId(), index);
}
public void removeIndex(String indexId) {
indexes.remove(indexId);
}
// public Collection<IndexSpecification> getIndexes() {
// return indexes.values();
// }
//
// public IndexSpecification getIndex(String indexId) {
// return indexes.get(indexId);
// }
//
// public void addIndex(IndexSpecification index) {
// indexes.put(index.getIndexId(), index);
// }
/**
* Adds a column family.
@ -510,13 +524,13 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor>, I
s.append(FAMILIES);
s.append(" => ");
s.append(families.values());
if (!indexes.isEmpty()) {
// Don't emit if empty. Has to do w/ transactional hbase.
s.append(", ");
s.append("INDEXES");
s.append(" => ");
s.append(indexes.values());
}
// if (!indexes.isEmpty()) {
// // Don't emit if empty. Has to do w/ transactional hbase.
// s.append(", ");
// s.append("INDEXES");
// s.append(" => ");
// s.append(indexes.values());
// }
s.append('}');
return s.toString();
}
@ -581,16 +595,16 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor>, I
c.readFields(in);
families.put(c.getName(), c);
}
indexes.clear();
// indexes.clear();
if (version < 4) {
return;
}
int numIndexes = in.readInt();
for (int i = 0; i < numIndexes; i++) {
IndexSpecification index = new IndexSpecification();
index.readFields(in);
addIndex(index);
}
// int numIndexes = in.readInt();
// for (int i = 0; i < numIndexes; i++) {
// IndexSpecification index = new IndexSpecification();
// index.readFields(in);
// addIndex(index);
// }
}
public void write(DataOutput out) throws IOException {
@ -610,10 +624,10 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor>, I
HColumnDescriptor family = it.next();
family.write(out);
}
out.writeInt(indexes.size());
for(IndexSpecification index : indexes.values()) {
index.write(out);
}
// out.writeInt(indexes.size());
// for(IndexSpecification index : indexes.values()) {
// index.write(out);
// }
}
// Comparable
@ -654,6 +668,13 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor>, I
return Collections.unmodifiableCollection(this.families.values());
}
/**
* @return Immutable sorted set of the keys of the families.
*/
public Set<byte[]> getFamiliesKeys() {
return Collections.unmodifiableSet(this.families.keySet());
}
@TOJSON(fieldName = "columns")
public HColumnDescriptor[] getColumnFamilies() {
return getFamilies().toArray(new HColumnDescriptor[0]);
@ -689,22 +710,22 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor>, I
/** Table descriptor for <core>-ROOT-</code> catalog table */
public static final HTableDescriptor ROOT_TABLEDESC = new HTableDescriptor(
HConstants.ROOT_TABLE_NAME,
new HColumnDescriptor[] { new HColumnDescriptor(HConstants.COLUMN_FAMILY,
new HColumnDescriptor[] { new HColumnDescriptor(HConstants.CATALOG_FAMILY,
10, // Ten is arbitrary number. Keep versions to help debuggging.
Compression.Algorithm.NONE.getName(), false, true, 8 * 1024,
Integer.MAX_VALUE, HConstants.FOREVER, false) });
HConstants.FOREVER, false) });
/** Table descriptor for <code>.META.</code> catalog table */
public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor(
HConstants.META_TABLE_NAME, new HColumnDescriptor[] {
new HColumnDescriptor(HConstants.COLUMN_FAMILY,
new HColumnDescriptor(HConstants.CATALOG_FAMILY,
10, // Ten is arbitrary number. Keep versions to help debuggging.
Compression.Algorithm.NONE.getName(), false, true, 8 * 1024,
Integer.MAX_VALUE, HConstants.FOREVER, false),
new HColumnDescriptor(HConstants.COLUMN_FAMILY_HISTORIAN,
HConstants.FOREVER, false),
new HColumnDescriptor(HConstants.CATALOG_HISTORIAN_FAMILY,
HConstants.ALL_VERSIONS, Compression.Algorithm.NONE.getName(),
false, false, 8 * 1024,
Integer.MAX_VALUE, HConstants.WEEK_IN_SECONDS, false)});
HConstants.WEEK_IN_SECONDS, false)});
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.rest.xml.IOutputXML#toXML()

File diff suppressed because it is too large Load Diff

View File

@ -29,8 +29,10 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.util.Bytes;
@ -57,17 +59,17 @@ public class RegionHistorian implements HConstants {
"EEE, d MMM yyyy HH:mm:ss");
private static enum HistorianColumnKey {
REGION_CREATION ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"creation")),
REGION_OPEN ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"open")),
REGION_SPLIT ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"split")),
REGION_COMPACTION ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"compaction")),
REGION_FLUSH ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"flush")),
REGION_ASSIGNMENT ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"assignment"));
private static enum HistorianQualifierKey {
REGION_CREATION ( Bytes.toBytes("creation")),
REGION_OPEN ( Bytes.toBytes("open")),
REGION_SPLIT ( Bytes.toBytes("split")),
REGION_COMPACTION ( Bytes.toBytes("compaction")),
REGION_FLUSH ( Bytes.toBytes("flush")),
REGION_ASSIGNMENT ( Bytes.toBytes("assignment"));
byte[] key;
byte[] key;
HistorianColumnKey(byte[] key) {
HistorianQualifierKey(byte[] key) {
this.key = key;
}
}
@ -113,15 +115,17 @@ public class RegionHistorian implements HConstants {
* moment to retrieve all version and to have the column key information.
* To be changed when HTable.getRow handles versions.
*/
for (HistorianColumnKey keyEnu : HistorianColumnKey.values()) {
for (HistorianQualifierKey keyEnu : HistorianQualifierKey.values()) {
byte[] columnKey = keyEnu.key;
Cell[] cells = this.metaTable.get(Bytes.toBytes(regionName),
columnKey, ALL_VERSIONS);
if (cells != null) {
for (Cell cell : cells) {
informations.add(historian.new RegionHistoryInformation(cell
.getTimestamp(), Bytes.toString(columnKey).split(":")[1], Bytes
.toString(cell.getValue())));
Get get = new Get(Bytes.toBytes(regionName));
get.addColumn(CATALOG_HISTORIAN_FAMILY, columnKey);
get.setMaxVersions(ALL_VERSIONS);
Result result = this.metaTable.get(get);
if (result != null) {
for(KeyValue kv : result.raw()) {
informations.add(historian.new RegionHistoryInformation(
kv.getTimestamp(), columnKey, kv.getValue()));
}
}
}
@ -138,7 +142,7 @@ public class RegionHistorian implements HConstants {
* @param serverName
*/
public void addRegionAssignment(HRegionInfo info, String serverName) {
add(HistorianColumnKey.REGION_ASSIGNMENT.key, "Region assigned to server "
add(HistorianQualifierKey.REGION_ASSIGNMENT.key, "Region assigned to server "
+ serverName, info);
}
@ -147,7 +151,7 @@ public class RegionHistorian implements HConstants {
* @param info
*/
public void addRegionCreation(HRegionInfo info) {
add(HistorianColumnKey.REGION_CREATION.key, "Region creation", info);
add(HistorianQualifierKey.REGION_CREATION.key, "Region creation", info);
}
/**
@ -156,7 +160,7 @@ public class RegionHistorian implements HConstants {
* @param address
*/
public void addRegionOpen(HRegionInfo info, HServerAddress address) {
add(HistorianColumnKey.REGION_OPEN.key, "Region opened on server : "
add(HistorianQualifierKey.REGION_OPEN.key, "Region opened on server : "
+ address.getHostname(), info);
}
@ -171,7 +175,7 @@ public class RegionHistorian implements HConstants {
HRegionInfo newInfo2) {
HRegionInfo[] infos = new HRegionInfo[] { newInfo1, newInfo2 };
for (HRegionInfo info : infos) {
add(HistorianColumnKey.REGION_SPLIT.key, SPLIT_PREFIX +
add(HistorianQualifierKey.REGION_SPLIT.key, SPLIT_PREFIX +
oldInfo.getRegionNameAsString(), info);
}
}
@ -188,7 +192,7 @@ public class RegionHistorian implements HConstants {
// such danger compacting; compactions are not allowed when
// Flusher#flushSomeRegions is run.
if (LOG.isDebugEnabled()) {
add(HistorianColumnKey.REGION_COMPACTION.key,
add(HistorianQualifierKey.REGION_COMPACTION.key,
"Region compaction completed in " + timeTaken, info);
}
}
@ -211,9 +215,8 @@ public class RegionHistorian implements HConstants {
* @param text
* @param info
*/
private void add(byte[] column,
String text, HRegionInfo info) {
add(column, text, info, LATEST_TIMESTAMP);
private void add(byte [] qualifier, String text, HRegionInfo info) {
add(qualifier, text, info, LATEST_TIMESTAMP);
}
/**
@ -223,18 +226,19 @@ public class RegionHistorian implements HConstants {
* @param info
* @param timestamp
*/
private void add(byte[] column,
String text, HRegionInfo info, long timestamp) {
private void add(byte [] qualifier, String text, HRegionInfo info,
long timestamp) {
if (!isOnline()) {
// Its a noop
return;
}
if (!info.isMetaRegion()) {
BatchUpdate batch = new BatchUpdate(info.getRegionName());
batch.setTimestamp(timestamp);
batch.put(column, Bytes.toBytes(text));
Put put = new Put(info.getRegionName());
put.setTimeStamp(timestamp);
put.add(HConstants.CATALOG_HISTORIAN_FAMILY, qualifier,
Bytes.toBytes(text));
try {
this.metaTable.commit(batch);
this.metaTable.put(put);
} catch (IOException ioe) {
LOG.warn("Unable to '" + text + "'", ioe);
}
@ -252,34 +256,35 @@ public class RegionHistorian implements HConstants {
private long timestamp;
private String event;
private byte [] event = null;
private String description;
private byte [] description = null;
/**
* @param timestamp
* @param event
* @param description
*/
public RegionHistoryInformation(long timestamp, String event,
String description) {
public RegionHistoryInformation(long timestamp, byte [] event,
byte [] description) {
this.timestamp = timestamp;
this.event = event;
this.description = description;
}
public int compareTo(RegionHistoryInformation otherInfo) {
return -1 * Long.valueOf(timestamp).compareTo(otherInfo.getTimestamp());
}
/** @return the event */
public String getEvent() {
return event;
return Bytes.toString(event);
}
/** @return the description */
public String getDescription() {
return description;
return Bytes.toString(description);
}
/** @return the timestamp */

View File

@ -23,6 +23,7 @@ import java.util.Comparator;
import org.apache.hadoop.io.Writable;
public interface WritableComparator<T> extends Writable, Comparator<T> {
// No methods, just bring the two interfaces together
}
/**
* Interface that brings writable and comparable together
*/
public interface WritableComparator<T> extends Writable, Comparator<T> {}

View File

@ -0,0 +1,312 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Used to perform Delete operations on a single row.
* <p>
* To delete an entire row, instantiate a Delete object with the row
* to delete. To further define the scope of what to delete, perform
* additional methods as outlined below.
* <p>
* To delete specific families, execute {@link #deleteFamily(byte []) deleteFamily}
* for each family to delete.
* <p>
* To delete multiple versions of specific columns, execute
* {@link #deleteColumns(byte [],byte []) deleteColumns}
* for each column to delete.
* <p>
* To delete specific versions of specific columns, execute
* {@link #deleteColumn(byte [],byte [],long) deleteColumn}
* for each column version to delete.
* <p>
* Specifying timestamps calling constructor, deleteFamily, and deleteColumns
* will delete all versions with a timestamp less than or equal to that
* specified. Specifying a timestamp to deleteColumn will delete versions
* only with a timestamp equal to that specified.
* <p>The timestamp passed to the constructor is only used ONLY for delete of
* rows. For anything less -- a deleteColumn, deleteColumns or
* deleteFamily -- then you need to use the method overrides that take a
* timestamp. The constructor timestamp is not referenced.
*/
public class Delete implements Writable {
private byte [] row = null;
// This ts is only used when doing a deleteRow. Anything less,
private long ts;
private long lockId = -1L;
private final Map<byte [], List<KeyValue>> familyMap =
new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
/** Constructor for Writable. DO NOT USE */
public Delete() {
this(null);
}
/**
* Create a Delete operation for the specified row.
* <p>
* If no further operations are done, this will delete everything
* associated with the specified row (all versions of all columns in all
* families).
* @param row row key
*/
public Delete(byte [] row) {
this(row, HConstants.LATEST_TIMESTAMP, null);
}
/**
* Create a Delete operation for the specified row and timestamp, using
* an optional row lock.
* <p>
* If no further operations are done, this will delete all columns in all
* families of the specified row with a timestamp less than or equal to the
* specified timestamp.
* @param row row key
* @param timestamp maximum version timestamp
* @param rowLock previously acquired row lock, or null
*/
public Delete(byte [] row, long timestamp, RowLock rowLock) {
this.row = row;
this.ts = timestamp;
if (rowLock != null) {
this.lockId = rowLock.getLockId();
}
}
/**
* Delete all versions of all columns of the specified family.
* <p>
* Overrides previous calls to deleteColumn and deleteColumns for the
* specified family.
* @param family family name
*/
public void deleteFamily(byte [] family) {
this.deleteFamily(family, HConstants.LATEST_TIMESTAMP);
}
/**
* Delete all columns of the specified family with a timestamp less than
* or equal to the specified timestamp.
* <p>
* Overrides previous calls to deleteColumn and deleteColumns for the
* specified family.
* @param family family name
* @param timestamp maximum version timestamp
*/
public void deleteFamily(byte [] family, long timestamp) {
List<KeyValue> list = familyMap.get(family);
if(list == null) {
list = new ArrayList<KeyValue>();
} else if(!list.isEmpty()) {
list.clear();
}
list.add(new KeyValue(row, family, null, timestamp, KeyValue.Type.DeleteFamily));
familyMap.put(family, list);
}
/**
* Delete all versions of the specified column.
* @param family family name
* @param qualifier column qualifier
*/
public void deleteColumns(byte [] family, byte [] qualifier) {
this.deleteColumns(family, qualifier, HConstants.LATEST_TIMESTAMP);
}
/**
* Delete all versions of the specified column with a timestamp less than
* or equal to the specified timestamp.
* @param family family name
* @param qualifier column qualifier
* @param timestamp maximum version timestamp
*/
public void deleteColumns(byte [] family, byte [] qualifier, long timestamp) {
List<KeyValue> list = familyMap.get(family);
if (list == null) {
list = new ArrayList<KeyValue>();
}
list.add(new KeyValue(this.row, family, qualifier, timestamp,
KeyValue.Type.DeleteColumn));
familyMap.put(family, list);
}
/**
* Delete the latest version of the specified column.
* This is an expensive call in that on the server-side, it first does a
* get to find the latest versions timestamp. Then it adds a delete using
* the fetched cells timestamp.
* @param family family name
* @param qualifier column qualifier
*/
public void deleteColumn(byte [] family, byte [] qualifier) {
this.deleteColumn(family, qualifier, HConstants.LATEST_TIMESTAMP);
}
/**
* Delete the specified version of the specified column.
* @param family family name
* @param qualifier column qualifier
* @param timestamp version timestamp
*/
public void deleteColumn(byte [] family, byte [] qualifier, long timestamp) {
List<KeyValue> list = familyMap.get(family);
if(list == null) {
list = new ArrayList<KeyValue>();
}
list.add(new KeyValue(
this.row, family, qualifier, timestamp, KeyValue.Type.Delete));
familyMap.put(family, list);
}
/**
* Delete the latest version of the specified column, given in
* <code>family:qualifier</code> notation.
* @param column colon-delimited family and qualifier
*/
public void deleteColumn(byte [] column) {
byte [][] parts = KeyValue.parseColumn(column);
this.deleteColumn(parts[0], parts[1], HConstants.LATEST_TIMESTAMP);
}
/**
* Method for retrieving the delete's familyMap
* @return familyMap
*/
public Map<byte [], List<KeyValue>> getFamilyMap() {
return this.familyMap;
}
/**
* Method for retrieving the delete's row
* @return row
*/
public byte [] getRow() {
return this.row;
}
/**
* Method for retrieving the delete's RowLock
* @return RowLock
*/
public RowLock getRowLock() {
return new RowLock(this.row, this.lockId);
}
/**
* Method for retrieving the delete's lockId
* @return
*/
public long getLockId() {
return this.lockId;
}
/**
* Method for retrieving the delete's timestamp
* @return timestamp
*/
public long getTimeStamp() {
return this.ts;
}
/**
* @return string
*/
@Override
public String toString() {
StringBuffer sb = new StringBuffer();
sb.append("row=");
sb.append(Bytes.toString(this.row));
sb.append(", ts=");
sb.append(this.ts);
sb.append(", families={");
boolean moreThanOne = false;
for(Map.Entry<byte [], List<KeyValue>> entry : this.familyMap.entrySet()) {
if(moreThanOne) {
sb.append(", ");
} else {
moreThanOne = true;
}
sb.append("(family=");
sb.append(Bytes.toString(entry.getKey()));
sb.append(", keyvalues=(");
boolean moreThanOneB = false;
for(KeyValue kv : entry.getValue()) {
if(moreThanOneB) {
sb.append(", ");
} else {
moreThanOneB = true;
}
sb.append(kv.toString());
}
sb.append(")");
}
sb.append("}");
return sb.toString();
}
//Writable
public void readFields(final DataInput in) throws IOException {
this.row = Bytes.readByteArray(in);
this.ts = in.readLong();
this.lockId = in.readLong();
this.familyMap.clear();
int numFamilies = in.readInt();
for(int i=0;i<numFamilies;i++) {
byte [] family = Bytes.readByteArray(in);
int numColumns = in.readInt();
List<KeyValue> list = new ArrayList<KeyValue>(numColumns);
for(int j=0;j<numColumns;j++) {
KeyValue kv = new KeyValue();
kv.readFields(in);
list.add(kv);
}
this.familyMap.put(family, list);
}
}
public void write(final DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.row);
out.writeLong(this.ts);
out.writeLong(this.lockId);
out.writeInt(familyMap.size());
for(Map.Entry<byte [], List<KeyValue>> entry : familyMap.entrySet()) {
Bytes.writeByteArray(out, entry.getKey());
List<KeyValue> list = entry.getValue();
out.writeInt(list.size());
for(KeyValue kv : list) {
kv.write(out);
}
}
}
}

View File

@ -0,0 +1,398 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
/**
* Used to perform Get operations on a single row.
* <p>
* To get everything for a row, instantiate a Get object with the row to get.
* To further define the scope of what to get, perform additional methods as
* outlined below.
* <p>
* To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily}
* for each family to retrieve.
* <p>
* To get specific columns, execute {@link #addColumn(byte[], byte[]) addColumn}
* for each column to retrieve.
* <p>
* To only retrieve columns within a specific range of version timestamps,
* execute {@link #setTimeRange(long, long) setTimeRange}.
* <p>
* To only retrieve columns with a specific timestamp, execute
* {@link #setTimeStamp(long) setTimestamp}.
* <p>
* To limit the number of versions of each column to be returned, execute
* {@link #setMaxVersions(int) setMaxVersions}.
* <p>
* To add a filter, execute {@link #setFilter(RowFilterInterface) setFilter}.
*/
public class Get implements Writable {
private byte [] row = null;
private long lockId = -1L;
private int maxVersions = 1;
private RowFilterInterface filter = null;
private TimeRange tr = new TimeRange();
private Map<byte [], NavigableSet<byte []>> familyMap =
new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
/** Constructor for Writable. DO NOT USE */
public Get() {}
/**
* Create a Get operation for the specified row.
* <p>
* If no further operations are done, this will get the latest version of
* all columns in all families of the specified row.
* @param row row key
*/
public Get(byte [] row) {
this(row, null);
}
/**
* Create a Get operation for the specified row, using an existing row lock.
* <p>
* If no further operations are done, this will get the latest version of
* all columns in all families of the specified row.
* @param row row key
* @param rowLock previously acquired row lock, or null
*/
public Get(byte [] row, RowLock rowLock) {
this.row = row;
if(rowLock != null) {
this.lockId = rowLock.getLockId();
}
}
/**
* Get all columns from the specified family.
* <p>
* Overrides previous calls to addColumn for this family.
* @param family family name
* @return the Get object
*/
public Get addFamily(byte [] family) {
familyMap.remove(family);
familyMap.put(family, null);
return this;
}
/**
* Get the column from the specific family with the specified qualifier.
* <p>
* Overrides previous calls to addFamily for this family.
* @param family family name
* @param qualifier column qualifier
* @return the Get objec
*/
public Get addColumn(byte [] family, byte [] qualifier) {
NavigableSet<byte []> set = familyMap.get(family);
if(set == null) {
set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
}
set.add(qualifier);
familyMap.put(family, set);
return this;
}
/**
* Adds an array of columns specified the old format, family:qualifier.
* <p>
* Overrides previous calls to addFamily for any families in the input.
* @param columns array of columns, formatted as <pre>family:qualifier</pre>
*/
public Get addColumns(byte [][] columns) {
if (columns == null) return this;
for(int i = 0; i < columns.length; i++) {
try {
addColumn(columns[i]);
} catch(Exception e) {}
}
return this;
}
/**
* @param column Old format column.
* @return This.
*/
public Get addColumn(final byte [] column) {
if (column == null) return this;
byte [][] split = KeyValue.parseColumn(column);
addColumn(split[0], split[1]);
return this;
}
/**
* Get versions of columns only within the specified timestamp range,
* [minStamp, maxStamp).
* @param minStamp minimum timestamp value, inclusive
* @param maxStamp maximum timestamp value, exclusive
* @throws IOException if invalid time range
*/
public Get setTimeRange(long minStamp, long maxStamp)
throws IOException {
tr = new TimeRange(minStamp, maxStamp);
return this;
}
/**
* Get versions of columns with the specified timestamp.
* @param timestamp version timestamp
*/
public Get setTimeStamp(long timestamp) {
try {
tr = new TimeRange(timestamp, timestamp+1);
} catch(IOException e) {
// Will never happen
}
return this;
}
/**
* Get all available versions.
*/
public Get setMaxVersions() {
this.maxVersions = Integer.MAX_VALUE;
return this;
}
/**
* Get up to the specified number of versions of each column.
* @param maxVersions maximum versions for each column
* @throws IOException if invalid number of versions
*/
public Get setMaxVersions(int maxVersions) throws IOException {
if(maxVersions <= 0) {
throw new IOException("maxVersions must be positive");
}
this.maxVersions = maxVersions;
return this;
}
/**
* Apply the specified server-side filter when performing the Get.
* @param filter filter to run on the server
*/
public Get setFilter(RowFilterInterface filter) {
this.filter = filter;
return this;
}
/** Accessors */
/**
* Method for retrieving the get's row
* @return row
*/
public byte [] getRow() {
return this.row;
}
/**
* Method for retrieving the get's RowLock
* @return RowLock
*/
public RowLock getRowLock() {
return new RowLock(this.row, this.lockId);
}
/**
* Method for retrieving the get's lockId
* @return lockId
*/
public long getLockId() {
return this.lockId;
}
/**
* Method for retrieving the get's maximum number of version
* @return the maximum number of version to fetch for this get
*/
public int getMaxVersions() {
return this.maxVersions;
}
/**
* Method for retrieving the get's TimeRange
* @return timeRange
*/
public TimeRange getTimeRange() {
return this.tr;
}
/**
* Method for retrieving the keys in the familyMap
* @return keys in the current familyMap
*/
public Set<byte[]> familySet() {
return this.familyMap.keySet();
}
/**
* Method for retrieving the number of families to get from
* @return number of families
*/
public int numFamilies() {
return this.familyMap.size();
}
/**
* Method for checking if any families have been inserted into this Get
* @return true if familyMap is non empty false otherwise
*/
public boolean hasFamilies() {
return !this.familyMap.isEmpty();
}
/**
* Method for retrieving the get's familyMap
* @return familyMap
*/
public Map<byte[],NavigableSet<byte[]>> getFamilyMap() {
return this.familyMap;
}
/**
* @return String
*/
@Override
public String toString() {
StringBuffer sb = new StringBuffer();
sb.append("row=");
sb.append(Bytes.toString(this.row));
sb.append(", maxVersions=");
sb.append("" + this.maxVersions);
sb.append(", timeRange=");
sb.append("[" + this.tr.getMin() + "," + this.tr.getMax() + ")");
sb.append(", families=");
if(this.familyMap.size() == 0) {
sb.append("ALL");
return sb.toString();
}
boolean moreThanOne = false;
for(Map.Entry<byte [], NavigableSet<byte[]>> entry :
this.familyMap.entrySet()) {
if(moreThanOne) {
sb.append("), ");
} else {
moreThanOne = true;
sb.append("{");
}
sb.append("(family=");
sb.append(Bytes.toString(entry.getKey()));
sb.append(", columns=");
if(entry.getValue() == null) {
sb.append("ALL");
} else {
sb.append("{");
boolean moreThanOneB = false;
for(byte [] column : entry.getValue()) {
if(moreThanOneB) {
sb.append(", ");
} else {
moreThanOneB = true;
}
sb.append(Bytes.toString(column));
}
sb.append("}");
}
}
sb.append("}");
return sb.toString();
}
//Writable
public void readFields(final DataInput in)
throws IOException {
this.row = Bytes.readByteArray(in);
this.lockId = in.readLong();
this.maxVersions = in.readInt();
boolean hasFilter = in.readBoolean();
if(hasFilter) {
this.filter =
(RowFilterInterface)HbaseObjectWritable.readObject(in, null);
}
this.tr = new TimeRange();
tr.readFields(in);
int numFamilies = in.readInt();
this.familyMap =
new TreeMap<byte [],NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
for(int i=0; i<numFamilies; i++) {
byte [] family = Bytes.readByteArray(in);
boolean hasColumns = in.readBoolean();
NavigableSet<byte []> set = null;
if(hasColumns) {
int numColumns = in.readInt();
set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
for(int j=0; j<numColumns; j++) {
byte [] qualifier = Bytes.readByteArray(in);
set.add(qualifier);
}
}
this.familyMap.put(family, set);
}
}
public void write(final DataOutput out)
throws IOException {
Bytes.writeByteArray(out, this.row);
out.writeLong(this.lockId);
out.writeInt(this.maxVersions);
if(this.filter == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
HbaseObjectWritable.writeObject(out, this.filter,
RowFilterInterface.class, null);
}
tr.write(out);
out.writeInt(familyMap.size());
for(Map.Entry<byte [], NavigableSet<byte []>> entry :
familyMap.entrySet()) {
Bytes.writeByteArray(out, entry.getKey());
NavigableSet<byte []> columnSet = entry.getValue();
if(columnSet == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeInt(columnSet.size());
for(byte [] qualifier : columnSet) {
Bytes.writeByteArray(out, qualifier);
}
}
}
}
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.Map;
import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -34,8 +35,9 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.RegionException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
@ -51,7 +53,8 @@ import org.apache.hadoop.ipc.RemoteException;
*/
public class HBaseAdmin {
private final Log LOG = LogFactory.getLog(this.getClass().getName());
private final HConnection connection;
// private final HConnection connection;
final HConnection connection;
private volatile HBaseConfiguration conf;
private final long pause;
private final int numRetries;
@ -121,11 +124,13 @@ public class HBaseAdmin {
return this.connection.listTables();
}
public HTableDescriptor getTableDescriptor(final String tableName)
throws IOException {
return getTableDescriptor(Bytes.toBytes(tableName));
}
/**
* Method for getting the tableDescriptor
* @param tableName as a byte []
* @return the tableDescriptor
* @throws IOException
*/
public HTableDescriptor getTableDescriptor(final byte [] tableName)
throws IOException {
return this.connection.getHTableDescriptor(tableName);
@ -238,19 +243,22 @@ public class HBaseAdmin {
for (int tries = 0; tries < numRetries; tries++) {
long scannerId = -1L;
try {
scannerId =
server.openScanner(firstMetaServer.getRegionInfo().getRegionName(),
HConstants.COL_REGIONINFO_ARRAY, tableName,
HConstants.LATEST_TIMESTAMP, null);
RowResult values = server.next(scannerId);
Scan scan = new Scan().addColumn(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
scannerId = server.openScanner(
firstMetaServer.getRegionInfo().getRegionName(),
scan);
Result values = server.next(scannerId);
if (values == null || values.size() == 0) {
break;
}
boolean found = false;
for (Map.Entry<byte [], Cell> e: values.entrySet()) {
if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) {
NavigableMap<byte[], byte[]> infoValues = values.getFamilyMap(HConstants.CATALOG_FAMILY);
for (Map.Entry<byte [], byte []> e: infoValues.entrySet()) {
if (Bytes.equals(e.getKey(), HConstants.REGIONINFO_QUALIFIER)) {
info = (HRegionInfo) Writables.getWritable(
e.getValue().getValue(), info);
e.getValue(), info);
if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
found = true;
@ -566,7 +574,7 @@ public class HBaseAdmin {
newargs[i + xtraArgsCount] = args[i];
}
}
modifyTable(HConstants.META_TABLE_NAME, HConstants.MODIFY_CLOSE_REGION,
modifyTable(HConstants.META_TABLE_NAME, HConstants.Modify.CLOSE_REGION,
newargs);
}
@ -589,7 +597,7 @@ public class HBaseAdmin {
* @throws IOException
*/
public void flush(final byte [] tableNameOrRegionName) throws IOException {
modifyTable(tableNameOrRegionName, HConstants.MODIFY_TABLE_FLUSH);
modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_FLUSH);
}
/**
@ -611,7 +619,7 @@ public class HBaseAdmin {
* @throws IOException
*/
public void compact(final byte [] tableNameOrRegionName) throws IOException {
modifyTable(tableNameOrRegionName, HConstants.MODIFY_TABLE_COMPACT);
modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_COMPACT);
}
/**
@ -635,7 +643,7 @@ public class HBaseAdmin {
*/
public void majorCompact(final byte [] tableNameOrRegionName)
throws IOException {
modifyTable(tableNameOrRegionName, HConstants.MODIFY_TABLE_MAJOR_COMPACT);
modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_MAJOR_COMPACT);
}
/**
@ -657,7 +665,7 @@ public class HBaseAdmin {
* @throws IOException
*/
public void split(final byte [] tableNameOrRegionName) throws IOException {
modifyTable(tableNameOrRegionName, HConstants.MODIFY_TABLE_SPLIT);
modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_SPLIT);
}
/*
@ -667,7 +675,8 @@ public class HBaseAdmin {
* @param op
* @throws IOException
*/
private void modifyTable(final byte [] tableNameOrRegionName, final int op)
private void modifyTable(final byte [] tableNameOrRegionName,
final HConstants.Modify op)
throws IOException {
if (tableNameOrRegionName == null) {
throw new IllegalArgumentException("Pass a table name or region name");
@ -689,7 +698,7 @@ public class HBaseAdmin {
*/
public void modifyTable(final byte [] tableName, HTableDescriptor htd)
throws IOException {
modifyTable(tableName, HConstants.MODIFY_TABLE_SET_HTD, htd);
modifyTable(tableName, HConstants.Modify.TABLE_SET_HTD, htd);
}
/**
@ -702,7 +711,8 @@ public class HBaseAdmin {
* @param args operation specific arguments
* @throws IOException
*/
public void modifyTable(final byte [] tableName, int op, Object... args)
public void modifyTable(final byte [] tableName, HConstants.Modify op,
Object... args)
throws IOException {
if (this.master == null) {
throw new MasterNotRunningException("master has been shut down");
@ -715,7 +725,7 @@ public class HBaseAdmin {
Writable[] arr = null;
try {
switch (op) {
case HConstants.MODIFY_TABLE_SET_HTD:
case TABLE_SET_HTD:
if (args == null || args.length < 1 ||
!(args[0] instanceof HTableDescriptor)) {
throw new IllegalArgumentException("SET_HTD requires a HTableDescriptor");
@ -725,10 +735,10 @@ public class HBaseAdmin {
this.master.modifyTable(tableName, op, arr);
break;
case HConstants.MODIFY_TABLE_COMPACT:
case HConstants.MODIFY_TABLE_SPLIT:
case HConstants.MODIFY_TABLE_MAJOR_COMPACT:
case HConstants.MODIFY_TABLE_FLUSH:
case TABLE_COMPACT:
case TABLE_SPLIT:
case TABLE_MAJOR_COMPACT:
case TABLE_FLUSH:
if (args != null && args.length > 0) {
arr = new Writable[1];
if (args[0] instanceof byte[]) {
@ -745,7 +755,7 @@ public class HBaseAdmin {
this.master.modifyTable(tableName, op, arr);
break;
case HConstants.MODIFY_CLOSE_REGION:
case CLOSE_REGION:
if (args == null || args.length < 1) {
throw new IllegalArgumentException("Requires at least a region name");
}

View File

@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
@ -190,6 +189,6 @@ public interface HConnection {
* @param tableName The name of the table
* @throws IOException
*/
public void processBatchOfRows(ArrayList<BatchUpdate> list, byte[] tableName)
public void processBatchOfRows(ArrayList<Put> list, byte[] tableName)
throws IOException;
}

View File

@ -1,5 +1,5 @@
/**
* Copyright 2007 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -38,15 +38,12 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
@ -338,9 +335,9 @@ public class HConnectionManager implements HConstants {
MetaScannerVisitor visitor = new MetaScannerVisitor() {
public boolean processRow(RowResult rowResult) throws IOException {
public boolean processRow(Result result) throws IOException {
HRegionInfo info = Writables.getHRegionInfo(
rowResult.get(COL_REGIONINFO));
result.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
// Only examine the rows where the startKey is zero length
if (info != null && info.getStartKey().length == 0) {
@ -387,12 +384,13 @@ public class HConnectionManager implements HConstants {
HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES);
byte[] endKey = null;
HRegionInfo currentRegion = null;
Scan scan = new Scan(startKey);
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
ScannerCallable s = new ScannerCallable(this,
(Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME),
HConstants.COL_REGIONINFO_ARRAY, startKey,
HConstants.LATEST_TIMESTAMP, null
);
scan.getStartRow(),
scan);
try {
// Open scanner
getRegionServerWithRetries(s);
@ -402,27 +400,25 @@ public class HConnectionManager implements HConstants {
startKey = oldRegion.getEndKey();
}
currentRegion = s.getHRegionInfo();
RowResult r = null;
RowResult[] rrs = null;
Result r = null;
Result [] rrs = null;
while ((rrs = getRegionServerWithRetries(s)) != null) {
r = rrs[0];
Cell c = r.get(HConstants.COL_REGIONINFO);
if (c != null) {
byte[] value = c.getValue();
if (value != null) {
HRegionInfo info = Writables.getHRegionInfoOrNull(value);
if (info != null) {
if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
rowsScanned += 1;
rowsOffline += info.isOffline() ? 1 : 0;
}
byte [] value = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (value != null) {
HRegionInfo info = Writables.getHRegionInfoOrNull(value);
if (info != null) {
if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
rowsScanned += 1;
rowsOffline += info.isOffline() ? 1 : 0;
}
}
}
}
endKey = currentRegion.getEndKey();
} while (!(endKey == null || HStoreKey.equalsTwoRowKeys(endKey,
HConstants.EMPTY_BYTE_ARRAY)));
} while (!(endKey == null ||
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)));
}
finally {
s.setClose();
@ -440,9 +436,9 @@ public class HConnectionManager implements HConstants {
protected HTableDescriptorFinder(byte[] tableName) {
this.tableName = tableName;
}
public boolean processRow(RowResult rowResult) throws IOException {
public boolean processRow(Result rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfo(
rowResult.get(HConstants.COL_REGIONINFO));
rowResult.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
HTableDescriptor desc = info.getTableDesc();
if (Bytes.compareTo(desc.getName(), tableName) == 0) {
result = desc;
@ -554,21 +550,22 @@ public class HConnectionManager implements HConstants {
getHRegionConnection(metaLocation.getServerAddress());
// Query the root or meta region for the location of the meta region
RowResult regionInfoRow = server.getClosestRowBefore(
Result regionInfoRow = server.getClosestRowBefore(
metaLocation.getRegionInfo().getRegionName(), metaKey,
HConstants.COLUMN_FAMILY);
HConstants.CATALOG_FAMILY);
if (regionInfoRow == null) {
throw new TableNotFoundException(Bytes.toString(tableName));
}
Cell value = regionInfoRow.get(COL_REGIONINFO);
if (value == null || value.getValue().length == 0) {
byte [] value = regionInfoRow.getValue(CATALOG_FAMILY,
REGIONINFO_QUALIFIER);
if (value == null || value.length == 0) {
throw new IOException("HRegionInfo was null or empty in " +
Bytes.toString(parentTable));
}
// convert the row result into the HRegionLocation we need!
HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(
value.getValue(), new HRegionInfo());
value, new HRegionInfo());
// possible we got a region of a different table...
if (!Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) {
throw new TableNotFoundException(
@ -579,8 +576,11 @@ public class HConnectionManager implements HConstants {
regionInfo.getRegionNameAsString());
}
String serverAddress =
Writables.cellToString(regionInfoRow.get(COL_SERVER));
value = regionInfoRow.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
String serverAddress = "";
if(value != null) {
serverAddress = Bytes.toString(value);
}
if (serverAddress.equals("")) {
throw new NoServerForRegionException("No server address listed " +
"in " + Bytes.toString(parentTable) + " for region " +
@ -680,8 +680,8 @@ public class HConnectionManager implements HConstants {
// this one. the exception case is when the endkey is EMPTY_START_ROW,
// signifying that the region we're checking is actually the last
// region in the table.
if (HStoreKey.equalsTwoRowKeys(endKey, HConstants.EMPTY_END_ROW) ||
HStoreKey.getComparator(tableName).compareRows(endKey, row) > 0) {
if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ||
KeyValue.getRowComparator(tableName).compare(endKey, row) > 0) {
return possibleRegion;
}
}
@ -718,7 +718,7 @@ public class HConnectionManager implements HConstants {
// by nature of the map, we know that the start key has to be <
// otherwise it wouldn't be in the headMap.
if (HStoreKey.getComparator(tableName).compareRows(endKey, row) <= 0) {
if (KeyValue.getRowComparator(tableName).compare(endKey, row) <= 0) {
// delete any matching entry
HRegionLocation rl =
tableLocations.remove(matchingRegions.lastKey());
@ -978,15 +978,15 @@ public class HConnectionManager implements HConstants {
return location;
}
public void processBatchOfRows(ArrayList<BatchUpdate> list, byte[] tableName)
public void processBatchOfRows(ArrayList<Put> list, byte[] tableName)
throws IOException {
if (list.isEmpty()) {
return;
}
boolean retryOnlyOne = false;
int tries = 0;
Collections.sort(list);
List<BatchUpdate> tempUpdates = new ArrayList<BatchUpdate>();
Collections.sort(list);
List<Put> currentPuts = new ArrayList<Put>();
HRegionLocation location =
getRegionLocationForRowWithRetries(tableName, list.get(0).getRow(),
false);
@ -994,8 +994,8 @@ public class HConnectionManager implements HConstants {
byte [] region = currentRegion;
boolean isLastRow = false;
for (int i = 0; i < list.size() && tries < numRetries; i++) {
BatchUpdate batchUpdate = list.get(i);
tempUpdates.add(batchUpdate);
Put put = list.get(i);
currentPuts.add(put);
isLastRow = (i + 1) == list.size();
if (!isLastRow) {
location = getRegionLocationForRowWithRetries(tableName,
@ -1003,19 +1003,19 @@ public class HConnectionManager implements HConstants {
region = location.getRegionInfo().getRegionName();
}
if (!Bytes.equals(currentRegion, region) || isLastRow || retryOnlyOne) {
final BatchUpdate[] updates = tempUpdates.toArray(new BatchUpdate[0]);
final Put [] puts = currentPuts.toArray(new Put[0]);
int index = getRegionServerWithRetries(new ServerCallable<Integer>(
this, tableName, batchUpdate.getRow()) {
this, tableName, put.getRow()) {
public Integer call() throws IOException {
int i = server.batchUpdates(location.getRegionInfo()
.getRegionName(), updates);
int i = server.put(location.getRegionInfo()
.getRegionName(), puts);
return i;
}
});
if (index != -1) {
if (tries == numRetries - 1) {
throw new RetriesExhaustedException("Some server",
currentRegion, batchUpdate.getRow(),
currentRegion, put.getRow(),
tries, new ArrayList<Throwable>());
}
long sleepTime = getPauseTime(tries);
@ -1031,7 +1031,7 @@ public class HConnectionManager implements HConstants {
} catch (InterruptedException e) {
// continue
}
i = i - updates.length + index;
i = i - puts.length + index;
retryOnlyOne = true;
location = getRegionLocationForRowWithRetries(tableName,
list.get(i + 1).getRow(), true);
@ -1041,7 +1041,7 @@ public class HConnectionManager implements HConstants {
retryOnlyOne = false;
}
currentRegion = region;
tempUpdates.clear();
currentPuts.clear();
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -47,6 +47,7 @@ public class HTablePool {
/**
* Get a shared table pool.
* @param config
* @param tableName the table name
* @return the table pool
*/

View File

@ -5,7 +5,6 @@ import java.io.IOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -49,14 +48,14 @@ class MetaScanner implements HConstants {
// Scan over each meta region
ScannerCallable callable = null;
do {
callable = new ScannerCallable(connection, META_TABLE_NAME,
COLUMN_FAMILY_ARRAY, startRow, LATEST_TIMESTAMP, null);
Scan scan = new Scan(startRow).addFamily(CATALOG_FAMILY);
callable = new ScannerCallable(connection, META_TABLE_NAME, scan.getStartRow(), scan);
// Open scanner
connection.getRegionServerWithRetries(callable);
try {
RowResult r = null;
Result r = null;
do {
RowResult [] rrs = connection.getRegionServerWithRetries(callable);
Result [] rrs = connection.getRegionServerWithRetries(callable);
if (rrs == null || rrs.length == 0 || rrs[0].size() == 0) {
break;
}
@ -85,6 +84,6 @@ class MetaScanner implements HConstants {
* @return A boolean to know if it should continue to loop in the region
* @throws IOException
*/
public boolean processRow(RowResult rowResult) throws IOException;
public boolean processRow(Result rowResult) throws IOException;
}
}

View File

@ -0,0 +1,305 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Used to perform Put operations for a single row.
* <p>
* To perform a Put, instantiate a Put object with the row to insert to and
* for each column to be inserted, execute {@link #add(byte[], byte[], byte[]) add} or
* {@link #add(byte[], byte[], long, byte[]) add} if setting the timestamp.
*/
public class Put implements HeapSize, Writable, Comparable<Put> {
private byte [] row = null;
private long timestamp = HConstants.LATEST_TIMESTAMP;
private long lockId = -1L;
private Map<byte [], List<KeyValue>> familyMap =
new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
/** Constructor for Writable. DO NOT USE */
public Put() {}
/**
* Create a Put operation for the specified row.
* @param row row key
*/
public Put(byte [] row) {
this(row, null);
}
/**
* Create a Put operation for the specified row, using an existing row lock.
* @param row row key
* @param rowLock previously acquired row lock, or null
*/
public Put(byte [] row, RowLock rowLock) {
if(row == null || row.length > HConstants.MAX_ROW_LENGTH) {
throw new IllegalArgumentException("Row key is invalid");
}
this.row = row;
if(rowLock != null) {
this.lockId = rowLock.getLockId();
}
}
/**
* Copy constructor. Creates a Put operation cloned from the specified Put.
* @param putToCopy put to copy
*/
public Put(Put putToCopy) {
this(putToCopy.getRow(), putToCopy.getRowLock());
this.familyMap =
new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
for(Map.Entry<byte [], List<KeyValue>> entry :
putToCopy.getFamilyMap().entrySet()) {
this.familyMap.put(entry.getKey(), entry.getValue());
}
}
/**
* Add the specified column and value to this Put operation.
* @param family family name
* @param qualifier column qualifier
* @param value column value
*/
public void add(byte [] family, byte [] qualifier, byte [] value) {
add(family, qualifier, this.timestamp, value);
}
/**
* Add the specified column and value, with the specified timestamp as
* its version to this Put operation.
* @param column Old style column name with family and qualifier put together
* with a colon.
* @param timestamp version timestamp
* @param value column value
*/
public void add(byte [] column, long timestamp, byte [] value) {
byte [][] parts = KeyValue.parseColumn(column);
add(parts[0], parts[1], timestamp, value);
}
/**
* Add the specified column and value, with the specified timestamp as
* its version to this Put operation.
* @param family family name
* @param qualifier column qualifier
* @param timestamp version timestamp
* @param value column value
*/
public void add(byte [] family, byte [] qualifier, long timestamp, byte [] value) {
List<KeyValue> list = familyMap.get(family);
if(list == null) {
list = new ArrayList<KeyValue>();
}
KeyValue kv = new KeyValue(this.row, family, qualifier, timestamp,
KeyValue.Type.Put, value);
list.add(kv);
familyMap.put(family, list);
}
/**
* Add the specified KeyValue to this Put operation.
* @param kv
*/
public void add(KeyValue kv) {
byte [] family = kv.getFamily();
List<KeyValue> list = familyMap.get(family);
if(list == null) {
list = new ArrayList<KeyValue>();
}
list.add(kv);
familyMap.put(family, list);
}
/**
* Method for retrieving the put's familyMap
* @return familyMap
*/
public Map<byte [], List<KeyValue>> getFamilyMap() {
return this.familyMap;
}
/**
* Method for retrieving the put's row
* @return row
*/
public byte [] getRow() {
return this.row;
}
/**
* Method for retrieving the put's RowLock
* @return RowLock
*/
public RowLock getRowLock() {
return new RowLock(this.row, this.lockId);
}
/**
* Method for retrieving the put's lockId
* @return lockId
*/
public long getLockId() {
return this.lockId;
}
/**
* Method to check if the familyMap is empty
* @return true if empty, false otherwise
*/
public boolean isEmpty() {
return familyMap.isEmpty();
}
/**
* Method for setting the timestamp
* @param timestamp
*/
public void setTimeStamp(long timestamp) {
this.timestamp = timestamp;
}
public int numFamilies() {
return familyMap.size();
}
public int size() {
int size = 0;
for(List<KeyValue> kvList : this.familyMap.values()) {
size += kvList.size();
}
return size;
}
/**
* @return String
*/
@Override
public String toString() {
StringBuffer sb = new StringBuffer();
sb.append("row=");
sb.append(Bytes.toString(this.row));
sb.append(", families={");
boolean moreThanOne = false;
for(Map.Entry<byte [], List<KeyValue>> entry : this.familyMap.entrySet()) {
if(moreThanOne) {
sb.append(", ");
} else {
moreThanOne = true;
}
sb.append("(family=");
sb.append(Bytes.toString(entry.getKey()));
sb.append(", keyvalues=(");
boolean moreThanOneB = false;
for(KeyValue kv : entry.getValue()) {
if(moreThanOneB) {
sb.append(", ");
} else {
moreThanOneB = true;
}
sb.append(kv.toString());
}
sb.append(")");
}
sb.append("}");
return sb.toString();
}
public int compareTo(Put p) {
return Bytes.compareTo(this.getRow(), p.getRow());
}
//HeapSize
public long heapSize() {
long totalSize = 0;
for(Map.Entry<byte [], List<KeyValue>> entry : this.familyMap.entrySet()) {
for(KeyValue kv : entry.getValue()) {
totalSize += kv.heapSize();
}
}
return totalSize;
}
//Writable
public void readFields(final DataInput in)
throws IOException {
this.row = Bytes.readByteArray(in);
this.timestamp = in.readLong();
this.lockId = in.readLong();
int numFamilies = in.readInt();
this.familyMap =
new TreeMap<byte [],List<KeyValue>>(Bytes.BYTES_COMPARATOR);
for(int i=0;i<numFamilies;i++) {
byte [] family = Bytes.readByteArray(in);
int numKeys = in.readInt();
List<KeyValue> keys = new ArrayList<KeyValue>(numKeys);
int totalLen = in.readInt();
byte [] buf = new byte[totalLen];
int offset = 0;
for(int j=0;j<numKeys;j++) {
int keyLength = in.readInt();
in.readFully(buf, offset, keyLength);
keys.add(new KeyValue(buf, offset, keyLength));
offset += keyLength;
}
this.familyMap.put(family, keys);
}
}
public void write(final DataOutput out)
throws IOException {
Bytes.writeByteArray(out, this.row);
out.writeLong(this.timestamp);
out.writeLong(this.lockId);
out.writeInt(familyMap.size());
for(Map.Entry<byte [], List<KeyValue>> entry : familyMap.entrySet()) {
Bytes.writeByteArray(out, entry.getKey());
List<KeyValue> keys = entry.getValue();
out.writeInt(keys.size());
int totalLen = 0;
for(KeyValue kv : keys) {
totalLen += kv.getLength();
}
out.writeInt(totalLen);
for(KeyValue kv : keys) {
out.writeInt(kv.getLength());
out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
}
}
}
}

View File

@ -0,0 +1,479 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.SplitKeyValue;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
/**
* Single row result of a {@link Get} or {@link Scan} query.
*/
public class Result implements Writable {
private KeyValue [] kvs = null;
private NavigableMap<byte[],
NavigableMap<byte[], NavigableMap<Long, byte[]>>> familyMap = null;
// We're not using java serialization. Transient here is just a marker to say
// that this is where we cache row if we're ever asked for it.
private transient byte [] row = null;
/**
* Constructor used for Writable.
*/
public Result() {}
/**
* Instantiate a Result with the specified array of KeyValues.
* @param kvs array of KeyValues
*/
public Result(KeyValue [] kvs) {
if(kvs != null && kvs.length > 0) {
this.kvs = kvs;
}
}
/**
* Instantiate a Result with the specified List of KeyValues.
* @param kvs List of KeyValues
*/
public Result(List<KeyValue> kvs) {
this(kvs.toArray(new KeyValue[0]));
}
/**
* Method for retrieving the row that this result is for
* @return row
*/
public synchronized byte [] getRow() {
if (this.row == null) {
this.row =
this.kvs == null || this.kvs.length == 0? null: this.kvs[0].getRow();
}
return this.row;
}
/**
* Directly return the unsorted array of KeyValues in this Result.
* @return unsorted array of KeyValues
*/
public KeyValue[] raw() {
return kvs;
}
/**
* Return a sorted list of the KeyValues in this result.
* @return
*/
public List<KeyValue> list() {
return Arrays.asList(sorted());
}
/**
* Returns a sorted array of KeyValues in this Result.
* <p>
* Note: Sorting is done in place, so the backing array will be sorted
* after calling this method.
* @return sorted array of KeyValues
*/
public KeyValue[] sorted() {
if(isEmpty()) {
return null;
}
Arrays.sort(kvs, (Comparator<KeyValue>)KeyValue.COMPARATOR);
return kvs;
}
/**
* Map of families to all versions of its qualifiers and values.
* <p>
* Returns a three level Map of the form:
* <code>Map<family,Map<qualifier,Map<timestamp,value>>></code>
* <p>
* Note: All other map returning methods make use of this map internally.
* @return map from families to qualifiers to versions
*/
public NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> getMap() {
if(this.familyMap != null) {
return this.familyMap;
}
if(isEmpty()) {
return null;
}
this.familyMap =
new TreeMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>
(Bytes.BYTES_COMPARATOR);
for(KeyValue kv : this.kvs) {
SplitKeyValue splitKV = kv.split();
byte [] family = splitKV.getFamily();
NavigableMap<byte[], NavigableMap<Long, byte[]>> columnMap =
familyMap.get(family);
if(columnMap == null) {
columnMap = new TreeMap<byte[], NavigableMap<Long, byte[]>>
(Bytes.BYTES_COMPARATOR);
familyMap.put(family, columnMap);
}
byte [] qualifier = splitKV.getQualifier();
NavigableMap<Long, byte[]> versionMap = columnMap.get(qualifier);
if(versionMap == null) {
versionMap = new TreeMap<Long, byte[]>(new Comparator<Long>() {
public int compare(Long l1, Long l2) {
return l2.compareTo(l1);
}
});
columnMap.put(qualifier, versionMap);
}
Long timestamp = Bytes.toLong(splitKV.getTimestamp());
byte [] value = splitKV.getValue();
versionMap.put(timestamp, value);
}
return this.familyMap;
}
/**
* Map of families to their most recent qualifiers and values.
* <p>
* Returns a two level Map of the form: <code>Map<family,Map<qualifier,value>></code>
* <p>
* The most recent version of each qualifier will be used.
* @return map from families to qualifiers and value
*/
public NavigableMap<byte[], NavigableMap<byte[], byte[]>> getNoVersionMap() {
if(this.familyMap == null) {
getMap();
}
if(isEmpty()) {
return null;
}
NavigableMap<byte[], NavigableMap<byte[], byte[]>> returnMap =
new TreeMap<byte[], NavigableMap<byte[], byte[]>>(Bytes.BYTES_COMPARATOR);
for(Map.Entry<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>
familyEntry : familyMap.entrySet()) {
NavigableMap<byte[], byte[]> qualifierMap =
new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
for(Map.Entry<byte[], NavigableMap<Long, byte[]>> qualifierEntry :
familyEntry.getValue().entrySet()) {
byte [] value =
qualifierEntry.getValue().get(qualifierEntry.getValue().firstKey());
qualifierMap.put(qualifierEntry.getKey(), value);
}
returnMap.put(familyEntry.getKey(), qualifierMap);
}
return returnMap;
}
/**
* Map of qualifiers to values.
* <p>
* Returns a Map of the form: <code>Map<qualifier,value></code>
* @return map of qualifiers to values
*/
public NavigableMap<byte[], byte[]> getFamilyMap(byte [] family) {
if(this.familyMap == null) {
getMap();
}
if(isEmpty()) {
return null;
}
NavigableMap<byte[], byte[]> returnMap =
new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
NavigableMap<byte[], NavigableMap<Long, byte[]>> qualifierMap =
familyMap.get(family);
if(qualifierMap == null) {
return returnMap;
}
for(Map.Entry<byte[], NavigableMap<Long, byte[]>> entry :
qualifierMap.entrySet()) {
byte [] value =
entry.getValue().get(entry.getValue().firstKey());
returnMap.put(entry.getKey(), value);
}
return returnMap;
}
/**
* Get the latest version of the specified column.
* @param family family name
* @param qualifier column qualifier
* @return value of latest version of column, null if none found
*/
public byte [] getValue(byte [] family, byte [] qualifier) {
Map.Entry<Long,byte[]> entry = getKeyValue(family, qualifier);
return entry == null ?null :entry.getValue();
}
public Cell getCellValue(byte[] family, byte[] qualifier) {
Map.Entry<Long,byte[]> val = getKeyValue(family, qualifier);
return new Cell(val.getValue(), val.getKey());
}
/**
* @return First KeyValue in this Result as a Cell or null if empty.
*/
public Cell getCellValue() {
return isEmpty()? null: new Cell(kvs[0].getValue(), kvs[0].getTimestamp());
}
/**
* @return This Result as array of Cells or null if empty.
*/
public Cell [] getCellValues() {
if (isEmpty()) return null;
Cell [] results = new Cell[kvs.length];
for (int i = 0; i < kvs.length; i++) {
results[i] = new Cell(kvs[i].getValue(), kvs[i].getTimestamp());
}
return results;
}
private Map.Entry<Long,byte[]> getKeyValue(byte[] family, byte[] qualifier) {
if(this.familyMap == null) {
getMap();
}
if(isEmpty()) {
return null;
}
NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap =
familyMap.get(family);
if(qualifierMap == null) {
return null;
}
NavigableMap<Long, byte[]> versionMap =
getVersionMap(qualifierMap, qualifier);
if(versionMap == null) {
return null;
}
return versionMap.firstEntry();
}
private NavigableMap<Long, byte[]> getVersionMap(
NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap, byte [] qualifier) {
if(qualifier != null) {
return qualifierMap.get(qualifier);
} else {
return qualifierMap.get(new byte[0]);
}
}
/**
* Get the latest version of the specified column,
* using <pre>family:qualifier</pre> notation.
* @param column column in family:qualifier notation
* @return value of latest version of column, null if none found
*/
public byte [] getValue(byte [] column) {
try {
byte [][] split = KeyValue.parseColumn(column);
return getValue(split[0], split[1]);
} catch(Exception e) {
return null;
}
}
/**
* Checks for existence of the specified column.
* @param family family name
* @param qualifier column qualifier
* @return true if at least one value exists in the result, false if not
*/
public boolean containsColumn(byte [] family, byte [] qualifier) {
if(this.familyMap == null) {
getMap();
}
if(isEmpty()) {
return false;
}
NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap =
familyMap.get(family);
if(qualifierMap == null) {
return false;
}
NavigableMap<Long, byte[]> versionMap = getVersionMap(qualifierMap, qualifier);
if(versionMap == null) {
return false;
}
return true;
}
/**
* Returns this Result in the old return format, {@link RowResult}.
* @return a RowResult
*/
public RowResult getRowResult() {
return RowResult.createRowResult(Arrays.asList(kvs));
}
/**
* Returns the value of the first column in the Result.
* @return value of the first column
*/
public byte [] value() {
if(isEmpty()) {
return null;
}
return kvs[0].getValue();
}
/**
* Check if the underlying KeyValue [] is empty or not
* @return true if empty
*/
public boolean isEmpty() {
return (this.kvs == null || this.kvs.length == 0);
}
/**
*
* @return the size of the underlying KeyValue []
*/
public int size() {
return (this.kvs == null ? 0 : this.kvs.length);
}
/**
* @return String
*/
@Override
public String toString() {
StringBuffer sb = new StringBuffer();
sb.append("keyvalues=");
if(isEmpty()) {
sb.append("NONE");
return sb.toString();
}
sb.append("{");
boolean moreThanOne = false;
for(KeyValue kv : this.kvs) {
if(moreThanOne) {
sb.append(", ");
} else {
moreThanOne = true;
}
sb.append(kv.toString());
}
sb.append("}");
return sb.toString();
}
//Writable
public void readFields(final DataInput in)
throws IOException {
int numKeys = in.readInt();
this.kvs = new KeyValue[numKeys];
if(numKeys == 0) {
return;
}
int totalBuffer = in.readInt();
byte [] buf = new byte[totalBuffer];
int offset = 0;
for(int i=0; i<numKeys; i++) {
int keyLength = in.readInt();
in.readFully(buf, offset, keyLength);
kvs[i] = new KeyValue(buf, offset, keyLength);
offset += keyLength;
}
}
public void write(final DataOutput out)
throws IOException {
if(isEmpty()) {
out.writeInt(0);
} else {
int len = this.kvs.length;
out.writeInt(len);
int totalLen = 0;
for(KeyValue kv : kvs) {
totalLen += kv.getLength();
}
out.writeInt(totalLen);
for(KeyValue kv : kvs) {
out.writeInt(kv.getLength());
out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
}
}
}
public static void writeArray(final DataOutput out, Result [] results)
throws IOException {
if(results == null || results.length == 0) {
out.writeInt(0);
return;
}
out.writeInt(results.length);
int bufLen = 0;
for(Result result : results) {
if(result == null || result.isEmpty()) {
continue;
}
for(KeyValue key : result.raw()) {
bufLen += key.getLength();
}
}
out.writeInt(bufLen);
for(Result result : results) {
if(result == null || result.isEmpty()) {
out.writeInt(0);
continue;
}
out.writeInt(result.size());
for(KeyValue kv : result.raw()) {
out.writeInt(kv.getLength());
out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
}
}
}
public static Result [] readArray(final DataInput in)
throws IOException {
int numResults = in.readInt();
if(numResults == 0) {
return new Result[0];
}
Result [] results = new Result[numResults];
int bufSize = in.readInt();
byte [] buf = new byte[bufSize];
int offset = 0;
for(int i=0;i<numResults;i++) {
int numKeys = in.readInt();
KeyValue [] keys = new KeyValue[numKeys];
for(int j=0;j<numKeys;j++) {
int keyLen = in.readInt();
in.readFully(buf, offset, keyLen);
keys[j] = new KeyValue(buf, offset, keyLen);
offset += keyLen;
}
results[i] = new Result(keys);
}
return results;
}
}

View File

@ -0,0 +1,52 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.Closeable;
import java.io.IOException;
import org.apache.hadoop.hbase.client.Result;
/**
* Interface for client-side scanning.
* Go to {@link HTable} to obtain instances.
*/
public interface ResultScanner extends Closeable, Iterable<Result> {
/**
* Grab the next row's worth of values. The scanner will return a Result.
* @return Result object if there is another row, null if the scanner is
* exhausted.
* @throws IOException
*/
public Result next() throws IOException;
/**
* @param nbRows number of rows to return
* @return Between zero and <param>nbRows</param> Results
* @throws IOException
*/
public Result [] next(int nbRows) throws IOException;
/**
* Closes the scanner and releases any resources it has allocated
*/
public void close();
}

View File

@ -0,0 +1,456 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Map;
import java.util.NavigableSet;
import java.util.TreeMap;
import java.util.TreeSet;
/**
* Used to perform Scan operations.
* <p>
* All operations are identical to {@link Get} with the exception of
* instantiation. Rather than specifying a single row, an optional startRow
* and stopRow may be defined. If rows are not specified, the Scanner will
* iterate over all rows.
* <p>
* To scan everything for each row, instantiate a Scan object.
* To further define the scope of what to get when scanning, perform additional
* methods as outlined below.
* <p>
* To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily}
* for each family to retrieve.
* <p>
* To get specific columns, execute {@link #addColumn(byte[], byte[]) addColumn}
* for each column to retrieve.
* <p>
* To only retrieve columns within a specific range of version timestamps,
* execute {@link #setTimeRange(long, long) setTimeRange}.
* <p>
* To only retrieve columns with a specific timestamp, execute
* {@link #setTimeStamp(long) setTimestamp}.
* <p>
* To limit the number of versions of each column to be returned, execute
* {@link #setMaxVersions(int) setMaxVersions}.
* <p>
* To add a filter, execute {@link #setFilter(org.apache.hadoop.hbase.filter.Filter) setFilter}.
*/
public class Scan implements Writable {
private byte [] startRow = HConstants.EMPTY_START_ROW;
private byte [] stopRow = HConstants.EMPTY_END_ROW;
private int maxVersions = 1;
private Filter filter = null;
private RowFilterInterface oldFilter = null;
private TimeRange tr = new TimeRange();
private Map<byte [], NavigableSet<byte []>> familyMap =
new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
/**
* Create a Scan operation across all rows.
*/
public Scan() {}
public Scan(byte [] startRow, Filter filter) {
this(startRow);
this.filter = filter;
}
/**
* Create a Scan operation starting at the specified row.
* <p>
* If the specified row does not exist, the Scanner will start from the
* next closest row after the specified row.
* @param startRow row to start scanner at or after
*/
public Scan(byte [] startRow) {
this.startRow = startRow;
}
/**
* Create a Scan operation for the range of rows specified.
* @param startRow row to start scanner at or after (inclusive)
* @param stopRow row to stop scanner before (exclusive)
*/
public Scan(byte [] startRow, byte [] stopRow) {
this.startRow = startRow;
this.stopRow = stopRow;
}
/**
* Get all columns from the specified family.
* <p>
* Overrides previous calls to addColumn for this family.
* @param family family name
*/
public Scan addFamily(byte [] family) {
familyMap.remove(family);
familyMap.put(family, null);
return this;
}
/**
* Get the column from the specified family with the specified qualifier.
* <p>
* Overrides previous calls to addFamily for this family.
* @param family family name
* @param qualifier column qualifier
*/
public Scan addColumn(byte [] family, byte [] qualifier) {
NavigableSet<byte []> set = familyMap.get(family);
if(set == null) {
set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
}
set.add(qualifier);
familyMap.put(family, set);
return this;
}
/**
* Adds an array of columns specified the old format, family:qualifier.
* <p>
* Overrides previous calls to addFamily for any families in the input.
* @param columns array of columns, formatted as <pre>family:qualifier</pre>
*/
public Scan addColumns(byte [][] columns) {
for(int i=0; i<columns.length; i++) {
try {
byte [][] split = KeyValue.parseColumn(columns[i]);
addColumn(split[0], split[1]);
} catch(Exception e) {}
}
return this;
}
/**
* Get versions of columns only within the specified timestamp range,
* [minStamp, maxStamp).
* @param minStamp minimum timestamp value, inclusive
* @param maxStamp maximum timestamp value, exclusive
* @throws IOException if invalid time range
*/
public Scan setTimeRange(long minStamp, long maxStamp)
throws IOException {
tr = new TimeRange(minStamp, maxStamp);
return this;
}
/**
* Get versions of columns with the specified timestamp.
* @param timestamp version timestamp
*/
public Scan setTimeStamp(long timestamp) {
try {
tr = new TimeRange(timestamp, timestamp+1);
} catch(IOException e) {
// Will never happen
}
return this;
}
/**
* Set the start row.
* @param startRow
*/
public Scan setStartRow(byte [] startRow) {
this.startRow = startRow;
return this;
}
/**
* Set the stop row.
* @param stopRow
*/
public Scan setStopRow(byte [] stopRow) {
this.stopRow = stopRow;
return this;
}
/**
* Get all available versions.
*/
public Scan setMaxVersions() {
this.maxVersions = Integer.MAX_VALUE;
return this;
}
/**
* Get up to the specified number of versions of each column.
* @param maxVersions maximum versions for each column
* @throws IOException if invalid number of versions
*/
public Scan setMaxVersions(int maxVersions) {
this.maxVersions = maxVersions;
return this;
}
/**
* Apply the specified server-side filter when performing the Scan.
* @param filter filter to run on the server
*/
public Scan setFilter(Filter filter) {
this.filter = filter;
return this;
}
/**
* Set an old-style filter interface to use. Note: not all features of the
* old style filters are supported.
* @deprecated
* @param filter
* @return
*/
public Scan setOldFilter(RowFilterInterface filter) {
oldFilter = filter;
return this;
}
/**
* Setting the familyMap
* @param familyMap
*/
public Scan setFamilyMap(Map<byte [], NavigableSet<byte []>> familyMap) {
this.familyMap = familyMap;
return this;
}
/**
* Getting the familyMap
* @return familyMap
*/
public Map<byte [], NavigableSet<byte []>> getFamilyMap() {
return this.familyMap;
}
/**
* @return the number of families in familyMap
*/
public int numFamilies() {
if(hasFamilies()) {
return this.familyMap.size();
}
return 0;
}
/**
* @return true if familyMap is non empty, false otherwise
*/
public boolean hasFamilies() {
return !this.familyMap.isEmpty();
}
/**
* @return the keys of the familyMap
*/
public byte[][] getFamilies() {
if(hasFamilies()) {
return this.familyMap.keySet().toArray(new byte[0][0]);
}
return null;
}
/**
* @return the startrow
*/
public byte [] getStartRow() {
return this.startRow;
}
/**
* @return the stoprow
*/
public byte [] getStopRow() {
return this.stopRow;
}
/**
* @return the max number of versions to fetch
*/
public int getMaxVersions() {
return this.maxVersions;
}
/**
* @return TimeRange
*/
public TimeRange getTimeRange() {
return this.tr;
}
/**
* @return RowFilter
*/
public Filter getFilter() {
return filter;
}
/**
* Get the old style filter, if there is one.
* @deprecated
* @return null or instance
*/
public RowFilterInterface getOldFilter() {
return oldFilter;
}
/**
* @return true is a filter has been specified, false if not
*/
public boolean hasFilter() {
return filter != null || oldFilter != null;
}
/**
* @return String
*/
@Override
public String toString() {
StringBuffer sb = new StringBuffer();
sb.append("startRow=");
sb.append(Bytes.toString(this.startRow));
sb.append(", stopRow=");
sb.append(Bytes.toString(this.stopRow));
sb.append(", maxVersions=");
sb.append("" + this.maxVersions);
sb.append(", timeRange=");
sb.append("[" + this.tr.getMin() + "," + this.tr.getMax() + ")");
sb.append(", families=");
if(this.familyMap.size() == 0) {
sb.append("ALL");
return sb.toString();
}
boolean moreThanOne = false;
for(Map.Entry<byte [], NavigableSet<byte[]>> entry : this.familyMap.entrySet()) {
if(moreThanOne) {
sb.append("), ");
} else {
moreThanOne = true;
sb.append("{");
}
sb.append("(family=");
sb.append(Bytes.toString(entry.getKey()));
sb.append(", columns=");
if(entry.getValue() == null) {
sb.append("ALL");
} else {
sb.append("{");
boolean moreThanOneB = false;
for(byte [] column : entry.getValue()) {
if(moreThanOneB) {
sb.append(", ");
} else {
moreThanOneB = true;
}
sb.append(Bytes.toString(column));
}
sb.append("}");
}
}
sb.append("}");
return sb.toString();
}
//Writable
public void readFields(final DataInput in)
throws IOException {
this.startRow = Bytes.readByteArray(in);
this.stopRow = Bytes.readByteArray(in);
this.maxVersions = in.readInt();
boolean hasFilter = in.readBoolean();
if(hasFilter) {
this.filter = (Filter)HbaseObjectWritable.readObject(in,
new Configuration());
}
boolean hasOldFilter = in.readBoolean();
if (hasOldFilter) {
this.oldFilter = (RowFilterInterface)HbaseObjectWritable.readObject(in,
new Configuration());
}
this.tr = new TimeRange();
tr.readFields(in);
int numFamilies = in.readInt();
this.familyMap =
new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
for(int i=0; i<numFamilies; i++) {
byte [] family = Bytes.readByteArray(in);
int numColumns = in.readInt();
TreeSet<byte []> set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
for(int j=0; j<numColumns; j++) {
byte [] qualifier = Bytes.readByteArray(in);
set.add(qualifier);
}
this.familyMap.put(family, set);
}
}
public void write(final DataOutput out)
throws IOException {
Bytes.writeByteArray(out, this.startRow);
Bytes.writeByteArray(out, this.stopRow);
out.writeInt(this.maxVersions);
if(this.filter == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
HbaseObjectWritable.writeObject(out, this.filter,
Filter.class, null);
}
if (this.oldFilter == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
HbaseObjectWritable.writeObject(out, this.oldFilter,
RowFilterInterface.class, null);
}
tr.write(out);
out.writeInt(familyMap.size());
for(Map.Entry<byte [], NavigableSet<byte []>> entry : familyMap.entrySet()) {
Bytes.writeByteArray(out, entry.getKey());
NavigableSet<byte []> columnSet = entry.getValue();
if(columnSet != null){
out.writeInt(columnSet.size());
for(byte [] qualifier : columnSet) {
Bytes.writeByteArray(out, qualifier);
}
} else {
out.writeInt(0);
}
}
}
}

View File

@ -1,5 +1,5 @@
/**
* Copyright 2008 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -21,34 +21,32 @@ package org.apache.hadoop.hbase.client;
import java.io.Closeable;
import java.io.IOException;
import org.apache.hadoop.hbase.io.RowResult;
/**
* Interface for client-side scanning.
* Go to {@link HTable} to obtain instances.
* @deprecated See {@link ResultScanner}
*/
public interface Scanner extends Closeable, Iterable<RowResult> {
/**
* Grab the next row's worth of values. The scanner will return a RowResult
* that contains both the row's key and a map of byte[] column names to Cell
* value objects. The data returned will only contain the most recent data
* value for each row that is not newer than the target time passed when the
* scanner was created.
* Grab the next row's worth of values.
* @return RowResult object if there is another row, null if the scanner is
* exhausted.
* @throws IOException
*/
public RowResult next() throws IOException;
/**
* @param nbRows number of rows to return
* @return Between zero and <param>nbRows</param> RowResults
* @return Between zero and <param>nbRows</param> Results
* @throws IOException
*/
public RowResult [] next(int nbRows) throws IOException;
/**
* Closes the scanner and releases any resources it has allocated
*/
public void close();
public void close();
}

View File

@ -23,37 +23,31 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.io.RowResult;
/**
* Retries scanner operations such as create, next, etc.
* Used by {@link Scanner}s made by {@link HTable}.
* Used by {@link ResultScanner}s made by {@link HTable}.
*/
public class ScannerCallable extends ServerCallable<RowResult[]> {
public class ScannerCallable extends ServerCallable<Result[]> {
private long scannerId = -1L;
private boolean instantiated = false;
private boolean closed = false;
private final byte [][] columns;
private final long timestamp;
private final RowFilterInterface filter;
private Scan scan;
private byte [] startRow;
private int caching = 1;
/**
* @param connection
* @param tableName
* @param columns
* @param startRow
* @param timestamp
* @param filter
* @param scan
*/
public ScannerCallable (HConnection connection, byte [] tableName, byte [][] columns,
byte [] startRow, long timestamp, RowFilterInterface filter) {
public ScannerCallable (HConnection connection, byte [] tableName,
byte [] startRow, Scan scan) {
super(connection, tableName, startRow);
this.columns = columns;
this.timestamp = timestamp;
this.filter = filter;
this.scan = scan;
this.startRow = startRow;
}
/**
@ -71,7 +65,7 @@ public class ScannerCallable extends ServerCallable<RowResult[]> {
/**
* @see java.util.concurrent.Callable#call()
*/
public RowResult[] call() throws IOException {
public Result [] call() throws IOException {
if (scannerId != -1L && closed) {
server.close(scannerId);
scannerId = -1L;
@ -79,28 +73,19 @@ public class ScannerCallable extends ServerCallable<RowResult[]> {
// open the scanner
scannerId = openScanner();
} else {
RowResult [] rrs = server.next(scannerId, caching);
return rrs.length == 0 ? null : rrs;
Result [] rrs = server.next(scannerId, caching);
return rrs == null || rrs.length == 0? null: rrs;
}
return null;
}
protected long openScanner() throws IOException {
return server.openScanner(
this.location.getRegionInfo().getRegionName(), columns, row,
timestamp, filter);
this.location.getRegionInfo().getRegionName(), scan);
}
protected byte [][] getColumns() {
return columns;
}
protected long getTimestamp() {
return timestamp;
}
protected RowFilterInterface getFilter() {
return filter;
protected Scan getScan() {
return scan;
}
/**

View File

@ -55,14 +55,6 @@ public class UnmodifyableHColumnDescriptor extends HColumnDescriptor {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
/**
* @see org.apache.hadoop.hbase.HColumnDescriptor#setMaxValueLength(int)
*/
@Override
public void setMaxValueLength(int maxLength) {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
/**
* @see org.apache.hadoop.hbase.HColumnDescriptor#setTimeToLive(int)
*/

View File

@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
//import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
/**
* Read-only table descriptor.
@ -37,10 +37,14 @@ public class UnmodifyableHTableDescriptor extends HTableDescriptor {
* Create an unmodifyable copy of an HTableDescriptor
* @param desc
*/
// UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
// super(desc.getName(), getUnmodifyableFamilies(desc), desc.getIndexes(), desc.getValues());
// }
UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
super(desc.getName(), getUnmodifyableFamilies(desc), desc.getIndexes(), desc.getValues());
super(desc.getName(), getUnmodifyableFamilies(desc), desc.getValues());
}
/*
* @param desc
* @return Families as unmodifiable array.
@ -122,11 +126,11 @@ public class UnmodifyableHTableDescriptor extends HTableDescriptor {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}
/**
* @see org.apache.hadoop.hbase.HTableDescriptor#addIndex(org.apache.hadoop.hbase.client.tableindexed.IndexSpecification)
*/
@Override
public void addIndex(IndexSpecification index) {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}
// /**
// * @see org.apache.hadoop.hbase.HTableDescriptor#addIndex(org.apache.hadoop.hbase.client.tableindexed.IndexSpecification)
// */
// @Override
// public void addIndex(IndexSpecification index) {
// throw new UnsupportedOperationException("HTableDescriptor is read-only");
// }
}

View File

@ -1,29 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.util.Map;
import org.apache.hadoop.io.Writable;
public interface IndexKeyGenerator extends Writable {
byte [] createIndexKey(byte [] rowKey, Map<byte [], byte []> columns);
}

View File

@ -1,47 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.IOException;
/**
* Thrown when asking for an index that does not exist.
*/
public class IndexNotFoundException extends IOException {
private static final long serialVersionUID = 6533971528557000965L;
public IndexNotFoundException() {
super();
}
public IndexNotFoundException(String arg0) {
super(arg0);
}
public IndexNotFoundException(Throwable arg0) {
super(arg0.getMessage());
}
public IndexNotFoundException(String arg0, Throwable arg1) {
super(arg0+arg1.getMessage());
}
}

View File

@ -1,190 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.ObjectWritable;
import org.apache.hadoop.io.Writable;
/** Holds the specification for a single secondary index. */
public class IndexSpecification implements Writable {
// Columns that are indexed (part of the indexRowKey)
private byte[][] indexedColumns;
// Constructs the
private IndexKeyGenerator keyGenerator;
// Additional columns mapped into the indexed row. These will be available for
// filters when scanning the index.
private byte[][] additionalColumns;
private byte[][] allColumns;
// Id of this index, unique within a table.
private String indexId;
/** Construct an "simple" index spec for a single column.
* @param indexId
* @param indexedColumn
*/
public IndexSpecification(String indexId, byte[] indexedColumn) {
this(indexId, new byte[][] { indexedColumn }, null,
new SimpleIndexKeyGenerator(indexedColumn));
}
/**
* Construct an index spec by specifying everything.
*
* @param indexId
* @param indexedColumns
* @param additionalColumns
* @param keyGenerator
*/
public IndexSpecification(String indexId, byte[][] indexedColumns,
byte[][] additionalColumns, IndexKeyGenerator keyGenerator) {
this.indexId = indexId;
this.indexedColumns = indexedColumns;
this.additionalColumns = additionalColumns;
this.keyGenerator = keyGenerator;
this.makeAllColumns();
}
public IndexSpecification() {
// For writable
}
private void makeAllColumns() {
this.allColumns = new byte[indexedColumns.length
+ (additionalColumns == null ? 0 : additionalColumns.length)][];
System.arraycopy(indexedColumns, 0, allColumns, 0, indexedColumns.length);
if (additionalColumns != null) {
System.arraycopy(additionalColumns, 0, allColumns, indexedColumns.length,
additionalColumns.length);
}
}
/**
* Get the indexedColumns.
*
* @return Return the indexedColumns.
*/
public byte[][] getIndexedColumns() {
return indexedColumns;
}
/**
* Get the keyGenerator.
*
* @return Return the keyGenerator.
*/
public IndexKeyGenerator getKeyGenerator() {
return keyGenerator;
}
/**
* Get the additionalColumns.
*
* @return Return the additionalColumns.
*/
public byte[][] getAdditionalColumns() {
return additionalColumns;
}
/**
* Get the indexId.
*
* @return Return the indexId.
*/
public String getIndexId() {
return indexId;
}
public byte[][] getAllColumns() {
return allColumns;
}
public boolean containsColumn(byte[] column) {
for (byte[] col : allColumns) {
if (Bytes.equals(column, col)) {
return true;
}
}
return false;
}
public byte[] getIndexedTableName(byte[] baseTableName) {
return Bytes.add(baseTableName, Bytes.toBytes("-" + indexId));
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
indexId = in.readUTF();
int numIndexedCols = in.readInt();
indexedColumns = new byte[numIndexedCols][];
for (int i = 0; i < numIndexedCols; i++) {
indexedColumns[i] = Bytes.readByteArray(in);
}
int numAdditionalCols = in.readInt();
additionalColumns = new byte[numAdditionalCols][];
for (int i = 0; i < numAdditionalCols; i++) {
additionalColumns[i] = Bytes.readByteArray(in);
}
makeAllColumns();
HBaseConfiguration conf = new HBaseConfiguration();
keyGenerator = (IndexKeyGenerator) ObjectWritable.readObject(in, conf);
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
out.writeUTF(indexId);
out.writeInt(indexedColumns.length);
for (byte[] col : indexedColumns) {
Bytes.writeByteArray(out, col);
}
if (additionalColumns != null) {
out.writeInt(additionalColumns.length);
for (byte[] col : additionalColumns) {
Bytes.writeByteArray(out, col);
}
} else {
out.writeInt(0);
}
HBaseConfiguration conf = new HBaseConfiguration();
ObjectWritable
.writeObject(out, keyGenerator, IndexKeyGenerator.class, conf);
}
/** {@inheritDoc} */
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("ID => ");
sb.append(indexId);
return sb.toString();
}
}

View File

@ -1,224 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Scanner;
import org.apache.hadoop.hbase.client.transactional.TransactionalTable;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.util.Bytes;
/** HTable extended with indexed support. */
public class IndexedTable extends TransactionalTable {
// FIXME, these belong elsewhere
static final byte[] INDEX_COL_FAMILY_NAME = Bytes.toBytes("__INDEX__");
static final byte[] INDEX_COL_FAMILY = Bytes.add(
INDEX_COL_FAMILY_NAME, new byte[] { HStoreKey.COLUMN_FAMILY_DELIMITER });
public static final byte[] INDEX_BASE_ROW_COLUMN = Bytes.add(
INDEX_COL_FAMILY, Bytes.toBytes("ROW"));
static final Log LOG = LogFactory.getLog(IndexedTable.class);
private Map<String, HTable> indexIdToTable = new HashMap<String, HTable>();
public IndexedTable(final HBaseConfiguration conf, final byte[] tableName)
throws IOException {
super(conf, tableName);
for (IndexSpecification spec : super.getTableDescriptor().getIndexes()) {
indexIdToTable.put(spec.getIndexId(), new HTable(conf, spec
.getIndexedTableName(tableName)));
}
}
/**
* Open up an indexed scanner. Results will come back in the indexed order,
* but will contain RowResults from the original table.
*
* @param indexId the id of the index to use
* @param indexStartRow (created from the IndexKeyGenerator)
* @param indexColumns in the index table
* @param indexFilter filter to run on the index'ed table. This can only use
* columns that have been added to the index.
* @param baseColumns from the original table
* @return scanner
* @throws IOException
* @throws IndexNotFoundException
*/
public Scanner getIndexedScanner(String indexId, final byte[] indexStartRow,
byte[][] indexColumns, final RowFilterInterface indexFilter,
final byte[][] baseColumns) throws IOException, IndexNotFoundException {
IndexSpecification indexSpec = super.getTableDescriptor().getIndex(indexId);
if (indexSpec == null) {
throw new IndexNotFoundException("Index " + indexId
+ " not defined in table "
+ super.getTableDescriptor().getNameAsString());
}
verifyIndexColumns(indexColumns, indexSpec);
// TODO, verify/remove index columns from baseColumns
HTable indexTable = indexIdToTable.get(indexId);
byte[][] allIndexColumns;
if (indexColumns != null) {
allIndexColumns = new byte[indexColumns.length + 1][];
System
.arraycopy(indexColumns, 0, allIndexColumns, 0, indexColumns.length);
allIndexColumns[indexColumns.length] = INDEX_BASE_ROW_COLUMN;
} else {
byte[][] allColumns = indexSpec.getAllColumns();
allIndexColumns = new byte[allColumns.length + 1][];
System.arraycopy(allColumns, 0, allIndexColumns, 0, allColumns.length);
allIndexColumns[allColumns.length] = INDEX_BASE_ROW_COLUMN;
}
Scanner indexScanner = indexTable.getScanner(allIndexColumns,
indexStartRow, indexFilter);
return new ScannerWrapper(indexScanner, baseColumns);
}
private void verifyIndexColumns(byte[][] requestedColumns,
IndexSpecification indexSpec) {
if (requestedColumns == null) {
return;
}
for (byte[] requestedColumn : requestedColumns) {
boolean found = false;
for (byte[] indexColumn : indexSpec.getAllColumns()) {
if (Bytes.equals(requestedColumn, indexColumn)) {
found = true;
break;
}
}
if (!found) {
throw new RuntimeException("Column [" + Bytes.toString(requestedColumn)
+ "] not in index " + indexSpec.getIndexId());
}
}
}
private class ScannerWrapper implements Scanner {
private Scanner indexScanner;
private byte[][] columns;
public ScannerWrapper(Scanner indexScanner, byte[][] columns) {
this.indexScanner = indexScanner;
this.columns = columns;
}
/** {@inheritDoc} */
public RowResult next() throws IOException {
RowResult[] result = next(1);
if (result == null || result.length < 1)
return null;
return result[0];
}
/** {@inheritDoc} */
public RowResult[] next(int nbRows) throws IOException {
RowResult[] indexResult = indexScanner.next(nbRows);
if (indexResult == null) {
return null;
}
RowResult[] result = new RowResult[indexResult.length];
for (int i = 0; i < indexResult.length; i++) {
RowResult row = indexResult[i];
byte[] baseRow = row.get(INDEX_BASE_ROW_COLUMN).getValue();
LOG.debug("next index row [" + Bytes.toString(row.getRow())
+ "] -> base row [" + Bytes.toString(baseRow) + "]");
HbaseMapWritable<byte[], Cell> colValues =
new HbaseMapWritable<byte[], Cell>();
if (columns != null && columns.length > 0) {
LOG.debug("Going to base table for remaining columns");
RowResult baseResult = IndexedTable.this.getRow(baseRow, columns);
if (baseResult != null) {
colValues.putAll(baseResult);
}
}
for (Entry<byte[], Cell> entry : row.entrySet()) {
byte[] col = entry.getKey();
if (HStoreKey.matchingFamily(INDEX_COL_FAMILY_NAME, col)) {
continue;
}
colValues.put(col, entry.getValue());
}
result[i] = new RowResult(baseRow, colValues);
}
return result;
}
/** {@inheritDoc} */
public void close() {
indexScanner.close();
}
/** {@inheritDoc} */
public Iterator<RowResult> iterator() {
// FIXME, copied from HTable.ClientScanner. Extract this to common base
// class?
return new Iterator<RowResult>() {
RowResult next = null;
public boolean hasNext() {
if (next == null) {
try {
next = ScannerWrapper.this.next();
return next != null;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
return true;
}
public RowResult next() {
if (!hasNext()) {
return null;
}
RowResult temp = next;
next = null;
return temp;
}
public void remove() {
throw new UnsupportedOperationException();
}
};
}
}
}

View File

@ -1,154 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.IOException;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ColumnNameParseException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.regionserver.tableindexed.IndexMaintenanceUtils;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Extension of HBaseAdmin that creates indexed tables.
*
*/
public class IndexedTableAdmin extends HBaseAdmin {
private static final Log LOG = LogFactory.getLog(IndexedTableAdmin.class);
/**
* Constructor
*
* @param conf Configuration object
* @throws MasterNotRunningException
*/
public IndexedTableAdmin(HBaseConfiguration conf)
throws MasterNotRunningException {
super(conf);
}
/**
* Creates a new table
*
* @param desc table descriptor for table
*
* @throws IllegalArgumentException if the table name is reserved
* @throws MasterNotRunningException if master is not running
* @throws TableExistsException if table already exists (If concurrent
* threads, the table may have been created between test-for-existence and
* attempt-at-creation).
* @throws IOException
*/
@Override
public void createTable(HTableDescriptor desc) throws IOException {
super.createTable(desc);
this.createIndexTables(desc);
}
private void createIndexTables(HTableDescriptor tableDesc) throws IOException {
byte[] baseTableName = tableDesc.getName();
for (IndexSpecification indexSpec : tableDesc.getIndexes()) {
HTableDescriptor indexTableDesc = createIndexTableDesc(baseTableName,
indexSpec);
super.createTable(indexTableDesc);
}
}
private HTableDescriptor createIndexTableDesc(byte[] baseTableName,
IndexSpecification indexSpec) throws ColumnNameParseException {
HTableDescriptor indexTableDesc = new HTableDescriptor(indexSpec
.getIndexedTableName(baseTableName));
Set<byte[]> families = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
families.add(IndexedTable.INDEX_COL_FAMILY);
for (byte[] column : indexSpec.getAllColumns()) {
families.add(Bytes.add(HStoreKey.getFamily(column),
new byte[] { HStoreKey.COLUMN_FAMILY_DELIMITER }));
}
for (byte[] colFamily : families) {
indexTableDesc.addFamily(new HColumnDescriptor(colFamily));
}
return indexTableDesc;
}
/** Remove an index for a table.
* @throws IOException
*
*/
public void removeIndex(byte[] baseTableName, String indexId) throws IOException {
super.disableTable(baseTableName);
HTableDescriptor desc = super.getTableDescriptor(baseTableName);
IndexSpecification spec = desc.getIndex(indexId);
desc.removeIndex(indexId);
this.disableTable(spec.getIndexedTableName(baseTableName));
this.deleteTable(spec.getIndexedTableName(baseTableName));
super.modifyTable(baseTableName, desc);
super.enableTable(baseTableName);
}
/** Add an index to a table. */
public void addIndex(byte []baseTableName, IndexSpecification indexSpec) throws IOException {
LOG.warn("Adding index to existing table ["+Bytes.toString(baseTableName)+"], this may take a long time");
// TODO, make table read-only
LOG.warn("Not putting table in readonly, if its being written to, the index may get out of sync");
HTableDescriptor indexTableDesc = createIndexTableDesc(baseTableName, indexSpec);
super.createTable(indexTableDesc);
super.disableTable(baseTableName);
HTableDescriptor desc = super.getTableDescriptor(baseTableName);
desc.addIndex(indexSpec);
super.modifyTable(baseTableName, desc);
super.enableTable(baseTableName);
reIndexTable(baseTableName, indexSpec);
}
private void reIndexTable(byte[] baseTableName, IndexSpecification indexSpec) throws IOException {
HTable baseTable = new HTable(baseTableName);
HTable indexTable = new HTable(indexSpec.getIndexedTableName(baseTableName));
for (RowResult rowResult : baseTable.getScanner(indexSpec.getAllColumns())) {
SortedMap<byte[], byte[]> columnValues = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
for (Entry<byte[], Cell> entry : rowResult.entrySet()) {
columnValues.put(entry.getKey(), entry.getValue().getValue());
}
if (IndexMaintenanceUtils.doesApplyToIndex(indexSpec, columnValues)) {
BatchUpdate indexUpdate = IndexMaintenanceUtils.createIndexUpdate(indexSpec, rowResult.getRow(), columnValues);
indexTable.commit(indexUpdate);
}
}
}
}

View File

@ -1,46 +0,0 @@
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
<html>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<head />
<body bgcolor="white">
This package provides support for secondary indexing by maintaining a separate, "index", table for each index.
The IndexSpecification class provides the metadata for the index. This includes:
<li> the columns that contribute to the index key,
<li> additional columns to put in the index table (and are thus made available to filters on the index table),
<br> and
<li> an IndexKeyGenerator which constructs the index-row-key from the indexed column(s) and the original row.
IndexesSpecifications can be added to a table's metadata (HTableDescriptor) before the table is constructed.
Afterwards, updates and deletes to the original table will trigger the updates in the index, and
the indexes can be scanned using the API on IndexedTable.
For a simple example, look at the unit test in org.apache.hadoop.hbase.client.tableIndexed.
<p> To enable the indexing, modify hbase-site.xml to turn on the
IndexedRegionServer. This is done by setting
<i>hbase.regionserver.class</i> to
<i>org.apache.hadoop.hbase.ipc.IndexedRegionInterface</i> and
<i>hbase.regionserver.impl </i> to
<i>org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegionServer</i>
</body>
</html>

View File

@ -1,71 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Random;
/**
* A local, in-memory implementation of the transaction logger. Does not provide a global view, so
* it can't be relighed on by
*
*/
public class LocalTransactionLogger implements TransactionLogger {
private static LocalTransactionLogger instance;
/**
* Creates singleton if it does not exist
*
* @return reference to singleton
*/
public synchronized static LocalTransactionLogger getInstance() {
if (instance == null) {
instance = new LocalTransactionLogger();
}
return instance;
}
private Random random = new Random();
private Map<Long, TransactionStatus> transactionIdToStatusMap = Collections
.synchronizedMap(new HashMap<Long, TransactionStatus>());
private LocalTransactionLogger() {
// Enforce singlton
}
/** @return random longs to minimize possibility of collision */
public long createNewTransactionLog() {
long id = random.nextLong();
transactionIdToStatusMap.put(id, TransactionStatus.PENDING);
return id;
}
public TransactionStatus getStatusForTransaction(final long transactionId) {
return transactionIdToStatusMap.get(transactionId);
}
public void setStatusForTransaction(final long transactionId,
final TransactionStatus status) {
transactionIdToStatusMap.put(transactionId, status);
}
}

View File

@ -1,59 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
/**
* Simple interface used to provide a log about transaction status. Written to
* by the client, and read by regionservers in case of failure.
*
*/
public interface TransactionLogger {
/** Transaction status values */
enum TransactionStatus {
/** Transaction is pending */
PENDING,
/** Transaction was committed */
COMMITTED,
/** Transaction was aborted */
ABORTED
}
/**
* Create a new transaction log. Return the transaction's globally unique id.
* Log's initial value should be PENDING
*
* @return transaction id
*/
long createNewTransactionLog();
/**
* @param transactionId
* @return transaction status
*/
TransactionStatus getStatusForTransaction(long transactionId);
/**
* @param transactionId
* @param status
*/
void setStatusForTransaction(long transactionId, TransactionStatus status);
}

View File

@ -1,152 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
import org.apache.hadoop.ipc.RemoteException;
/**
* Transaction Manager. Responsible for committing transactions.
*
*/
public class TransactionManager {
static final Log LOG = LogFactory.getLog(TransactionManager.class);
private final HConnection connection;
private final TransactionLogger transactionLogger;
/**
* @param conf
*/
public TransactionManager(final HBaseConfiguration conf) {
this(LocalTransactionLogger.getInstance(), conf);
}
/**
* @param transactionLogger
* @param conf
*/
public TransactionManager(final TransactionLogger transactionLogger,
final HBaseConfiguration conf) {
this.transactionLogger = transactionLogger;
connection = HConnectionManager.getConnection(conf);
}
/**
* Called to start a transaction.
*
* @return new transaction state
*/
public TransactionState beginTransaction() {
long transactionId = transactionLogger.createNewTransactionLog();
LOG.debug("Begining transaction " + transactionId);
return new TransactionState(transactionId);
}
/**
* Try and commit a transaction.
*
* @param transactionState
* @throws IOException
* @throws CommitUnsuccessfulException
*/
public void tryCommit(final TransactionState transactionState)
throws CommitUnsuccessfulException, IOException {
LOG.debug("atempting to commit trasaction: " + transactionState.toString());
try {
for (HRegionLocation location : transactionState
.getParticipatingRegions()) {
TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection
.getHRegionConnection(location.getServerAddress());
boolean canCommit = transactionalRegionServer.commitRequest(location
.getRegionInfo().getRegionName(), transactionState
.getTransactionId());
if (LOG.isTraceEnabled()) {
LOG.trace("Region ["
+ location.getRegionInfo().getRegionNameAsString() + "] votes "
+ (canCommit ? "to commit" : "to abort") + " transaction "
+ transactionState.getTransactionId());
}
if (!canCommit) {
LOG.debug("Aborting [" + transactionState.getTransactionId() + "]");
abort(transactionState, location);
throw new CommitUnsuccessfulException();
}
}
LOG.debug("Commiting [" + transactionState.getTransactionId() + "]");
transactionLogger.setStatusForTransaction(transactionState
.getTransactionId(), TransactionLogger.TransactionStatus.COMMITTED);
for (HRegionLocation location : transactionState
.getParticipatingRegions()) {
TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection
.getHRegionConnection(location.getServerAddress());
transactionalRegionServer.commit(location.getRegionInfo()
.getRegionName(), transactionState.getTransactionId());
}
} catch (RemoteException e) {
LOG.debug("Commit of transaction [" + transactionState.getTransactionId()
+ "] was unsucsessful", e);
// FIXME, think about the what ifs
throw new CommitUnsuccessfulException(e);
}
// Tran log can be deleted now ...
}
/**
* Abort a s transaction.
*
* @param transactionState
* @throws IOException
*/
public void abort(final TransactionState transactionState) throws IOException {
abort(transactionState, null);
}
private void abort(final TransactionState transactionState,
final HRegionLocation locationToIgnore) throws IOException {
transactionLogger.setStatusForTransaction(transactionState
.getTransactionId(), TransactionLogger.TransactionStatus.ABORTED);
for (HRegionLocation location : transactionState.getParticipatingRegions()) {
if (locationToIgnore != null && location.equals(locationToIgnore)) {
continue;
}
TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection
.getHRegionConnection(location.getServerAddress());
transactionalRegionServer.abort(location.getRegionInfo().getRegionName(),
transactionState.getTransactionId());
}
}
}

View File

@ -1,51 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
class TransactionScannerCallable extends ScannerCallable {
private TransactionState transactionState;
TransactionScannerCallable(final TransactionState transactionState,
final HConnection connection, final byte[] tableName,
final byte[][] columns, final byte[] startRow, final long timestamp,
final RowFilterInterface filter) {
super(connection, tableName, columns, startRow, timestamp, filter);
this.transactionState = transactionState;
}
@Override
protected long openScanner() throws IOException {
if (transactionState.addRegion(location)) {
((TransactionalRegionInterface) server).beginTransaction(transactionState
.getTransactionId(), location.getRegionInfo().getRegionName());
}
return ((TransactionalRegionInterface) server).openScanner(transactionState
.getTransactionId(), this.location.getRegionInfo().getRegionName(),
getColumns(), row, getTimestamp(), getFilter());
}
}

View File

@ -1,78 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.util.HashSet;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionLocation;
/**
* Holds client-side transaction information. Client's use them as opaque
* objects passed around to transaction operations.
*
*/
public class TransactionState {
static final Log LOG = LogFactory.getLog(TransactionState.class);
private final long transactionId;
private Set<HRegionLocation> participatingRegions = new HashSet<HRegionLocation>();
TransactionState(final long transactionId) {
this.transactionId = transactionId;
}
boolean addRegion(final HRegionLocation hregion) {
boolean added = participatingRegions.add(hregion);
if (added) {
LOG.debug("Adding new hregion ["
+ hregion.getRegionInfo().getRegionNameAsString()
+ "] to transaction [" + transactionId + "]");
}
return added;
}
Set<HRegionLocation> getParticipatingRegions() {
return participatingRegions;
}
/**
* Get the transactionId.
*
* @return Return the transactionId.
*/
public long getTransactionId() {
return transactionId;
}
/**
* @see java.lang.Object#toString()
*/
@Override
public String toString() {
return "id: " + transactionId + ", particpants: "
+ participatingRegions.size();
}
}

View File

@ -1,428 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Scanner;
import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.client.ServerCallable;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
/**
* Table with transactional support.
*
*/
public class TransactionalTable extends HTable {
/**
* @param conf
* @param tableName
* @throws IOException
*/
public TransactionalTable(final HBaseConfiguration conf,
final String tableName) throws IOException {
super(conf, tableName);
}
/**
* @param conf
* @param tableName
* @throws IOException
*/
public TransactionalTable(final HBaseConfiguration conf,
final byte[] tableName) throws IOException {
super(conf, tableName);
}
private static abstract class TransactionalServerCallable<T> extends
ServerCallable<T> {
protected TransactionState transactionState;
protected TransactionalRegionInterface getTransactionServer() {
return (TransactionalRegionInterface) server;
}
protected void recordServer() throws IOException {
if (transactionState.addRegion(location)) {
getTransactionServer().beginTransaction(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName());
}
}
/**
* @param connection
* @param tableName
* @param row
* @param transactionState
*/
public TransactionalServerCallable(final HConnection connection,
final byte[] tableName, final byte[] row,
final TransactionState transactionState) {
super(connection, tableName, row);
this.transactionState = transactionState;
}
}
/**
* Get a single value for the specified row and column
*
* @param transactionState
* @param row row key
* @param column column name
* @return value for specified row/column
* @throws IOException
*/
public Cell get(final TransactionState transactionState, final byte[] row,
final byte[] column) throws IOException {
return super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<Cell>(super.getConnection(), super
.getTableName(), row, transactionState) {
public Cell call() throws IOException {
recordServer();
return getTransactionServer().get(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), row, column);
}
});
}
/**
* Get the specified number of versions of the specified row and column
*
* @param transactionState
* @param row - row key
* @param column - column name
* @param numVersions - number of versions to retrieve
* @return - array byte values
* @throws IOException
*/
public Cell[] get(final TransactionState transactionState, final byte[] row,
final byte[] column, final int numVersions) throws IOException {
Cell[] values = null;
values = super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<Cell[]>(super.getConnection(), super
.getTableName(), row, transactionState) {
public Cell[] call() throws IOException {
recordServer();
return getTransactionServer().get(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), row, column,
numVersions);
}
});
return values;
}
/**
* Get the specified number of versions of the specified row and column with
* the specified timestamp.
*
* @param transactionState
* @param row - row key
* @param column - column name
* @param timestamp - timestamp
* @param numVersions - number of versions to retrieve
* @return - array of values that match the above criteria
* @throws IOException
*/
public Cell[] get(final TransactionState transactionState, final byte[] row,
final byte[] column, final long timestamp, final int numVersions)
throws IOException {
Cell[] values = null;
values = super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<Cell[]>(super.getConnection(), super
.getTableName(), row, transactionState) {
public Cell[] call() throws IOException {
recordServer();
return getTransactionServer().get(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), row, column,
timestamp, numVersions);
}
});
return values;
}
/**
* Get all the data for the specified row at the latest timestamp
*
* @param transactionState
* @param row row key
* @return RowResult is empty if row does not exist.
* @throws IOException
*/
public RowResult getRow(final TransactionState transactionState,
final byte[] row) throws IOException {
return getRow(transactionState, row, HConstants.LATEST_TIMESTAMP);
}
/**
* Get all the data for the specified row at a specified timestamp
*
* @param transactionState
* @param row row key
* @param ts timestamp
* @return RowResult is empty if row does not exist.
* @throws IOException
*/
public RowResult getRow(final TransactionState transactionState,
final byte[] row, final long ts) throws IOException {
return super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<RowResult>(super.getConnection(), super
.getTableName(), row, transactionState) {
public RowResult call() throws IOException {
recordServer();
return getTransactionServer().getRow(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), row, ts);
}
});
}
/**
* Get selected columns for the specified row at the latest timestamp
*
* @param transactionState
* @param row row key
* @param columns Array of column names you want to retrieve.
* @return RowResult is empty if row does not exist.
* @throws IOException
*/
public RowResult getRow(final TransactionState transactionState,
final byte[] row, final byte[][] columns) throws IOException {
return getRow(transactionState, row, columns, HConstants.LATEST_TIMESTAMP);
}
/**
* Get selected columns for the specified row at a specified timestamp
*
* @param transactionState
* @param row row key
* @param columns Array of column names you want to retrieve.
* @param ts timestamp
* @return RowResult is empty if row does not exist.
* @throws IOException
*/
public RowResult getRow(final TransactionState transactionState,
final byte[] row, final byte[][] columns, final long ts)
throws IOException {
return super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<RowResult>(super.getConnection(), super
.getTableName(), row, transactionState) {
public RowResult call() throws IOException {
recordServer();
return getTransactionServer().getRow(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), row, columns, ts);
}
});
}
/**
* Delete all cells that match the passed row and whose timestamp is equal-to
* or older than the passed timestamp.
*
* @param transactionState
* @param row Row to update
* @param ts Delete all cells of the same timestamp or older.
* @throws IOException
*/
public void deleteAll(final TransactionState transactionState,
final byte[] row, final long ts) throws IOException {
super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<Boolean>(super.getConnection(), super
.getTableName(), row, transactionState) {
public Boolean call() throws IOException {
recordServer();
getTransactionServer().deleteAll(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), row, ts);
return null;
}
});
}
/**
* Get a scanner on the current table starting at first row. Return the
* specified columns.
*
* @param transactionState
* @param columns columns to scan. If column name is a column family, all
* columns of the specified column family are returned. Its also possible to
* pass a regex in the column qualifier. A column qualifier is judged to be a
* regex if it contains at least one of the following characters:
* <code>\+|^&*$[]]}{)(</code>.
* @return scanner
* @throws IOException
*/
public Scanner getScanner(final TransactionState transactionState,
final byte[][] columns) throws IOException {
return getScanner(transactionState, columns, HConstants.EMPTY_START_ROW,
HConstants.LATEST_TIMESTAMP, null);
}
/**
* Get a scanner on the current table starting at the specified row. Return
* the specified columns.
*
* @param transactionState
* @param columns columns to scan. If column name is a column family, all
* columns of the specified column family are returned. Its also possible to
* pass a regex in the column qualifier. A column qualifier is judged to be a
* regex if it contains at least one of the following characters:
* <code>\+|^&*$[]]}{)(</code>.
* @param startRow starting row in table to scan
* @return scanner
* @throws IOException
*/
public Scanner getScanner(final TransactionState transactionState,
final byte[][] columns, final byte[] startRow) throws IOException {
return getScanner(transactionState, columns, startRow,
HConstants.LATEST_TIMESTAMP, null);
}
/**
* Get a scanner on the current table starting at the specified row. Return
* the specified columns.
*
* @param transactionState
* @param columns columns to scan. If column name is a column family, all
* columns of the specified column family are returned. Its also possible to
* pass a regex in the column qualifier. A column qualifier is judged to be a
* regex if it contains at least one of the following characters:
* <code>\+|^&*$[]]}{)(</code>.
* @param startRow starting row in table to scan
* @param timestamp only return results whose timestamp <= this value
* @return scanner
* @throws IOException
*/
public Scanner getScanner(final TransactionState transactionState,
final byte[][] columns, final byte[] startRow, final long timestamp)
throws IOException {
return getScanner(transactionState, columns, startRow, timestamp, null);
}
/**
* Get a scanner on the current table starting at the specified row. Return
* the specified columns.
*
* @param transactionState
* @param columns columns to scan. If column name is a column family, all
* columns of the specified column family are returned. Its also possible to
* pass a regex in the column qualifier. A column qualifier is judged to be a
* regex if it contains at least one of the following characters:
* <code>\+|^&*$[]]}{)(</code>.
* @param startRow starting row in table to scan
* @param filter a row filter using row-key regexp and/or column data filter.
* @return scanner
* @throws IOException
*/
public Scanner getScanner(final TransactionState transactionState,
final byte[][] columns, final byte[] startRow,
final RowFilterInterface filter) throws IOException {
return getScanner(transactionState, columns, startRow,
HConstants.LATEST_TIMESTAMP, filter);
}
/**
* Get a scanner on the current table starting at the specified row. Return
* the specified columns.
*
* @param transactionState
* @param columns columns to scan. If column name is a column family, all
* columns of the specified column family are returned. Its also possible to
* pass a regex in the column qualifier. A column qualifier is judged to be a
* regex if it contains at least one of the following characters:
* <code>\+|^&*$[]]}{)(</code>.
* @param startRow starting row in table to scan
* @param timestamp only return results whose timestamp <= this value
* @param filter a row filter using row-key regexp and/or column data filter.
* @return scanner
* @throws IOException
*/
public Scanner getScanner(final TransactionState transactionState,
final byte[][] columns, final byte[] startRow, final long timestamp,
final RowFilterInterface filter) throws IOException {
ClientScanner scanner = new TransactionalClientScanner(transactionState, columns, startRow,
timestamp, filter);
scanner.initialize();
return scanner;
}
/**
* Commit a BatchUpdate to the table.
*
* @param transactionState
* @param batchUpdate
* @throws IOException
*/
public synchronized void commit(final TransactionState transactionState,
final BatchUpdate batchUpdate) throws IOException {
super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<Boolean>(super.getConnection(), super
.getTableName(), batchUpdate.getRow(), transactionState) {
public Boolean call() throws IOException {
recordServer();
getTransactionServer().batchUpdate(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), batchUpdate);
return null;
}
});
}
protected class TransactionalClientScanner extends HTable.ClientScanner {
private TransactionState transactionState;
protected TransactionalClientScanner(
final TransactionState transactionState, final byte[][] columns,
final byte[] startRow, final long timestamp,
final RowFilterInterface filter) {
super(columns, startRow, timestamp, filter);
this.transactionState = transactionState;
}
@Override
protected ScannerCallable getScannerCallable(
final byte[] localStartKey, int caching) {
TransactionScannerCallable t =
new TransactionScannerCallable(transactionState, getConnection(),
getTableName(), getColumns(), localStartKey, getTimestamp(),
getFilter());
t.setCaching(caching);
return t;
}
}
}

View File

@ -1,43 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import org.apache.hadoop.hbase.DoNotRetryIOException;
/**
* Thrown if a region server is passed an unknown transaction id
*/
public class UnknownTransactionException extends DoNotRetryIOException {
private static final long serialVersionUID = 698575374929591099L;
/** constructor */
public UnknownTransactionException() {
super();
}
/**
* Constructor
* @param s message
*/
public UnknownTransactionException(String s) {
super(s);
}
}

View File

@ -1,61 +0,0 @@
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
<html>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<head />
<body bgcolor="white">
This package provides support for atomic transactions. Transactions can
span multiple regions. Transaction writes are applied when committing a
transaction. At commit time, the transaction is examined to see if it
can be applied while still maintaining atomicity. This is done by
looking for conflicts with the transactions that committed while the
current transaction was running. This technique is known as optimistic
concurrency control (OCC) because it relies on the assumption that
transactions will mostly not have conflicts with each other.
<p>
For more details on OCC, see the paper <i> On Optimistic Methods for Concurrency Control </i>
by Kung and Robinson available
<a href=http://www.seas.upenn.edu/~zives/cis650/papers/opt-cc.pdf> here </a>.
<p> To enable transactions, modify hbase-site.xml to turn on the
TransactionalRegionServer. This is done by setting
<i>hbase.regionserver.class</i> to
<i>org.apache.hadoop.hbase.ipc.TransactionalRegionInterface</i> and
<i>hbase.regionserver.impl </i> to
<i>org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegionServer</i>
<p>
The read set claimed by a transactional scanner is determined from the start and
end keys which the scanner is opened with.
<h3> Known Issues </h3>
Recovery in the face of hregion server failure
is not fully implemented. Thus, you cannot rely on the transactional
properties in the face of node failure.
</body>
</html>

View File

@ -33,12 +33,15 @@ import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.io.ObjectWritable;
/**
* This filter is a no-op in HBase 0.20. Don't use it.
*
* This filter is used to filter based on the value of a given column. It takes
* an operator (equal, greater, not equal, etc) and either a byte [] value or a
* byte [] comparator. If we have a byte [] value then we just do a
* lexicographic compare. If this is not sufficient (eg you want to deserialize
* a long and then compare it to a fixed long value), then you can pass in your
* own comparator instead.
* @deprecated Use filters that are rooted on @{link Filter} instead
*/
public class ColumnValueFilter implements RowFilterInterface {
/** Comparison operators. */

View File

@ -0,0 +1,111 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.filter;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.hbase.KeyValue;
/**
* Interface for row and column filters directly applied within the regionserver.
* A filter can expect the following call sequence:
*<ul>
* <li>{@link #reset()}</li>
* <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
* <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
* if false, we will also call</li>
* <li>{@link #filterKeyValue(KeyValue)} -> true to drop this key/value</li>
* <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
* filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
* </li>
* </ul>
*
* Filter instances are created one per region/scan. This interface replaces
* the old RowFilterInterface.
*/
public interface Filter extends Writable {
/**
* Reset the state of the filter between rows.
*/
public void reset();
/**
* Filters a row based on the row key. If this returns true, the entire
* row will be excluded. If false, each KeyValue in the row will be
* passed to {@link #filterKeyValue(KeyValue)} below.
*
* @param buffer buffer containing row key
* @param offset offset into buffer where row key starts
* @param length length of the row key
* @return true, remove entire row, false, include the row (maybe).
*/
public boolean filterRowKey(byte [] buffer, int offset, int length);
/**
* If this returns true, the scan will terminate.
*
* @return true to end scan, false to continue.
*/
public boolean filterAllRemaining();
/**
* A way to filter based on the column family, column qualifier and/or the
* column value. Return code is described below. This allows filters to
* filter only certain number of columns, then terminate without matching ever
* column.
*
* If your filter returns <code>ReturnCode.NEXT_ROW</code>, it should return
* <code>ReturnCode.NEXT_ROW</code> until {@link #reset()} is called
* just in case the caller calls for the next row.
*
* @param v the KeyValue in question
* @return code as described below
* @see {@link Filter.ReturnCode}
*/
public ReturnCode filterKeyValue(KeyValue v);
/**
* Return codes for filterValue().
*/
public enum ReturnCode {
/**
* Include the KeyValue
*/
INCLUDE,
/**
* Skip this KeyValue
*/
SKIP,
/**
* Done with columns, skip to next row. Note that filterRow() will
* still be called.
*/
NEXT_ROW,
};
/**
* Last chance to veto row based on previous {@link #filterKeyValue(KeyValue)}
* calls. The filter needs to retain state then return a particular value for
* this call if they wish to exclude a row if a certain column is missing
* (for example).
* @return true to exclude row, false to include row.
*/
public boolean filterRow();
}

View File

@ -24,8 +24,10 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* Subclass of StopRowFilter that filters rows > the stop row,
* making it include up to the last row but no further.
*
* @deprecated Use filters that are rooted on @{link Filter} instead
*/
public class InclusiveStopRowFilter extends StopRowFilter{
public class InclusiveStopRowFilter extends StopRowFilter {
/**
* Default constructor, filters nothing. Required though for RPC
* deserialization.
@ -46,12 +48,17 @@ public class InclusiveStopRowFilter extends StopRowFilter{
*/
@Override
public boolean filterRowKey(final byte [] rowKey) {
return filterRowKey(rowKey, 0, rowKey.length);
}
public boolean filterRowKey(byte []rowKey, int offset, int length) {
if (rowKey == null) {
if (getStopRowKey() == null) {
return true;
}
return false;
}
return Bytes.compareTo(getStopRowKey(), rowKey) < 0;
}
return Bytes.compareTo(getStopRowKey(), 0, getStopRowKey().length,
rowKey, offset, length) < 0;
}
}

View File

@ -0,0 +1,92 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.filter;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.KeyValue;
/**
* Implementation of Filter interface that limits results to a specific page
* size. It terminates scanning once the number of filter-passed results is >=
* the given page size.
*
* <p>
* Note that this filter cannot guarantee that the number of results returned
* to a client are <= page size. This is because the filter is applied
* separately on different region servers. It does however optimize the scan of
* individual HRegions by making sure that the page size is never exceeded
* locally.
* </p>
*/
public class PageFilter implements Filter {
private long pageSize = Long.MAX_VALUE;
private int rowsAccepted = 0;
/**
* Default constructor, filters nothing. Required though for RPC
* deserialization.
*/
public PageFilter() {
super();
}
/**
* Constructor that takes a maximum page size.
*
* @param pageSize Maximum result size.
*/
public PageFilter(final long pageSize) {
this.pageSize = pageSize;
}
public void reset() {
rowsAccepted = 0;
}
public boolean filterAllRemaining() {
return this.rowsAccepted >= this.pageSize;
}
public boolean filterRowKey(byte[] rowKey, int offset, int length) {
return filterAllRemaining();
}
public void readFields(final DataInput in) throws IOException {
this.pageSize = in.readLong();
}
public void write(final DataOutput out) throws IOException {
out.writeLong(pageSize);
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
this.rowsAccepted++;
return filterAllRemaining()? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
}
@Override
public boolean filterRow() {
return filterAllRemaining();
}
}

View File

@ -40,6 +40,8 @@ import org.apache.hadoop.hbase.io.Cell;
* individual HRegions by making sure that the page size is never exceeded
* locally.
* </p>
*
* @deprecated Use filters that are rooted on @{link Filter} instead
*/
public class PageRowFilter implements RowFilterInterface {

View File

@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* RowFilterInterface that filters everything that does not match a prefix
*
* @deprecated Use filters that are rooted on @{link Filter} instead
*/
public class PrefixRowFilter implements RowFilterInterface {
protected byte[] prefix;

View File

@ -30,7 +30,18 @@ import org.apache.hadoop.io.Writable;
*
* Interface used for row-level filters applied to HRegion.HScanner scan
* results during calls to next().
* TODO: Make Filters use proper comparator comparing rows.
*
* In HBase 0.20, not all of the functions will be called, thus filters which depend
* on them will not work as advertised!
*
* Specifically, you can only count on the following methods to be called:
* boolean filterRowKey(final byte [] rowKey, final int offset, final int length);
* boolean filterAllRemaining();
*
* Complex filters that depend in more need to be rewritten to work with @{link Filter}
*
* Write new filters to use the @{link Filter} API instead.
* @deprecated Use filters that are rooted on @{link Filter} instead
*/
public interface RowFilterInterface extends Writable {
/**

View File

@ -38,6 +38,10 @@ import org.apache.hadoop.io.ObjectWritable;
* which will be evaluated with a specified boolean operator MUST_PASS_ALL
* (!AND) or MUST_PASS_ONE (!OR). Since you can use RowFilterSets as children
* of RowFilterSet, you can create a hierarchy of filters to be evaluated.
*
* It is highly likely this construct will no longer work!
*
* @deprecated Use filters that are rooted on @{link Filter} instead
*/
public class RowFilterSet implements RowFilterInterface {

View File

@ -0,0 +1,89 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.filter;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.DataOutput;
import java.io.IOException;
import java.io.DataInput;
/**
* A Filter that stops after the given row. There is no "RowStopFilter" because the Scan
* spec allows you to specify a stop row.
*
* Use this filter to include the stop row, eg: [A,Z].
*/
public class RowInclusiveStopFilter implements Filter {
private byte [] stopRowKey;
public RowInclusiveStopFilter() {
super();
}
public RowInclusiveStopFilter(final byte [] stopRowKey) {
this.stopRowKey = stopRowKey;
}
@Override
public void reset() {
// noop, no state
}
@Override
public boolean filterRowKey(byte[] buffer, int offset, int length) {
if (buffer == null) {
if (this.stopRowKey == null) {
return true; //filter...
}
return false;
}
// if stopRowKey is <= buffer, then true, filter row.
return Bytes.compareTo(stopRowKey, 0, stopRowKey.length, buffer, offset, length) < 0;
}
@Override
public boolean filterAllRemaining() {
return false;
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
// include everything.
return ReturnCode.INCLUDE;
}
@Override
public boolean filterRow() {
return false;
}
@Override
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.stopRowKey);
}
@Override
public void readFields(DataInput in) throws IOException {
this.stopRowKey = Bytes.readByteArray(in);
}
}

View File

@ -1,5 +1,5 @@
/**
* Copyright 2008 The Apache Software Foundation
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -17,43 +17,64 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Map;
package org.apache.hadoop.hbase.filter;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
/** Creates indexed keys for a single column....
*
*/
public class SimpleIndexKeyGenerator implements IndexKeyGenerator {
import java.io.DataOutput;
import java.io.IOException;
import java.io.DataInput;
private byte [] column;
public SimpleIndexKeyGenerator(byte [] column) {
this.column = column;
}
public SimpleIndexKeyGenerator() {
// For Writable
}
/** {@inheritDoc} */
public byte[] createIndexKey(byte[] rowKey, Map<byte[], byte[]> columns) {
return Bytes.add(columns.get(column), rowKey);
public class RowPrefixFilter implements Filter {
protected byte [] prefix;
public RowPrefixFilter(final byte [] prefix) {
this.prefix = prefix;
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
column = Bytes.readByteArray(in);
public RowPrefixFilter() {
}
/** {@inheritDoc} */
@Override
public void reset() {
}
@Override
public boolean filterRowKey(byte[] buffer, int offset, int length) {
if (buffer == null)
return true;
if (length < prefix.length)
return true;
// if they are equal, return false => pass row
// else return true, filter row
return Bytes.compareTo(buffer, offset, prefix.length, prefix, 0, prefix.length) != 0;
}
@Override
public boolean filterAllRemaining() {
return false;
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
return ReturnCode.INCLUDE;
}
@Override
public boolean filterRow() {
return false;
}
@Override
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, column);
Bytes.writeByteArray(out, prefix);
}
@Override
public void readFields(DataInput in) throws IOException {
prefix = Bytes.readByteArray(in);
}
}

View File

@ -0,0 +1,96 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.filter;
import org.apache.hadoop.hbase.KeyValue;
import java.io.DataOutput;
import java.io.IOException;
import java.io.DataInput;
/**
* A wrapper filter that filters everything after the first filtered row.
*/
public class RowWhileMatchFilter implements Filter {
private boolean filterAllRemaining = false;
private Filter filter;
public RowWhileMatchFilter() {
super();
}
public RowWhileMatchFilter(Filter filter) {
this.filter = filter;
}
@Override
public void reset() {
// no state.
}
private void changeFAR(boolean value) {
filterAllRemaining = filterAllRemaining || value;
}
@Override
public boolean filterRowKey(byte[] buffer, int offset, int length) {
changeFAR(filter.filterRowKey(buffer, offset, length));
return filterAllRemaining();
}
@Override
public boolean filterAllRemaining() {
return this.filterAllRemaining || this.filter.filterAllRemaining();
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
ReturnCode c = filter.filterKeyValue(v);
changeFAR(c != ReturnCode.INCLUDE);
return c;
}
@Override
public boolean filterRow() {
return false;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeUTF(this.filter.getClass().getName());
this.filter.write(out);
}
@Override
public void readFields(DataInput in) throws IOException {
String className = in.readUTF();
try {
this.filter = (Filter)(Class.forName(className).newInstance());
this.filter.readFields(in);
} catch (InstantiationException e) {
throw new RuntimeException("Failed deserialize.", e);
} catch (IllegalAccessException e) {
throw new RuntimeException("Failed deserialize.", e);
} catch (ClassNotFoundException e) {
throw new RuntimeException("Failed deserialize.", e);
}
}
}

View File

@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* Implementation of RowFilterInterface that filters out rows greater than or
* equal to a specified rowKey.
*
* @deprecated Use filters that are rooted on @{link Filter} instead
*/
public class StopRowFilter implements RowFilterInterface {
private byte [] stopRowKey;

View File

@ -34,6 +34,8 @@ import org.apache.hadoop.hbase.io.Cell;
* filter(..) methods or filterNotNull(SortedMap<Text, byte[]>), this wrapper's
* filterAllRemaining() will return true. All filtering methods will
* thereafter defer to the result of filterAllRemaining().
*
* @deprecated Use filters that are rooted on @{link Filter} instead
*/
public class WhileMatchRowFilter implements RowFilterInterface {
private boolean filterAllRemaining = false;

View File

@ -17,7 +17,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**Provides row-level filters applied to HRegion scan results during calls to {@link org.apache.hadoop.hbase.client.Scanner#next()}.
/**Provides row-level filters applied to HRegion scan results during calls to {@link org.apache.hadoop.hbase.client.ResultScanner#next()}.
<p>Use {@link org.apache.hadoop.hbase.filter.StopRowFilter} to stop the scan once rows exceed the supplied row key.
Filters will not stop the scan unless hosted inside of a {@link org.apache.hadoop.hbase.filter.WhileMatchRowFilter}.

View File

@ -33,7 +33,8 @@ import org.apache.hadoop.io.Writable;
* This object is purposely bare-bones because many instances are created
* during bulk uploads. We have one class for DELETEs and PUTs rather than
* a class per type because it makes the serialization easier.
* @see BatchUpdate
* @see BatchUpdate
* @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API.
*/
public class BatchOperation implements Writable, HeapSize {
/**

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.io.WritableComparable;
* There is one BatchUpdate object per server, so a series of batch operations
* can result in multiple BatchUpdate objects if the batch contains rows that
* are served by multiple region servers.
* @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API.
*/
public class BatchUpdate
implements WritableComparable<BatchUpdate>, Iterable<BatchOperation>, HeapSize {

View File

@ -46,6 +46,7 @@ import agilejson.TOJSON;
* stored with together as a result for get and getRow methods. This promotes
* the timestamp of a cell to a first-class value, making it easy to take note
* of temporal data. Cell is used all the way from HStore up to HTable.
* @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API.
*/
public class Cell implements Writable, Iterable<Map.Entry<Long, byte[]>>,
ISerializable {
@ -228,7 +229,8 @@ public class Cell implements Writable, Iterable<Map.Entry<Long, byte[]>>,
* TODO: This is the glue between old way of doing things and the new.
* Herein we are converting our clean KeyValues to Map of Cells.
*/
public static HbaseMapWritable<byte [], Cell> createCells(final List<KeyValue> results) {
public static HbaseMapWritable<byte [], Cell> createCells(
final List<KeyValue> results) {
HbaseMapWritable<byte [], Cell> cells =
new HbaseMapWritable<byte [], Cell>();
// Walking backward through the list of results though it has no effect
@ -275,4 +277,4 @@ public class Cell implements Writable, Iterable<Map.Entry<Long, byte[]>>,
throws HBaseRestException {
serializer.serializeCell(this);
}
}
}

View File

@ -45,7 +45,7 @@ public interface CodeToClassAndBack {
/**
* Class list for supported classes
*/
public Class[] classList = {byte[].class, Cell.class};
public Class<?>[] classList = {byte[].class, Cell.class};
/**
* The static loader that is used instead of the static constructor in
@ -58,8 +58,8 @@ public interface CodeToClassAndBack {
* Class that loads the static maps with their values.
*/
public class InternalStaticLoader{
InternalStaticLoader(Class[] classList, Map<Byte, Class<?>> CODE_TO_CLASS,
Map<Class<?>, Byte> CLASS_TO_CODE){
InternalStaticLoader(Class<?>[] classList,
Map<Byte,Class<?>> CODE_TO_CLASS, Map<Class<?>, Byte> CLASS_TO_CODE){
byte code = 1;
for(int i=0; i<classList.length; i++){
CLASS_TO_CODE.put(classList[i], code);

View File

@ -32,11 +32,18 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HMsg;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.filter.RowFilterSet;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
@ -47,6 +54,7 @@ import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -89,52 +97,49 @@ public class HbaseObjectWritable implements Writable, Configurable {
addToMap(Float.TYPE, code++);
addToMap(Double.TYPE, code++);
addToMap(Void.TYPE, code++);
// Other java types
addToMap(String.class, code++);
addToMap(byte [].class, code++);
addToMap(byte [][].class, code++);
// Hadoop types
addToMap(Text.class, code++);
addToMap(Writable.class, code++);
addToMap(Writable [].class, code++);
addToMap(HbaseMapWritable.class, code++);
addToMap(NullInstance.class, code++);
try {
addToMap(Class.forName("[Lorg.apache.hadoop.io.Text;"), code++);
} catch (ClassNotFoundException e) {
e.printStackTrace();
}
// Hbase types
addToMap(HServerInfo.class, code++);
addToMap(HMsg.class, code++);
addToMap(HTableDescriptor.class, code++);
addToMap(HColumnDescriptor.class, code++);
addToMap(HConstants.Modify.class, code++);
addToMap(HMsg.class, code++);
addToMap(HMsg[].class, code++);
addToMap(RowFilterInterface.class, code++);
addToMap(RowFilterSet.class, code++);
addToMap(HRegion.class, code++);
addToMap(HRegion[].class, code++);
addToMap(HRegionInfo.class, code++);
addToMap(BatchUpdate.class, code++);
addToMap(HServerAddress.class, code++);
try {
addToMap(Class.forName("[Lorg.apache.hadoop.hbase.HMsg;"), code++);
} catch (ClassNotFoundException e) {
e.printStackTrace();
}
addToMap(Cell.class, code++);
try {
addToMap(Class.forName("[Lorg.apache.hadoop.hbase.io.Cell;"), code++);
} catch (ClassNotFoundException e) {
e.printStackTrace();
}
addToMap(RowResult.class, code++);
addToMap(HRegionInfo[].class, code++);
addToMap(HServerAddress.class, code++);
addToMap(HServerInfo.class, code++);
addToMap(HTableDescriptor.class, code++);
addToMap(MapWritable.class, code++);
try {
addToMap(Class.forName("[Lorg.apache.hadoop.hbase.io.RowResult;"), code++);
} catch (ClassNotFoundException e) {
e.printStackTrace();
}
addToMap(BatchUpdate[].class, code++);
//
// HBASE-880
//
addToMap(ClusterStatus.class, code++);
addToMap(Delete.class, code++);
addToMap(Get.class, code++);
addToMap(KeyValue.class, code++);
addToMap(KeyValue[].class, code++);
addToMap(Put.class, code++);
addToMap(Put[].class, code++);
addToMap(Result.class, code++);
addToMap(Result[].class, code++);
addToMap(Scan.class, code++);
}
private Class<?> declaredClass;
@ -228,6 +233,12 @@ public class HbaseObjectWritable implements Writable, Configurable {
Byte code = CLASS_TO_CODE.get(c);
if (code == null) {
LOG.error("Unsupported type " + c);
StackTraceElement[] els = new Exception().getStackTrace();
for(StackTraceElement elem : els) {
LOG.error(elem.getMethodName());
}
// new Exception().getStackTrace()[0].getMethodName());
// throw new IOException(new Exception().getStackTrace()[0].getMethodName());
throw new UnsupportedOperationException("No code for unexpected " + c);
}
out.writeByte(code);
@ -261,6 +272,8 @@ public class HbaseObjectWritable implements Writable, Configurable {
// byte-at-a-time we were previously doing.
if (declClass.equals(byte [].class)) {
Bytes.writeByteArray(out, (byte [])instanceObj);
} else if(declClass.equals(Result [].class)) {
Result.writeArray(out, (Result [])instanceObj);
} else {
int length = Array.getLength(instanceObj);
out.writeInt(length);
@ -363,6 +376,8 @@ public class HbaseObjectWritable implements Writable, Configurable {
} else if (declaredClass.isArray()) { // array
if (declaredClass.equals(byte [].class)) {
instance = Bytes.readByteArray(in);
} else if(declaredClass.equals(Result [].class)) {
instance = Result.readArray(in);
} else {
int length = in.readInt();
instance = Array.newInstance(declaredClass.getComponentType(), length);

View File

@ -21,11 +21,24 @@ package org.apache.hadoop.hbase.io;
/**
* Implementations can be asked for an estimate of their size in bytes.
* <p>
* Useful for sizing caches. Its a given that implementation approximations
* probably do not account for 32 vs 64 bit nor for different VM implemenations.
* do not account for 32 vs 64 bit nor for different VM implementations.
* <p>
* An Object's size is determined by the non-static data members in it,
* as well as the fixed {@link OBJECT} overhead.
* <p>
* For example:
* <pre>
* public class SampleObject implements HeapSize {
*
* int [] numbers;
* int x;
* }
* </pre>
*/
public interface HeapSize {
/** Reference size is 8 bytes on 64-bit, 4 bytes on 32-bit */
static final int REFERENCE = 8;
@ -49,10 +62,12 @@ public interface HeapSize {
static final int LONG = 8;
/** Array overhead */
static final int BYTE_ARRAY = REFERENCE;
static final int ARRAY = 3 * REFERENCE;
static final int MULTI_ARRAY = (4 * REFERENCE) + ARRAY;
/** Byte arrays are fixed size below plus its length, 8 byte aligned */
static final int BYTE_ARRAY = 3 * REFERENCE;
static final int BLOCK_SIZE_TAX = 8;
static final int BYTE_BUFFER = 56;

View File

@ -122,10 +122,22 @@ implements WritableComparable<ImmutableBytesWritable> {
return this.length;
}
/**
* @return the current length of the buffer. same as getSize()
*/
//Should probably deprecate getSize() so that we keep the same calls for all
//byte []
public int getLength() {
return getSize();
if (this.bytes == null) {
throw new IllegalStateException("Uninitialiized. Null constructor " +
"called w/o accompaying readFields invocation");
}
return this.length;
}
/**
* @return offset
*/
public int getOffset(){
return this.offset;
}

View File

@ -65,10 +65,17 @@ public class Reference implements Writable {
this(null, Range.bottom);
}
/**
*
* @return Range
*/
public Range getFileRegion() {
return this.region;
}
/**
* @return splitKey
*/
public byte [] getSplitKey() {
return splitkey;
}

View File

@ -32,7 +32,6 @@ import java.util.Set;
import java.util.SortedMap;
import java.util.TreeSet;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.rest.descriptors.RestCell;
import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
@ -46,11 +45,13 @@ import agilejson.TOJSON;
/**
* Holds row name and then a map of columns to cells.
* @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API.
*/
public class RowResult implements Writable, SortedMap<byte [], Cell>,
Comparable<RowResult>, ISerializable {
private byte [] row = null;
private final HbaseMapWritable<byte [], Cell> cells;
private final byte [] COL_REGIONINFO = Bytes.toBytes("info:regioninfo");
/** default constructor for writable */
public RowResult() {
@ -102,6 +103,11 @@ public class RowResult implements Writable, SortedMap<byte [], Cell>,
return cells.containsKey(key);
}
/**
* Check if the key can be found in this RowResult
* @param key
* @return true if key id found, false if not
*/
public boolean containsKey(String key) {
return cells.containsKey(Bytes.toBytes(key));
}
@ -175,6 +181,16 @@ public class RowResult implements Writable, SortedMap<byte [], Cell>,
public Cell get(String key) {
return get(Bytes.toBytes(key));
}
/**
* Get a cell using seperate family, columnQualifier arguments.
* @param family
* @param columnQualifier
* @return
*/
public Cell get(byte [] family, byte [] columnQualifier) {
return get(Bytes.add(family, KeyValue.COLUMN_FAMILY_DELIM_ARRAY, columnQualifier));
}
public Comparator<? super byte[]> comparator() {
@ -245,7 +261,7 @@ public class RowResult implements Writable, SortedMap<byte [], Cell>,
sb.append(Long.toString(e.getValue().getTimestamp()));
sb.append(", value=");
byte [] v = e.getValue().getValue();
if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) {
if (Bytes.equals(e.getKey(), this.COL_REGIONINFO)) {
try {
sb.append(Writables.getHRegionInfo(v).toString());
} catch (IOException ioe) {

View File

@ -0,0 +1,172 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.io;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Represents an interval of version timestamps.
* <p>
* Evaluated according to minStamp <= timestamp < maxStamp
* or [minStamp,maxStamp) in interval notation.
* <p>
* Only used internally; should not be accessed directly by clients.
*/
public class TimeRange implements Writable {
private long minStamp = 0L;
private long maxStamp = Long.MAX_VALUE;
private boolean allTime = false;
/**
* Default constructor.
* Represents interval [0, Long.MAX_VALUE) (allTime)
*/
public TimeRange() {
allTime = true;
}
/**
* Represents interval [minStamp, Long.MAX_VALUE)
* @param minStamp the minimum timestamp value, inclusive
*/
public TimeRange(long minStamp) {
this.minStamp = minStamp;
}
/**
* Represents interval [minStamp, Long.MAX_VALUE)
* @param minStamp the minimum timestamp value, inclusive
*/
public TimeRange(byte [] minStamp) {
this.minStamp = Bytes.toLong(minStamp);
}
/**
* Represents interval [minStamp, maxStamp)
* @param minStamp the minimum timestamp, inclusive
* @param maxStamp the maximum timestamp, exclusive
* @throws IOException
*/
public TimeRange(long minStamp, long maxStamp)
throws IOException {
if(maxStamp < minStamp) {
throw new IOException("maxStamp is smaller than minStamp");
}
this.minStamp = minStamp;
this.maxStamp = maxStamp;
}
/**
* Represents interval [minStamp, maxStamp)
* @param minStamp the minimum timestamp, inclusive
* @param maxStamp the maximum timestamp, exclusive
* @throws IOException
*/
public TimeRange(byte [] minStamp, byte [] maxStamp)
throws IOException {
this(Bytes.toLong(minStamp), Bytes.toLong(maxStamp));
}
/**
* @return the smallest timestamp that should be considered
*/
public long getMin() {
return minStamp;
}
/**
* @return the biggest timestamp that should be considered
*/
public long getMax() {
return maxStamp;
}
/**
* Check if the specified timestamp is within this TimeRange.
* <p>
* Returns true if within interval [minStamp, maxStamp), false
* if not.
* @param bytes timestamp to check
* @param offset offset into the bytes
* @return true if within TimeRange, false if not
*/
public boolean withinTimeRange(byte [] bytes, int offset) {
if(allTime) return true;
return withinTimeRange(Bytes.toLong(bytes, offset));
}
/**
* Check if the specified timestamp is within this TimeRange.
* <p>
* Returns true if within interval [minStamp, maxStamp), false
* if not.
* @param timestamp timestamp to check
* @return true if within TimeRange, false if not
*/
public boolean withinTimeRange(long timestamp) {
if(allTime) return true;
// check if >= minStamp
return (minStamp <= timestamp && timestamp < maxStamp);
}
/**
* Check if the specified timestamp is within this TimeRange.
* <p>
* Returns true if within interval [minStamp, maxStamp), false
* if not.
* @param timestamp timestamp to check
* @return true if within TimeRange, false if not
*/
public boolean withinOrAfterTimeRange(long timestamp) {
if(allTime) return true;
// check if >= minStamp
return (timestamp >= minStamp);
}
@Override
public String toString() {
StringBuffer sb = new StringBuffer();
sb.append("maxStamp=");
sb.append(this.maxStamp);
sb.append(", minStamp=");
sb.append(this.minStamp);
return sb.toString();
}
//Writable
public void readFields(final DataInput in) throws IOException {
this.minStamp = in.readLong();
this.maxStamp = in.readLong();
this.allTime = in.readBoolean();
}
public void write(final DataOutput out) throws IOException {
out.writeLong(minStamp);
out.writeLong(maxStamp);
out.writeBoolean(this.allTime);
}
}

View File

@ -104,11 +104,6 @@ import org.apache.hadoop.io.compress.Decompressor;
* <pre>&lt;fileinfo>&lt;trailer></pre>. That is, there are not data nor meta
* blocks present.
* <p>
* TODO: Bloomfilters. Need to add hadoop 0.20. first since it has bug fixes
* on the hadoop bf package.
* * TODO: USE memcmp by default? Write the keys out in an order that allows
* my using this -- reverse the timestamp.
* TODO: Add support for fast-gzip and for lzo.
* TODO: Do scanners need to be able to take a start and end row?
* TODO: Should BlockIndex know the name of its file? Should it have a Path
* that points at its file say for the case where an index lives apart from
@ -465,8 +460,12 @@ public class HFile {
* Add key/value to file.
* Keys must be added in an order that agrees with the Comparator passed
* on construction.
* @param key Key to add. Cannot be empty nor null.
* @param value Value to add. Cannot be empty nor null.
* @param key
* @param koffset
* @param klength
* @param value
* @param voffset
* @param vlength
* @throws IOException
*/
public void append(final byte [] key, final int koffset, final int klength,
@ -1039,6 +1038,9 @@ public class HFile {
}
public KeyValue getKeyValue() {
if(this.block == null) {
return null;
}
return new KeyValue(this.block.array(),
this.block.arrayOffset() + this.block.position() - 8);
}

View File

@ -24,6 +24,9 @@ public class SimpleBlockCache implements BlockCache {
private ReferenceQueue q = new ReferenceQueue();
public int dumps = 0;
/**
* Constructor
*/
public SimpleBlockCache() {
super();
}
@ -36,6 +39,9 @@ public class SimpleBlockCache implements BlockCache {
}
}
/**
* @return the size
*/
public synchronized int size() {
processQueue();
return cache.size();

View File

@ -390,6 +390,7 @@ public class HBaseRPC {
* @param addr
* @param conf
* @param maxAttempts
* @param timeout
* @return proxy
* @throws IOException
*/

View File

@ -86,7 +86,8 @@ public abstract class HBaseServer {
public static final Log LOG =
LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer");
protected static final ThreadLocal<HBaseServer> SERVER = new ThreadLocal<HBaseServer>();
protected static final ThreadLocal<HBaseServer> SERVER =
new ThreadLocal<HBaseServer>();
/** Returns the server instance called under or null. May be called under
* {@link #call(Writable, long)} implementations, and under {@link Writable}
@ -128,10 +129,11 @@ public abstract class HBaseServer {
private int handlerCount; // number of handler threads
protected Class<? extends Writable> paramClass; // class of call parameters
protected int maxIdleTime; // the maximum idle time after
// which a client may be disconnected
protected int thresholdIdleConnections; // the number of idle connections
// after which we will start
// cleaning up idle
// which a client may be
// disconnected
protected int thresholdIdleConnections; // the number of idle
// connections after which we
// will start cleaning up idle
// connections
int maxConnectionsToNuke; // the max number of
// connections to nuke
@ -173,8 +175,9 @@ public abstract class HBaseServer {
try {
socket.bind(address, backlog);
} catch (BindException e) {
BindException bindException = new BindException("Problem binding to " + address
+ " : " + e.getMessage());
BindException bindException =
new BindException("Problem binding to " + address + " : " +
e.getMessage());
bindException.initCause(e);
throw bindException;
} catch (SocketException e) {
@ -297,7 +300,6 @@ public abstract class HBaseServer {
public void run() {
LOG.info(getName() + ": starting");
SERVER.set(HBaseServer.this);
long lastPurgeTime = 0; // last check for old calls.
while (running) {
SelectionKey key = null;

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.io.Writable;
@ -109,7 +110,7 @@ public interface HMasterInterface extends HBaseRPCProtocolVersion {
* @param args
* @throws IOException
*/
public void modifyTable(byte[] tableName, int op, Writable[] args)
public void modifyTable(byte[] tableName, HConstants.Modify op, Writable[] args)
throws IOException;
/**

View File

@ -1,5 +1,5 @@
/**
* Copyright 2007 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -21,15 +21,15 @@ package org.apache.hadoop.hbase.ipc;
import java.io.IOException;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
/**
* Clients interact with HRegionServers using a handle to the HRegionInterface.
@ -49,21 +49,6 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion {
public HRegionInfo getRegionInfo(final byte [] regionName)
throws NotServingRegionException;
/**
* Get the specified number of versions of the specified row and column with
* the specified timestamp.
*
* @param regionName region name
* @param row row key
* @param column column key
* @param timestamp timestamp
* @param numVersions number of versions to return
* @return array of values
* @throws IOException
*/
public Cell[] get(final byte [] regionName, final byte [] row,
final byte [] column, final long timestamp, final int numVersions)
throws IOException;
/**
* Return all the data for the row that matches <i>row</i> exactly,
@ -71,159 +56,104 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion {
*
* @param regionName region name
* @param row row key
* @param columnFamily Column family to look for row in.
* @param family Column family to look for row in.
* @return map of values
* @throws IOException
*/
public RowResult getClosestRowBefore(final byte [] regionName,
final byte [] row, final byte [] columnFamily)
public Result getClosestRowBefore(final byte [] regionName,
final byte [] row, final byte [] family)
throws IOException;
/**
* Get selected columns for the specified row at a given timestamp.
*
* @param regionName region name
* @param row row key
* @param columns columns to get
* @param ts time stamp
* @param numVersions number of versions
* @param lockId lock id
* @return map of values
* @return the regions served by this regionserver
*/
public HRegion [] getOnlineRegionsAsArray();
/**
* Perform Get operation.
* @param regionName name of region to get from
* @param get Get operation
* @return Result
* @throws IOException
*/
public RowResult getRow(final byte [] regionName, final byte [] row,
final byte[][] columns, final long ts,
final int numVersions, final long lockId)
throws IOException;
public Result get(byte [] regionName, Get get) throws IOException;
/**
* Applies a batch of updates via one RPC
*
* @param regionName name of the region to update
* @param b BatchUpdate
* @param lockId lock id
* Perform exists operation.
* @param regionName name of region to get from
* @param get Get operation describing cell to test
* @return true if exists
* @throws IOException
*/
public void batchUpdate(final byte [] regionName, final BatchUpdate b,
final long lockId)
throws IOException;
/**
* Applies a batch of updates via one RPC for many rows
*
* @param regionName name of the region to update
* @param b BatchUpdate[]
* @throws IOException
* @return number of updates applied
*/
public int batchUpdates(final byte[] regionName, final BatchUpdate[] b)
throws IOException;
/**
* Applies a batch of updates to one row atomically via one RPC
* if the columns specified in expectedValues match
* the given values in expectedValues
*
* @param regionName name of the region to update
* @param b BatchUpdate
* @param expectedValues map of column names to expected data values.
* @return true if update was applied
* @throws IOException
*/
public boolean checkAndSave(final byte [] regionName, final BatchUpdate b,
final HbaseMapWritable<byte[],byte[]> expectedValues)
throws IOException;
public boolean exists(byte [] regionName, Get get) throws IOException;
/**
* Delete all cells that match the passed row and column and whose timestamp
* is equal-to or older than the passed timestamp.
*
* @param regionName region name
* @param row row key
* @param column column key
* @param timestamp Delete all entries that have this timestamp or older
* @param lockId lock id
* Put data into the specified region
* @param regionName
* @param put the data to be put
* @throws IOException
*/
public void deleteAll(byte [] regionName, byte [] row, byte [] column,
long timestamp, long lockId)
throws IOException;
/**
* Delete all cells that match the passed row and whose
* timestamp is equal-to or older than the passed timestamp.
*
* @param regionName region name
* @param row row key
* @param timestamp Delete all entries that have this timestamp or older
* @param lockId lock id
* @throws IOException
*/
public void deleteAll(byte [] regionName, byte [] row, long timestamp,
long lockId)
public void put(final byte [] regionName, final Put put)
throws IOException;
/**
* Delete all cells that match the passed row & the column regex and whose
* timestamp is equal-to or older than the passed timestamp.
* Put an array of puts into the specified region
* @param regionName
* @param puts
* @return
* @throws IOException
*/
public int put(final byte[] regionName, final Put [] puts)
throws IOException;
/**
* Deletes all the KeyValues that match those found in the Delete object,
* if their ts <= to the Delete. In case of a delete with a specific ts it
* only deletes that specific KeyValue.
* @param regionName
* @param delete
* @throws IOException
*/
public void delete(final byte[] regionName, final Delete delete)
throws IOException;
/**
* Atomically checks if a row/family/qualifier value match the expectedValue.
* If it does, it adds the put.
*
* @param regionName
* @param row
* @param colRegex
* @param timestamp
* @param lockId
* @param family
* @param qualifier
* @param value the expected value
* @param put
* @throws IOException
* @return true if the new put was execute, false otherwise
*/
public void deleteAllByRegex(byte [] regionName, byte [] row, String colRegex,
long timestamp, long lockId)
throws IOException;
/**
* Delete all cells for a row with matching column family with timestamps
* less than or equal to <i>timestamp</i>.
*
* @param regionName The name of the region to operate on
* @param row The row to operate on
* @param family The column family to match
* @param timestamp Timestamp to match
* @param lockId lock id
* @throws IOException
*/
public void deleteFamily(byte [] regionName, byte [] row, byte [] family,
long timestamp, long lockId)
public boolean checkAndPut(final byte[] regionName, final byte [] row,
final byte [] family, final byte [] qualifier, final byte [] value,
final Put put)
throws IOException;
/**
* Delete all cells for a row with matching column family regex with
* timestamps less than or equal to <i>timestamp</i>.
* Atomically increments a column value. If the column value isn't long-like,
* this could throw an exception.
*
* @param regionName The name of the region to operate on
* @param row The row to operate on
* @param familyRegex column family regex
* @param timestamp Timestamp to match
* @param lockId lock id
* @param regionName
* @param row
* @param family
* @param qualifier
* @param amount
* @return new incremented column value
* @throws IOException
*/
public void deleteFamilyByRegex(byte [] regionName, byte [] row, String familyRegex,
long timestamp, long lockId)
public long incrementColumnValue(byte [] regionName, byte [] row,
byte [] family, byte [] qualifier, long amount)
throws IOException;
/**
* Returns true if any cells exist for the given coordinate.
*
* @param regionName The name of the region
* @param row The row
* @param column The column, or null for any
* @param timestamp The timestamp, or LATEST_TIMESTAMP for any
* @param lockID lock id
* @return true if the row exists, false otherwise
* @throws IOException
*/
public boolean exists(byte [] regionName, byte [] row, byte [] column,
long timestamp, long lockID)
throws IOException;
//
// remote scanner interface
//
@ -232,20 +162,11 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion {
* Opens a remote scanner with a RowFilter.
*
* @param regionName name of region to scan
* @param columns columns to scan. If column name is a column family, all
* columns of the specified column family are returned. Its also possible
* to pass a regex for column family name. A column name is judged to be
* regex if it contains at least one of the following characters:
* <code>\+|^&*$[]]}{)(</code>.
* @param startRow starting row to scan
* @param timestamp only return values whose timestamp is <= this value
* @param filter RowFilter for filtering results at the row-level.
*
* @param scan configured scan object
* @return scannerId scanner identifier used in other calls
* @throws IOException
*/
public long openScanner(final byte [] regionName, final byte [][] columns,
final byte [] startRow, long timestamp, RowFilterInterface filter)
public long openScanner(final byte [] regionName, final Scan scan)
throws IOException;
/**
@ -254,7 +175,7 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion {
* @return map of values
* @throws IOException
*/
public RowResult next(long scannerId) throws IOException;
public Result next(long scannerId) throws IOException;
/**
* Get the next set of values
@ -263,7 +184,7 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion {
* @return map of values
* @throws IOException
*/
public RowResult[] next(long scannerId, int numberOfRows) throws IOException;
public Result [] next(long scannerId, int numberOfRows) throws IOException;
/**
* Close a scanner
@ -272,7 +193,7 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion {
* @throws IOException
*/
public void close(long scannerId) throws IOException;
/**
* Opens a remote row lock.
*
@ -294,19 +215,6 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion {
public void unlockRow(final byte [] regionName, final long lockId)
throws IOException;
/**
* Atomically increments a column value. If the column value isn't long-like, this could
* throw an exception.
*
* @param regionName
* @param row
* @param column
* @param amount
* @return new incremented column value
* @throws IOException
*/
public long incrementColumnValue(byte [] regionName, byte [] row,
byte [] column, long amount) throws IOException;
/**
* Method used when a master is taking the place of another failed one.

View File

@ -19,7 +19,8 @@ package org.apache.hadoop.hbase.ipc;
import java.io.IOException;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
@ -136,33 +137,24 @@ public interface TransactionalRegionInterface extends HRegionInterface {
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param delete
* @param timestamp Delete all entries that have this timestamp or older
* @throws IOException
*/
public void deleteAll(long transactionId, byte[] regionName, byte[] row,
long timestamp) throws IOException;
public void delete(long transactionId, byte [] regionName, Delete delete)
throws IOException;
/**
* Opens a remote scanner with a RowFilter.
*
* @param transactionId
* @param regionName name of region to scan
* @param columns columns to scan. If column name is a column family, all
* columns of the specified column family are returned. Its also possible to
* pass a regex for column family name. A column name is judged to be regex if
* it contains at least one of the following characters:
* <code>\+|^&*$[]]}{)(</code>.
* @param startRow starting row to scan
* @param timestamp only return values whose timestamp is <= this value
* @param filter RowFilter for filtering results at the row-level.
*
* @param scan
* @return scannerId scanner identifier used in other calls
* @throws IOException
*/
public long openScanner(final long transactionId, final byte[] regionName,
final byte[][] columns, final byte[] startRow, long timestamp,
RowFilterInterface filter) throws IOException;
Scan scan) throws IOException;
/**
* Applies a batch of updates via one RPC

View File

@ -1,5 +1,5 @@
/**
* Copyright 2008 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -28,7 +28,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Scanner;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.filter.RowFilterSet;
import org.apache.hadoop.hbase.filter.StopRowFilter;
@ -73,7 +75,7 @@ import org.apache.hadoop.util.StringUtils;
* </pre>
*/
public abstract class TableInputFormatBase
implements InputFormat<ImmutableBytesWritable, RowResult> {
implements InputFormat<ImmutableBytesWritable, Result> {
final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
private byte [][] inputColumns;
private HTable table;
@ -84,12 +86,12 @@ implements InputFormat<ImmutableBytesWritable, RowResult> {
* Iterate over an HBase table data, return (Text, RowResult) pairs
*/
protected class TableRecordReader
implements RecordReader<ImmutableBytesWritable, RowResult> {
implements RecordReader<ImmutableBytesWritable, Result> {
private byte [] startRow;
private byte [] endRow;
private byte [] lastRow;
private RowFilterInterface trrRowFilter;
private Scanner scanner;
private ResultScanner scanner;
private HTable htable;
private byte [][] trrInputColumns;
@ -106,16 +108,21 @@ implements InputFormat<ImmutableBytesWritable, RowResult> {
new HashSet<RowFilterInterface>();
rowFiltersSet.add(new WhileMatchRowFilter(new StopRowFilter(endRow)));
rowFiltersSet.add(trrRowFilter);
this.scanner = this.htable.getScanner(trrInputColumns, startRow,
new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL,
rowFiltersSet));
Scan scan = new Scan(startRow);
scan.addColumns(trrInputColumns);
// scan.setFilter(new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL,
// rowFiltersSet));
this.scanner = this.htable.getScanner(scan);
} else {
this.scanner =
this.htable.getScanner(trrInputColumns, firstRow, endRow);
Scan scan = new Scan(firstRow, endRow);
scan.addColumns(trrInputColumns);
this.scanner = this.htable.getScanner(scan);
}
} else {
this.scanner =
this.htable.getScanner(trrInputColumns, firstRow, trrRowFilter);
Scan scan = new Scan(firstRow);
scan.addColumns(trrInputColumns);
// scan.setFilter(trrRowFilter);
this.scanner = this.htable.getScanner(scan);
}
}
@ -182,8 +189,8 @@ implements InputFormat<ImmutableBytesWritable, RowResult> {
*
* @see org.apache.hadoop.mapred.RecordReader#createValue()
*/
public RowResult createValue() {
return new RowResult();
public Result createValue() {
return new Result();
}
public long getPos() {
@ -203,9 +210,9 @@ implements InputFormat<ImmutableBytesWritable, RowResult> {
* @return true if there was more data
* @throws IOException
*/
public boolean next(ImmutableBytesWritable key, RowResult value)
public boolean next(ImmutableBytesWritable key, Result value)
throws IOException {
RowResult result;
Result result;
try {
result = this.scanner.next();
} catch (UnknownScannerException e) {
@ -232,7 +239,7 @@ implements InputFormat<ImmutableBytesWritable, RowResult> {
* @see org.apache.hadoop.mapred.InputFormat#getRecordReader(InputSplit,
* JobConf, Reporter)
*/
public RecordReader<ImmutableBytesWritable, RowResult> getRecordReader(
public RecordReader<ImmutableBytesWritable, Result> getRecordReader(
InputSplit split, JobConf job, Reporter reporter)
throws IOException {
TableSplit tSplit = (TableSplit) split;

View File

@ -1,5 +1,5 @@
/**
* Copyright 2007 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.mapred.FileAlreadyExistsException;
@ -40,7 +41,7 @@ import org.apache.hadoop.util.Progressable;
* Convert Map/Reduce output and write it to an HBase table
*/
public class TableOutputFormat extends
FileOutputFormat<ImmutableBytesWritable, BatchUpdate> {
FileOutputFormat<ImmutableBytesWritable, Put> {
/** JobConf parameter that specifies the output table */
public static final String OUTPUT_TABLE = "hbase.mapred.outputtable";
@ -51,7 +52,7 @@ FileOutputFormat<ImmutableBytesWritable, BatchUpdate> {
* and write to an HBase table
*/
protected static class TableRecordWriter
implements RecordWriter<ImmutableBytesWritable, BatchUpdate> {
implements RecordWriter<ImmutableBytesWritable, Put> {
private HTable m_table;
/**
@ -69,8 +70,8 @@ FileOutputFormat<ImmutableBytesWritable, BatchUpdate> {
}
public void write(ImmutableBytesWritable key,
BatchUpdate value) throws IOException {
m_table.commit(new BatchUpdate(value));
Put value) throws IOException {
m_table.put(new Put(value));
}
}

View File

@ -38,8 +38,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -147,16 +148,18 @@ abstract class BaseScanner extends Chore implements HConstants {
// Array to hold list of split parents found. Scan adds to list. After
// scan we go check if parents can be removed.
Map<HRegionInfo, RowResult> splitParents =
new HashMap<HRegionInfo, RowResult>();
Map<HRegionInfo, Result> splitParents =
new HashMap<HRegionInfo, Result>();
List<byte []> emptyRows = new ArrayList<byte []>();
int rows = 0;
try {
regionServer = master.connection.getHRegionConnection(region.getServer());
scannerId = regionServer.openScanner(region.getRegionName(),
COLUMN_FAMILY_ARRAY, EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null);
new Scan().addFamily(HConstants.CATALOG_FAMILY));
while (true) {
RowResult values = regionServer.next(scannerId);
Result values = regionServer.next(scannerId);
if (values == null || values.size() == 0) {
break;
}
@ -165,8 +168,16 @@ abstract class BaseScanner extends Chore implements HConstants {
emptyRows.add(values.getRow());
continue;
}
String serverName = Writables.cellToString(values.get(COL_SERVER));
long startCode = Writables.cellToLong(values.get(COL_STARTCODE));
String serverName = "";
byte [] val = values.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
if( val != null) {
serverName = Bytes.toString(val);
}
long startCode = 0L;
val = values.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER);
if(val != null) {
startCode = Bytes.toLong(val);
}
// Note Region has been assigned.
checkAssigned(info, serverName, startCode);
@ -213,7 +224,7 @@ abstract class BaseScanner extends Chore implements HConstants {
// Take a look at split parents to see if any we can clean up.
if (splitParents.size() > 0) {
for (Map.Entry<HRegionInfo, RowResult> e : splitParents.entrySet()) {
for (Map.Entry<HRegionInfo, Result> e : splitParents.entrySet()) {
HRegionInfo hri = e.getKey();
cleanupSplits(region.getRegionName(), regionServer, hri, e.getValue());
}
@ -250,13 +261,13 @@ abstract class BaseScanner extends Chore implements HConstants {
*/
private boolean cleanupSplits(final byte [] metaRegionName,
final HRegionInterface srvr, final HRegionInfo parent,
RowResult rowContent)
Result rowContent)
throws IOException {
boolean result = false;
boolean hasReferencesA = hasReferences(metaRegionName, srvr,
parent.getRegionName(), rowContent, COL_SPLITA);
parent.getRegionName(), rowContent, CATALOG_FAMILY, SPLITA_QUALIFIER);
boolean hasReferencesB = hasReferences(metaRegionName, srvr,
parent.getRegionName(), rowContent, COL_SPLITB);
parent.getRegionName(), rowContent, CATALOG_FAMILY, SPLITB_QUALIFIER);
if (!hasReferencesA && !hasReferencesB) {
LOG.info("Deleting region " + parent.getRegionNameAsString() +
" (encoded=" + parent.getEncodedName() +
@ -283,15 +294,16 @@ abstract class BaseScanner extends Chore implements HConstants {
*/
private boolean hasReferences(final byte [] metaRegionName,
final HRegionInterface srvr, final byte [] parent,
RowResult rowContent, final byte [] splitColumn)
Result rowContent, final byte [] splitFamily, byte [] splitQualifier)
throws IOException {
boolean result = false;
HRegionInfo split =
Writables.getHRegionInfo(rowContent.get(splitColumn));
Writables.getHRegionInfo(rowContent.getValue(splitFamily, splitQualifier));
if (split == null) {
return result;
}
Path tabledir = new Path(this.master.rootdir, split.getTableDesc().getNameAsString());
Path tabledir =
new Path(this.master.rootdir, split.getTableDesc().getNameAsString());
for (HColumnDescriptor family: split.getTableDesc().getFamilies()) {
Path p = Store.getStoreHomedir(tabledir, split.getEncodedName(),
family.getName());
@ -320,10 +332,10 @@ abstract class BaseScanner extends Chore implements HConstants {
" no longer has references to " + Bytes.toString(parent));
}
BatchUpdate b = new BatchUpdate(parent);
b.delete(splitColumn);
srvr.batchUpdate(metaRegionName, b, -1L);
Delete delete = new Delete(parent);
delete.deleteColumns(splitFamily, splitQualifier);
srvr.delete(metaRegionName, delete);
return result;
}

View File

@ -27,6 +27,8 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.util.Writables;
@ -78,11 +80,13 @@ class ChangeTableState extends TableOperation {
}
// Update meta table
BatchUpdate b = new BatchUpdate(i.getRegionName());
updateRegionInfo(b, i);
b.delete(COL_SERVER);
b.delete(COL_STARTCODE);
server.batchUpdate(m.getRegionName(), b, -1L);
Put put = updateRegionInfo(i);
server.put(m.getRegionName(), put);
Delete delete = new Delete(i.getRegionName());
delete.deleteColumns(CATALOG_FAMILY, SERVER_QUALIFIER);
delete.deleteColumns(CATALOG_FAMILY, STARTCODE_QUALIFIER);
server.delete(m.getRegionName(), delete);
if (LOG.isDebugEnabled()) {
LOG.debug("Updated columns in row: " + i.getRegionNameAsString());
}
@ -125,9 +129,11 @@ class ChangeTableState extends TableOperation {
servedRegions.clear();
}
protected void updateRegionInfo(final BatchUpdate b, final HRegionInfo i)
protected Put updateRegionInfo(final HRegionInfo i)
throws IOException {
i.setOffline(!online);
b.put(COL_REGIONINFO, Writables.getBytes(i));
Put put = new Put(i.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(i));
return put;
}
}

View File

@ -25,7 +25,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.util.Writables;
@ -47,9 +47,9 @@ abstract class ColumnOperation extends TableOperation {
protected void updateRegionInfo(HRegionInterface server, byte [] regionName,
HRegionInfo i) throws IOException {
BatchUpdate b = new BatchUpdate(i.getRegionName());
b.put(COL_REGIONINFO, Writables.getBytes(i));
server.batchUpdate(regionName, b, -1L);
Put put = new Put(i.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(i));
server.put(regionName, put);
if (LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + i.getRegionNameAsString());
}

View File

@ -1,5 +1,5 @@
/**
* Copyright 2007 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -23,16 +23,15 @@ import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.lang.management.RuntimeMXBean;
import java.lang.reflect.Constructor;
import java.net.InetAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.DelayQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.PriorityBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
@ -51,19 +50,20 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.RegionHistorian;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ServerConnection;
import org.apache.hadoop.hbase.client.ServerConnectionManager;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
import org.apache.hadoop.hbase.ipc.HBaseServer;
@ -714,12 +714,14 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
byte [] metaRegionName = m.getRegionName();
HRegionInterface srvr = connection.getHRegionConnection(m.getServer());
byte[] firstRowInTable = Bytes.toBytes(tableName + ",,");
long scannerid = srvr.openScanner(metaRegionName, COL_REGIONINFO_ARRAY,
firstRowInTable, LATEST_TIMESTAMP, null);
Scan scan = new Scan(firstRowInTable);
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
long scannerid = srvr.openScanner(metaRegionName, scan);
try {
RowResult data = srvr.next(scannerid);
Result data = srvr.next(scannerid);
if (data != null && data.size() > 0) {
HRegionInfo info = Writables.getHRegionInfo(data.get(COL_REGIONINFO));
HRegionInfo info = Writables.getHRegionInfo(
data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
if (info.getTableDesc().getNameAsString().equals(tableName)) {
// A region for this table already exists. Ergo table exists.
throw new TableExistsException(tableName);
@ -752,7 +754,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
public void deleteColumn(final byte [] tableName, final byte [] c)
throws IOException {
new DeleteColumn(this, tableName, HStoreKey.getFamily(c)).process();
new DeleteColumn(this, tableName, KeyValue.parseColumn(c)[0]).process();
}
public void enableTable(final byte [] tableName) throws IOException {
@ -778,23 +780,23 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
for (MetaRegion m: regions) {
byte [] metaRegionName = m.getRegionName();
HRegionInterface srvr = connection.getHRegionConnection(m.getServer());
Scan scan = new Scan(firstRowInTable);
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
scan.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER);
long scannerid =
srvr.openScanner(metaRegionName,
new byte[][] {COL_REGIONINFO, COL_SERVER},
firstRowInTable,
LATEST_TIMESTAMP,
null);
srvr.openScanner(metaRegionName, scan);
try {
while (true) {
RowResult data = srvr.next(scannerid);
Result data = srvr.next(scannerid);
if (data == null || data.size() <= 0)
break;
HRegionInfo info = Writables.getHRegionInfo(data.get(COL_REGIONINFO));
HRegionInfo info = Writables.getHRegionInfo(
data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
if (Bytes.compareTo(info.getTableDesc().getName(), tableName) == 0) {
Cell cell = data.get(COL_SERVER);
if (cell != null) {
byte [] value = data.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
if (value != null) {
HServerAddress server =
new HServerAddress(Bytes.toString(cell.getValue()));
new HServerAddress(Bytes.toString(value));
result.add(new Pair<HRegionInfo,HServerAddress>(info, server));
}
} else {
@ -816,25 +818,25 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
byte [] firstRowInTable = Bytes.toBytes(Bytes.toString(tableName) + ",,");
byte [] metaRegionName = m.getRegionName();
HRegionInterface srvr = connection.getHRegionConnection(m.getServer());
Scan scan = new Scan(firstRowInTable);
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
scan.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER);
long scannerid =
srvr.openScanner(metaRegionName,
new byte[][] {COL_REGIONINFO, COL_SERVER},
firstRowInTable,
LATEST_TIMESTAMP,
null);
srvr.openScanner(metaRegionName, scan);
try {
while (true) {
RowResult data = srvr.next(scannerid);
Result data = srvr.next(scannerid);
if (data == null || data.size() <= 0)
break;
HRegionInfo info = Writables.getHRegionInfo(data.get(COL_REGIONINFO));
HRegionInfo info = Writables.getHRegionInfo(
data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
if (Bytes.compareTo(info.getTableDesc().getName(), tableName) == 0) {
if ((Bytes.compareTo(info.getStartKey(), rowKey) >= 0) &&
(Bytes.compareTo(info.getEndKey(), rowKey) < 0)) {
Cell cell = data.get(COL_SERVER);
if (cell != null) {
byte [] value = data.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
if (value != null) {
HServerAddress server =
new HServerAddress(Bytes.toString(cell.getValue()));
new HServerAddress(Bytes.toString(value));
return new Pair<HRegionInfo,HServerAddress>(info, server);
}
}
@ -857,15 +859,17 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
for (MetaRegion m: regions) {
byte [] metaRegionName = m.getRegionName();
HRegionInterface srvr = connection.getHRegionConnection(m.getServer());
RowResult data = srvr.getRow(metaRegionName, regionName,
new byte[][] {COL_REGIONINFO, COL_SERVER},
HConstants.LATEST_TIMESTAMP, 1, -1L);
Get get = new Get(regionName);
get.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
get.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER);
Result data = srvr.get(metaRegionName, get);
if(data == null || data.size() <= 0) continue;
HRegionInfo info = Writables.getHRegionInfo(data.get(COL_REGIONINFO));
Cell cell = data.get(COL_SERVER);
if(cell != null) {
HRegionInfo info = Writables.getHRegionInfo(
data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
byte [] value = data.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
if(value != null) {
HServerAddress server =
new HServerAddress(Bytes.toString(cell.getValue()));
new HServerAddress(Bytes.toString(value));
return new Pair<HRegionInfo,HServerAddress>(info, server);
}
}
@ -876,15 +880,18 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
* Get row from meta table.
* @param row
* @param columns
* @return RowResult
* @return Result
* @throws IOException
*/
protected RowResult getFromMETA(final byte [] row, final byte [][] columns)
protected Result getFromMETA(final byte [] row, final byte [] family)
throws IOException {
MetaRegion meta = this.regionManager.getMetaRegionForRow(row);
HRegionInterface srvr = getMETAServer(meta);
return srvr.getRow(meta.getRegionName(), row, columns,
HConstants.LATEST_TIMESTAMP, 1, -1);
Get get = new Get(row);
get.addFamily(family);
return srvr.get(meta.getRegionName(), get);
}
/*
@ -897,10 +904,11 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
return this.connection.getHRegionConnection(meta.getServer());
}
public void modifyTable(final byte[] tableName, int op, Writable[] args)
public void modifyTable(final byte[] tableName, HConstants.Modify op,
Writable[] args)
throws IOException {
switch (op) {
case MODIFY_TABLE_SET_HTD:
case TABLE_SET_HTD:
if (args == null || args.length < 1 ||
!(args[0] instanceof HTableDescriptor))
throw new IOException("SET_HTD request requires an HTableDescriptor");
@ -909,10 +917,10 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
new ModifyTableMeta(this, tableName, htd).process();
break;
case MODIFY_TABLE_SPLIT:
case MODIFY_TABLE_COMPACT:
case MODIFY_TABLE_MAJOR_COMPACT:
case MODIFY_TABLE_FLUSH:
case TABLE_SPLIT:
case TABLE_COMPACT:
case TABLE_MAJOR_COMPACT:
case TABLE_FLUSH:
if (args != null && args.length > 0) {
if (!(args[0] instanceof ImmutableBytesWritable))
throw new IOException(
@ -936,7 +944,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
}
break;
case MODIFY_CLOSE_REGION:
case CLOSE_REGION:
if (args == null || args.length < 1 || args.length > 2) {
throw new IOException("Requires at least a region name; " +
"or cannot have more than region name and servername");
@ -947,12 +955,13 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
if (args.length == 2) {
servername = Bytes.toString(((ImmutableBytesWritable)args[1]).get());
}
// Need hri
RowResult rr = getFromMETA(regionname, HConstants.COLUMN_FAMILY_ARRAY);
// Need hri
Result rr = getFromMETA(regionname, HConstants.CATALOG_FAMILY);
HRegionInfo hri = getHRegionInfo(rr.getRow(), rr);
if (servername == null) {
// Get server from the .META. if it wasn't passed as argument
servername = Writables.cellToString(rr.get(COL_SERVER));
servername =
Bytes.toString(rr.getValue(CATALOG_FAMILY, SERVER_QUALIFIER));
}
LOG.info("Marking " + hri.getRegionNameAsString() +
" as closed on " + servername + "; cleaning SERVER + STARTCODE; " +
@ -995,7 +1004,8 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
public HBaseConfiguration getConfiguration() {
return this.conf;
}
// TODO ryan rework this function
/*
* Get HRegionInfo from passed META map of row values.
* Returns null if none found (and logs fact that expected COL_REGIONINFO
@ -1005,22 +1015,24 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
* @return Null or found HRegionInfo.
* @throws IOException
*/
HRegionInfo getHRegionInfo(final byte [] row, final Map<byte [], Cell> map)
HRegionInfo getHRegionInfo(final byte [] row, final Result res)
throws IOException {
Cell regioninfo = map.get(COL_REGIONINFO);
byte [] regioninfo = res.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
if (regioninfo == null) {
StringBuilder sb = new StringBuilder();
for (byte [] e: map.keySet()) {
NavigableMap<byte[], byte[]> infoMap = res.getFamilyMap(CATALOG_FAMILY);
for (byte [] e: infoMap.keySet()) {
if (sb.length() > 0) {
sb.append(", ");
}
sb.append(Bytes.toString(e));
sb.append(Bytes.toString(CATALOG_FAMILY) + ":" + Bytes.toString(e));
}
LOG.warn(Bytes.toString(COL_REGIONINFO) + " is empty for row: " +
LOG.warn(Bytes.toString(CATALOG_FAMILY) + ":" +
Bytes.toString(REGIONINFO_QUALIFIER) + " is empty for row: " +
Bytes.toString(row) + "; has keys: " + sb.toString());
return null;
}
return Writables.getHRegionInfo(regioninfo.getValue());
return Writables.getHRegionInfo(regioninfo);
}
/*
@ -1065,7 +1077,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
System.exit(0);
}
@SuppressWarnings("null")
protected static void doMain(String [] args,
Class<? extends HMaster> masterClass) {

View File

@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.util.Bytes;
@ -50,9 +51,9 @@ class ModifyTableMeta extends TableOperation {
protected void updateRegionInfo(HRegionInterface server, byte [] regionName,
HRegionInfo i)
throws IOException {
BatchUpdate b = new BatchUpdate(i.getRegionName());
b.put(COL_REGIONINFO, Writables.getBytes(i));
server.batchUpdate(regionName, b, -1L);
Put put = new Put(i.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(i));
server.put(regionName, put);
LOG.debug("updated HTableDescriptor for region " + i.getRegionNameAsString());
}

View File

@ -25,6 +25,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.RegionHistorian;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.util.Bytes;
@ -78,11 +81,12 @@ class ProcessRegionOpen extends ProcessRegionStatusChange {
" in region " + Bytes.toString(metaRegionName) +
" with startcode " + serverInfo.getStartCode() + " and server " +
serverInfo.getServerAddress());
BatchUpdate b = new BatchUpdate(regionInfo.getRegionName());
b.put(COL_SERVER,
Put p = new Put(regionInfo.getRegionName());
p.add(CATALOG_FAMILY, SERVER_QUALIFIER,
Bytes.toBytes(serverInfo.getServerAddress().toString()));
b.put(COL_STARTCODE, Bytes.toBytes(serverInfo.getStartCode()));
server.batchUpdate(metaRegionName, b, -1L);
p.add(CATALOG_FAMILY, STARTCODE_QUALIFIER,
Bytes.toBytes(serverInfo.getStartCode()));
server.put(metaRegionName, p);
if (!historian.isOnline()) {
// This is safest place to do the onlining of the historian in
// the master. When we get to here, we know there is a .META.

View File

@ -1,5 +1,5 @@
/**
* Copyright 2008 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -113,7 +115,7 @@ class ProcessServerShutdown extends RegionServerOperation {
List<byte []> emptyRows = new ArrayList<byte []>();
try {
while (true) {
RowResult values = null;
Result values = null;
try {
values = server.next(scannerId);
} catch (IOException e) {
@ -129,8 +131,10 @@ class ProcessServerShutdown extends RegionServerOperation {
// shutdown server but that would mean that we'd reassign regions that
// were already out being assigned, ones that were product of a split
// that happened while the shutdown was being processed.
String serverAddress = Writables.cellToString(values.get(COL_SERVER));
long startCode = Writables.cellToLong(values.get(COL_STARTCODE));
String serverAddress =
Bytes.toString(values.getValue(CATALOG_FAMILY, SERVER_QUALIFIER));
long startCode =
Bytes.toLong(values.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER));
String serverName = null;
if (serverAddress != null && serverAddress.length() > 0) {
serverName = HServerInfo.getServerName(serverAddress, startCode);
@ -145,6 +149,7 @@ class ProcessServerShutdown extends RegionServerOperation {
Bytes.toString(row));
}
// HRegionInfo info = master.getHRegionInfo(row, values.rowResult());
HRegionInfo info = master.getHRegionInfo(row, values);
if (info == null) {
emptyRows.add(row);
@ -221,9 +226,10 @@ class ProcessServerShutdown extends RegionServerOperation {
LOG.debug("process server shutdown scanning root region on " +
master.getRootRegionLocation().getBindAddress());
}
Scan scan = new Scan();
scan.addFamily(CATALOG_FAMILY);
long scannerId = server.openScanner(
HRegionInfo.ROOT_REGIONINFO.getRegionName(), COLUMN_FAMILY_ARRAY,
EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null);
HRegionInfo.ROOT_REGIONINFO.getRegionName(), scan);
scanMetaRegion(server, scannerId,
HRegionInfo.ROOT_REGIONINFO.getRegionName());
return true;
@ -240,9 +246,10 @@ class ProcessServerShutdown extends RegionServerOperation {
LOG.debug("process server shutdown scanning " +
Bytes.toString(m.getRegionName()) + " on " + m.getServer());
}
long scannerId =
server.openScanner(m.getRegionName(), COLUMN_FAMILY_ARRAY,
EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null);
Scan scan = new Scan();
scan.addFamily(CATALOG_FAMILY);
long scannerId = server.openScanner(
HRegionInfo.ROOT_REGIONINFO.getRegionName(), scan);
scanMetaRegion(server, scannerId, m.getRegionName());
return true;
}

View File

@ -49,12 +49,12 @@ import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionHistorian;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.HMsg;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
@ -723,9 +723,12 @@ class RegionManager implements HConstants {
// 3. Insert into meta
HRegionInfo info = region.getRegionInfo();
byte [] regionName = region.getRegionName();
BatchUpdate b = new BatchUpdate(regionName);
b.put(COL_REGIONINFO, Writables.getBytes(info));
server.batchUpdate(metaRegionName, b, -1L);
Put put = new Put(regionName);
byte [] infoBytes = Writables.getBytes(info);
String infoString = new String(infoBytes);
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(info));
server.put(metaRegionName, put);
// 4. Close the new region to flush it to disk. Close its log file too.
region.close();
@ -1204,18 +1207,21 @@ class RegionManager implements HConstants {
* @param op
*/
public void startAction(byte[] regionName, HRegionInfo info,
HServerAddress server, int op) {
HServerAddress server, HConstants.Modify op) {
if (LOG.isDebugEnabled()) {
LOG.debug("Adding operation " + op + " from tasklist");
}
switch (op) {
case HConstants.MODIFY_TABLE_SPLIT:
case TABLE_SPLIT:
startAction(regionName, info, server, this.regionsToSplit);
break;
case HConstants.MODIFY_TABLE_COMPACT:
case TABLE_COMPACT:
startAction(regionName, info, server, this.regionsToCompact);
break;
case HConstants.MODIFY_TABLE_MAJOR_COMPACT:
case TABLE_MAJOR_COMPACT:
startAction(regionName, info, server, this.regionsToMajorCompact);
break;
case HConstants.MODIFY_TABLE_FLUSH:
case TABLE_FLUSH:
startAction(regionName, info, server, this.regionsToFlush);
break;
default:
@ -1233,18 +1239,21 @@ class RegionManager implements HConstants {
* @param regionName
* @param op
*/
public void endAction(byte[] regionName, int op) {
public void endAction(byte[] regionName, HConstants.Modify op) {
if (LOG.isDebugEnabled()) {
LOG.debug("Removing operation " + op + " from tasklist");
}
switch (op) {
case HConstants.MODIFY_TABLE_SPLIT:
case TABLE_SPLIT:
this.regionsToSplit.remove(regionName);
break;
case HConstants.MODIFY_TABLE_COMPACT:
case TABLE_COMPACT:
this.regionsToCompact.remove(regionName);
break;
case HConstants.MODIFY_TABLE_MAJOR_COMPACT:
case TABLE_MAJOR_COMPACT:
this.regionsToMajorCompact.remove(regionName);
break;
case HConstants.MODIFY_TABLE_FLUSH:
case TABLE_FLUSH:
this.regionsToFlush.remove(regionName);
break;
default:
@ -1267,6 +1276,9 @@ class RegionManager implements HConstants {
* @param returnMsgs
*/
public void applyActions(HServerInfo serverInfo, ArrayList<HMsg> returnMsgs) {
if (LOG.isDebugEnabled()) {
LOG.debug("Applying operation in tasklists to region");
}
applyActions(serverInfo, returnMsgs, this.regionsToCompact,
HMsg.Type.MSG_REGION_COMPACT);
applyActions(serverInfo, returnMsgs, this.regionsToSplit,

View File

@ -31,10 +31,10 @@ import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
/**
* Abstract base class for operations that need to examine all HRegionInfo
@ -80,26 +80,28 @@ abstract class TableOperation implements HConstants {
// Open a scanner on the meta region
byte [] tableNameMetaStart =
Bytes.toBytes(Bytes.toString(tableName) + ",,");
long scannerId = server.openScanner(m.getRegionName(),
COLUMN_FAMILY_ARRAY, tableNameMetaStart, HConstants.LATEST_TIMESTAMP, null);
Scan scan = new Scan(tableNameMetaStart).addFamily(CATALOG_FAMILY);
long scannerId = server.openScanner(m.getRegionName(), scan);
List<byte []> emptyRows = new ArrayList<byte []>();
try {
while (true) {
RowResult values = server.next(scannerId);
if(values == null || values.size() == 0) {
Result values = server.next(scannerId);
if(values == null || values.isEmpty()) {
break;
}
HRegionInfo info = this.master.getHRegionInfo(values.getRow(), values);
if (info == null) {
emptyRows.add(values.getRow());
LOG.error(Bytes.toString(COL_REGIONINFO) + " not found on " +
LOG.error(Bytes.toString(CATALOG_FAMILY) + ":" +
Bytes.toString(REGIONINFO_QUALIFIER) + " not found on " +
Bytes.toString(values.getRow()));
continue;
}
String serverAddress = Writables.cellToString(values.get(COL_SERVER));
long startCode = Writables.cellToLong(values.get(COL_STARTCODE));
String serverAddress =
Bytes.toString(values.getValue(CATALOG_FAMILY, SERVER_QUALIFIER));
long startCode =
Bytes.toLong(values.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER));
String serverName = null;
if (serverAddress != null && serverAddress.length() > 0) {
serverName = HServerInfo.getServerName(serverAddress, startCode);

View File

@ -0,0 +1,112 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
/**
* Simple wrapper for a byte buffer and a counter. Does not copy.
* <p>
* NOT thread-safe because it is not used in a multi-threaded context, yet.
*/
public class ColumnCount {
private byte [] bytes;
private int offset;
private int length;
private int count;
/**
* Constructor
* @param column the qualifier to count the versions for
*/
public ColumnCount(byte [] column) {
this(column, 0);
}
/**
* Constructor
* @param column the qualifier to count the versions for
* @param count initial count
*/
public ColumnCount(byte [] column, int count) {
this(column, 0, column.length, count);
}
/**
* Constuctor
* @param column the qualifier to count the versions for
* @param offset in the passed buffer where to start the qualifier from
* @param length of the qualifier
* @param count initial count
*/
public ColumnCount(byte [] column, int offset, int length, int count) {
this.bytes = column;
this.offset = offset;
this.length = length;
this.count = count;
}
/**
* @return the buffer
*/
public byte [] getBuffer(){
return this.bytes;
}
/**
* @return the offset
*/
public int getOffset(){
return this.offset;
}
/**
* @return the length
*/
public int getLength(){
return this.length;
}
/**
* Decrement the current version count
* @return current count
*/
public int decrement() {
return --count;
}
/**
* Increment the current version count
* @return current count
*/
public int increment() {
return ++count;
}
/**
* Check to see if needed to fetch more versions
* @param max
* @return true if more versions are needed, false otherwise
*/
public boolean needMore(int max) {
if(this.count < max) {
return true;
}
return false;
}
}

View File

@ -0,0 +1,78 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
/**
* Implementing classes of this interface will be used for the tracking
* and enforcement of columns and numbers of versions during the course of a
* Get or Scan operation.
* <p>
* Currently there are two different types of Store/Family-level queries.
* <ul><li>{@link ExplicitColumnTracker} is used when the query specifies
* one or more column qualifiers to return in the family.
* <li>{@link WildcardColumnTracker} is used when the query asks for all
* qualifiers within the family.
* <p>
* This class is utilized by {@link QueryMatcher} through two methods:
* <ul><li>{@link checkColumn} is called when a Put satisfies all other
* conditions of the query. This method returns a {@link MatchCode} to define
* what action should be taken.
* <li>{@link update} is called at the end of every StoreFile or Memcache.
* <p>
* This class is NOT thread-safe as queries are never multi-threaded
*/
public interface ColumnTracker {
/**
* Keeps track of the number of versions for the columns asked for
* @param bytes
* @param offset
* @param length
* @return
*/
public MatchCode checkColumn(byte [] bytes, int offset, int length);
/**
* Updates internal variables in between files
*/
public void update();
/**
* Resets the Matcher
*/
public void reset();
/**
*
* @return
*/
public boolean done();
/**
* Used by matcher and scan/get to get a hint of the next column
* to seek to after checkColumn() returns SKIP. Returns the next interesting
* column we want, or NULL there is none (wildcard scanner).
*
* Implementations aren't required to return anything useful unless the most recent
* call was to checkColumn() and the return code was SKIP. This is pretty implementation
* detail-y, but optimizations are like that.
*
* @return null, or a ColumnCount that we should seek to
*/
public ColumnCount getColumnHint();
}

View File

@ -30,11 +30,11 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
@ -202,18 +202,21 @@ class CompactSplitThread extends Thread implements HConstants {
// Inform the HRegionServer that the parent HRegion is no-longer online.
this.server.removeFromOnlineRegions(oldRegionInfo);
BatchUpdate update = new BatchUpdate(oldRegionInfo.getRegionName());
update.put(COL_REGIONINFO, Writables.getBytes(oldRegionInfo));
update.put(COL_SPLITA, Writables.getBytes(newRegions[0].getRegionInfo()));
update.put(COL_SPLITB, Writables.getBytes(newRegions[1].getRegionInfo()));
t.commit(update);
Put put = new Put(oldRegionInfo.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER,
Writables.getBytes(oldRegionInfo));
put.add(CATALOG_FAMILY, SPLITA_QUALIFIER,
Writables.getBytes(newRegions[0].getRegionInfo()));
put.add(CATALOG_FAMILY, SPLITB_QUALIFIER,
Writables.getBytes(newRegions[0].getRegionInfo()));
t.put(put);
// Add new regions to META
for (int i = 0; i < newRegions.length; i++) {
update = new BatchUpdate(newRegions[i].getRegionName());
update.put(COL_REGIONINFO, Writables.getBytes(
put = new Put(newRegions[i].getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(
newRegions[i].getRegionInfo()));
t.commit(update);
t.put(put);
}
// Now tell the master about the new regions

View File

@ -0,0 +1,120 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Class that provides static method needed when putting deletes into memcache
*/
public class DeleteCompare {
/**
* Return codes from deleteCompare.
*/
enum DeleteCode {
/**
* Do nothing. Move to next KV in Memcache
*/
SKIP,
/**
* Add to the list of deletes.
*/
DELETE,
/**
* Stop looking at KVs in Memcache. Finalize.
*/
DONE
}
/**
* Method used when putting deletes into memcache to remove all the previous
* entries that are affected by this Delete
* @param mem
* @param deleteBuffer
* @param deleteRowOffset
* @param deleteRowLength
* @param deleteQualifierOffset
* @param deleteQualifierLength
* @param deleteTimeOffset
* @param deleteType
* @param comparator
* @return SKIP if current KeyValue should not be deleted, DELETE if
* current KeyValue should be deleted and DONE when the current KeyValue is
* out of the Deletes range
*/
public static DeleteCode deleteCompare(KeyValue mem, byte [] deleteBuffer,
int deleteRowOffset, short deleteRowLength, int deleteQualifierOffset,
int deleteQualifierLength, int deleteTimeOffset, byte deleteType,
KeyValue.KeyComparator comparator) {
//Parsing new KeyValue
byte [] memBuffer = mem.getBuffer();
int memOffset = mem.getOffset();
//Getting key lengths
int memKeyLen = Bytes.toInt(memBuffer, memOffset);
memOffset += Bytes.SIZEOF_INT;
//Skipping value lengths
memOffset += Bytes.SIZEOF_INT;
//Getting row lengths
short memRowLen = Bytes.toShort(memBuffer, memOffset);
memOffset += Bytes.SIZEOF_SHORT;
int res = comparator.compareRows(memBuffer, memOffset, memRowLen,
deleteBuffer, deleteRowOffset, deleteRowLength);
if(res > 0) {
return DeleteCode.DONE;
} else if(res < 0){
System.out.println("SKIPPING ROW");
return DeleteCode.SKIP;
}
memOffset += memRowLen;
//Getting family lengths
byte memFamLen = memBuffer[memOffset];
memOffset += Bytes.SIZEOF_BYTE + memFamLen;
//Get column lengths
int memQualifierLen = memKeyLen - memRowLen - memFamLen -
Bytes.SIZEOF_SHORT - Bytes.SIZEOF_BYTE - Bytes.SIZEOF_LONG -
Bytes.SIZEOF_BYTE;
//Compare timestamp
int tsOffset = memOffset + memQualifierLen;
int timeRes = Bytes.compareTo(memBuffer, tsOffset, Bytes.SIZEOF_LONG,
deleteBuffer, deleteTimeOffset, Bytes.SIZEOF_LONG);
if(deleteType == KeyValue.Type.DeleteFamily.getCode()) {
if(timeRes <= 0){
return DeleteCode.DELETE;
}
return DeleteCode.SKIP;
}
//Compare columns
res = Bytes.compareTo(memBuffer, memOffset, memQualifierLen,
deleteBuffer, deleteQualifierOffset, deleteQualifierLength);
if(res < 0) {
return DeleteCode.SKIP;
} else if(res > 0) {
return DeleteCode.DONE;
}
// same column, compare the time.
if(timeRes == 0) {
return DeleteCode.DELETE;
} else if (timeRes < 0) {
if(deleteType == KeyValue.Type.DeleteColumn.getCode()) {
return DeleteCode.DELETE;
}
return DeleteCode.DONE;
} else {
System.out.println("SKIPPING TS");
return DeleteCode.SKIP;
}
}
}

View File

@ -0,0 +1,97 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
/**
* This interface is used for the tracking and enforcement of Deletes
* during the course of a Get or Scan operation.
* <p>
* This class is utilized through three methods:
* <ul><li>{@link add} when encountering a Delete
* <li>{@link isDeleted} when checking if a Put KeyValue has been deleted
* <li>{@link update} when reaching the end of a StoreFile
*/
public interface DeleteTracker {
/**
* Add the specified KeyValue to the list of deletes to check against for
* this row operation.
* <p>
* This is called when a Delete is encountered in a StoreFile.
* @param buffer KeyValue buffer
* @param qualifierOffset column qualifier offset
* @param qualifierLength column qualifier length
* @param timestamp timestamp
* @param type delete type as byte
*/
public void add(byte [] buffer, int qualifierOffset, int qualifierLength,
long timestamp, byte type);
/**
* Check if the specified KeyValue buffer has been deleted by a previously
* seen delete.
* @param buffer KeyValue buffer
* @param qualifierOffset column qualifier offset
* @param qualifierLength column qualifier length
* @param timestamp timestamp
* @return true is the specified KeyValue is deleted, false if not
*/
public boolean isDeleted(byte [] buffer, int qualifierOffset,
int qualifierLength, long timestamp);
/**
* @return true if there are no current delete, false otherwise
*/
public boolean isEmpty();
/**
* Called at the end of every StoreFile.
* <p>
* Many optimized implementations of Trackers will require an update at
* when the end of each StoreFile is reached.
*/
public void update();
/**
* Called between rows.
* <p>
* This clears everything as if a new DeleteTracker was instantiated.
*/
public void reset();
/**
* Return codes for comparison of two Deletes.
* <p>
* The codes tell the merging function what to do.
* <p>
* INCLUDE means add the specified Delete to the merged list.
* NEXT means move to the next element in the specified list(s).
*/
enum DeleteCompare {
INCLUDE_OLD_NEXT_OLD,
INCLUDE_OLD_NEXT_BOTH,
INCLUDE_NEW_NEXT_NEW,
INCLUDE_NEW_NEXT_BOTH,
NEXT_OLD,
NEXT_NEW
}
}

View File

@ -0,0 +1,157 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
/**
* This class is used for the tracking and enforcement of columns and numbers
* of versions during the course of a Get or Scan operation, when explicit
* column qualifiers have been asked for in the query.
*
* With a little magic (see {@link ScanQueryMatcher}), we can use this matcher
* for both scans and gets. The main difference is 'next' and 'done' collapse
* for the scan case (since we see all columns in order), and we only reset
* between rows.
*
* <p>
* This class is utilized by {@link QueryMatcher} through two methods:
* <ul><li>{@link checkColumn} is called when a Put satisfies all other
* conditions of the query. This method returns a {@link MatchCode} to define
* what action should be taken.
* <li>{@link update} is called at the end of every StoreFile or Memcache.
* <p>
* This class is NOT thread-safe as queries are never multi-threaded
*/
public class ExplicitColumnTracker implements ColumnTracker {
private int maxVersions;
private List<ColumnCount> columns;
private int index;
private ColumnCount column;
private NavigableSet<byte[]> origColumns;
/**
* Default constructor.
* @param columns columns specified user in query
* @param maxVersions maximum versions to return per column
*/
public ExplicitColumnTracker(NavigableSet<byte[]> columns, int maxVersions) {
this.maxVersions = maxVersions;
this.origColumns = columns;
reset();
}
/**
* Done when there are no more columns to match against.
*/
public boolean done() {
return this.columns.size() == 0;
}
public ColumnCount getColumnHint() {
return this.column;
}
/**
* Checks against the parameters of the query and the columns which have
* already been processed by this query.
* @param bytes KeyValue buffer
* @param offset offset to the start of the qualifier
* @param length length of the qualifier
* @return MatchCode telling QueryMatcher what action to take
*/
public MatchCode checkColumn(byte [] bytes, int offset, int length) {
// No more columns left, we are done with this query
if(this.columns.size() == 0) {
return MatchCode.DONE; // done_row
}
// No more columns to match against, done with storefile
if(this.column == null) {
return MatchCode.NEXT; // done_row
}
// Compare specific column to current column
int ret = Bytes.compareTo(column.getBuffer(), column.getOffset(),
column.getLength(), bytes, offset, length);
// Matches, decrement versions left and include
if(ret == 0) {
if(this.column.decrement() == 0) {
// Done with versions for this column
this.columns.remove(this.index);
if(this.columns.size() == this.index) {
// Will not hit any more columns in this storefile
this.column = null;
} else {
this.column = this.columns.get(this.index);
}
}
return MatchCode.INCLUDE;
}
// Specified column is bigger than current column
// Move down current column and check again
if(ret <= -1) {
if(++this.index == this.columns.size()) {
// No more to match, do not include, done with storefile
return MatchCode.NEXT; // done_row
}
this.column = this.columns.get(this.index);
return checkColumn(bytes, offset, length);
}
// Specified column is smaller than current column
// Skip
return MatchCode.SKIP; // skip to next column, with hint?
}
/**
* Called at the end of every StoreFile or Memcache.
*/
public void update() {
if(this.columns.size() != 0) {
this.index = 0;
this.column = this.columns.get(this.index);
} else {
this.index = -1;
this.column = null;
}
}
// Called between every row.
public void reset() {
buildColumnList(this.origColumns);
this.index = 0;
this.column = this.columns.get(this.index);
}
private void buildColumnList(NavigableSet<byte[]> columns) {
this.columns = new ArrayList<ColumnCount>(columns.size());
for(byte [] column : columns) {
this.columns.add(new ColumnCount(column,maxVersions));
}
}
}

View File

@ -28,10 +28,16 @@ import java.io.IOException;
class FailedLogCloseException extends IOException {
private static final long serialVersionUID = 1759152841462990925L;
/**
*
*/
public FailedLogCloseException() {
super();
}
/**
* @param arg0
*/
public FailedLogCloseException(String arg0) {
super(arg0);
}

View File

@ -0,0 +1,405 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
/**
* This class is responsible for the tracking and enforcement of Deletes
* during the course of a Get operation.
* <p>
* This class is utilized through three methods:
* <ul><li>{@link add} when encountering a Delete
* <li>{@link isDeleted} when checking if a Put KeyValue has been deleted
* <li>{@link update} when reaching the end of a StoreFile
* <p>
* This class is NOT thread-safe as queries are never multi-threaded
*/
public class GetDeleteTracker implements DeleteTracker {
private long familyStamp = -1L;
protected List<Delete> deletes = null;
private List<Delete> newDeletes = new ArrayList<Delete>();
private Iterator<Delete> iterator;
private Delete delete = null;
private KeyValue.KeyComparator comparator;
/**
* Constructor
* @param comparator
*/
public GetDeleteTracker(KeyValue.KeyComparator comparator) {
this.comparator = comparator;
}
/**
* Add the specified KeyValue to the list of deletes to check against for
* this row operation.
* <p>
* This is called when a Delete is encountered in a StoreFile.
* @param kv
* @param type
* @param timestamp
*/
@Override
public void add(byte [] buffer, int qualifierOffset, int qualifierLength,
long timestamp, byte type) {
if(type == KeyValue.Type.DeleteFamily.getCode()) {
if(timestamp > familyStamp) {
familyStamp = timestamp;
}
return;
}
if(timestamp > familyStamp) {
this.newDeletes.add(new Delete(buffer, qualifierOffset, qualifierLength,
type, timestamp));
}
}
/**
* Check if the specified KeyValue buffer has been deleted by a previously
* seen delete.
* @param buffer KeyValue buffer
* @param qualifierOffset column qualifier offset
* @param qualifierLength column qualifier length
* @param timestamp timestamp
* @return true is the specified KeyValue is deleted, false if not
*/
@Override
public boolean isDeleted(byte [] buffer, int qualifierOffset,
int qualifierLength, long timestamp) {
// Check against DeleteFamily
if(timestamp <= familyStamp) {
return true;
}
// Check if there are other deletes
if(this.delete == null) {
return false;
}
// Check column
int ret = comparator.compareRows(buffer, qualifierOffset, qualifierLength,
this.delete.buffer, this.delete.qualifierOffset,
this.delete.qualifierLength);
if(ret <= -1) {
// Have not reached the next delete yet
return false;
} else if(ret >= 1) {
// Deletes an earlier column, need to move down deletes
if(this.iterator.hasNext()) {
this.delete = this.iterator.next();
} else {
this.delete = null;
return false;
}
return isDeleted(buffer, qualifierOffset, qualifierLength, timestamp);
}
// Check Timestamp
if(timestamp > this.delete.timestamp) {
return false;
}
// Check Type
switch(KeyValue.Type.codeToType(this.delete.type)) {
case Delete:
boolean equal = timestamp == this.delete.timestamp;
if(this.iterator.hasNext()) {
this.delete = this.iterator.next();
} else {
this.delete = null;
}
if(equal){
return true;
}
// timestamp < this.delete.timestamp
// Delete of an explicit column newer than current
return isDeleted(buffer, qualifierOffset, qualifierLength, timestamp);
case DeleteColumn:
return true;
}
// should never reach this
return false;
}
@Override
public boolean isEmpty() {
if(this.familyStamp == 0L && this.delete == null) {
return true;
}
return false;
}
@Override
public void reset() {
this.deletes = null;
this.delete = null;
this.newDeletes = new ArrayList<Delete>();
this.familyStamp = 0L;
this.iterator = null;
}
/**
* Called at the end of every StoreFile.
* <p>
* Many optimized implementations of Trackers will require an update at
* when the end of each StoreFile is reached.
*/
@Override
public void update() {
// If no previous deletes, use new deletes and return
if(this.deletes == null || this.deletes.size() == 0) {
finalize(this.newDeletes);
return;
}
// If no new delete, retain previous deletes and return
if(this.newDeletes.size() == 0) {
return;
}
// Merge previous deletes with new deletes
List<Delete> mergeDeletes =
new ArrayList<Delete>(this.newDeletes.size());
int oldIndex = 0;
int newIndex = 0;
Delete newDelete = newDeletes.get(oldIndex);
Delete oldDelete = deletes.get(oldIndex);
while(true) {
switch(compareDeletes(oldDelete,newDelete)) {
case NEXT_NEW: {
if(++newIndex == newDeletes.size()) {
// Done with new, add the rest of old to merged and return
mergeDown(mergeDeletes, deletes, oldIndex);
finalize(mergeDeletes);
return;
}
newDelete = this.newDeletes.get(newIndex);
break;
}
case INCLUDE_NEW_NEXT_NEW: {
mergeDeletes.add(newDelete);
if(++newIndex == newDeletes.size()) {
// Done with new, add the rest of old to merged and return
mergeDown(mergeDeletes, deletes, oldIndex);
finalize(mergeDeletes);
return;
}
newDelete = this.newDeletes.get(newIndex);
break;
}
case INCLUDE_NEW_NEXT_BOTH: {
mergeDeletes.add(newDelete);
++oldIndex;
++newIndex;
if(oldIndex == deletes.size()) {
if(newIndex == newDeletes.size()) {
finalize(mergeDeletes);
return;
}
mergeDown(mergeDeletes, newDeletes, newIndex);
finalize(mergeDeletes);
return;
} else if(newIndex == newDeletes.size()) {
mergeDown(mergeDeletes, deletes, oldIndex);
finalize(mergeDeletes);
return;
}
oldDelete = this.deletes.get(oldIndex);
newDelete = this.newDeletes.get(newIndex);
break;
}
case INCLUDE_OLD_NEXT_BOTH: {
mergeDeletes.add(oldDelete);
++oldIndex;
++newIndex;
if(oldIndex == deletes.size()) {
if(newIndex == newDeletes.size()) {
finalize(mergeDeletes);
return;
}
mergeDown(mergeDeletes, newDeletes, newIndex);
finalize(mergeDeletes);
return;
} else if(newIndex == newDeletes.size()) {
mergeDown(mergeDeletes, deletes, oldIndex);
finalize(mergeDeletes);
return;
}
oldDelete = this.deletes.get(oldIndex);
newDelete = this.newDeletes.get(newIndex);
break;
}
case INCLUDE_OLD_NEXT_OLD: {
mergeDeletes.add(oldDelete);
if(++oldIndex == deletes.size()) {
mergeDown(mergeDeletes, newDeletes, newIndex);
finalize(mergeDeletes);
return;
}
oldDelete = this.deletes.get(oldIndex);
break;
}
case NEXT_OLD: {
if(++oldIndex == deletes.size()) {
// Done with old, add the rest of new to merged and return
mergeDown(mergeDeletes, newDeletes, newIndex);
finalize(mergeDeletes);
return;
}
oldDelete = this.deletes.get(oldIndex);
}
}
}
}
private void finalize(List<Delete> mergeDeletes) {
this.deletes = mergeDeletes;
this.newDeletes = new ArrayList<Delete>();
if(this.deletes.size() > 0){
this.iterator = deletes.iterator();
this.delete = iterator.next();
}
}
private void mergeDown(List<Delete> mergeDeletes, List<Delete> srcDeletes,
int srcIndex) {
int index = srcIndex;
while(index < srcDeletes.size()) {
mergeDeletes.add(srcDeletes.get(index++));
}
}
protected DeleteCompare compareDeletes(Delete oldDelete, Delete newDelete) {
// Compare columns
// Just compairing qualifier portion, can keep on using Bytes.compareTo().
int ret = Bytes.compareTo(oldDelete.buffer, oldDelete.qualifierOffset,
oldDelete.qualifierLength, newDelete.buffer, newDelete.qualifierOffset,
newDelete.qualifierLength);
if(ret <= -1) {
return DeleteCompare.INCLUDE_OLD_NEXT_OLD;
} else if(ret >= 1) {
return DeleteCompare.INCLUDE_NEW_NEXT_NEW;
}
// Same column
// Branches below can be optimized. Keeping like this until testing
// is complete.
if(oldDelete.type == newDelete.type) {
// the one case where we can merge 2 deletes -> 1 delete.
if(oldDelete.type == KeyValue.Type.Delete.getCode()){
if(oldDelete.timestamp > newDelete.timestamp) {
return DeleteCompare.INCLUDE_OLD_NEXT_OLD;
} else if(oldDelete.timestamp < newDelete.timestamp) {
return DeleteCompare.INCLUDE_NEW_NEXT_NEW;
} else {
return DeleteCompare.INCLUDE_OLD_NEXT_BOTH;
}
}
if(oldDelete.timestamp < newDelete.timestamp) {
return DeleteCompare.INCLUDE_NEW_NEXT_BOTH;
}
return DeleteCompare.INCLUDE_OLD_NEXT_BOTH;
}
// old delete is more specific than the new delete.
// if the olddelete is newer then the newdelete, we have to
// keep it
if(oldDelete.type < newDelete.type) {
if(oldDelete.timestamp > newDelete.timestamp) {
return DeleteCompare.INCLUDE_OLD_NEXT_OLD;
} else if(oldDelete.timestamp < newDelete.timestamp) {
return DeleteCompare.NEXT_OLD;
} else {
return DeleteCompare.NEXT_OLD;
}
}
// new delete is more specific than the old delete.
if(oldDelete.type > newDelete.type) {
if(oldDelete.timestamp > newDelete.timestamp) {
return DeleteCompare.NEXT_NEW;
} else if(oldDelete.timestamp < newDelete.timestamp) {
return DeleteCompare.INCLUDE_NEW_NEXT_NEW;
} else {
return DeleteCompare.NEXT_NEW;
}
}
// Should never reach,
// throw exception for assertion?
throw new RuntimeException("GetDeleteTracker:compareDelete reached terminal state");
}
/**
* Internal class used to store the necessary information for a Delete.
* <p>
* Rather than reparsing the KeyValue, or copying fields, this class points
* to the underlying KeyValue buffer with pointers to the qualifier and fields
* for type and timestamp. No parsing work is done in DeleteTracker now.
* <p>
* Fields are public because they are accessed often, directly, and only
* within this class.
*/
protected class Delete {
byte [] buffer;
int qualifierOffset;
int qualifierLength;
byte type;
long timestamp;
/**
* Constructor
* @param buffer
* @param qualifierOffset
* @param qualifierLength
* @param type
* @param timestamp
*/
public Delete(byte [] buffer, int qualifierOffset, int qualifierLength,
byte type, long timestamp) {
this.buffer = buffer;
this.qualifierOffset = qualifierOffset;
this.qualifierLength = qualifierLength;
this.type = type;
this.timestamp = timestamp;
}
}
}

View File

@ -1,214 +0,0 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ColumnNameParseException;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Abstract base class that implements the InternalScanner.
*/
public abstract class HAbstractScanner implements InternalScanner {
final Log LOG = LogFactory.getLog(this.getClass().getName());
// Pattern to determine if a column key is a regex
static final Pattern isRegexPattern =
Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$");
/** The kind of match we are doing on a column: */
private static enum MATCH_TYPE {
/** Just check the column family name */
FAMILY_ONLY,
/** Column family + matches regex */
REGEX,
/** Literal matching */
SIMPLE
}
private final List<ColumnMatcher> matchers = new ArrayList<ColumnMatcher>();
// True when scanning is done
protected volatile boolean scannerClosed = false;
// The timestamp to match entries against
protected final long timestamp;
private boolean wildcardMatch = false;
private boolean multipleMatchers = false;
/** Constructor for abstract base class */
protected HAbstractScanner(final long timestamp,
final NavigableSet<byte []> columns)
throws IOException {
this.timestamp = timestamp;
for (byte [] column: columns) {
ColumnMatcher matcher = new ColumnMatcher(column);
this.wildcardMatch = matcher.isWildCardMatch();
matchers.add(matcher);
this.multipleMatchers = !matchers.isEmpty();
}
}
/**
* For a particular column, find all the matchers defined for the column.
* Compare the column family and column key using the matchers. The first one
* that matches returns true. If no matchers are successful, return false.
*
* @param kv KeyValue to test
* @return true if any of the matchers for the column match the column family
* and the column key.
*
* @throws IOException
*/
protected boolean columnMatch(final KeyValue kv)
throws IOException {
if (matchers == null) {
return false;
}
for(int m = 0; m < this.matchers.size(); m++) {
if (this.matchers.get(m).matches(kv)) {
return true;
}
}
return false;
}
public boolean isWildcardScanner() {
return this.wildcardMatch;
}
public boolean isMultipleMatchScanner() {
return this.multipleMatchers;
}
public abstract boolean next(List<KeyValue> results)
throws IOException;
/**
* This class provides column matching functions that are more sophisticated
* than a simple string compare. There are three types of matching:
* <ol>
* <li>Match on the column family name only</li>
* <li>Match on the column family + column key regex</li>
* <li>Simple match: compare column family + column key literally</li>
* </ul>
*/
private static class ColumnMatcher {
private boolean wildCardmatch;
private MATCH_TYPE matchType;
private byte [] family;
private Pattern columnMatcher;
// Column without delimiter so easy compare to KeyValue column
private byte [] col;
private int familylength = 0;
ColumnMatcher(final byte [] col) throws IOException {
byte [][] parse = parseColumn(col);
// Make up column without delimiter
byte [] columnWithoutDelimiter =
new byte [parse[0].length + parse[1].length];
System.arraycopy(parse[0], 0, columnWithoutDelimiter, 0, parse[0].length);
System.arraycopy(parse[1], 0, columnWithoutDelimiter, parse[0].length,
parse[1].length);
// First position has family. Second has qualifier.
byte [] qualifier = parse[1];
try {
if (qualifier == null || qualifier.length == 0) {
this.matchType = MATCH_TYPE.FAMILY_ONLY;
this.family = parse[0];
this.wildCardmatch = true;
} else if (isRegexPattern.matcher(Bytes.toString(qualifier)).matches()) {
this.matchType = MATCH_TYPE.REGEX;
this.columnMatcher =
Pattern.compile(Bytes.toString(columnWithoutDelimiter));
this.wildCardmatch = true;
} else {
this.matchType = MATCH_TYPE.SIMPLE;
this.col = columnWithoutDelimiter;
this.familylength = parse[0].length;
this.wildCardmatch = false;
}
} catch(Exception e) {
throw new IOException("Column: " + Bytes.toString(col) + ": " +
e.getMessage());
}
}
/**
* @param kv
* @return
* @throws IOException
*/
boolean matches(final KeyValue kv) throws IOException {
if (this.matchType == MATCH_TYPE.SIMPLE) {
return kv.matchingColumnNoDelimiter(this.col, this.familylength);
} else if(this.matchType == MATCH_TYPE.FAMILY_ONLY) {
return kv.matchingFamily(this.family);
} else if (this.matchType == MATCH_TYPE.REGEX) {
// Pass a column without the delimiter since thats whats we're
// expected to match.
int o = kv.getColumnOffset();
int l = kv.getColumnLength(o);
String columnMinusQualifier = Bytes.toString(kv.getBuffer(), o, l);
return this.columnMatcher.matcher(columnMinusQualifier).matches();
} else {
throw new IOException("Invalid match type: " + this.matchType);
}
}
boolean isWildCardMatch() {
return this.wildCardmatch;
}
/**
* @param c Column name
* @return Return array of size two whose first element has the family
* prefix of passed column <code>c</code> and whose second element is the
* column qualifier.
* @throws ColumnNameParseException
*/
public static byte [][] parseColumn(final byte [] c)
throws ColumnNameParseException {
final byte [][] result = new byte [2][];
// TODO: Change this so don't do parse but instead use the comparator
// inside in KeyValue which just looks at column family.
final int index = KeyValue.getFamilyDelimiterIndex(c, 0, c.length);
if (index == -1) {
throw new ColumnNameParseException("Impossible column name: " + c);
}
result[0] = new byte [index];
System.arraycopy(c, 0, result[0], 0, index);
final int len = c.length - (index + 1);
result[1] = new byte[len];
System.arraycopy(c, index + 1 /*Skip delimiter*/, result[1], 0,
len);
return result;
}
}
}

View File

@ -100,7 +100,7 @@ import org.apache.hadoop.io.compress.DefaultCodec;
public class HLog implements HConstants, Syncable {
static final Log LOG = LogFactory.getLog(HLog.class);
private static final String HLOG_DATFILE = "hlog.dat.";
static final byte [] METACOLUMN = Bytes.toBytes("METACOLUMN:");
static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
static final byte [] METAROW = Bytes.toBytes("METAROW");
private final FileSystem fs;
private final Path dir;
@ -701,8 +701,8 @@ public class HLog implements HConstants, Syncable {
}
private KeyValue completeCacheFlushLogEdit() {
return new KeyValue(METAROW, METACOLUMN, System.currentTimeMillis(),
COMPLETE_CACHE_FLUSH);
return new KeyValue(METAROW, METAFAMILY, null,
System.currentTimeMillis(), COMPLETE_CACHE_FLUSH);
}
/**
@ -716,11 +716,11 @@ public class HLog implements HConstants, Syncable {
}
/**
* @param column
* @param family
* @return true if the column is a meta column
*/
public static boolean isMetaColumn(byte [] column) {
return Bytes.equals(METACOLUMN, column);
public static boolean isMetaFamily(byte [] family) {
return Bytes.equals(METAFAMILY, family);
}
/**
@ -870,6 +870,7 @@ public class HLog implements HConstants, Syncable {
Executors.newFixedThreadPool(DEFAULT_NUMBER_LOG_WRITER_THREAD);
for (final byte[] key : logEntries.keySet()) {
Thread thread = new Thread(Bytes.toString(key)) {
@Override
public void run() {
LinkedList<HLogEntry> entries = logEntries.get(key);
LOG.debug("Thread got " + entries.size() + " to process");

View File

@ -87,6 +87,9 @@ public class HLogKey implements WritableComparable<HLogKey>, HeapSize {
return logSeqNum;
}
/**
* @return the write time
*/
public long getWriteTime() {
return this.writeTime;
}

File diff suppressed because it is too large Load Diff

View File

@ -1,5 +1,5 @@
/**
* Copyright 2007 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -29,7 +29,6 @@ import java.lang.reflect.Field;
import java.net.BindException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
@ -38,7 +37,6 @@ import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Random;
import java.util.Set;
import java.util.SortedMap;
@ -79,13 +77,13 @@ import org.apache.hadoop.hbase.UnknownRowLockException;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.HMsg.Type;
import org.apache.hadoop.hbase.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ServerConnection;
import org.apache.hadoop.hbase.client.ServerConnectionManager;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
@ -758,8 +756,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
* @return RegionLoad instance.
* @throws IOException
*/
private HServerLoad.RegionLoad createRegionLoad(final HRegion r)
throws IOException {
private HServerLoad.RegionLoad createRegionLoad(final HRegion r) {
byte[] name = r.getRegionName();
int stores = 0;
int storefiles = 0;
@ -782,8 +779,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
* @return An instance of RegionLoad.
* @throws IOException
*/
public HServerLoad.RegionLoad createRegionLoad(final byte [] regionName)
throws IOException {
public HServerLoad.RegionLoad createRegionLoad(final byte [] regionName) {
return createRegionLoad(this.onlineRegions.get(Bytes.mapKey(regionName)));
}
@ -1080,12 +1076,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
for(Map.Entry<byte [], Store> ee: r.stores.entrySet()) {
Store store = ee.getValue();
storefiles += store.getStorefilesCount();
try {
storefileIndexSize += store.getStorefilesIndexSize();
} catch (IOException ex) {
LOG.warn("error getting store file index size for " + store +
": " + StringUtils.stringifyException(ex));
}
storefileIndexSize += store.getStorefilesIndexSize();
}
}
}
@ -1630,7 +1621,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
super(Thread.currentThread().getName() + ".regionCloser." + r.toString());
this.r = r;
}
@Override
public void run() {
try {
@ -1701,46 +1692,9 @@ public class HRegionServer implements HConstants, HRegionInterface,
return getRegion(regionName).getRegionInfo();
}
public Cell [] get(final byte [] regionName, final byte [] row,
final byte [] column, final long timestamp, final int numVersions)
throws IOException {
checkOpen();
requestCount.incrementAndGet();
try {
List<KeyValue> results =
getRegion(regionName).get(row, column, timestamp, numVersions);
return Cell.createSingleCellArray(results);
} catch (Throwable t) {
throw convertThrowableToIOE(cleanup(t));
}
}
public RowResult getRow(final byte [] regionName, final byte [] row,
final byte [][] columns, final long ts,
final int numVersions, final long lockId)
throws IOException {
checkOpen();
requestCount.incrementAndGet();
try {
// convert the columns array into a set so it's easy to check later.
NavigableSet<byte []> columnSet = null;
if (columns != null) {
columnSet = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
columnSet.addAll(Arrays.asList(columns));
}
HRegion region = getRegion(regionName);
HbaseMapWritable<byte [], Cell> result =
region.getFull(row, columnSet, ts, numVersions, getLockFromId(lockId));
if (result == null || result.isEmpty())
return null;
return new RowResult(row, result);
} catch (Throwable t) {
throw convertThrowableToIOE(cleanup(t));
}
}
public RowResult getClosestRowBefore(final byte [] regionName,
final byte [] row, final byte [] columnFamily)
public Result getClosestRowBefore(final byte [] regionName,
final byte [] row, final byte [] family)
throws IOException {
checkOpen();
requestCount.incrementAndGet();
@ -1748,49 +1702,41 @@ public class HRegionServer implements HConstants, HRegionInterface,
// locate the region we're operating on
HRegion region = getRegion(regionName);
// ask the region for all the data
RowResult rr = region.getClosestRowBefore(row, columnFamily);
return rr;
Result r = region.getClosestRowBefore(row, family);
return r;
} catch (Throwable t) {
throw convertThrowableToIOE(cleanup(t));
}
}
public RowResult next(final long scannerId) throws IOException {
RowResult[] rrs = next(scannerId, 1);
return rrs.length == 0 ? null : rrs[0];
}
public RowResult [] next(final long scannerId, int nbRows) throws IOException {
/** {@inheritDoc} */
public Result get(byte [] regionName, Get get) throws IOException {
checkOpen();
List<List<KeyValue>> results = new ArrayList<List<KeyValue>>();
requestCount.incrementAndGet();
try {
String scannerName = String.valueOf(scannerId);
InternalScanner s = scanners.get(scannerName);
if (s == null) {
throw new UnknownScannerException("Name: " + scannerName);
}
this.leases.renewLease(scannerName);
for (int i = 0; i < nbRows; i++) {
requestCount.incrementAndGet();
// Collect values to be returned here
List<KeyValue> values = new ArrayList<KeyValue>();
while (s.next(values)) {
if (!values.isEmpty()) {
// Row has something in it. Return the value.
results.add(values);
break;
}
}
}
return RowResult.createRowResultArray(results);
} catch (Throwable t) {
HRegion region = getRegion(regionName);
return region.get(get, getLockFromId(get.getLockId()));
} catch(Throwable t) {
throw convertThrowableToIOE(cleanup(t));
}
}
public void batchUpdate(final byte [] regionName, BatchUpdate b, long lockId)
public boolean exists(byte [] regionName, Get get) throws IOException {
checkOpen();
requestCount.incrementAndGet();
try {
HRegion region = getRegion(regionName);
Result r = region.get(get, getLockFromId(get.getLockId()));
return r != null && !r.isEmpty();
} catch(Throwable t) {
throw convertThrowableToIOE(cleanup(t));
}
}
public void put(final byte [] regionName, final Put put)
throws IOException {
if (b.getRow() == null)
if (put.getRow() == null)
throw new IllegalArgumentException("update has null row");
checkOpen();
@ -1798,24 +1744,24 @@ public class HRegionServer implements HConstants, HRegionInterface,
HRegion region = getRegion(regionName);
try {
cacheFlusher.reclaimMemcacheMemory();
region.batchUpdate(b, getLockFromId(b.getRowLock()));
region.put(put, getLockFromId(put.getLockId()));
} catch (Throwable t) {
throw convertThrowableToIOE(cleanup(t));
}
}
public int batchUpdates(final byte[] regionName, final BatchUpdate [] b)
public int put(final byte[] regionName, final Put [] puts)
throws IOException {
int i = 0;
checkOpen();
try {
HRegion region = getRegion(regionName);
this.cacheFlusher.reclaimMemcacheMemory();
Integer[] locks = new Integer[b.length];
for (i = 0; i < b.length; i++) {
Integer[] locks = new Integer[puts.length];
for (i = 0; i < puts.length; i++) {
this.requestCount.incrementAndGet();
locks[i] = getLockFromId(b[i].getRowLock());
region.batchUpdate(b[i], locks[i]);
locks[i] = getLockFromId(puts[i].getLockId());
region.put(puts[i], locks[i]);
}
} catch(WrongRegionException ex) {
return i;
@ -1827,38 +1773,49 @@ public class HRegionServer implements HConstants, HRegionInterface,
return -1;
}
public boolean checkAndSave(final byte [] regionName, final BatchUpdate b,
final HbaseMapWritable<byte[],byte[]> expectedValues)
throws IOException {
if (b.getRow() == null)
throw new IllegalArgumentException("update has null row");
/**
*
* @param regionName
* @param row
* @param family
* @param qualifier
* @param value the expected value
* @param put
* @throws IOException
* @return true if the new put was execute, false otherwise
*/
public boolean checkAndPut(final byte[] regionName, final byte [] row,
final byte [] family, final byte [] qualifier, final byte [] value,
final Put put) throws IOException{
//Getting actual value
Get get = new Get(row);
get.addColumn(family, qualifier);
checkOpen();
this.requestCount.incrementAndGet();
HRegion region = getRegion(regionName);
try {
cacheFlusher.reclaimMemcacheMemory();
return region.checkAndSave(b,
expectedValues,getLockFromId(b.getRowLock()), true);
return region.checkAndPut(row, family, qualifier, value, put,
getLockFromId(put.getLockId()), true);
} catch (Throwable t) {
throw convertThrowableToIOE(cleanup(t));
}
}
//
// remote scanner interface
//
public long openScanner(byte [] regionName, byte [][] cols, byte [] firstRow,
final long timestamp, final RowFilterInterface filter)
public long openScanner(byte [] regionName, Scan scan)
throws IOException {
checkOpen();
NullPointerException npe = null;
if (regionName == null) {
npe = new NullPointerException("regionName is null");
} else if (cols == null) {
npe = new NullPointerException("columns to scan is null");
} else if (firstRow == null) {
npe = new NullPointerException("firstRow for scanner is null");
} else if (scan == null) {
npe = new NullPointerException("scan is null");
}
if (npe != null) {
throw new IOException("Invalid arguments to openScanner", npe);
@ -1866,8 +1823,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
requestCount.incrementAndGet();
try {
HRegion r = getRegion(regionName);
InternalScanner s =
r.getScanner(cols, firstRow, timestamp, filter);
InternalScanner s = r.getScanner(scan);
long scannerId = addScanner(s);
return scannerId;
} catch (Throwable t) {
@ -1886,6 +1842,47 @@ public class HRegionServer implements HConstants, HRegionInterface,
createLease(scannerName, new ScannerListener(scannerName));
return scannerId;
}
public Result next(final long scannerId) throws IOException {
Result [] res = next(scannerId, 1);
if(res == null || res.length == 0) {
return null;
}
return res[0];
}
public Result [] next(final long scannerId, int nbRows) throws IOException {
checkOpen();
List<Result> results = new ArrayList<Result>();
try {
long start = System.currentTimeMillis();
String scannerName = String.valueOf(scannerId);
InternalScanner s = scanners.get(scannerName);
if (s == null) {
throw new UnknownScannerException("Name: " + scannerName);
}
this.leases.renewLease(scannerName);
for (int i = 0; i < nbRows; i++) {
requestCount.incrementAndGet();
// Collect values to be returned here
List<KeyValue> values = new ArrayList<KeyValue>();
boolean moreRows = s.next(values);
if(!values.isEmpty()) {
results.add(new Result(values));
}
if(!moreRows) {
break;
}
}
LOG.debug("Result[]next time: " + (System.currentTimeMillis()-start) + " (ms)");
return results.toArray(new Result[0]);
} catch (Throwable t) {
throw convertThrowableToIOE(cleanup(t));
}
}
public void close(final long scannerId) throws IOException {
try {
@ -1937,45 +1934,23 @@ public class HRegionServer implements HConstants, HRegionInterface,
// Methods that do the actual work for the remote API
//
public void deleteAll(final byte [] regionName, final byte [] row,
final byte [] column, final long timestamp, final long lockId)
public void delete(final byte [] regionName, final Delete delete)
throws IOException {
HRegion region = getRegion(regionName);
region.deleteAll(row, column, timestamp, getLockFromId(lockId));
}
public void deleteAll(final byte [] regionName, final byte [] row,
final long timestamp, final long lockId)
throws IOException {
HRegion region = getRegion(regionName);
region.deleteAll(row, timestamp, getLockFromId(lockId));
}
public void deleteAllByRegex(byte[] regionName, byte[] row, String colRegex,
long timestamp, long lockId) throws IOException {
getRegion(regionName).deleteAllByRegex(row, colRegex, timestamp,
getLockFromId(lockId));
}
public void deleteFamily(byte [] regionName, byte [] row, byte [] family,
long timestamp, final long lockId)
throws IOException{
getRegion(regionName).deleteFamily(row, family, timestamp,
getLockFromId(lockId));
}
public void deleteFamilyByRegex(byte[] regionName, byte[] row, String familyRegex,
long timestamp, long lockId) throws IOException {
getRegion(regionName).deleteFamilyByRegex(row, familyRegex, timestamp,
getLockFromId(lockId));
}
public boolean exists(byte[] regionName, byte[] row, byte[] column,
long timestamp, long lockId)
throws IOException {
return getRegion(regionName).exists(row, column, timestamp,
getLockFromId(lockId));
checkOpen();
try {
boolean writeToWAL = true;
this.cacheFlusher.reclaimMemcacheMemory();
this.requestCount.incrementAndGet();
Integer lock = getLockFromId(delete.getLockId());
HRegion region = getRegion(regionName);
region.delete(delete, lock, writeToWAL);
} catch(WrongRegionException ex) {
} catch (NotServingRegionException ex) {
} catch (Throwable t) {
throw convertThrowableToIOE(cleanup(t));
}
}
public long lockRow(byte [] regionName, byte [] row)
throws IOException {
@ -2023,7 +1998,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
* @return intId Integer row lock used internally in HRegion
* @throws IOException Thrown if this is not a valid client lock id.
*/
private Integer getLockFromId(long lockId)
Integer getLockFromId(long lockId)
throws IOException {
if(lockId == -1L) {
return null;
@ -2147,6 +2122,10 @@ public class HRegionServer implements HConstants, HRegionInterface,
return Collections.unmodifiableCollection(onlineRegions.values());
}
public HRegion [] getOnlineRegionsAsArray() {
return getOnlineRegions().toArray(new HRegion[0]);
}
/**
* @return The HRegionInfos from online regions sorted
*/
@ -2410,7 +2389,6 @@ public class HRegionServer implements HConstants, HRegionInterface,
} catch (Throwable t) {
LOG.error( "Can not start region server because "+
StringUtils.stringifyException(t) );
System.exit(-1);
}
break;
}
@ -2426,39 +2404,20 @@ public class HRegionServer implements HConstants, HRegionInterface,
}
}
/**
* @param args
*/
public static void main(String [] args) {
Configuration conf = new HBaseConfiguration();
@SuppressWarnings("unchecked")
Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
.getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
doMain(args, regionServerClass);
}
/** {@inheritDoc} */
public long incrementColumnValue(byte[] regionName, byte[] row,
byte[] column, long amount) throws IOException {
public long incrementColumnValue(byte [] regionName, byte [] row,
byte [] family, byte [] qualifier, long amount)
throws IOException {
checkOpen();
NullPointerException npe = null;
if (regionName == null) {
npe = new NullPointerException("regionName is null");
} else if (row == null) {
npe = new NullPointerException("row is null");
} else if (column == null) {
npe = new NullPointerException("column is null");
}
if (npe != null) {
IOException io = new IOException(
"Invalid arguments to incrementColumnValue", npe);
throw io;
throw new IOException("Invalid arguments to incrementColumnValue " +
"regionName is null");
}
requestCount.incrementAndGet();
try {
HRegion region = getRegion(regionName);
return region.incrementColumnValue(row, column, amount);
return region.incrementColumnValue(row, family, qualifier, amount);
} catch (IOException e) {
checkFileSystem();
throw e;
@ -2479,4 +2438,17 @@ public class HRegionServer implements HConstants, HRegionInterface,
public HServerInfo getHServerInfo() throws IOException {
return serverInfo;
}
/**
* @param args
*/
public static void main(String [] args) {
Configuration conf = new HBaseConfiguration();
@SuppressWarnings("unchecked")
Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
.getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
doMain(args, regionServerClass);
}
}

View File

@ -1,5 +1,5 @@
/**
* Copyright 2007 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -41,11 +41,9 @@ import org.apache.hadoop.hbase.KeyValue;
*/
public interface InternalScanner extends Closeable {
/**
* Grab the next row's worth of values. The scanner will return the most
* recent data value for each row that is not newer than the target time
* passed when the scanner was created.
* Grab the next row's worth of values.
* @param results
* @return true if data was returned
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException
*/
public boolean next(List<KeyValue> results)
@ -55,11 +53,5 @@ public interface InternalScanner extends Closeable {
* Closes the scanner and releases any resources it has allocated
* @throws IOException
*/
public void close() throws IOException;
/** @return true if the scanner is matching a column family or regex */
public boolean isWildcardScanner();
/** @return true if the scanner is matching multiple column family members */
public boolean isMultipleMatchScanner();
public void close() throws IOException;
}

View File

@ -0,0 +1,197 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.PriorityQueue;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
/**
* Implements a heap merge across any number of KeyValueScanners.
* <p>
* Implements KeyValueScanner itself.
* <p>
* This class is used at the Region level to merge across Stores
* and at the Store level to merge across the Memcache and StoreFiles.
* <p>
* In the Region case, we also need InternalScanner.next(List), so this class
* also implements InternalScanner. WARNING: As is, if you try to use this
* as an InternalScanner at the Store level, you will get runtime exceptions.
*/
public class KeyValueHeap implements KeyValueScanner, InternalScanner {
private PriorityQueue<KeyValueScanner> heap;
private KeyValueScanner current = null;
private KVScannerComparator comparator;
/**
* Constructor
* @param scanners
* @param comparator
*/
public KeyValueHeap(KeyValueScanner [] scanners, KVComparator comparator) {
this.comparator = new KVScannerComparator(comparator);
this.heap = new PriorityQueue<KeyValueScanner>(scanners.length,
this.comparator);
for(KeyValueScanner scanner : scanners) {
if(scanner.peek() != null) {
this.heap.add(scanner);
}
}
this.current = heap.poll();
}
public KeyValue peek() {
if(this.current == null) {
return null;
}
return this.current.peek();
}
public KeyValue next() {
if(this.current == null) {
return null;
}
KeyValue kvReturn = this.current.next();
KeyValue kvNext = this.current.peek();
if(kvNext == null) {
this.current.close();
this.current = this.heap.poll();
} else {
KeyValueScanner topScanner = this.heap.peek();
if(topScanner == null ||
this.comparator.compare(kvNext, topScanner.peek()) > 0) {
this.heap.add(this.current);
this.current = this.heap.poll();
}
}
return kvReturn;
}
/**
* Gets the next row of keys from the top-most scanner.
* <p>
* This method takes care of updating the heap.
* <p>
* This can ONLY be called when you are using Scanners that implement
* InternalScanner as well as KeyValueScanner (a {@link StoreScanner}).
* @return true if there are more keys, false if all scanners are done
*/
public boolean next(List<KeyValue> result) throws IOException {
InternalScanner currentAsInternal = (InternalScanner)this.current;
currentAsInternal.next(result);
KeyValue pee = this.current.peek();
if(pee == null) {
this.current.close();
} else {
this.heap.add(this.current);
}
this.current = this.heap.poll();
return (this.current != null);
}
private class KVScannerComparator implements Comparator<KeyValueScanner> {
private KVComparator kvComparator;
/**
* Constructor
* @param kvComparator
*/
public KVScannerComparator(KVComparator kvComparator) {
this.kvComparator = kvComparator;
}
public int compare(KeyValueScanner left, KeyValueScanner right) {
return compare(left.peek(), right.peek());
}
/**
* Compares two KeyValue
* @param left
* @param right
* @return less than 0 if left is smaller, 0 if equal etc..
*/
public int compare(KeyValue left, KeyValue right) {
return this.kvComparator.compare(left, right);
}
/**
* @return KVComparator
*/
public KVComparator getComparator() {
return this.kvComparator;
}
}
public void close() {
if(this.current != null) {
this.current.close();
}
KeyValueScanner scanner;
while((scanner = this.heap.poll()) != null) {
scanner.close();
}
}
/**
* Seeks all scanners at or below the specified seek key. If we earlied-out
* of a row, we may end up skipping values that were never reached yet.
* Rather than iterating down, we want to give the opportunity to re-seek.
* <p>
* As individual scanners may run past their ends, those scanners are
* automatically closed and removed from the heap.
* @param seekKey KeyValue to seek at or after
* @return true if KeyValues exist at or after specified key, false if not
*/
public boolean seek(KeyValue seekKey) {
if(this.current == null) {
return false;
}
this.heap.add(this.current);
this.current = null;
KeyValueScanner scanner;
while((scanner = this.heap.poll()) != null) {
KeyValue topKey = scanner.peek();
if(comparator.getComparator().compare(seekKey, topKey) <= 0) { // Correct?
// Top KeyValue is at-or-after Seek KeyValue
this.current = scanner;
return true;
}
if(!scanner.seek(seekKey)) {
scanner.close();
} else {
this.heap.add(scanner);
}
}
// Heap is returning empty, scanner is done
return false;
}
/**
* @return the current Heap
*/
public PriorityQueue<KeyValueScanner> getHeap() {
return this.heap;
}
}

View File

@ -1,5 +1,5 @@
/**
* Copyright 2008 The Apache Software Foundation
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -17,40 +17,35 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
package org.apache.hadoop.hbase.regionserver;
/** Thrown when a transaction cannot be committed.
*
import org.apache.hadoop.hbase.KeyValue;
/**
* Scanner that returns the next KeyValue.
*/
public class CommitUnsuccessfulException extends Exception {
private static final long serialVersionUID = 7062921444531109202L;
/** Default Constructor */
public CommitUnsuccessfulException() {
super();
}
public interface KeyValueScanner {
/**
* @param arg0 message
* @param arg1 cause
* Look at the next KeyValue in this scanner, but do not iterate scanner.
* @return the next KeyValue
*/
public CommitUnsuccessfulException(String arg0, Throwable arg1) {
super(arg0, arg1);
}
public KeyValue peek();
/**
* @param arg0 message
* Return the next KeyValue in this scanner, iterating the scanner
* @return the next KeyValue
*/
public CommitUnsuccessfulException(String arg0) {
super(arg0);
}
public KeyValue next();
/**
* @param arg0 cause
* Seek the scanner at or after the specified KeyValue.
* @param key
* @return true if scanner has values left, false if end of scanner
*/
public CommitUnsuccessfulException(Throwable arg0) {
super(arg0);
}
}
public boolean seek(KeyValue key);
/**
* Close the KeyValue scanner.
*/
public void close();
}

View File

@ -1,5 +1,5 @@
/**
* Copyright 2008 The Apache Software Foundation
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -27,22 +27,19 @@ import java.rmi.UnexpectedException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
import java.util.SortedMap;
import java.util.SortedSet;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.regionserver.HRegion.Counter;
import org.apache.hadoop.hbase.regionserver.DeleteCompare.DeleteCode;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -82,7 +79,7 @@ class Memcache {
// TODO: Fix this guess by studying jprofiler
private final static int ESTIMATED_KV_HEAP_TAX = 60;
/**
* Default constructor. Used for tests.
*/
@ -202,7 +199,86 @@ class Memcache {
}
return size;
}
/**
* Write a delete
* @param delete
* @return approximate size of the passed key and value.
*/
long delete(final KeyValue delete) {
long size = 0;
this.lock.readLock().lock();
//Have to find out what we want to do here, to find the fastest way of
//removing things that are under a delete.
//Actions that will take place here are:
//1. Insert a delete and remove all the affected entries already in memcache
//2. In the case of a Delete and the matching put is found then don't insert
// the delete
//TODO Would be nice with if we had an iterator for this, so we could remove
//things that needs to be removed while iterating and don't have to go
//back and do it afterwards
try {
boolean notpresent = false;
List<KeyValue> deletes = new ArrayList<KeyValue>();
SortedSet<KeyValue> tailSet = this.memcache.tailSet(delete);
//Parse the delete, so that it is only done once
byte [] deleteBuffer = delete.getBuffer();
int deleteOffset = delete.getOffset();
int deleteKeyLen = Bytes.toInt(deleteBuffer, deleteOffset);
deleteOffset += Bytes.SIZEOF_INT + Bytes.SIZEOF_INT;
short deleteRowLen = Bytes.toShort(deleteBuffer, deleteOffset);
deleteOffset += Bytes.SIZEOF_SHORT;
int deleteRowOffset = deleteOffset;
deleteOffset += deleteRowLen;
byte deleteFamLen = deleteBuffer[deleteOffset];
deleteOffset += Bytes.SIZEOF_BYTE + deleteFamLen;
int deleteQualifierOffset = deleteOffset;
int deleteQualifierLen = deleteKeyLen - deleteRowLen - deleteFamLen -
Bytes.SIZEOF_SHORT - Bytes.SIZEOF_BYTE - Bytes.SIZEOF_LONG -
Bytes.SIZEOF_BYTE;
deleteOffset += deleteQualifierLen;
int deleteTimestampOffset = deleteOffset;
deleteOffset += Bytes.SIZEOF_LONG;
byte deleteType = deleteBuffer[deleteOffset];
//Comparing with tail from memcache
for(KeyValue mem : tailSet) {
DeleteCode res = DeleteCompare.deleteCompare(mem, deleteBuffer,
deleteRowOffset, deleteRowLen, deleteQualifierOffset,
deleteQualifierLen, deleteTimestampOffset, deleteType,
comparator.getRawComparator());
if(res == DeleteCode.DONE) {
break;
} else if (res == DeleteCode.DELETE) {
deletes.add(mem);
} // SKIP
}
//Delete all the entries effected by the last added delete
for(KeyValue del : deletes) {
notpresent = this.memcache.remove(del);
size -= heapSize(del, notpresent);
}
//Adding the delete to memcache
notpresent = this.memcache.add(delete);
size += heapSize(delete, notpresent);
} finally {
this.lock.readLock().unlock();
}
return size;
}
/*
* Calculate how the memcache size has changed, approximately. Be careful.
* If class changes, be sure to change the size calculation.
@ -218,43 +294,6 @@ class Memcache {
ESTIMATED_KV_HEAP_TAX + 57 + kv.getLength(): 0; // Guess no change in size.
}
/**
* Look back through all the backlog TreeMaps to find the target.
* @param kv
* @param numVersions
* @return Set of KeyValues. Empty size not null if no results.
*/
List<KeyValue> get(final KeyValue kv, final int numVersions) {
List<KeyValue> results = new ArrayList<KeyValue>();
get(kv, numVersions, results,
new TreeSet<KeyValue>(this.comparatorIgnoreType),
System.currentTimeMillis());
return results;
}
/**
* Look back through all the backlog TreeMaps to find the target.
* @param key
* @param versions
* @param results
* @param deletes Pass a Set that has a Comparator that ignores key type.
* @param now
* @return True if enough versions.
*/
boolean get(final KeyValue key, final int versions,
List<KeyValue> results, final NavigableSet<KeyValue> deletes,
final long now) {
this.lock.readLock().lock();
try {
if (get(this.memcache, key, versions, results, deletes, now)) {
return true;
}
return get(this.snapshot, key, versions , results, deletes, now);
} finally {
this.lock.readLock().unlock();
}
}
/**
* @param kv Find the row that comes after this one. If null, we return the
* first.
@ -307,86 +346,6 @@ class Memcache {
return result;
}
/**
* Return all the available columns for the given key. The key indicates a
* row and timestamp, but not a column name.
* @param origin Where to start searching. Specifies a row and timestamp.
* Columns are specified in following arguments.
* @param columns Pass null for all columns else the wanted subset.
* @param columnPattern Column pattern to match.
* @param numVersions number of versions to retrieve
* @param versionsCount Map of KV to Count. Uses a Comparator that doesn't
* look at timestamps so only Row/Column are compared.
* @param deletes Pass a Set that has a Comparator that ignores key type.
* @param results Where to stick row results found.
* @return True if we found enough results for passed <code>columns</code>
* and <code>numVersions</code>.
*/
boolean getFull(final KeyValue key, NavigableSet<byte []> columns,
final Pattern columnPattern,
int numVersions, final Map<KeyValue, HRegion.Counter> versionsCount,
final NavigableSet<KeyValue> deletes,
final List<KeyValue> results, final long now) {
this.lock.readLock().lock();
try {
// Used to be synchronized but now with weak iteration, no longer needed.
if (getFull(this.memcache, key, columns, columnPattern, numVersions,
versionsCount, deletes, results, now)) {
// Has enough results.
return true;
}
return getFull(this.snapshot, key, columns, columnPattern, numVersions,
versionsCount, deletes, results, now);
} finally {
this.lock.readLock().unlock();
}
}
/*
* @param set
* @param target Where to start searching.
* @param columns
* @param versions
* @param versionCounter
* @param deletes Pass a Set that has a Comparator that ignores key type.
* @param keyvalues
* @return True if enough results found.
*/
private boolean getFull(final ConcurrentSkipListSet<KeyValue> set,
final KeyValue target, final Set<byte []> columns,
final Pattern columnPattern,
final int versions, final Map<KeyValue, HRegion.Counter> versionCounter,
final NavigableSet<KeyValue> deletes, List<KeyValue> keyvalues,
final long now) {
boolean hasEnough = false;
if (target == null) {
return hasEnough;
}
NavigableSet<KeyValue> tailset = set.tailSet(target);
if (tailset == null || tailset.isEmpty()) {
return hasEnough;
}
// TODO: This loop same as in HStore.getFullFromStoreFile. Make sure they
// are the same.
for (KeyValue kv: tailset) {
// Make sure we have not passed out the row. If target key has a
// column on it, then we are looking explicit key+column combination. If
// we've passed it out, also break.
if (target.isEmptyColumn()? !this.comparator.matchingRows(target, kv):
!this.comparator.matchingRowColumn(target, kv)) {
break;
}
if (!Store.getFullCheck(this.comparator, target, kv, columns, columnPattern)) {
continue;
}
if (Store.doKeyValue(kv, versions, versionCounter, columns, deletes, now,
this.ttl, keyvalues, tailset)) {
hasEnough = true;
break;
}
}
return hasEnough;
}
/**
* @param row Row to look for.
@ -554,45 +513,6 @@ class Memcache {
}
}
/*
* Examine a single map for the desired key.
*
* TODO - This is kinda slow. We need a data structure that allows for
* proximity-searches, not just precise-matches.
*
* @param set
* @param key
* @param results
* @param versions
* @param keyvalues
* @param deletes Pass a Set that has a Comparator that ignores key type.
* @param now
* @return True if enough versions.
*/
private boolean get(final ConcurrentSkipListSet<KeyValue> set,
final KeyValue key, final int versions,
final List<KeyValue> keyvalues,
final NavigableSet<KeyValue> deletes,
final long now) {
NavigableSet<KeyValue> tailset = set.tailSet(key);
if (tailset.isEmpty()) {
return false;
}
boolean enoughVersions = false;
for (KeyValue kv : tailset) {
if (this.comparator.matchingRowColumn(kv, key)) {
if (Store.doKeyValue(kv, versions, deletes, now, this.ttl, keyvalues,
tailset)) {
enoughVersions = true;
break;
}
} else {
// By L.N. HBASE-684, map is sorted, so we can't find match any more.
break;
}
}
return enoughVersions;
}
/*
* @param set
@ -621,93 +541,160 @@ class Memcache {
/**
* @return a scanner over the keys in the Memcache
*/
InternalScanner getScanner(long timestamp,
final NavigableSet<byte []> targetCols, final byte [] firstRow)
throws IOException {
KeyValueScanner getScanner() {
this.lock.readLock().lock();
try {
return new MemcacheScanner(timestamp, targetCols, firstRow);
return new MemcacheScanner();
} finally {
this.lock.readLock().unlock();
}
}
//////////////////////////////////////////////////////////////////////////////
// MemcacheScanner implements the InternalScanner.
// It lets the caller scan the contents of the Memcache.
//////////////////////////////////////////////////////////////////////////////
private class MemcacheScanner extends HAbstractScanner {
private KeyValue current;
private final NavigableSet<byte []> columns;
private final NavigableSet<KeyValue> deletes;
private final Map<KeyValue, Counter> versionCounter;
private final long now = System.currentTimeMillis();
MemcacheScanner(final long timestamp, final NavigableSet<byte []> columns,
final byte [] firstRow)
throws IOException {
// Call to super will create ColumnMatchers and whether this is a regex
// scanner or not. Will also save away timestamp. Also sorts rows.
super(timestamp, columns);
this.deletes = new TreeSet<KeyValue>(comparatorIgnoreType);
this.versionCounter =
new TreeMap<KeyValue, Counter>(comparatorIgnoreTimestamp);
this.current = KeyValue.createFirstOnRow(firstRow, timestamp);
// If we're being asked to scan explicit columns rather than all in
// a family or columns that match regexes, cache the sorted array of
// columns.
this.columns = isWildcardScanner()? null: columns;
//
// HBASE-880/1249/1304
//
/**
* Perform a single-row Get on the memcache and snapshot, placing results
* into the specified KV list.
* <p>
* This will return true if it is determined that the query is complete
* and it is not necessary to check any storefiles after this.
* <p>
* Otherwise, it will return false and you should continue on.
* @param startKey Starting KeyValue
* @param matcher Column matcher
* @param result List to add results to
* @return true if done with store (early-out), false if not
* @throws IOException
*/
public boolean get(QueryMatcher matcher, List<KeyValue> result)
throws IOException {
this.lock.readLock().lock();
try {
if(internalGet(this.memcache, matcher, result) || matcher.isDone()) {
return true;
}
matcher.update();
if(internalGet(this.snapshot, matcher, result) || matcher.isDone()) {
return true;
}
return false;
} finally {
this.lock.readLock().unlock();
}
}
/**
*
* @param set memcache or snapshot
* @param matcher query matcher
* @param result list to add results to
* @return true if done with store (early-out), false if not
* @throws IOException
*/
private boolean internalGet(SortedSet<KeyValue> set, QueryMatcher matcher,
List<KeyValue> result) throws IOException {
if(set.isEmpty()) return false;
// Seek to startKey
SortedSet<KeyValue> tailSet = set.tailSet(matcher.getStartKey());
for (KeyValue kv : tailSet) {
QueryMatcher.MatchCode res = matcher.match(kv);
switch(res) {
case INCLUDE:
result.add(kv);
break;
case SKIP:
break;
case NEXT:
return false;
case DONE:
return true;
default:
throw new RuntimeException("Unexpected " + res);
}
}
return false;
}
//////////////////////////////////////////////////////////////////////////////
// MemcacheScanner implements the KeyValueScanner.
// It lets the caller scan the contents of the Memcache.
// This behaves as if it were a real scanner but does not maintain position
// in the Memcache tree.
//////////////////////////////////////////////////////////////////////////////
@Override
public boolean next(final List<KeyValue> keyvalues)
throws IOException {
if (this.scannerClosed) {
protected class MemcacheScanner implements KeyValueScanner {
private KeyValue current = null;
private List<KeyValue> result = new ArrayList<KeyValue>();
private int idx = 0;
MemcacheScanner() {}
public boolean seek(KeyValue key) {
try {
if(key == null) {
close();
return false;
}
current = key;
return cacheNextRow();
} catch(Exception e) {
close();
return false;
}
while (keyvalues.isEmpty() && this.current != null) {
// Deletes are per row.
if (!deletes.isEmpty()) {
deletes.clear();
}
public KeyValue peek() {
if(idx >= result.size()) {
if(!cacheNextRow()) {
return null;
}
if (!versionCounter.isEmpty()) {
versionCounter.clear();
}
// The getFull will take care of expired and deletes inside memcache.
// The first getFull when row is the special empty bytes will return
// nothing so we go around again. Alternative is calling a getNextRow
// if row is null but that looks like it would take same amount of work
// so leave it for now.
getFull(this.current, isWildcardScanner()? null: this.columns, null, 1,
versionCounter, deletes, keyvalues, this.now);
for (KeyValue bb: keyvalues) {
if (isWildcardScanner()) {
// Check the results match. We only check columns, not timestamps.
// We presume that timestamps have been handled properly when we
// called getFull.
if (!columnMatch(bb)) {
keyvalues.remove(bb);
}
}
}
// Add any deletes found so they are available to the StoreScanner#next.
if (!this.deletes.isEmpty()) {
keyvalues.addAll(deletes);
}
this.current = getNextRow(this.current);
// Change current to be column-less and to have the scanners' now. We
// do this because first item on 'next row' may not have the scanners'
// now time which will cause trouble down in getFull; same reason no
// column.
if (this.current != null) this.current = this.current.cloneRow(this.now);
return peek();
}
return !keyvalues.isEmpty();
return result.get(idx);
}
public KeyValue next() {
if(idx >= result.size()) {
if(!cacheNextRow()) {
return null;
}
return next();
}
return result.get(idx++);
}
boolean cacheNextRow() {
NavigableSet<KeyValue> keys;
try {
keys = memcache.tailSet(current);
} catch(Exception e) {
close();
return false;
}
if(keys == null || keys.isEmpty()) {
close();
return false;
}
current = null;
byte [] row = keys.first().getRow();
for(KeyValue key : keys) {
if(comparator.compareRows(key, row) != 0) {
current = key;
break;
}
result.add(key);
}
return true;
}
public void close() {
if (!scannerClosed) {
scannerClosed = true;
current = null;
idx = 0;
if(!result.isEmpty()) {
result.clear();
}
}
}
@ -721,8 +708,7 @@ class Memcache {
* @throws InterruptedException
* @throws IOException
*/
public static void main(String [] args)
throws InterruptedException, IOException {
public static void main(String [] args) {
RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());

View File

@ -62,11 +62,11 @@ class MemcacheFlusher extends Thread implements FlushRequester {
protected final long globalMemcacheLimit;
protected final long globalMemcacheLimitLowMark;
public static final float DEFAULT_UPPER = 0.4f;
public static final float DEFAULT_LOWER = 0.25f;
public static final String UPPER_KEY =
private static final float DEFAULT_UPPER = 0.4f;
private static final float DEFAULT_LOWER = 0.25f;
private static final String UPPER_KEY =
"hbase.regionserver.globalMemcache.upperLimit";
public static final String LOWER_KEY =
private static final String LOWER_KEY =
"hbase.regionserver.globalMemcache.lowerLimit";
private long blockingStoreFilesNumber;
private long blockingWaitTime;

View File

@ -0,0 +1,119 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.List;
import java.io.IOException;
/**
* A scanner that does a minor compaction at the same time. Doesn't need to
* implement ChangedReadersObserver, since it doesn't scan memcache, only store files
* and optionally the memcache-snapshot.
*/
public class MinorCompactingStoreScanner implements KeyValueScanner, InternalScanner {
private QueryMatcher matcher;
private KeyValueHeap heap;
MinorCompactingStoreScanner(Store store,
KeyValueScanner [] scanners) {
Scan scan = new Scan();
// No max version, no ttl matching, start at first row, all columns.
matcher = new ScanQueryMatcher(scan, store.getFamily().getName(),
null, Long.MAX_VALUE, store.comparator.getRawComparator(),
store.versionsToReturn(Integer.MAX_VALUE));
for (KeyValueScanner scanner : scanners ) {
scanner.seek(matcher.getStartKey());
}
heap = new KeyValueHeap(scanners, store.comparator);
}
MinorCompactingStoreScanner(String cfName, KeyValue.KVComparator comparator,
KeyValueScanner [] scanners) {
Scan scan = new Scan();
matcher = new ScanQueryMatcher(scan, Bytes.toBytes(cfName),
null, Long.MAX_VALUE, comparator.getRawComparator(),
Integer.MAX_VALUE);
for (KeyValueScanner scanner : scanners ) {
scanner.seek(matcher.getStartKey());
}
heap = new KeyValueHeap(scanners, comparator);
}
public KeyValue peek() {
return heap.peek();
}
public KeyValue next() {
return heap.next();
}
@Override
public boolean seek(KeyValue key) {
// cant seek.
throw new UnsupportedOperationException("Can't seek a MinorCompactingStoreScanner");
}
@Override
public boolean next(List<KeyValue> results) throws IOException {
KeyValue peeked = heap.peek();
if (peeked == null) {
close();
return false;
}
matcher.setRow(peeked.getRow());
KeyValue kv;
while ((kv = heap.peek()) != null) {
// if delete type, output no matter what:
if (kv.getType() != KeyValue.Type.Put.getCode())
results.add(kv);
switch (matcher.match(kv)) {
case INCLUDE:
results.add(heap.next());
continue;
case DONE:
if (results.isEmpty()) {
matcher.setRow(heap.peek().getRow());
continue;
}
return true;
}
heap.next();
}
close();
return false;
}
public void close() {
heap.close();
}
}

View File

@ -0,0 +1,373 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.util.NavigableSet;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.util.Bytes;
/**
* This is the primary class used to process KeyValues during a Get or Scan
* operation.
* <p>
* It encapsulates the handling of the column and version input parameters to
* the query through a {@link ColumnTracker}.
* <p>
* Deletes are handled using the {@link DeleteTracker}.
* <p>
* All other query parameters are accessed from the client-specified Get.
* <p>
* The primary method used is {@link match} with the current KeyValue. It will
* return a {@link MatchCode}
*
* , deletes,
* versions,
*/
public class QueryMatcher {
/**
* {@link match} return codes. These instruct the scanner moving through
* Memcaches and StoreFiles what to do with the current KeyValue.
* <p>
* Additionally, this contains "early-out" language to tell the scanner to
* move on to the next File (Memcache or Storefile), or to return immediately.
*/
static enum MatchCode {
/**
* Include KeyValue in the returned result
*/
INCLUDE,
/**
* Do not include KeyValue in the returned result
*/
SKIP,
/**
* Do not include, jump to next StoreFile or Memcache (in time order)
*/
NEXT,
/**
* Do not include, return current result
*/
DONE,
/**
* These codes are used by the ScanQueryMatcher
*/
/**
* Done with the row, seek there.
*/
SEEK_NEXT_ROW,
/**
* Done with column, seek to next.
*/
SEEK_NEXT_COL,
/**
* Done with scan, thanks to the row filter.
*/
DONE_SCAN,
}
/** Keeps track of deletes */
protected DeleteTracker deletes;
/** Keeps track of columns and versions */
protected ColumnTracker columns;
/** Key to seek to in Memcache and StoreFiles */
protected KeyValue startKey;
/** Row comparator for the region this query is for */
KeyComparator rowComparator;
/** Row the query is on */
protected byte [] row;
/** TimeRange the query is for */
protected TimeRange tr;
/** Oldest allowed version stamp for TTL enforcement */
protected long oldestStamp;
/**
* Constructs a QueryMatcher for a Get.
* @param get
* @param row
* @param family
* @param columns
* @param ttl
* @param rowComparator
*/
public QueryMatcher(Get get, byte [] row, byte [] family,
NavigableSet<byte[]> columns, long ttl, KeyComparator rowComparator,
int maxVersions) {
this.row = row;
this.tr = get.getTimeRange();
this.oldestStamp = System.currentTimeMillis() - ttl;
this.rowComparator = rowComparator;
this.deletes = new GetDeleteTracker(rowComparator);
this.startKey = KeyValue.createFirstOnRow(row);
// Single branch to deal with two types of Gets (columns vs all in family)
if(columns == null || columns.size() == 0) {
this.columns = new WildcardColumnTracker(maxVersions);
} else {
this.columns = new ExplicitColumnTracker(columns, maxVersions);
}
}
// For the subclasses.
protected QueryMatcher() {
}
/**
* Constructs a copy of an existing QueryMatcher with a new row.
* @param matcher
* @param row
*/
public QueryMatcher(QueryMatcher matcher, byte [] row) {
this.row = row;
this.tr = matcher.getTimeRange();
this.oldestStamp = matcher.getOldestStamp();
this.rowComparator = matcher.getRowComparator();
this.columns = matcher.getColumnTracker();
this.deletes = matcher.getDeleteTracker();
this.startKey = matcher.getStartKey();
reset();
}
/**
* Main method for ColumnMatcher.
* <p>
* Determines whether the specified KeyValue should be included in the
* result or not.
* <p>
* Contains additional language to early-out of the current file or to
* return immediately.
* <p>
* Things to be checked:<ul>
* <li>Row
* <li>TTL
* <li>Type
* <li>TimeRange
* <li>Deletes
* <li>Column
* <li>Versions
* @param kv KeyValue to check
* @return MatchCode: include, skip, next, done
*/
public MatchCode match(KeyValue kv) {
if(this.columns.done()) {
return MatchCode.DONE; // done_row
}
// Directly act on KV buffer
byte [] bytes = kv.getBuffer();
int offset = kv.getOffset();
int keyLength = Bytes.toInt(bytes, offset);
offset += KeyValue.ROW_OFFSET;
short rowLength = Bytes.toShort(bytes, offset);
offset += Bytes.SIZEOF_SHORT;
// scanners are relying on us to check the row first, and return
// "NEXT" when we are there.
/* Check ROW
* If past query's row, go to next StoreFile
* If not reached query's row, go to next KeyValue
*/
int ret = this.rowComparator.compareRows(row, 0, row.length,
bytes, offset, rowLength);
if(ret <= -1) {
// Have reached the next row
return MatchCode.NEXT; // got_to_next_row (end)
} else if(ret >= 1) {
// At a previous row
return MatchCode.SKIP; // skip_to_cur_row
}
offset += rowLength;
byte familyLength = bytes[offset];
offset += Bytes.SIZEOF_BYTE + familyLength;
int columnLength = keyLength + KeyValue.ROW_OFFSET -
(offset - kv.getOffset()) - KeyValue.TIMESTAMP_TYPE_SIZE;
int columnOffset = offset;
offset += columnLength;
/* Check TTL
* If expired, go to next KeyValue
*/
long timestamp = Bytes.toLong(bytes, offset);
if(isExpired(timestamp)) {
// reached the expired part, for scans, this indicates we're done.
return MatchCode.NEXT; // done_row
}
offset += Bytes.SIZEOF_LONG;
/* Check TYPE
* If a delete within (or after) time range, add to deletes
* Move to next KeyValue
*/
byte type = bytes[offset];
// if delete type == delete family, return done_row
if(isDelete(type)) {
if(tr.withinOrAfterTimeRange(timestamp)) {
this.deletes.add(bytes, columnOffset, columnLength, timestamp, type);
}
return MatchCode.SKIP; // skip the delete cell.
}
/* Check TimeRange
* If outside of range, move to next KeyValue
*/
if(!tr.withinTimeRange(timestamp)) {
return MatchCode.SKIP; // optimization chances here.
}
/* Check Deletes
* If deleted, move to next KeyValue
*/
if(!deletes.isEmpty() && deletes.isDeleted(bytes, columnOffset,
columnLength, timestamp)) {
// 2 types of deletes:
// affects 1 cell or 1 column, so just skip the keyvalues.
// - delete family, so just skip to the next row.
return MatchCode.SKIP;
}
/* Check Column and Versions
* Returns a MatchCode directly, identical language
* If matched column without enough versions, include
* If enough versions of this column or does not match, skip
* If have moved past
* If enough versions of everything,
*/
return columns.checkColumn(bytes, columnOffset, columnLength);
}
// should be in KeyValue.
protected boolean isDelete(byte type) {
return (type != KeyValue.Type.Put.getCode());
}
protected boolean isExpired(long timestamp) {
return (timestamp < oldestStamp);
}
/**
* If matcher returns SEEK_NEXT_COL you may be able
* to get a hint of the next column to seek to - call this.
* If it returns null, there is no hint.
*
* @return immediately after match returns SEEK_NEXT_COL - null if no hint,
* else the next column we want
*/
public ColumnCount getSeekColumn() {
return this.columns.getColumnHint();
}
/**
* Called after reading each section (memcache, snapshot, storefiles).
* <p>
* This method will update the internal structures to be accurate for
* the next section.
*/
public void update() {
this.deletes.update();
this.columns.update();
}
/**
* Resets the current columns and deletes
*/
public void reset() {
this.deletes.reset();
this.columns.reset();
}
/**
* Set current row
* @param row
*/
public void setRow(byte [] row) {
this.row = row;
}
/**
*
* @return the start key
*/
public KeyValue getStartKey() {
return this.startKey;
}
/**
* @return the TimeRange
*/
public TimeRange getTimeRange() {
return this.tr;
}
/**
* @return the oldest stamp
*/
public long getOldestStamp() {
return this.oldestStamp;
}
/**
* @return current KeyComparator
*/
public KeyComparator getRowComparator() {
return this.rowComparator;
}
/**
* @return ColumnTracker
*/
public ColumnTracker getColumnTracker() {
return this.columns;
}
/**
* @return DeleteTracker
*/
public DeleteTracker getDeleteTracker() {
return this.deletes;
}
/**
*
* @return
*/
public boolean isDone() {
return this.columns.done();
}
}

Some files were not shown because too many files have changed in this diff Show More