HBASE-465 Fix javadoc for all public declarations

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@693597 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jim Kellerman 2008-09-09 20:36:49 +00:00
parent 3b540886df
commit e93249b444
135 changed files with 591 additions and 566 deletions

View File

@ -82,6 +82,7 @@ Release 0.18.0 - Unreleased
shell or thrift server, etc. (Jonathan Gray via Jim Kellerman)
HBASE-871 Major compaction periodicity should be specifyable at the column
family level, not cluster wide (Jonathan Gray via Stack)
HBASE-465 Fix javadoc for all public declarations
NEW FEATURES
HBASE-787 Postgresql to HBase table replication example (Tim Sell via Stack)

View File

@ -49,7 +49,6 @@ public abstract class Chore extends Thread {
this.stop = s;
}
/** {@inheritDoc} */
@Override
public void run() {
try {

View File

@ -22,11 +22,16 @@ package org.apache.hadoop.hbase;
/**
* Thrown if issue with passed column name.
*/
@SuppressWarnings("serial")
public class ColumnNameParseException extends DoNotRetryIOException {
/** default constructor */
public ColumnNameParseException() {
super();
}
/**
* @param message
*/
public ColumnNameParseException(String message) {
super(message);
}

View File

@ -25,11 +25,18 @@ import java.io.IOException;
* Subclass if exception is not meant to be retried: e.g.
* {@link UnknownScannerException}
*/
@SuppressWarnings("serial")
public class DoNotRetryIOException extends IOException {
/**
* default constructor
*/
public DoNotRetryIOException() {
super();
}
/**
* @param message
*/
public DoNotRetryIOException(String message) {
super(message);
}

View File

@ -21,11 +21,18 @@ import java.io.IOException;
* Thrown during flush if the possibility snapshot content was not properly
* persisted into store files. Response should include replay of hlog content.
*/
@SuppressWarnings("serial")
public class DroppedSnapshotException extends IOException {
/**
* @param msg
*/
public DroppedSnapshotException(String msg) {
super(msg);
}
/**
* default constructor
*/
public DroppedSnapshotException() {
super();
}

View File

@ -62,13 +62,13 @@ public class HColumnDescriptor implements WritableComparable {
BLOCK
}
public static final String COMPRESSION = "COMPRESSION";
public static final String BLOCKCACHE = "BLOCKCACHE";
public static final String LENGTH = "LENGTH";
public static final String TTL = "TTL";
public static final String BLOOMFILTER = "BLOOMFILTER";
public static final String FOREVER = "FOREVER";
public static final String MAPFILE_INDEX_INTERVAL =
public static final String COMPRESSION = "COMPRESSION"; //TODO: change to protected
public static final String BLOCKCACHE = "BLOCKCACHE"; //TODO: change to protected
public static final String LENGTH = "LENGTH"; //TODO: change to protected
public static final String TTL = "TTL"; //TODO: change to protected
public static final String BLOOMFILTER = "BLOOMFILTER"; //TODO: change to protected
public static final String FOREVER = "FOREVER"; //TODO: change to protected
public static final String MAPFILE_INDEX_INTERVAL = //TODO: change to protected
"MAPFILE_INDEX_INTERVAL";
/**
@ -455,7 +455,6 @@ public class HColumnDescriptor implements WritableComparable {
setValue(MAPFILE_INDEX_INTERVAL, Integer.toString(interval));
}
/** {@inheritDoc} */
@Override
public String toString() {
StringBuffer s = new StringBuffer();
@ -476,13 +475,11 @@ public class HColumnDescriptor implements WritableComparable {
return s.toString();
}
/** {@inheritDoc} */
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
/** {@inheritDoc} */
@Override
public int hashCode() {
int result = Bytes.hashCode(this.name);
@ -493,7 +490,7 @@ public class HColumnDescriptor implements WritableComparable {
// Writable
/** {@inheritDoc} */
@SuppressWarnings("deprecation")
public void readFields(DataInput in) throws IOException {
int version = in.readByte();
if (version < 6) {
@ -543,7 +540,6 @@ public class HColumnDescriptor implements WritableComparable {
}
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
out.writeByte(COLUMN_DESCRIPTOR_VERSION);
Bytes.writeByteArray(out, this.name);
@ -557,7 +553,6 @@ public class HColumnDescriptor implements WritableComparable {
// Comparable
/** {@inheritDoc} */
public int compareTo(Object o) {
HColumnDescriptor other = (HColumnDescriptor)o;
int result = Bytes.compareTo(this.name, other.getName());

View File

@ -29,8 +29,12 @@ public interface HConstants {
/** long constant for zero */
static final Long ZERO_L = Long.valueOf(0L);
//TODO: NINES is only used in HBaseAdmin and HConnectionManager. Move to client
// package and change visibility to default
static final String NINES = "99999999999999";
//TODO: ZEROS is only used in HConnectionManager and MetaScanner. Move to
// client package and change visibility to default
static final String ZEROES = "00000000000000";
// For migration
@ -224,11 +228,18 @@ public interface HConstants {
* Unlimited time-to-live.
*/
static final int FOREVER = -1;
//TODO: HBASE_CLIENT_RETRIES_NUMBER_KEY is only used by TestMigrate. Move it
// there.
public static final String HBASE_CLIENT_RETRIES_NUMBER_KEY =
"hbase.client.retries.number";
//TODO: DEFAULT_CLIENT_RETRIES is not referenced anywhere. Remove it.
public static final int DEFAULT_CLIENT_RETRIES = 5;
//TODO: although the following are referenced widely to format strings for
// the shell. They really aren't a part of the public API. It would be
// nice if we could put them somewhere where they did not need to be
// public. They could have package visibility
public static final String NAME = "NAME";
public static final String VERSIONS = "VERSIONS";
public static final String IN_MEMORY = "IN_MEMORY";

View File

@ -40,6 +40,7 @@ public class HMsg implements Writable {
* Message types sent between master and regionservers
*/
public static enum Type {
/** null message */
MSG_NONE,
// Message types sent from master to region server
@ -100,14 +101,19 @@ public class HMsg implements Writable {
private byte[] message = null;
// Some useful statics. Use these rather than create a new HMsg each time.
//TODO: move the following to HRegionServer
public static final HMsg REPORT_EXITING = new HMsg(Type.MSG_REPORT_EXITING);
public static final HMsg REPORT_QUIESCED = new HMsg(Type.MSG_REPORT_QUIESCED);
//TODO: Move to o.a.h.h.master
public static final HMsg REGIONSERVER_QUIESCE =
new HMsg(Type.MSG_REGIONSERVER_QUIESCE);
//TODO: Move to o.a.h.h.master
public static final HMsg REGIONSERVER_STOP =
new HMsg(Type.MSG_REGIONSERVER_STOP);
//TODO: Move to o.a.h.h.master
public static final HMsg CALL_SERVER_STARTUP =
new HMsg(Type.MSG_CALL_SERVER_STARTUP);
//TODO: Move to o.a.h.h.master
public static final HMsg [] EMPTY_HMSG_ARRAY = new HMsg[0];
@ -160,6 +166,7 @@ public class HMsg implements Writable {
return this.info;
}
/** @return the type of message */
public Type getType() {
return this.type;
}
@ -171,14 +178,12 @@ public class HMsg implements Writable {
public boolean isType(final HMsg.Type other) {
return this.type.equals(other);
}
/** @return the message type */
public byte[] getMessage() {
return this.message;
}
/**
* {@inheritDoc}
*/
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
@ -215,9 +220,6 @@ public class HMsg implements Writable {
// Writable
//////////////////////////////////////////////////////////////////////////////
/**
* {@inheritDoc}
*/
public void write(DataOutput out) throws IOException {
out.writeInt(this.type.ordinal());
this.info.write(out);
@ -229,9 +231,6 @@ public class HMsg implements Writable {
}
}
/**
* {@inheritDoc}
*/
public void readFields(DataInput in) throws IOException {
int ordinal = in.readInt();
this.type = HMsg.Type.values()[ordinal];

View File

@ -65,6 +65,7 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
protected HTableDescriptor tableDesc = null;
private int hashCode = -1;
//TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
public static final int NO_HASH = -1;
private volatile int encodedName = NO_HASH;
@ -308,9 +309,6 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
this.offLine = offLine;
}
/**
* {@inheritDoc}
*/
@Override
public String toString() {
return "REGION => {" + HConstants.NAME + " => '" +
@ -323,17 +321,11 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
" TABLE => {" + this.tableDesc.toString() + "}";
}
/**
* {@inheritDoc}
*/
@Override
public boolean equals(Object o) {
return this.compareTo(o) == 0;
}
/**
* {@inheritDoc}
*/
@Override
public int hashCode() {
return this.hashCode;
@ -349,7 +341,6 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
// Writable
//
/** {@inheritDoc} */
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
@ -363,7 +354,6 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
out.writeInt(hashCode);
}
/** {@inheritDoc} */
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
@ -382,9 +372,6 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
// Comparable
//
/**
* {@inheritDoc}
*/
public int compareTo(Object o) {
HRegionInfo other = (HRegionInfo) o;
if (other == null) {

View File

@ -39,26 +39,17 @@ public class HRegionLocation implements Comparable {
this.serverAddress = serverAddress;
}
/**
* {@inheritDoc}
*/
@Override
public String toString() {
return "address: " + this.serverAddress.toString() + ", regioninfo: " +
this.regionInfo;
}
/**
* {@inheritDoc}
*/
@Override
public boolean equals(Object o) {
return this.compareTo(o) == 0;
}
/**
* {@inheritDoc}
*/
@Override
public int hashCode() {
int result = this.regionInfo.hashCode();
@ -80,9 +71,6 @@ public class HRegionLocation implements Comparable {
// Comparable
//
/**
* {@inheritDoc}
*/
public int compareTo(Object o) {
HRegionLocation other = (HRegionLocation) o;
int result = this.regionInfo.compareTo(other.regionInfo);

View File

@ -109,25 +109,16 @@ public class HServerAddress implements WritableComparable {
return address;
}
/**
* {@inheritDoc}
*/
@Override
public String toString() {
return (stringValue == null ? "" : stringValue);
}
/**
* {@inheritDoc}
*/
@Override
public boolean equals(Object o) {
return this.compareTo(o) == 0;
}
/**
* {@inheritDoc}
*/
@Override
public int hashCode() {
int result = this.address.hashCode();
@ -139,9 +130,6 @@ public class HServerAddress implements WritableComparable {
// Writable
//
/**
* {@inheritDoc}
*/
public void readFields(DataInput in) throws IOException {
String bindAddress = in.readUTF();
int port = in.readInt();
@ -156,9 +144,6 @@ public class HServerAddress implements WritableComparable {
}
}
/**
* {@inheritDoc}
*/
public void write(DataOutput out) throws IOException {
if(address == null) {
out.writeUTF("");
@ -174,9 +159,6 @@ public class HServerAddress implements WritableComparable {
// Comparable
//
/**
* {@inheritDoc}
*/
public int compareTo(Object o) {
HServerAddress that = (HServerAddress)o;
// Addresses as Strings may not compare though address is for the one

View File

@ -115,20 +115,17 @@ public class HServerInfo implements WritableComparable {
this.startCode = startCode;
}
/** {@inheritDoc} */
@Override
public String toString() {
return "address: " + this.serverAddress + ", startcode: " + this.startCode
+ ", load: (" + this.load.toString() + ")";
}
/** {@inheritDoc} */
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
/** {@inheritDoc} */
@Override
public int hashCode() {
int result = this.serverAddress.hashCode();
@ -140,7 +137,6 @@ public class HServerInfo implements WritableComparable {
// Writable
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.serverAddress.readFields(in);
this.startCode = in.readLong();
@ -148,7 +144,6 @@ public class HServerInfo implements WritableComparable {
this.infoPort = in.readInt();
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
this.serverAddress.write(out);
out.writeLong(this.startCode);
@ -156,7 +151,6 @@ public class HServerInfo implements WritableComparable {
out.writeInt(this.infoPort);
}
/** {@inheritDoc} */
public int compareTo(Object o) {
HServerInfo that = (HServerInfo)o;
int result = getServerAddress().compareTo(that.getServerAddress());

View File

@ -74,7 +74,6 @@ public class HServerLoad implements WritableComparable {
return numberOfRegions;
}
/** {@inheritDoc} */
@Override
public String toString() {
return toString(1);
@ -89,13 +88,11 @@ public class HServerLoad implements WritableComparable {
return "requests: " + numberOfRequests/msgInterval + " regions: " + numberOfRegions;
}
/** {@inheritDoc} */
@Override
public boolean equals(Object o) {
return compareTo(o) == 0;
}
/** {@inheritDoc} */
@Override
public int hashCode() {
int result = Integer.valueOf(numberOfRequests).hashCode();
@ -137,13 +134,11 @@ public class HServerLoad implements WritableComparable {
// Writable
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
numberOfRequests = in.readInt();
numberOfRegions = in.readInt();
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
out.writeInt(numberOfRequests);
out.writeInt(numberOfRegions);
@ -151,7 +146,6 @@ public class HServerLoad implements WritableComparable {
// Comparable
/** {@inheritDoc} */
public int compareTo(Object o) {
HServerLoad other = (HServerLoad) o;
return this.getLoad() - other.getLoad();

View File

@ -315,20 +315,17 @@ public class HStoreKey implements WritableComparable {
delimiterIndex) == 0;
}
/** {@inheritDoc} */
@Override
public String toString() {
return Bytes.toString(this.row) + "/" + Bytes.toString(this.column) + "/" +
timestamp;
}
/** {@inheritDoc} */
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
/** {@inheritDoc} */
@Override
public int hashCode() {
int result = Bytes.hashCode(this.row);
@ -339,7 +336,6 @@ public class HStoreKey implements WritableComparable {
// Comparable
/** {@inheritDoc} */
public int compareTo(Object o) {
return compareTo(this.regionInfo, this, (HStoreKey)o);
}
@ -556,14 +552,12 @@ public class HStoreKey implements WritableComparable {
// Writable
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.row);
Bytes.writeByteArray(out, this.column);
out.writeLong(timestamp);
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.row = Bytes.readByteArray(in);
this.column = Bytes.readByteArray(in);
@ -576,12 +570,15 @@ public class HStoreKey implements WritableComparable {
public static class HStoreKeyWritableComparator extends WritableComparator {
private final HRegionInfo hri;
/** @param hri */
public HStoreKeyWritableComparator(final HRegionInfo hri) {
super(HStoreKey.class);
this.hri = hri;
}
public int compare(final WritableComparable left, final WritableComparable right) {
@Override
public int compare(final WritableComparable left,
final WritableComparable right) {
return compareTo(this.hri, (HStoreKey)left, (HStoreKey)right);
}
}

View File

@ -65,6 +65,8 @@ public class HTableDescriptor implements WritableComparable {
// Table metadata
protected Map<ImmutableBytesWritable,ImmutableBytesWritable> values =
new HashMap<ImmutableBytesWritable,ImmutableBytesWritable>();
//TODO: Why can't the following be private? They are only used within this class.
public static final String FAMILIES = "FAMILIES";
@ -79,6 +81,8 @@ public class HTableDescriptor implements WritableComparable {
public static final boolean DEFAULT_READONLY = false;
public static final int DEFAULT_MEMCACHE_FLUSH_SIZE = 1024*1024*64;
// End TODO:
// Key is hash of the family name.
private final Map<Integer, HColumnDescriptor> families =
@ -434,13 +438,11 @@ public class HTableDescriptor implements WritableComparable {
return s.toString();
}
/** {@inheritDoc} */
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
/** {@inheritDoc} */
@Override
public int hashCode() {
int result = Bytes.hashCode(this.name);
@ -456,7 +458,6 @@ public class HTableDescriptor implements WritableComparable {
// Writable
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
int version = in.readInt();
if (version < 3)
@ -484,7 +485,6 @@ public class HTableDescriptor implements WritableComparable {
}
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
out.writeInt(TABLE_DESCRIPTOR_VERSION);
Bytes.writeByteArray(out, name);
@ -506,7 +506,6 @@ public class HTableDescriptor implements WritableComparable {
// Comparable
/** {@inheritDoc} */
public int compareTo(Object o) {
HTableDescriptor other = (HTableDescriptor) o;
int result = Bytes.compareTo(this.name, other.name);

View File

@ -19,11 +19,19 @@
*/
package org.apache.hadoop.hbase;
/**
* Reports a problem with a lease
*/
@SuppressWarnings("serial")
public class LeaseException extends DoNotRetryIOException {
/** default constructor */
public LeaseException() {
super();
}
/**
* @param message
*/
public LeaseException(String message) {
super(message);
}

View File

@ -66,7 +66,6 @@ public class Leases extends Thread {
this.leaseCheckFrequency = leaseCheckFrequency;
}
/** {@inheritDoc} */
@Override
public void run() {
while (!stopRequested || (stopRequested && leaseQueue.size() > 0) ) {
@ -149,13 +148,18 @@ public class Leases extends Thread {
* Thrown if we are asked create a lease but lease on passed name already
* exists.
*/
@SuppressWarnings("serial")
public static class LeaseStillHeldException extends IOException {
private final String leaseName;
/**
* @param name
*/
public LeaseStillHeldException(final String name) {
this.leaseName = name;
}
/** @return name of lease */
public String getName() {
return this.leaseName;
}
@ -218,25 +222,21 @@ public class Leases extends Thread {
return this.listener;
}
/** {@inheritDoc} */
@Override
public boolean equals(Object obj) {
return this.hashCode() == ((Lease) obj).hashCode();
}
/** {@inheritDoc} */
@Override
public int hashCode() {
return this.leaseName.hashCode();
}
/** {@inheritDoc} */
public long getDelay(TimeUnit unit) {
return unit.convert(this.expirationTime - System.currentTimeMillis(),
TimeUnit.MILLISECONDS);
}
/** {@inheritDoc} */
public int compareTo(Delayed o) {
long delta = this.getDelay(TimeUnit.MILLISECONDS) -
o.getDelay(TimeUnit.MILLISECONDS);

View File

@ -84,6 +84,7 @@ public class LocalHBaseCluster implements HConstants {
* @param noRegionServers Count of regionservers to start.
* @throws IOException
*/
@SuppressWarnings("unchecked")
public LocalHBaseCluster(final HBaseConfiguration conf,
final int noRegionServers)
throws IOException {
@ -284,6 +285,10 @@ public class LocalHBaseCluster implements HConstants {
" " + this.regionThreads.size() + " region server(s)");
}
/**
* @param t
* @throws InterruptedException
*/
public void threadDumpingJoin(final Thread t) throws InterruptedException {
if (t == null) {
return;

View File

@ -56,6 +56,8 @@ public class RegionHistorian implements HConstants {
private static SimpleDateFormat dateFormat = new SimpleDateFormat(
"EEE, d MMM yyyy HH:mm:ss");
//TODO: Why is this public? Appears to only apply internally.
public static enum HistorianColumnKey {
REGION_CREATION ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"creation")),
REGION_OPEN ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"open")),
@ -132,6 +134,7 @@ public class RegionHistorian implements HConstants {
/**
* Method to add a creation event to the row in the .META table
* @param info
* @param serverName
*/
public void addRegionAssignment(HRegionInfo info, String serverName) {
add(HistorianColumnKey.REGION_ASSIGNMENT.key, "Region assigned to server "
@ -175,6 +178,7 @@ public class RegionHistorian implements HConstants {
/**
* Method to add a compaction event to the row in the .META table
* @param info
* @param timeTaken
*/
public void addRegionCompaction(final HRegionInfo info,
final String timeTaken) {
@ -191,6 +195,7 @@ public class RegionHistorian implements HConstants {
/**
* Method to add a flush event to the row in the .META table
* @param info
* @param timeTaken
*/
public void addRegionFlush(HRegionInfo info,
@SuppressWarnings("unused") String timeTaken) {
@ -249,6 +254,11 @@ public class RegionHistorian implements HConstants {
private String description;
/**
* @param timestamp
* @param event
* @param description
*/
public RegionHistoryInformation(long timestamp, String event,
String description) {
this.timestamp = timestamp;
@ -256,21 +266,21 @@ public class RegionHistorian implements HConstants {
this.description = description;
}
/**
* Returns the inverse value of Long.compareTo
*/
public int compareTo(RegionHistoryInformation otherInfo) {
return -1 * Long.valueOf(timestamp).compareTo(otherInfo.getTimestamp());
}
/** @return the event */
public String getEvent() {
return event;
}
/** @return the description */
public String getDescription() {
return description;
}
/** @return the timestamp */
public long getTimestamp() {
return timestamp;
}

View File

@ -173,7 +173,6 @@ public class HConnectionManager implements HConstants {
return this.pause * HConstants.RETRY_BACKOFF[tries];
}
/** {@inheritDoc} */
public HMasterInterface getMaster() throws MasterNotRunningException {
HServerAddress masterLocation = null;
synchronized (this.masterLock) {
@ -224,7 +223,6 @@ public class HConnectionManager implements HConstants {
return this.master;
}
/** {@inheritDoc} */
public boolean isMasterRunning() {
if (this.master == null) {
try {
@ -237,7 +235,6 @@ public class HConnectionManager implements HConstants {
return true;
}
/** {@inheritDoc} */
public boolean tableExists(final byte [] tableName)
throws MasterNotRunningException {
getMaster();
@ -271,7 +268,6 @@ public class HConnectionManager implements HConstants {
Bytes.equals(n, META_TABLE_NAME);
}
/** {@inheritDoc} */
public HRegionLocation getRegionLocation(final byte [] name,
final byte [] row, boolean reload)
throws IOException {
@ -279,7 +275,6 @@ public class HConnectionManager implements HConstants {
return reload? relocateRegion(name, row): locateRegion(name, row);
}
/** {@inheritDoc} */
public HTableDescriptor[] listTables() throws IOException {
getMaster();
final HashSet<HTableDescriptor> uniqueTables =
@ -287,7 +282,6 @@ public class HConnectionManager implements HConstants {
MetaScannerVisitor visitor = new MetaScannerVisitor() {
/** {@inheritDoc} */
public boolean processRow(RowResult rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfo(
rowResult.get(COL_REGIONINFO));
@ -305,7 +299,6 @@ public class HConnectionManager implements HConstants {
return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
}
/** {@inheritDoc} */
public boolean isTableEnabled(byte[] tableName) throws IOException {
if (!tableExists(tableName)) {
throw new TableNotFoundException(Bytes.toString(tableName));
@ -371,6 +364,7 @@ public class HConnectionManager implements HConstants {
implements MetaScanner.MetaScannerVisitor {
byte[] tableName;
HTableDescriptor result;
//TODO: change visibility to protected
public HTableDescriptorFinder(byte[] tableName) {
this.tableName = tableName;
}
@ -389,7 +383,6 @@ public class HConnectionManager implements HConstants {
}
}
/** {@inheritDoc} */
public HTableDescriptor getHTableDescriptor(final byte[] tableName)
throws IOException {
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
@ -407,7 +400,6 @@ public class HConnectionManager implements HConstants {
return result;
}
/** {@inheritDoc} */
public HRegionLocation locateRegion(final byte [] tableName,
final byte [] row)
throws IOException{
@ -415,7 +407,6 @@ public class HConnectionManager implements HConstants {
return locateRegion(tableName, row, true);
}
/** {@inheritDoc} */
public HRegionLocation relocateRegion(final byte [] tableName,
final byte [] row)
throws IOException{
@ -724,7 +715,6 @@ public class HConnectionManager implements HConstants {
tableLocations.put(startKey, location);
}
/** {@inheritDoc} */
public HRegionInterface getHRegionConnection(HServerAddress regionServer)
throws IOException {
getMaster();
@ -850,7 +840,6 @@ public class HConnectionManager implements HConstants {
HRegionInfo.ROOT_REGIONINFO, rootRegionAddress);
}
/** {@inheritDoc} */
public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
throws IOException, RuntimeException {
getMaster();

View File

@ -1059,7 +1059,9 @@ public class HTable {
filter.validate(columns);
}
}
//TODO: change visibility to protected
public void initialize() throws IOException {
nextScanner();
}
@ -1144,7 +1146,6 @@ public class HTable {
return this.filter.filterAllRemaining();
}
/** {@inheritDoc} */
public RowResult next() throws IOException {
if (this.closed) {
return null;
@ -1160,9 +1161,6 @@ public class HTable {
return null;
}
/**
* {@inheritDoc}
*/
public void close() {
if (callable != null) {
callable.setClose();
@ -1179,7 +1177,6 @@ public class HTable {
closed = true;
}
/** {@inheritDoc} */
public Iterator<RowResult> iterator() {
return new Iterator<RowResult>() {
// The next RowResult, possibly pre-read

View File

@ -7,7 +7,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Bytes; //TODO: remove
/**
* Scanner class that contains the <code>.META.</code> table scanning logic

View File

@ -67,7 +67,6 @@ public class ScannerCallable extends ServerCallable<RowResult> {
}
}
/** {@inheritDoc} */
public RowResult call() throws IOException {
if (scannerId != -1L && closed) {
server.close(scannerId);

View File

@ -2,53 +2,68 @@ package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.HColumnDescriptor;
/**
* Immutable HColumnDescriptor
*/
public class UnmodifyableHColumnDescriptor extends HColumnDescriptor {
/**
* @param desc
*/
public UnmodifyableHColumnDescriptor (final HColumnDescriptor desc) {
super(desc);
}
@Override
@SuppressWarnings("unused")
public void setValue(byte[] key, byte[] value) {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setValue(String key, String value) {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setMaxVersions(int maxVersions) {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setInMemory(boolean inMemory) {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setBlockCacheEnabled(boolean blockCacheEnabled) {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setMaxValueLength(int maxLength) {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setTimeToLive(int timeToLive) {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setCompressionType(CompressionType type) {
throw new UnsupportedOperationException("HColumnDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setMapFileIndexInterval(int interval) {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
* Read-only table descriptor.
*/
public class UnmodifyableHTableDescriptor extends HTableDescriptor {
/** Default constructor */
public UnmodifyableHTableDescriptor() {
super();
}
@ -73,31 +74,37 @@ public class UnmodifyableHTableDescriptor extends HTableDescriptor {
}
@Override
@SuppressWarnings("unused")
public void setInMemory(boolean inMemory) {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setReadOnly(boolean readOnly) {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setValue(byte[] key, byte[] value) {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setValue(String key, String value) {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setMaxFileSize(long maxFileSize) {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}
@Override
@SuppressWarnings("unused")
public void setMemcacheFlushSize(int memcacheFlushSize) {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}

View File

@ -22,20 +22,32 @@ package org.apache.hadoop.hbase.client.transactional;
/** Thrown when a transaction cannot be committed.
*
*/
@SuppressWarnings("serial")
public class CommitUnsuccessfulException extends Exception {
/** Default Constructor */
public CommitUnsuccessfulException() {
super();
}
/**
* @param arg0 message
* @param arg1 cause
*/
public CommitUnsuccessfulException(String arg0, Throwable arg1) {
super(arg0, arg1);
}
/**
* @param arg0 message
*/
public CommitUnsuccessfulException(String arg0) {
super(arg0);
}
/**
* @param arg0 cause
*/
public CommitUnsuccessfulException(Throwable arg0) {
super(arg0);
}

View File

@ -33,6 +33,11 @@ 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();
@ -48,7 +53,7 @@ public class LocalTransactionLogger implements TransactionLogger {
// Enforce singlton
}
// Gives back random longs to minimize possibility of collision
/** @return random longs to minimize possibility of collision */
public long createNewTransactionLog() {
long id = random.nextLong();
transactionIdToStatusMap.put(id, TransactionStatus.PENDING);

View File

@ -26,8 +26,14 @@ package org.apache.hadoop.hbase.client.transactional;
*/
public interface TransactionLogger {
/** Transaction status values */
enum TransactionStatus {
PENDING, COMMITTED, ABORTED
/** Transaction is pending */
PENDING,
/** Transaction was committed */
COMMITTED,
/** Transaction was aborted */
ABORTED
}
/**
@ -38,8 +44,16 @@ public interface TransactionLogger {
*/
long createNewTransactionLog();
/**
* @param transactionId
* @return transaction status
*/
TransactionStatus getStatusForTransaction(long transactionId);
/**
* @param transactionId
* @param status
*/
void setStatusForTransaction(long transactionId, TransactionStatus status);
}

View File

@ -40,10 +40,17 @@ public class TransactionManager {
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;
@ -66,6 +73,7 @@ public class TransactionManager {
*
* @param transactionState
* @throws IOException
* @throws CommitUnsuccessfulException
*/
public void tryCommit(final TransactionState transactionState)
throws CommitUnsuccessfulException, IOException {

View File

@ -40,11 +40,21 @@ import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
*/
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);
@ -66,6 +76,12 @@ public class TransactionalTable extends HTable {
}
}
/**
* @param connection
* @param tableName
* @param row
* @param transactionState
*/
public TransactionalServerCallable(final HConnection connection,
final byte[] tableName, final byte[] row,
final TransactionState transactionState) {

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
/**
* Thrown if a region server is passed an unknown transaction id
*/
@SuppressWarnings("serial")
public class UnknownTransactionException extends DoNotRetryIOException {
/** constructor */

View File

@ -40,9 +40,20 @@ import org.apache.hadoop.io.ObjectWritable;
*/
public class ColumnValueFilter implements RowFilterInterface {
/** Comparison operator. */
/** Comparison operators. */
public enum CompareOp {
LESS, LESS_OR_EQUAL, EQUAL, NOT_EQUAL, GREATER_OR_EQUAL, GREATER;
/** less than */
LESS,
/** less than or equal to */
LESS_OR_EQUAL,
/** equals */
EQUAL,
/** not equal */
NOT_EQUAL,
/** greater than or equal to */
GREATER_OR_EQUAL,
/** greater than */
GREATER;
}
private byte[] columnName;
@ -82,12 +93,10 @@ public class ColumnValueFilter implements RowFilterInterface {
this.comparator = comparator;
}
/** {@inheritDoc} */
public boolean filterRowKey(@SuppressWarnings("unused") final byte[] rowKey) {
return false;
}
/** {@inheritDoc} */
public boolean filterColumn(@SuppressWarnings("unused") final byte[] rowKey,
final byte[] colKey, final byte[] data) {
if (!Arrays.equals(colKey, columnName)) {
@ -119,12 +128,10 @@ public class ColumnValueFilter implements RowFilterInterface {
}
}
/** {@inheritDoc} */
public boolean filterAllRemaining() {
return false;
}
/** {@inheritDoc} */
public boolean filterRow(final SortedMap<byte[], Cell> columns) {
// Don't let rows through if they don't have the column we are checking
return !columns.containsKey(columnName);
@ -141,28 +148,23 @@ public class ColumnValueFilter implements RowFilterInterface {
return b1.length - b2.length;
}
/** {@inheritDoc} */
public boolean processAlways() {
return false;
}
/** {@inheritDoc} */
public void reset() {
// Nothing.
}
/** {@inheritDoc} */
public void rowProcessed(@SuppressWarnings("unused") final boolean filtered,
@SuppressWarnings("unused") final byte[] key) {
// Nothing
}
/** {@inheritDoc} */
public void validate(@SuppressWarnings("unused") final byte[][] columns) {
// Nothing
}
/** {@inheritDoc} */
public void readFields(final DataInput in) throws IOException {
int valueLen = in.readInt();
if (valueLen > 0) {
@ -175,7 +177,6 @@ public class ColumnValueFilter implements RowFilterInterface {
new HBaseConfiguration());
}
/** {@inheritDoc} */
public void write(final DataOutput out) throws IOException {
if (value == null) {
out.writeInt(0);

View File

@ -41,7 +41,6 @@ public class InclusiveStopRowFilter extends StopRowFilter{
super(stopRowKey);
}
/** {@inheritDoc} */
@Override
public boolean filterRowKey(final byte [] rowKey) {
if (rowKey == null) {

View File

@ -61,23 +61,14 @@ public class PageRowFilter implements RowFilterInterface {
this.pageSize = pageSize;
}
/**
*
* {@inheritDoc}
*/
public void validate(@SuppressWarnings("unused") final byte [][] columns) {
// Doesn't filter columns
}
/**
*
* {@inheritDoc}
*/
public void reset() {
rowsAccepted = 0;
}
/** {@inheritDoc} */
public void rowProcessed(boolean filtered,
@SuppressWarnings("unused") byte [] rowKey) {
if (!filtered) {
@ -85,61 +76,33 @@ public class PageRowFilter implements RowFilterInterface {
}
}
/**
*
* {@inheritDoc}
*/
public boolean processAlways() {
return false;
}
/**
*
* {@inheritDoc}
*/
public boolean filterAllRemaining() {
return this.rowsAccepted > this.pageSize;
}
/**
*
* {@inheritDoc}
*/
public boolean filterRowKey(@SuppressWarnings("unused") final byte [] r) {
return filterAllRemaining();
}
/**
*
* {@inheritDoc}
*/
public boolean filterColumn(@SuppressWarnings("unused") final byte [] rowKey,
@SuppressWarnings("unused") final byte [] colKey,
@SuppressWarnings("unused") final byte[] data) {
return filterAllRemaining();
}
/**
*
* {@inheritDoc}
*/
public boolean filterRow(@SuppressWarnings("unused")
final SortedMap<byte [], Cell> columns) {
return filterAllRemaining();
}
/**
*
* {@inheritDoc}
*/
public void readFields(final DataInput in) throws IOException {
this.pageSize = in.readLong();
}
/**
*
* {@inheritDoc}
*/
public void write(final DataOutput out) throws IOException {
out.writeLong(pageSize);
}

View File

@ -86,13 +86,11 @@ public class RegExpRowFilter implements RowFilterInterface {
this.setColumnFilters(columnFilter);
}
/** {@inheritDoc} */
@SuppressWarnings("unused")
public void rowProcessed(boolean filtered, byte [] rowKey) {
//doesn't care
}
/** {@inheritDoc} */
public boolean processAlways() {
return false;
}
@ -134,36 +132,20 @@ public class RegExpRowFilter implements RowFilterInterface {
}
}
/**
*
* {@inheritDoc}
*/
public void reset() {
// Nothing to reset
}
/**
*
* {@inheritDoc}
*/
public boolean filterAllRemaining() {
return false;
}
/**
*
* {@inheritDoc}
*/
public boolean filterRowKey(final byte [] rowKey) {
return (filtersByRowKey() && rowKey != null)?
!getRowKeyPattern().matcher(Bytes.toString(rowKey)).matches():
false;
}
/**
*
* {@inheritDoc}
*/
public boolean filterColumn(final byte [] rowKey, final byte [] colKey,
final byte[] data) {
if (filterRowKey(rowKey)) {
@ -183,10 +165,6 @@ public class RegExpRowFilter implements RowFilterInterface {
return false;
}
/**
*
* {@inheritDoc}
*/
public boolean filterRow(final SortedMap<byte [], Cell> columns) {
for (Entry<byte [], Cell> col : columns.entrySet()) {
if (nullColumns.contains(col.getKey())
@ -225,10 +203,6 @@ public class RegExpRowFilter implements RowFilterInterface {
return rowKeyPattern;
}
/**
*
* {@inheritDoc}
*/
public void readFields(final DataInput in) throws IOException {
boolean hasRowKeyPattern = in.readBoolean();
if (hasRowKeyPattern) {
@ -255,10 +229,6 @@ public class RegExpRowFilter implements RowFilterInterface {
}
}
/**
*
* {@inheritDoc}
*/
public void validate(final byte [][] columns) {
Set<byte []> invalids = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
for (byte [] colKey : getFilterColumns()) {
@ -289,10 +259,6 @@ public class RegExpRowFilter implements RowFilterInterface {
return cols;
}
/**
*
* {@inheritDoc}
*/
public void write(final DataOutput out) throws IOException {
if (!filtersByRowKey()) {
out.writeBoolean(false);

View File

@ -80,28 +80,24 @@ public class RowFilterSet implements RowFilterInterface {
this.operator = operator;
}
/** {@inheritDoc} */
public void validate(final byte [][] columns) {
for (RowFilterInterface filter : filters) {
filter.validate(columns);
}
}
/** {@inheritDoc} */
public void reset() {
for (RowFilterInterface filter : filters) {
filter.reset();
}
}
/** {@inheritDoc} */
public void rowProcessed(boolean filtered, byte [] rowKey) {
for (RowFilterInterface filter : filters) {
filter.rowProcessed(filtered, rowKey);
}
}
/** {@inheritDoc} */
public boolean processAlways() {
for (RowFilterInterface filter : filters) {
if (filter.processAlways()) {
@ -111,7 +107,6 @@ public class RowFilterSet implements RowFilterInterface {
return false;
}
/** {@inheritDoc} */
public boolean filterAllRemaining() {
boolean result = operator == Operator.MUST_PASS_ONE;
for (RowFilterInterface filter : filters) {
@ -128,7 +123,6 @@ public class RowFilterSet implements RowFilterInterface {
return result;
}
/** {@inheritDoc} */
public boolean filterRowKey(final byte [] rowKey) {
boolean resultFound = false;
boolean result = operator == Operator.MUST_PASS_ONE;
@ -152,7 +146,6 @@ public class RowFilterSet implements RowFilterInterface {
return result;
}
/** {@inheritDoc} */
public boolean filterColumn(final byte [] rowKey, final byte [] colKey,
final byte[] data) {
boolean resultFound = false;
@ -179,7 +172,6 @@ public class RowFilterSet implements RowFilterInterface {
return result;
}
/** {@inheritDoc} */
public boolean filterRow(final SortedMap<byte [], Cell> columns) {
boolean resultFound = false;
boolean result = operator == Operator.MUST_PASS_ONE;
@ -203,7 +195,6 @@ public class RowFilterSet implements RowFilterInterface {
return result;
}
/** {@inheritDoc} */
public void readFields(final DataInput in) throws IOException {
Configuration conf = new HBaseConfiguration();
byte opByte = in.readByte();
@ -219,7 +210,6 @@ public class RowFilterSet implements RowFilterInterface {
}
}
/** {@inheritDoc} */
public void write(final DataOutput out) throws IOException {
Configuration conf = new HBaseConfiguration();
out.writeByte(operator.ordinal());

View File

@ -61,39 +61,27 @@ public class StopRowFilter implements RowFilterInterface {
return this.stopRowKey;
}
/**
*
* {@inheritDoc}
*/
public void validate(@SuppressWarnings("unused") final byte [][] columns) {
// Doesn't filter columns
}
/**
*
* {@inheritDoc}
*/
public void reset() {
// Nothing to reset
}
/** {@inheritDoc} */
@SuppressWarnings("unused")
public void rowProcessed(boolean filtered, byte [] rowKey) {
// Doesn't care
}
/** {@inheritDoc} */
public boolean processAlways() {
return false;
}
/** {@inheritDoc} */
public boolean filterAllRemaining() {
return false;
}
/** {@inheritDoc} */
public boolean filterRowKey(final byte [] rowKey) {
if (rowKey == null) {
if (this.stopRowKey == null) {
@ -105,8 +93,6 @@ public class StopRowFilter implements RowFilterInterface {
}
/**
* {@inheritDoc}
*
* Because StopRowFilter does not examine column information, this method
* defaults to calling the rowKey-only version of filter.
*/
@ -116,24 +102,19 @@ public class StopRowFilter implements RowFilterInterface {
return filterRowKey(rowKey);
}
/** {@inheritDoc}
*
/**
* Because StopRowFilter does not examine column information, this method
* defaults to calling filterAllRemaining().
*
* @param columns
*/
public boolean filterRow(@SuppressWarnings("unused")
final SortedMap<byte [], Cell> columns) {
return filterAllRemaining();
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.stopRowKey = Bytes.readByteArray(in);
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.stopRowKey);
}

View File

@ -63,13 +63,11 @@ public class WhileMatchRowFilter implements RowFilterInterface {
return this.filter;
}
/** {@inheritDoc} */
public void reset() {
this.filterAllRemaining = false;
this.filter.reset();
}
/** {@inheritDoc} */
public boolean processAlways() {
return true;
}
@ -85,20 +83,17 @@ public class WhileMatchRowFilter implements RowFilterInterface {
return this.filterAllRemaining || this.filter.filterAllRemaining();
}
/** {@inheritDoc} */
public boolean filterRowKey(final byte [] rowKey) {
changeFAR(this.filter.filterRowKey(rowKey));
return filterAllRemaining();
}
/** {@inheritDoc} */
public boolean filterColumn(final byte [] rowKey, final byte [] colKey,
final byte[] data) {
changeFAR(this.filter.filterColumn(rowKey, colKey, data));
return filterAllRemaining();
}
/** {@inheritDoc} */
public boolean filterRow(final SortedMap<byte [], Cell> columns) {
changeFAR(this.filter.filterRow(columns));
return filterAllRemaining();
@ -114,17 +109,14 @@ public class WhileMatchRowFilter implements RowFilterInterface {
this.filterAllRemaining = this.filterAllRemaining || value;
}
/** {@inheritDoc} */
public void rowProcessed(boolean filtered, byte [] rowKey) {
this.filter.rowProcessed(filtered, rowKey);
}
/** {@inheritDoc} */
public void validate(final byte [][] columns) {
this.filter.validate(columns);
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
String className = in.readUTF();
@ -144,7 +136,6 @@ public class WhileMatchRowFilter implements RowFilterInterface {
}
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
out.writeUTF(this.filter.getClass().getName());
this.filter.write(out);

View File

@ -78,6 +78,7 @@ public class BatchUpdate implements Writable, Iterable<BatchOperation> {
* Initialize a BatchUpdate operation on a row with a specific timestamp.
*
* @param row
* @param timestamp
*/
public BatchUpdate(final String row, long timestamp){
this(Bytes.toBytes(row), timestamp);
@ -87,6 +88,7 @@ public class BatchUpdate implements Writable, Iterable<BatchOperation> {
* Initialize a BatchUpdate operation on a row with a specific timestamp.
*
* @param row
* @param timestamp
*/
public BatchUpdate(final byte [] row, long timestamp){
this.row = row;
@ -100,7 +102,7 @@ public class BatchUpdate implements Writable, Iterable<BatchOperation> {
}
/**
* Return the timestamp this BatchUpdate will be committed with.
* @return the timestamp this BatchUpdate will be committed with.
*/
public long getTimestamp() {
return timestamp;
@ -108,6 +110,8 @@ public class BatchUpdate implements Writable, Iterable<BatchOperation> {
/**
* Set this BatchUpdate's timestamp.
*
* @param timestamp
*/
public void setTimestamp(long timestamp) {
this.timestamp = timestamp;

View File

@ -59,7 +59,7 @@ public class BlockFSInputStream extends FSInputStream {
* @param fileLength
* @param blockSize the size of each block in bytes.
*/
@SuppressWarnings("unchecked")
@SuppressWarnings({"unchecked", "serial"})
public BlockFSInputStream(InputStream in, long fileLength, int blockSize) {
this.in = in;
if (!(in instanceof Seekable) || !(in instanceof PositionedReadable)) {
@ -89,12 +89,12 @@ public class BlockFSInputStream extends FSInputStream {
}
@Override
public synchronized long getPos() throws IOException {
public synchronized long getPos() {
return pos;
}
@Override
public synchronized int available() throws IOException {
public synchronized int available() {
return (int) (fileLength - pos);
}
@ -108,6 +108,7 @@ public class BlockFSInputStream extends FSInputStream {
}
@Override
@SuppressWarnings("unused")
public synchronized boolean seekToNewSource(long targetPos)
throws IOException {
return false;
@ -194,6 +195,7 @@ public class BlockFSInputStream extends FSInputStream {
}
@Override
@SuppressWarnings("unused")
public void mark(int readLimit) {
// Do nothing
}

View File

@ -106,7 +106,6 @@ public class Cell implements Writable, Iterable<Cell> {
return timestamps[0];
}
/** {@inheritDoc} */
@Override
public String toString() {
if (this.values.length == 1) {
@ -132,7 +131,6 @@ public class Cell implements Writable, Iterable<Cell> {
// Writable
//
/** {@inheritDoc} */
public void readFields(final DataInput in) throws IOException {
int nvalues = in.readInt();
this.timestamps = new long[nvalues];
@ -145,7 +143,6 @@ public class Cell implements Writable, Iterable<Cell> {
}
}
/** {@inheritDoc} */
public void write(final DataOutput out) throws IOException {
out.writeInt(this.values.length);
for (int i = 0; i < this.timestamps.length; i++) {
@ -160,7 +157,6 @@ public class Cell implements Writable, Iterable<Cell> {
// Iterable
//
/** {@inheritDoc} */
public Iterator<Cell> iterator() {
return new CellIterator();
}
@ -169,18 +165,15 @@ public class Cell implements Writable, Iterable<Cell> {
CellIterator() {
}
/** {@inheritDoc} */
public boolean hasNext() {
return currentValue < values.length;
}
/** {@inheritDoc} */
public Cell next() {
currentValue += 1;
return new Cell(values[currentValue], timestamps[currentValue]);
}
/** {@inheritDoc} */
public void remove() throws UnsupportedOperationException {
throw new UnsupportedOperationException("remove is not supported");
}

View File

@ -45,8 +45,9 @@ import org.apache.hadoop.util.ReflectionUtils;
* if passed a Writable it has not already been told about. Its also been
* primed with hbase Writables. Keys are always byte arrays. Thats other
* difference from MapWritable.
* TODO: Have generics enforce V is a subclass of Writable and K is a byte []
* only.
*
* @param <K> key
* @param <V> value
*/
public class HbaseMapWritable <K, V>
implements SortedMap<byte [], V>, Writable, Configurable {
@ -89,47 +90,38 @@ implements SortedMap<byte [], V>, Writable, Configurable {
this.conf.set(conf);
}
/** {@inheritDoc} */
public void clear() {
instance.clear();
}
/** {@inheritDoc} */
public boolean containsKey(Object key) {
return instance.containsKey(key);
}
/** {@inheritDoc} */
public boolean containsValue(Object value) {
return instance.containsValue(value);
}
/** {@inheritDoc} */
public Set<Entry<byte [], V>> entrySet() {
return instance.entrySet();
}
/** {@inheritDoc} */
public V get(Object key) {
return instance.get(key);
}
/** {@inheritDoc} */
public boolean isEmpty() {
return instance.isEmpty();
}
/** {@inheritDoc} */
public Set<byte []> keySet() {
return instance.keySet();
}
/** {@inheritDoc} */
public int size() {
return instance.size();
}
/** {@inheritDoc} */
public Collection<V> values() {
return instance.values();
}
@ -193,7 +185,6 @@ implements SortedMap<byte [], V>, Writable, Configurable {
return this.instance.toString();
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
// Write out the number of entries in the map
out.writeInt(this.instance.size());
@ -206,7 +197,7 @@ implements SortedMap<byte [], V>, Writable, Configurable {
}
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
public void readFields(DataInput in) throws IOException {
// First clear the map. Otherwise we will just accumulate
// entries every time this method is called.

View File

@ -132,31 +132,43 @@ public class HbaseObjectWritable implements Writable, Configurable {
private Object instance;
private Configuration conf;
/** default constructor for writable */
public HbaseObjectWritable() {
super();
}
/**
* @param instance
*/
public HbaseObjectWritable(Object instance) {
set(instance);
}
/**
* @param declaredClass
* @param instance
*/
public HbaseObjectWritable(Class<?> declaredClass, Object instance) {
this.declaredClass = declaredClass;
this.instance = instance;
}
/** Return the instance, or null if none. */
/** @return the instance, or null if none. */
public Object get() { return instance; }
/** Return the class this is meant to be. */
/** @return the class this is meant to be. */
public Class<?> getDeclaredClass() { return declaredClass; }
/** Reset the instance. */
/**
* Reset the instance.
* @param instance
*/
public void set(Object instance) {
this.declaredClass = instance.getClass();
this.instance = instance;
}
@Override
public String toString() {
return "OW[class=" + declaredClass + ",value=" + instance + "]";
}
@ -172,14 +184,18 @@ public class HbaseObjectWritable implements Writable, Configurable {
private static class NullInstance extends Configured implements Writable {
Class<?> declaredClass;
/** default constructor for writable */
public NullInstance() { super(null); }
/**
* @param declaredClass
* @param conf
*/
public NullInstance(Class<?> declaredClass, Configuration conf) {
super(conf);
this.declaredClass = declaredClass;
}
@SuppressWarnings("boxing")
public void readFields(DataInput in) throws IOException {
this.declaredClass = CODE_TO_CLASS.get(in.readByte());
}
@ -195,7 +211,6 @@ public class HbaseObjectWritable implements Writable, Configurable {
* @param c
* @throws IOException
*/
@SuppressWarnings("boxing")
static void writeClassCode(final DataOutput out, final Class<?> c)
throws IOException {
Byte code = CLASS_TO_CODE.get(c);
@ -206,9 +221,16 @@ public class HbaseObjectWritable implements Writable, Configurable {
out.writeByte(code);
}
/** Write a {@link Writable}, {@link String}, primitive type, or an array of
* the preceding. */
@SuppressWarnings({ "boxing", "unchecked" })
/**
* Write a {@link Writable}, {@link String}, primitive type, or an array of
* the preceding.
* @param out
* @param instance
* @param declaredClass
* @param conf
* @throws IOException
*/
@SuppressWarnings("unchecked")
public static void writeObject(DataOutput out, Object instance,
Class declaredClass,
Configuration conf)
@ -273,16 +295,29 @@ public class HbaseObjectWritable implements Writable, Configurable {
}
/** Read a {@link Writable}, {@link String}, primitive type, or an array of
* the preceding. */
/**
* Read a {@link Writable}, {@link String}, primitive type, or an array of
* the preceding.
* @param in
* @param conf
* @return the object
* @throws IOException
*/
public static Object readObject(DataInput in, Configuration conf)
throws IOException {
return readObject(in, null, conf);
}
/** Read a {@link Writable}, {@link String}, primitive type, or an array of
* the preceding. */
@SuppressWarnings({ "unchecked", "boxing" })
/**
* Read a {@link Writable}, {@link String}, primitive type, or an array of
* the preceding.
* @param in
* @param objectWritable
* @param conf
* @return the object
* @throws IOException
*/
@SuppressWarnings("unchecked")
public static Object readObject(DataInput in,
HbaseObjectWritable objectWritable, Configuration conf)
throws IOException {
@ -353,7 +388,6 @@ public class HbaseObjectWritable implements Writable, Configurable {
return instance;
}
@SuppressWarnings("boxing")
private static void addToMap(final Class<?> clazz, final byte code) {
CLASS_TO_CODE.put(clazz, code);
CODE_TO_CLASS.put(code, clazz);

View File

@ -106,13 +106,11 @@ public class ImmutableBytesWritable implements WritableComparable {
}
/** {@inheritDoc} */
public void readFields(final DataInput in) throws IOException {
this.bytes = new byte[in.readInt()];
in.readFully(this.bytes, 0, this.bytes.length);
}
/** {@inheritDoc} */
public void write(final DataOutput out) throws IOException {
out.writeInt(this.bytes.length);
out.write(this.bytes, 0, this.bytes.length);
@ -120,7 +118,6 @@ public class ImmutableBytesWritable implements WritableComparable {
// Below methods copied from BytesWritable
/** {@inheritDoc} */
@Override
public int hashCode() {
return WritableComparator.hashBytes(bytes, this.bytes.length);
@ -150,7 +147,6 @@ public class ImmutableBytesWritable implements WritableComparable {
0, that.length);
}
/** {@inheritDoc} */
@Override
public boolean equals(Object right_obj) {
if (right_obj instanceof byte []) {
@ -162,7 +158,6 @@ public class ImmutableBytesWritable implements WritableComparable {
return false;
}
/** {@inheritDoc} */
@Override
public String toString() {
StringBuffer sb = new StringBuffer(3*this.bytes.length);
@ -192,7 +187,6 @@ public class ImmutableBytesWritable implements WritableComparable {
super(ImmutableBytesWritable.class);
}
/** {@inheritDoc} */
@Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
return comparator.compare(b1, s1, l1, b2, s2, l2);

View File

@ -43,12 +43,15 @@ public class RowResult implements Writable, SortedMap<byte [], Cell> {
private byte [] row = null;
private final HbaseMapWritable<byte [], Cell> cells;
/** default constructor for writable */
public RowResult() {
this(null, new HbaseMapWritable<byte [], Cell>());
}
/**
* Create a RowResult from a row and Cell map
* @param row
* @param m
*/
public RowResult (final byte [] row,
final HbaseMapWritable<byte [], Cell> m) {
@ -58,6 +61,7 @@ public class RowResult implements Writable, SortedMap<byte [], Cell> {
/**
* Get the row for this RowResult
* @return the row
*/
public byte [] getRow() {
return row;
@ -78,7 +82,7 @@ public class RowResult implements Writable, SortedMap<byte [], Cell> {
}
public Cell get(Object key) {
return (Cell)this.cells.get(key);
return this.cells.get(key);
}
public Cell remove(@SuppressWarnings("unused") Object key) {
@ -127,6 +131,8 @@ public class RowResult implements Writable, SortedMap<byte [], Cell> {
/**
* Get the Cell that corresponds to column
* @param column
* @return the Cell
*/
public Cell get(byte [] column) {
return this.cells.get(column);
@ -134,6 +140,8 @@ public class RowResult implements Writable, SortedMap<byte [], Cell> {
/**
* Get the Cell that corresponds to column, using a String key
* @param key
* @return the Cell
*/
public Cell get(String key) {
return get(Bytes.toBytes(key));

View File

@ -110,6 +110,7 @@ public interface HMasterInterface extends VersionedProtocol {
*
* @param tableName
* @param desc
* @throws IOException
*/
public void modifyTableMeta(byte[] tableName, HTableDescriptor desc)
throws IOException;

View File

@ -84,6 +84,8 @@ public interface HRegionInterface extends VersionedProtocol {
*
* @param regionName region name
* @param row row key
* @param columns columns to get
* @param ts time stamp
* @param lockId lock id
* @return map of values
* @throws IOException
@ -142,6 +144,7 @@ public interface HRegionInterface extends VersionedProtocol {
* @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)
@ -202,6 +205,7 @@ public interface HRegionInterface extends VersionedProtocol {
/**
* Releases a remote row lock.
*
* @param regionName
* @param lockId the lock id returned by lockRow
* @throws IOException
*/

View File

@ -88,25 +88,32 @@ public class HbaseRPC {
/** A method invocation, including the method name and its parameters.*/
private static class Invocation implements Writable, Configurable {
private String methodName;
@SuppressWarnings("unchecked")
private Class[] parameterClasses;
private Object[] parameters;
private Configuration conf;
/** default constructor */
public Invocation() {}
/**
* @param method
* @param parameters
*/
public Invocation(Method method, Object[] parameters) {
this.methodName = method.getName();
this.parameterClasses = method.getParameterTypes();
this.parameters = parameters;
}
/** The name of the method invoked. */
/** @return The name of the method invoked. */
public String getMethodName() { return methodName; }
/** The parameter classes. */
/** @return The parameter classes. */
@SuppressWarnings("unchecked")
public Class[] getParameterClasses() { return parameterClasses; }
/** The parameter instances. */
/** @return The parameter instances. */
public Object[] getParameters() { return parameters; }
public void readFields(DataInput in) throws IOException {
@ -129,6 +136,7 @@ public class HbaseRPC {
}
}
@Override
public String toString() {
StringBuffer buffer = new StringBuffer();
buffer.append(methodName);
@ -217,6 +225,12 @@ public class HbaseRPC {
private Client client;
private boolean isClosed = false;
/**
* @param address
* @param ticket
* @param conf
* @param factory
*/
public Invoker(InetSocketAddress address, UserGroupInformation ticket,
Configuration conf, SocketFactory factory) {
this.address = address;
@ -224,7 +238,8 @@ public class HbaseRPC {
this.client = CLIENTS.getClient(conf, factory);
}
public Object invoke(Object proxy, Method method, Object[] args)
public Object invoke(@SuppressWarnings("unused") Object proxy,
Method method, Object[] args)
throws Throwable {
long startTime = System.currentTimeMillis();
ObjectWritable value = (ObjectWritable)
@ -246,6 +261,7 @@ public class HbaseRPC {
/**
* A version mismatch for the RPC protocol.
*/
@SuppressWarnings("serial")
public static class VersionMismatch extends IOException {
private String interfaceName;
private long clientVersion;
@ -276,20 +292,30 @@ public class HbaseRPC {
}
/**
* Get the client's preferred version
* @return the client's preferred version
*/
public long getClientVersion() {
return clientVersion;
}
/**
* Get the server's agreed to version.
* @return the server's agreed to version.
*/
public long getServerVersion() {
return serverVersion;
}
}
/**
* @param protocol
* @param clientVersion
* @param addr
* @param conf
* @param maxAttempts
* @return proxy
* @throws IOException
*/
@SuppressWarnings("unchecked")
public static VersionedProtocol waitForProxy(Class protocol,
long clientVersion,
InetSocketAddress addr,
@ -319,16 +345,38 @@ public class HbaseRPC {
}
}
}
/** Construct a client-side proxy object that implements the named protocol,
* talking to a server at the named address. */
/**
* Construct a client-side proxy object that implements the named protocol,
* talking to a server at the named address.
*
* @param protocol
* @param clientVersion
* @param addr
* @param conf
* @param factory
* @return proxy
* @throws IOException
*/
public static VersionedProtocol getProxy(Class<?> protocol,
long clientVersion, InetSocketAddress addr, Configuration conf,
SocketFactory factory) throws IOException {
return getProxy(protocol, clientVersion, addr, null, conf, factory);
}
/** Construct a client-side proxy object that implements the named protocol,
* talking to a server at the named address. */
/**
* Construct a client-side proxy object that implements the named protocol,
* talking to a server at the named address.
*
* @param protocol
* @param clientVersion
* @param addr
* @param ticket
* @param conf
* @param factory
* @return proxy
* @throws IOException
*/
public static VersionedProtocol getProxy(Class<?> protocol,
long clientVersion, InetSocketAddress addr, UserGroupInformation ticket,
Configuration conf, SocketFactory factory) throws IOException {
@ -375,7 +423,16 @@ public class HbaseRPC {
}
}
/** Expert: Make multiple, parallel calls to a set of servers. */
/**
* Expert: Make multiple, parallel calls to a set of servers.
*
* @param method
* @param params
* @param addrs
* @param conf
* @return values
* @throws IOException
*/
public static Object[] call(Method method, Object[][] params,
InetSocketAddress[] addrs, Configuration conf)
throws IOException {
@ -403,15 +460,35 @@ public class HbaseRPC {
}
}
/** Construct a server for a protocol implementation instance listening on a
* port and address. */
/**
* Construct a server for a protocol implementation instance listening on a
* port and address.
*
* @param instance
* @param bindAddress
* @param port
* @param conf
* @return Server
* @throws IOException
*/
public static Server getServer(final Object instance, final String bindAddress, final int port, Configuration conf)
throws IOException {
return getServer(instance, bindAddress, port, 1, false, conf);
}
/** Construct a server for a protocol implementation instance listening on a
* port and address. */
/**
* Construct a server for a protocol implementation instance listening on a
* port and address.
*
* @param instance
* @param bindAddress
* @param port
* @param numHandlers
* @param verbose
* @param conf
* @return Server
* @throws IOException
*/
public static Server getServer(final Object instance, final String bindAddress, final int port,
final int numHandlers,
final boolean verbose, Configuration conf)
@ -425,11 +502,13 @@ public class HbaseRPC {
private Class<?> implementation;
private boolean verbose;
/** Construct an RPC server.
/**
* Construct an RPC server.
* @param instance the instance whose methods will be called
* @param conf the configuration to use
* @param bindAddress the address to bind on to listen for connection
* @param port the port to listen for connections on
* @throws IOException
*/
public Server(Object instance, Configuration conf, String bindAddress, int port)
throws IOException {
@ -451,6 +530,7 @@ public class HbaseRPC {
* @param port the port to listen for connections on
* @param numHandlers the number of method handler threads to run
* @param verbose whether each call should be logged
* @throws IOException
*/
public Server(Object instance, Configuration conf, String bindAddress, int port,
int numHandlers, boolean verbose) throws IOException {
@ -460,6 +540,7 @@ public class HbaseRPC {
this.verbose = verbose;
}
@Override
public Writable call(Writable param, long receivedTime) throws IOException {
try {
Invocation call = (Invocation)param;

View File

@ -25,6 +25,7 @@ public interface TransactionalRegionInterface extends HRegionInterface {
*
* @param transactionId
* @param regionName name of region
* @throws IOException
*/
public void beginTransaction(long transactionId, final byte[] regionName)
throws IOException;
@ -32,7 +33,8 @@ public interface TransactionalRegionInterface extends HRegionInterface {
/**
* Retrieve a single value from the specified region for the specified row and
* column keys
*
*
* @param transactionId
* @param regionName name of region
* @param row row key
* @param column column key
@ -45,6 +47,7 @@ public interface TransactionalRegionInterface extends HRegionInterface {
/**
* Get the specified number of versions of the specified row and column
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param column column key
@ -60,6 +63,7 @@ public interface TransactionalRegionInterface extends HRegionInterface {
* Get the specified number of versions of the specified row and column with
* the specified timestamp.
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param column column key
@ -75,8 +79,10 @@ public interface TransactionalRegionInterface extends HRegionInterface {
/**
* Get all the data for the specified row at a given timestamp
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param ts timestamp
* @return map of values
* @throws IOException
*/
@ -86,8 +92,11 @@ public interface TransactionalRegionInterface extends HRegionInterface {
/**
* Get selected columns for the specified row at a given timestamp.
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param columns colums to get
* @param ts timestamp
* @return map of values
* @throws IOException
*/
@ -98,8 +107,10 @@ public interface TransactionalRegionInterface extends HRegionInterface {
/**
* Get selected columns for the specified row at the latest timestamp.
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param columns columns to get
* @return map of values
* @throws IOException
*/
@ -110,6 +121,7 @@ public interface TransactionalRegionInterface extends HRegionInterface {
* Delete all cells that match the passed row and whose timestamp is equal-to
* or older than the passed timestamp.
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param timestamp Delete all entries that have this timestamp or older
@ -142,6 +154,7 @@ public interface TransactionalRegionInterface extends HRegionInterface {
/**
* Applies a batch of updates via one RPC
*
* @param transactionId
* @param regionName name of the region to update
* @param b BatchUpdate
* @throws IOException
@ -151,25 +164,31 @@ public interface TransactionalRegionInterface extends HRegionInterface {
/**
* Ask if we can commit the given transaction.
*
*
* @param regionName
* @param transactionId
* @return true if we can commit
* @throws IOException
*/
public boolean commitRequest(final byte[] regionName, long transactionId)
throws IOException;
/**
* Commit the transaction.
*
*
* @param regionName
* @param transactionId
* @throws IOException
*/
public void commit(final byte[] regionName, long transactionId)
throws IOException;
/**
* Abort the transaction.
*
*
* @param regionName
* @param transactionId
* @throws IOException
*/
public void abort(final byte[] regionName, long transactionId)
throws IOException;

View File

@ -67,7 +67,6 @@ public class GroupingTableMap extends TableMap<ImmutableBytesWritable,RowResult>
job.set(GROUP_COLUMNS, groupColumns);
}
/** {@inheritDoc} */
@Override
public void configure(JobConf job) {
super.configure(job);

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.mapred.Reporter;
* Write to table each key, record pair
*/
public class IdentityTableReduce extends TableReduce<ImmutableBytesWritable, BatchUpdate> {
@SuppressWarnings("unused")
private static final Log LOG =
LogFactory.getLog(IdentityTableReduce.class.getName());

View File

@ -45,7 +45,6 @@ public class IndexOutputFormat extends
FileOutputFormat<ImmutableBytesWritable, LuceneDocumentWrapper> {
static final Log LOG = LogFactory.getLog(IndexOutputFormat.class);
/** {@inheritDoc} */
@Override
public RecordWriter<ImmutableBytesWritable, LuceneDocumentWrapper>
getRecordWriter(final FileSystem fs, JobConf job, String name,

View File

@ -23,7 +23,7 @@ import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HConstants; //TODO: remove
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.RowResult;
@ -47,6 +47,7 @@ public class IndexTableReduce extends MapReduceBase implements
private IndexConfiguration indexConf;
@Override
public void configure(JobConf job) {
super.configure(job);
indexConf = new IndexConfiguration();
@ -59,13 +60,14 @@ public class IndexTableReduce extends MapReduceBase implements
}
}
@Override
public void close() throws IOException {
super.close();
}
public void reduce(ImmutableBytesWritable key, Iterator<RowResult> values,
OutputCollector<ImmutableBytesWritable, LuceneDocumentWrapper> output,
Reporter reporter)
@SuppressWarnings("unused") Reporter reporter)
throws IOException {
if (!values.hasNext()) {
return;

View File

@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.mapred;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.io.IOException; //TODO: remove
import org.apache.hadoop.io.Writable;
import org.apache.lucene.document.Document;
@ -33,19 +33,25 @@ import org.apache.lucene.document.Document;
class LuceneDocumentWrapper implements Writable {
private Document doc;
/**
* @param doc
*/
public LuceneDocumentWrapper(Document doc) {
this.doc = doc;
}
/**
* @return the document
*/
public Document get() {
return doc;
}
public void readFields(DataInput in) throws IOException {
public void readFields(@SuppressWarnings("unused") DataInput in) {
// intentionally left blank
}
public void write(DataOutput out) throws IOException {
public void write(@SuppressWarnings("unused") DataOutput out) {
// intentionally left blank
}
}

View File

@ -74,6 +74,11 @@ public class RowCounter extends TableMap<ImmutableBytesWritable, RowResult> impl
output.collect(row, EMPTY_RESULT_VALUE);
}
/**
* @param args
* @return the JobConf
* @throws IOException
*/
@SuppressWarnings({ "unused", "deprecation" })
public JobConf createSubmittableJob(String[] args) throws IOException {
JobConf c = new JobConf(getConf(), RowCounter.class);
@ -120,6 +125,10 @@ public class RowCounter extends TableMap<ImmutableBytesWritable, RowResult> impl
this.conf = c;
}
/**
* @param args
* @throws Exception
*/
public static void main(String[] args) throws Exception {
HBaseConfiguration c = new HBaseConfiguration();
int errCode = ToolRunner.run(c, new RowCounter(), args);

View File

@ -46,7 +46,6 @@ public class TableInputFormat extends TableInputFormatBase implements
*/
public static final String COLUMN_LIST = "hbase.mapred.tablecolumns";
/** {@inheritDoc} */
public void configure(JobConf job) {
Path[] tableNames = FileInputFormat.getInputPaths(job);
String colArg = job.get(COLUMN_LIST);
@ -63,7 +62,7 @@ public class TableInputFormat extends TableInputFormatBase implements
}
}
/** {@inheritDoc} */
@SuppressWarnings("deprecation")
public void validateInput(JobConf job) throws IOException {
// expecting exactly one path
Path [] tableNames = FileInputFormat.getInputPaths(job);

View File

@ -96,6 +96,7 @@ implements InputFormat<ImmutableBytesWritable, RowResult> {
/**
* Restart from survivable exceptions by creating a new scanner.
*
* @param firstRow
* @throws IOException
*/
public void restart(byte[] firstRow) throws IOException {
@ -163,8 +164,7 @@ implements InputFormat<ImmutableBytesWritable, RowResult> {
this.trrRowFilter = rowFilter;
}
/** {@inheritDoc} */
public void close() throws IOException {
public void close() {
this.scanner.close();
}
@ -186,14 +186,12 @@ implements InputFormat<ImmutableBytesWritable, RowResult> {
return new RowResult();
}
/** {@inheritDoc} */
public long getPos() {
// This should be the ordinal tuple in the range;
// not clear how to calculate...
return 0;
}
/** {@inheritDoc} */
public float getProgress() {
// Depends on the total number of tuples and getPos
return 0;

View File

@ -49,6 +49,8 @@ public abstract class TableMap<K extends WritableComparable, V extends Writable>
* @param table table name
* @param columns columns to scan
* @param mapper mapper class
* @param outputKeyClass
* @param outputValueClass
* @param job job configuration
*/
public static void initJob(String table, String columns,

View File

@ -63,19 +63,16 @@ FileOutputFormat<ImmutableBytesWritable, BatchUpdate> {
m_table = table;
}
/** {@inheritDoc} */
public void close(@SuppressWarnings("unused") Reporter reporter) {
// Nothing to do.
}
/** {@inheritDoc} */
public void write(@SuppressWarnings("unused") ImmutableBytesWritable key,
BatchUpdate value) throws IOException {
m_table.commit(value);
}
}
/** {@inheritDoc} */
@Override
@SuppressWarnings("unchecked")
public RecordWriter getRecordWriter(
@ -97,7 +94,6 @@ FileOutputFormat<ImmutableBytesWritable, BatchUpdate> {
return new TableRecordWriter(table);
}
/** {@inheritDoc} */
@Override
@SuppressWarnings("unused")
public void checkOutputSpecs(FileSystem ignored, JobConf job)

View File

@ -34,6 +34,9 @@ import org.apache.hadoop.mapred.Reporter;
/**
* Write a table, sorting by the input key
*
* @param <K> key class
* @param <V> value class
*/
@SuppressWarnings("unchecked")
public abstract class TableReduce<K extends WritableComparable, V extends Writable>

View File

@ -68,33 +68,28 @@ public class TableSplit implements InputSplit {
return m_endRow;
}
/** {@inheritDoc} */
public long getLength() {
// Not clear how to obtain this... seems to be used only for sorting splits
return 0;
}
/** {@inheritDoc} */
public String[] getLocations() {
// Return a random node from the cluster for now
return new String[] { };
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.m_tableName = Bytes.readByteArray(in);
this.m_startRow = Bytes.readByteArray(in);
this.m_endRow = Bytes.readByteArray(in);
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.m_tableName);
Bytes.writeByteArray(out, this.m_startRow);
Bytes.writeByteArray(out, this.m_endRow);
}
/** {@inheritDoc} */
@Override
public String toString() {
return Bytes.toString(m_tableName) +"," + Bytes.toString(m_startRow) +

View File

@ -22,9 +22,9 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystem; //TODO: remove
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.HStoreFile; //TODO: remove
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.ipc.HRegionInterface;

View File

@ -80,7 +80,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
static final Log LOG = LogFactory.getLog(HMaster.class.getName());
/** {@inheritDoc} */
public long getProtocolVersion(String protocol,
@SuppressWarnings("unused") long clientVersion)
throws IOException {
@ -525,7 +524,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
* HMasterRegionInterface
*/
/** {@inheritDoc} */
@SuppressWarnings("unused")
public MapWritable regionServerStartup(HServerInfo serverInfo)
throws IOException {
@ -560,7 +558,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
return mw;
}
/** {@inheritDoc} */
public HMsg[] regionServerReport(HServerInfo serverInfo, HMsg msgs[],
HRegionInfo[] mostLoadedRegions)
throws IOException {
@ -572,18 +569,15 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
* HMasterInterface
*/
/** {@inheritDoc} */
public boolean isMasterRunning() {
return !closed.get();
}
/** {@inheritDoc} */
public void shutdown() {
LOG.info("Cluster shutdown requested. Starting to quiesce servers");
this.shutdownRequested = true;
}
/** {@inheritDoc} */
public void createTable(HTableDescriptor desc)
throws IOException {
if (!isMasterRunning()) {
@ -640,7 +634,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
regionManager.createRegion(newRegion, srvr, metaRegionName);
}
/** {@inheritDoc} */
public void deleteTable(final byte [] tableName) throws IOException {
if (Bytes.equals(tableName, ROOT_TABLE_NAME)) {
throw new IOException("Can't delete root table");
@ -649,26 +642,22 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
LOG.info("deleted table: " + Bytes.toString(tableName));
}
/** {@inheritDoc} */
public void addColumn(byte [] tableName, HColumnDescriptor column)
throws IOException {
new AddColumn(this, tableName, column).process();
}
/** {@inheritDoc} */
public void modifyColumn(byte [] tableName, byte [] columnName,
HColumnDescriptor descriptor)
throws IOException {
new ModifyColumn(this, tableName, columnName, descriptor).process();
}
/** {@inheritDoc} */
public void deleteColumn(final byte [] tableName, final byte [] c)
throws IOException {
new DeleteColumn(this, tableName, HStoreKey.getFamily(c)).process();
}
/** {@inheritDoc} */
public void enableTable(final byte [] tableName) throws IOException {
if (Bytes.equals(tableName, ROOT_TABLE_NAME)) {
throw new IOException("Can't enable root table");
@ -676,7 +665,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
new ChangeTableState(this, tableName, true).process();
}
/** {@inheritDoc} */
public void disableTable(final byte [] tableName) throws IOException {
if (Bytes.equals(tableName, ROOT_TABLE_NAME)) {
throw new IOException("Can't disable root table");
@ -684,14 +672,12 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
new ChangeTableState(this, tableName, false).process();
}
/** {@inheritDoc} */
public void modifyTableMeta(final byte[] tableName, HTableDescriptor desc)
throws IOException
{
new ModifyTableMeta(this, tableName, desc).process();
}
/** {@inheritDoc} */
public HServerAddress findRootRegion() {
return regionManager.getRootRegionLocation();
}

View File

@ -49,7 +49,6 @@ public class MetaRegion implements Comparable<MetaRegion> {
this.startKey = startKey;
}
/** {@inheritDoc} */
@Override
public String toString() {
return "{regionname: " + Bytes.toString(this.regionName) +
@ -72,13 +71,11 @@ public class MetaRegion implements Comparable<MetaRegion> {
return startKey;
}
/** {@inheritDoc} */
@Override
public boolean equals(Object o) {
return o instanceof MetaRegion && this.compareTo((MetaRegion)o) == 0;
}
/** {@inheritDoc} */
@Override
public int hashCode() {
int result = this.regionName.hashCode();
@ -88,7 +85,6 @@ public class MetaRegion implements Comparable<MetaRegion> {
// Comparable
/** {@inheritDoc} */
public int compareTo(MetaRegion other) {
int result = Bytes.compareTo(this.regionName, other.getRegionName());
if(result == 0) {

View File

@ -45,7 +45,12 @@ class MetaScanner extends BaseScanner {
private final List<MetaRegion> metaRegionsToRescan =
new ArrayList<MetaRegion>();
/** Constructor */
/**
* Constructor
*
* @param master
* @param regionManager
*/
public MetaScanner(HMaster master, RegionManager regionManager) {
super(master, regionManager, false, master.metaRescanInterval, master.closed);
}

View File

@ -19,7 +19,7 @@
*/
package org.apache.hadoop.hbase.master;
import java.util.Map;
import java.util.Map; //TODO: remove
import java.io.IOException;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.ipc.HRegionInterface;

View File

@ -48,7 +48,6 @@ class ProcessRegionClose extends ProcessRegionStatusChange {
this.offlineRegion = offlineRegion;
}
/** {@inheritDoc} */
@Override
public String toString() {
return "ProcessRegionClose of " + this.regionInfo.getRegionNameAsString() +

View File

@ -52,7 +52,6 @@ class ProcessRegionOpen extends ProcessRegionStatusChange {
this.startCode = Bytes.toBytes(info.getStartCode());
}
/** {@inheritDoc} */
@Override
public String toString() {
return "PendingOpenOperation from " + serverAddress.toString();

View File

@ -20,7 +20,7 @@
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.io.UnsupportedEncodingException; //TODO: remove
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
@ -81,7 +81,6 @@ class ProcessServerShutdown extends RegionServerOperation {
this.oldLogDir = new Path(master.rootdir, dirName.toString());
}
/** {@inheritDoc} */
@Override
public String toString() {
return "ProcessServerShutdown of " + this.deadServer.toString();
@ -191,7 +190,6 @@ class ProcessServerShutdown extends RegionServerOperation {
super(m, master);
}
/** {@inheritDoc} */
public Boolean call() throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("process server shutdown scanning root region on " +
@ -211,7 +209,6 @@ class ProcessServerShutdown extends RegionServerOperation {
super(m, master);
}
/** {@inheritDoc} */
public Boolean call() throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("process server shutdown scanning " + m.getRegionName() +

View File

@ -43,13 +43,11 @@ abstract class RegionServerOperation implements Delayed, HConstants {
this.expire = System.currentTimeMillis() + this.master.leaseTimeout / 2;
}
/** {@inheritDoc} */
public long getDelay(TimeUnit unit) {
return unit.convert(this.expire - System.currentTimeMillis(),
TimeUnit.MILLISECONDS);
}
/** {@inheritDoc} */
public int compareTo(Delayed o) {
return Long.valueOf(getDelay(TimeUnit.MILLISECONDS)
- o.getDelay(TimeUnit.MILLISECONDS)).intValue();

View File

@ -26,7 +26,12 @@ import org.apache.hadoop.hbase.RemoteExceptionHandler;
/** Scanner for the <code>ROOT</code> HRegion. */
class RootScanner extends BaseScanner {
/** Constructor */
/**
* Constructor
*
* @param master
* @param regionManager
*/
public RootScanner(HMaster master, RegionManager regionManager) {
super(master, regionManager, true, master.metaRescanInterval, master.closed);
}

View File

@ -461,7 +461,8 @@ class ServerManager implements HConstants {
}
}
private void processRegionClose(HServerInfo serverInfo, HRegionInfo region) {
private void processRegionClose(
@SuppressWarnings("unused") HServerInfo serverInfo, HRegionInfo region) {
if (region.isRootRegion()) {
// Root region
if (region.isOffline()) {
@ -665,7 +666,6 @@ class ServerManager implements HConstants {
this.server = server;
}
/** {@inheritDoc} */
public void leaseExpired() {
LOG.info(server + " lease expired");
// Remove the server from the known servers list and update load info

View File

@ -72,7 +72,6 @@ abstract class TableOperation implements HConstants {
super(m, master);
}
/** {@inheritDoc} */
public Boolean call() throws IOException {
boolean tableExists = false;

View File

@ -57,62 +57,77 @@ public class BeforeThisStoreKey extends HStoreKey {
return false;
}
@Override
public byte[] getColumn() {
return this.beforeThisKey.getColumn();
}
@Override
public byte[] getRow() {
return this.beforeThisKey.getRow();
}
@Override
public long getSize() {
return this.beforeThisKey.getSize();
}
@Override
public long getTimestamp() {
return this.beforeThisKey.getTimestamp();
}
@Override
public int hashCode() {
return this.beforeThisKey.hashCode();
}
@Override
public boolean matchesRowCol(HStoreKey other) {
return this.beforeThisKey.matchesRowCol(other);
}
@Override
public boolean matchesRowFamily(HStoreKey that) {
return this.beforeThisKey.matchesRowFamily(that);
}
@Override
public boolean matchesWithoutColumn(HStoreKey other) {
return this.beforeThisKey.matchesWithoutColumn(other);
}
@Override
public void readFields(DataInput in) throws IOException {
this.beforeThisKey.readFields(in);
}
@Override
public void set(HStoreKey k) {
this.beforeThisKey.set(k);
}
@Override
public void setColumn(byte[] c) {
this.beforeThisKey.setColumn(c);
}
@Override
public void setRow(byte[] newrow) {
this.beforeThisKey.setRow(newrow);
}
@Override
public void setVersion(long timestamp) {
this.beforeThisKey.setVersion(timestamp);
}
@Override
public String toString() {
return this.beforeThisKey.toString();
}
@Override
public void write(DataOutput out) throws IOException {
this.beforeThisKey.write(out);
}

View File

@ -29,6 +29,7 @@ import java.io.IOException;
public interface ChangedReadersObserver {
/**
* Notify observers.
* @throws IOException
*/
void updateReaders() throws IOException;
}

View File

@ -67,7 +67,6 @@ class CompactSplitThread extends Thread implements HConstants {
20 * 1000);
}
/** {@inheritDoc} */
@Override
public void run() {
while (!server.isStopRequested()) {

View File

@ -77,7 +77,6 @@ class Flusher extends Thread implements FlushRequester {
globalMemcacheLimit / 2);
}
/** {@inheritDoc} */
@Override
public void run() {
while (!server.isStopRequested()) {
@ -109,7 +108,6 @@ class Flusher extends Thread implements FlushRequester {
LOG.info(getName() + " exiting");
}
/** {@inheritDoc} */
public void request(HRegion r) {
addRegion(r, System.currentTimeMillis());
}
@ -233,7 +231,8 @@ class Flusher extends Thread implements FlushRequester {
* @param r Region to add.
* @param now The 'now' to use. Set last flush time to this value.
*/
private void addRegion(final HRegion r, final long now) {
private void addRegion(final HRegion r,
@SuppressWarnings("unused") final long now) {
synchronized (regionsInQueue) {
if (!regionsInQueue.contains(r)) {
regionsInQueue.add(r);

View File

@ -172,17 +172,14 @@ public abstract class HAbstractScanner implements InternalScanner {
return false;
}
/** {@inheritDoc} */
public boolean isWildcardScanner() {
return this.wildcardMatch;
}
/** {@inheritDoc} */
public boolean isMultipleMatchScanner() {
return this.multipleMatchers;
}
/** {@inheritDoc} */
public abstract boolean next(HStoreKey key, SortedMap<byte [], Cell> results)
throws IOException;

View File

@ -205,6 +205,9 @@ public class HLog implements HConstants {
}
}
/**
* @return log sequence number
*/
public long getSequenceNumber() {
return logSeqNum;
}
@ -565,6 +568,10 @@ public class HLog implements HConstants {
this.cacheFlushLock.unlock();
}
/**
* @param column
* @return true if the column is a meta column
*/
public static boolean isMetaColumn(byte [] column) {
return Bytes.equals(METACOLUMN, column);
}

View File

@ -64,8 +64,16 @@ public class HLogEdit implements Writable, HConstants {
return (value == null)? false: deleteBytes.compareTo(value) == 0;
}
/** If transactional log entry, these are the op codes */
public enum TransactionalOperation {
START, WRITE, COMMIT, ABORT
/** start transaction */
START,
/** Equivalent to append in non-transactional environment */
WRITE,
/** Transaction commit entry */
COMMIT,
/** Abort transaction entry */
ABORT
}
private byte [] column;
@ -139,7 +147,8 @@ public class HLogEdit implements Writable, HConstants {
public long getTimestamp() {
return this.timestamp;
}
/** @return true if entry is a transactional entry */
public boolean isTransactionEntry() {
return isTransactionEntry;
}
@ -188,7 +197,6 @@ public class HLogEdit implements Writable, HConstants {
// Writable
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.column);
if (this.val == null) {
@ -205,7 +213,6 @@ public class HLogEdit implements Writable, HConstants {
}
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.column = Bytes.readByteArray(in);
this.val = new byte[in.readInt()];

View File

@ -66,42 +66,37 @@ public class HLogKey implements WritableComparable {
// A bunch of accessors
//////////////////////////////////////////////////////////////////////////////
/** @return region name */
public byte [] getRegionName() {
return regionName;
}
/** @return table name */
public byte [] getTablename() {
return tablename;
}
/** @return row key */
public byte [] getRow() {
return row;
}
/** @return log sequence number */
public long getLogSeqNum() {
return logSeqNum;
}
/**
* {@inheritDoc}
*/
@Override
public String toString() {
return Bytes.toString(tablename) + "/" + Bytes.toString(regionName) + "/" +
Bytes.toString(row) + "/" + logSeqNum;
}
/**
* {@inheritDoc}
*/
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
/**
* {@inheritDoc}
*/
@Override
public int hashCode() {
int result = this.regionName.hashCode();
@ -114,9 +109,6 @@ public class HLogKey implements WritableComparable {
// Comparable
//
/**
* {@inheritDoc}
*/
public int compareTo(Object o) {
HLogKey other = (HLogKey) o;
int result = Bytes.compareTo(this.regionName, other.regionName);
@ -141,9 +133,6 @@ public class HLogKey implements WritableComparable {
// Writable
//
/**
* {@inheritDoc}
*/
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.regionName);
Bytes.writeByteArray(out, this.tablename);
@ -151,9 +140,6 @@ public class HLogKey implements WritableComparable {
out.writeLong(logSeqNum);
}
/**
* {@inheritDoc}
*/
public void readFields(DataInput in) throws IOException {
this.regionName = Bytes.readByteArray(in);
this.tablename = Bytes.readByteArray(in);

View File

@ -1891,19 +1891,16 @@ public class HRegion implements HConstants {
}
}
/** {@inheritDoc} */
@Override
public boolean equals(Object o) {
return this.hashCode() == ((HRegion)o).hashCode();
}
/** {@inheritDoc} */
@Override
public int hashCode() {
return this.regionInfo.getRegionName().hashCode();
}
/** {@inheritDoc} */
@Override
public String toString() {
return this.regionInfo.getRegionNameAsString();
@ -1979,7 +1976,6 @@ public class HRegion implements HConstants {
activeScannerCount.incrementAndGet();
}
/** {@inheritDoc} */
@SuppressWarnings("null")
public boolean next(HStoreKey key, SortedMap<byte [], Cell> results)
throws IOException {
@ -2097,9 +2093,6 @@ public class HRegion implements HConstants {
}
}
/**
* {@inheritDoc}
*/
public void close() {
try {
for(int i = 0; i < scanners.length; i++) {
@ -2123,12 +2116,10 @@ public class HRegion implements HConstants {
}
}
/** {@inheritDoc} */
public boolean isWildcardScanner() {
throw new UnsupportedOperationException("Unimplemented on HScanner");
}
/** {@inheritDoc} */
public boolean isMultipleMatchScanner() {
throw new UnsupportedOperationException("Unimplemented on HScanner");
}

View File

@ -179,7 +179,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
this.instance = instance;
}
/** {@inheritDoc} */
@Override
public void run() {
LOG.info("Starting shutdown thread.");
@ -798,7 +797,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
}
/** {@inheritDoc} */
public void run() {
try {
while(!stopRequested.get()) {
@ -1002,14 +1000,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// HRegionInterface
//
/** {@inheritDoc} */
public HRegionInfo getRegionInfo(final byte [] regionName)
throws NotServingRegionException {
requestCount.incrementAndGet();
return getRegion(regionName).getRegionInfo();
}
/** {@inheritDoc} */
public Cell[] get(final byte [] regionName, final byte [] row,
final byte [] column, final long timestamp, final int numVersions)
throws IOException {
@ -1023,7 +1019,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
}
/** {@inheritDoc} */
public RowResult getRow(final byte [] regionName, final byte [] row,
final byte [][] columns, final long ts, final long lockId)
throws IOException {
@ -1050,7 +1045,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
}
/** {@inheritDoc} */
public RowResult getClosestRowBefore(final byte [] regionName,
final byte [] row)
throws IOException {
@ -1068,7 +1062,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
}
/** {@inheritDoc} */
public RowResult next(final long scannerId) throws IOException {
checkOpen();
requestCount.incrementAndGet();
@ -1103,7 +1096,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
}
/** {@inheritDoc} */
public void batchUpdate(final byte [] regionName, BatchUpdate b, long lockId)
throws IOException {
if (b.getRow() == null)
@ -1155,7 +1147,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// remote scanner interface
//
/** {@inheritDoc} */
public long openScanner(byte [] regionName, byte [][] cols, byte [] firstRow,
final long timestamp, final RowFilterInterface filter)
throws IOException {
@ -1200,7 +1191,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
return scannerId;
}
/** {@inheritDoc} */
public void close(final long scannerId) throws IOException {
checkOpen();
requestCount.incrementAndGet();
@ -1235,7 +1225,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
this.scannerName = n;
}
/** {@inheritDoc} */
public void leaseExpired() {
LOG.info("Scanner " + this.scannerName + " lease expired");
InternalScanner s = null;
@ -1256,7 +1245,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// Methods that do the actual work for the remote API
//
/** {@inheritDoc} */
public void deleteAll(final byte [] regionName, final byte [] row,
final byte [] column, final long timestamp, final long lockId)
throws IOException {
@ -1264,7 +1252,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
region.deleteAll(row, column, timestamp, getLockFromId(lockId));
}
/** {@inheritDoc} */
public void deleteAll(final byte [] regionName, final byte [] row,
final long timestamp, final long lockId)
throws IOException {
@ -1272,7 +1259,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
region.deleteAll(row, timestamp, getLockFromId(lockId));
}
/** {@inheritDoc} */
public void deleteFamily(byte [] regionName, byte [] row, byte [] family,
long timestamp, final long lockId)
throws IOException{
@ -1280,7 +1266,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
getLockFromId(lockId));
}
/** {@inheritDoc} */
public long lockRow(byte [] regionName, byte [] row)
throws IOException {
checkOpen();
@ -1346,7 +1331,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
return rl;
}
/** {@inheritDoc} */
public void unlockRow(byte [] regionName, long lockId)
throws IOException {
checkOpen();
@ -1397,7 +1381,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
this.region = region;
}
/** {@inheritDoc} */
public void leaseExpired() {
LOG.info("Row Lock " + this.lockName + " lease expired");
Integer r = null;
@ -1617,7 +1600,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
return regionsToCheck;
}
/** {@inheritDoc} */
public long getProtocolVersion(final String protocol,
@SuppressWarnings("unused") final long clientVersion)
throws IOException {

View File

@ -1913,7 +1913,6 @@ public class HStore implements HConstants {
}
}
/** {@inheritDoc} */
@Override
public String toString() {
return this.storeNameStr;

View File

@ -448,7 +448,6 @@ public class HStoreFile implements HConstants {
return (isReference())? l / 2: l;
}
/** {@inheritDoc} */
@Override
public String toString() {
return encodedRegionName + "/" + Bytes.toString(colFamily) + "/" + fileId +
@ -549,7 +548,6 @@ public class HStoreFile implements HConstants {
return this.encodedRegionName;
}
/** {@inheritDoc} */
@Override
public String toString() {
return encodedRegionName + "/" + fileid + "/" + region;
@ -557,7 +555,6 @@ public class HStoreFile implements HConstants {
// Make it serializable.
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
// Write out the encoded region name as a String. Doing it as a String
// keeps a Reference's serialziation backword compatible with
@ -571,7 +568,6 @@ public class HStoreFile implements HConstants {
midkey.write(out);
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.encodedRegionName = Integer.parseInt(in.readUTF());
fileid = in.readLong();
@ -671,6 +667,7 @@ public class HStoreFile implements HConstants {
* @param fs
* @param dirName
* @param compression
* @param hri
* @throws IOException
*/
public HbaseWriter(Configuration conf, FileSystem fs, String dirName,
@ -733,7 +730,6 @@ public class HStoreFile implements HConstants {
return filter;
}
/** {@inheritDoc} */
@Override
public Writable get(WritableComparable key, Writable val)
throws IOException {
@ -752,7 +748,6 @@ public class HStoreFile implements HConstants {
return null;
}
/** {@inheritDoc} */
@Override
public WritableComparable getClosest(WritableComparable key,
Writable val) throws IOException {
@ -836,7 +831,6 @@ public class HStoreFile implements HConstants {
}
}
/** {@inheritDoc} */
@Override
public void append(WritableComparable key, Writable val)
throws IOException {
@ -846,7 +840,6 @@ public class HStoreFile implements HConstants {
super.append(key, val);
}
/** {@inheritDoc} */
@Override
public synchronized void close() throws IOException {
super.close();
@ -923,7 +916,6 @@ public class HStoreFile implements HConstants {
}
}
/** {@inheritDoc} */
@Override
public synchronized void finalKey(WritableComparable key)
throws IOException {
@ -943,7 +935,6 @@ public class HStoreFile implements HConstants {
}
}
/** {@inheritDoc} */
@Override
public synchronized Writable get(WritableComparable key, Writable val)
throws IOException {
@ -951,7 +942,6 @@ public class HStoreFile implements HConstants {
return super.get(key, val);
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override
public synchronized WritableComparable getClosest(WritableComparable key,
@ -983,7 +973,6 @@ public class HStoreFile implements HConstants {
return closest;
}
/** {@inheritDoc} */
@SuppressWarnings("unused")
@Override
public synchronized WritableComparable midKey() throws IOException {
@ -991,7 +980,6 @@ public class HStoreFile implements HConstants {
return null;
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override
public synchronized boolean next(WritableComparable key, Writable val)
@ -1017,7 +1005,6 @@ public class HStoreFile implements HConstants {
return result;
}
/** {@inheritDoc} */
@Override
public synchronized void reset() throws IOException {
if (top) {
@ -1028,7 +1015,6 @@ public class HStoreFile implements HConstants {
super.reset();
}
/** {@inheritDoc} */
@Override
public synchronized boolean seek(WritableComparable key)
throws IOException {

View File

@ -104,7 +104,6 @@ class HStoreScanner implements InternalScanner {
return multipleMatchers;
}
/** {@inheritDoc} */
public boolean next(HStoreKey key, SortedMap<byte [], Cell> results)
throws IOException {
@ -261,7 +260,6 @@ class HStoreScanner implements InternalScanner {
}
}
/** {@inheritDoc} */
public void close() {
for(int i = 0; i < scanners.length; i++) {
if(scanners[i] != null) {

View File

@ -48,7 +48,6 @@ class LogRoller extends Thread implements LogRollListener {
lastLogRollTime = System.currentTimeMillis();
}
/** {@inheritDoc} */
@Override
public void run() {
while (!server.isStopRequested()) {
@ -93,7 +92,6 @@ class LogRoller extends Thread implements LogRollListener {
LOG.info("LogRoller exiting.");
}
/** {@inheritDoc} */
public void logRollRequested() {
synchronized (rollLog) {
rollLog.set(true);

View File

@ -742,7 +742,6 @@ class Memcache {
}
}
/** {@inheritDoc} */
@Override
public boolean next(HStoreKey key, SortedMap<byte [], Cell> results)
throws IOException {
@ -806,7 +805,6 @@ class Memcache {
return results.size() > 0;
}
/** {@inheritDoc} */
public void close() {
if (!scannerClosed) {
scannerClosed = true;

View File

@ -24,11 +24,16 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
/**
* Thrown if request for nonexistent column family.
*/
@SuppressWarnings("serial")
public class NoSuchColumnFamilyException extends DoNotRetryIOException {
/** default constructor */
public NoSuchColumnFamilyException() {
super();
}
/**
* @param message
*/
public NoSuchColumnFamilyException(String message) {
super(message);
}

View File

@ -368,7 +368,6 @@ implements ChangedReadersObserver {
// Implementation of ChangedReadersObserver
/** {@inheritDoc} */
public void updateReaders() throws IOException {
this.lock.writeLock().lock();
try {

View File

@ -35,6 +35,10 @@ class CleanOldTransactionsChore extends Chore {
private final TransactionalRegionServer regionServer;
/**
* @param regionServer
* @param stopRequest
*/
public CleanOldTransactionsChore(
final TransactionalRegionServer regionServer,
final AtomicBoolean stopRequest) {

View File

@ -69,6 +69,9 @@ class TransactionState {
private Integer sequenceNumber;
boolean hasScan = false;
//TODO: Why don't these methods and the class itself use default access?
// They are only referenced from within this package.
public TransactionState(final long transactionId,
final long rLogStartSequenceId) {
this.transactionId = transactionId;

View File

@ -62,6 +62,9 @@ class TransactionalHLogManager {
private final HRegionInfo regionInfo;
private final HBaseConfiguration conf;
/**
* @param region
*/
public TransactionalHLogManager(final TransactionalRegion region) {
this.hlog = region.getLog();
this.fileSystem = region.getFilesystem();
@ -78,6 +81,10 @@ class TransactionalHLogManager {
this.conf = conf;
}
/**
* @param transactionId
* @throws IOException
*/
public void writeStartToLog(final long transactionId) throws IOException {
HLogEdit logEdit;
logEdit = new HLogEdit(transactionId, HLogEdit.TransactionalOperation.START);
@ -85,6 +92,11 @@ class TransactionalHLogManager {
hlog.append(regionInfo, logEdit);
}
/**
* @param transactionId
* @param update
* @throws IOException
*/
public void writeUpdateToLog(final long transactionId,
final BatchUpdate update) throws IOException {
@ -98,6 +110,10 @@ class TransactionalHLogManager {
}
}
/**
* @param transactionId
* @throws IOException
*/
public void writeCommitToLog(final long transactionId) throws IOException {
HLogEdit logEdit;
logEdit = new HLogEdit(transactionId,
@ -106,6 +122,10 @@ class TransactionalHLogManager {
hlog.append(regionInfo, logEdit);
}
/**
* @param transactionId
* @throws IOException
*/
public void writeAbortToLog(final long transactionId) throws IOException {
HLogEdit logEdit;
logEdit = new HLogEdit(transactionId, HLogEdit.TransactionalOperation.ABORT);
@ -113,6 +133,14 @@ class TransactionalHLogManager {
hlog.append(regionInfo, logEdit);
}
/**
* @param reconstructionLog
* @param maxSeqID
* @param reporter
* @return map of batch updates
* @throws UnsupportedEncodingException
* @throws IOException
*/
public Map<Long, List<BatchUpdate>> getCommitsFromLog(
final Path reconstructionLog, final long maxSeqID,
final Progressable reporter) throws UnsupportedEncodingException,

View File

@ -103,6 +103,14 @@ class TransactionalRegion extends HRegion {
private TransactionalHLogManager logManager;
private final int oldTransactionFlushTrigger;
/**
* @param basedir
* @param log
* @param fs
* @param conf
* @param regionInfo
* @param flushListener
*/
public TransactionalRegion(final Path basedir, final HLog log,
final FileSystem fs, final HBaseConfiguration conf,
final HRegionInfo regionInfo, final FlushRequester flushListener) {
@ -160,6 +168,10 @@ class TransactionalRegion extends HRegion {
return minPendingStartSequenceId;
}
/**
* @param transactionId
* @throws IOException
*/
public void beginTransaction(final long transactionId) throws IOException {
String key = String.valueOf(transactionId);
if (transactionsById.get(key) != null) {
@ -332,6 +344,7 @@ class TransactionalRegion extends HRegion {
/**
* Add a write to the transaction. Does not get applied until commit process.
*
* @param transactionId
* @param b
* @throws IOException
*/
@ -346,7 +359,9 @@ class TransactionalRegion extends HRegion {
* Add a delete to the transaction. Does not get applied until commit process.
* FIXME, not sure about this approach
*
* @param b
* @param transactionId
* @param row
* @param timestamp
* @throws IOException
*/
public void deleteAll(final long transactionId, final byte[] row,
@ -370,6 +385,11 @@ class TransactionalRegion extends HRegion {
}
/**
* @param transactionId
* @return true if commit is successful
* @throws IOException
*/
public boolean commitRequest(final long transactionId) throws IOException {
synchronized (commitCheckLock) {
TransactionState state = getTransactionState(transactionId);
@ -419,7 +439,6 @@ class TransactionalRegion extends HRegion {
* Commit the transaction.
*
* @param transactionId
* @return
* @throws IOException
*/
public void commit(final long transactionId) throws IOException {
@ -446,7 +465,6 @@ class TransactionalRegion extends HRegion {
* Commit the transaction.
*
* @param transactionId
* @return
* @throws IOException
*/
public void abort(final long transactionId) throws IOException {
@ -594,7 +612,6 @@ class TransactionalRegion extends HRegion {
this.transactionName = n;
}
/** {@inheritDoc} */
public void leaseExpired() {
LOG.info("Transaction " + this.transactionName + " lease expired");
TransactionState s = null;
@ -627,6 +644,10 @@ class TransactionalRegion extends HRegion {
private long transactionId;
private InternalScanner scanner;
/**
* @param transactionId
* @param scanner
*/
public ScannerWrapper(final long transactionId,
final InternalScanner scanner) {
this.transactionId = transactionId;

View File

@ -60,12 +60,21 @@ public class TransactionalRegionServer extends HRegionServer implements
private final CleanOldTransactionsChore cleanOldTransactionsThread;
/**
* @param conf
* @throws IOException
*/
public TransactionalRegionServer(final HBaseConfiguration conf)
throws IOException {
this(new HServerAddress(conf.get(REGIONSERVER_ADDRESS,
DEFAULT_REGIONSERVER_ADDRESS)), conf);
}
/**
* @param address
* @param conf
* @throws IOException
*/
public TransactionalRegionServer(final HServerAddress address,
final HBaseConfiguration conf) throws IOException {
super(address, conf);
@ -73,7 +82,6 @@ public class TransactionalRegionServer extends HRegionServer implements
super.stopRequested);
}
/** {@inheritDoc} */
@Override
public long getProtocolVersion(final String protocol, final long clientVersion)
throws IOException {
@ -241,7 +249,6 @@ public class TransactionalRegionServer extends HRegionServer implements
}
/** {@inheritDoc} */
public void deleteAll(final long transactionId, final byte[] regionName,
final byte[] row, final long timestamp) throws IOException {
checkOpen();

View File

@ -57,8 +57,10 @@ import org.mortbay.jetty.servlet.WebApplicationContext;
* </ul>
* @see <a href="http://wiki.apache.org/lucene-hadoop/Hbase/HbaseRest">Hbase REST Specification</a>
*/
@SuppressWarnings("serial")
public class Dispatcher extends javax.servlet.http.HttpServlet
implements javax.servlet.Servlet {
@SuppressWarnings("unused")
private static final Log LOG = LogFactory.getLog(Dispatcher.class.getName());
private MetaHandler metaHandler;
private TableHandler tableHandler;
@ -75,6 +77,7 @@ implements javax.servlet.Servlet {
super();
}
@Override
public void init() throws ServletException {
super.init();
@ -92,6 +95,7 @@ implements javax.servlet.Servlet {
}
}
@Override
protected void doGet(HttpServletRequest request, HttpServletResponse response)
throws IOException, ServletException {
String [] pathSegments = getPathSegments(request);
@ -112,6 +116,7 @@ implements javax.servlet.Servlet {
}
}
@Override
protected void doPost(HttpServletRequest request, HttpServletResponse response)
throws IOException, ServletException {
String [] pathSegments = getPathSegments(request);
@ -145,6 +150,7 @@ implements javax.servlet.Servlet {
}
@Override
protected void doPut(HttpServletRequest request, HttpServletResponse response)
throws ServletException, IOException {
String [] pathSegments = getPathSegments(request);
@ -158,6 +164,7 @@ implements javax.servlet.Servlet {
}
}
@Override
protected void doDelete(HttpServletRequest request,
HttpServletResponse response)
throws IOException, ServletException {

View File

@ -12,6 +12,11 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.io.WritableUtils;
/**
* Utility class that handles byte arrays, conversions to/from other types,
* comparisons, hash code generation, manufacturing keys for HashMaps or
* HashSets, etc.
*/
public class Bytes {
/**
* Size of long in bytes

View File

@ -43,7 +43,7 @@ public class JenkinsHash {
private static long BYTE_MASK = 0x00000000000000ffL;
private static long rot(long val, int pos) {
return ((long)(Integer.rotateLeft(
return ((Integer.rotateLeft(
(int)(val & INT_MASK), pos)) & INT_MASK);
}
@ -93,6 +93,7 @@ public class JenkinsHash {
* <p>Use for hash table lookup, or anything where one collision in 2^^32 is
* acceptable. Do NOT use for cryptographic purposes.
*/
@SuppressWarnings("fallthrough")
public static int hash(byte[] key, int nbytes, int initval) {
int length = nbytes;
long a, b, c; // We use longs because we don't have unsigned ints

View File

@ -70,7 +70,6 @@ public class Merge extends Configured implements Tool {
this.mergeInfo = null;
}
/** {@inheritDoc} */
public int run(String[] args) throws Exception {
if (parseArgs(args) != 0) {
return -1;
@ -164,7 +163,6 @@ public class Merge extends Configured implements Tool {
this.region2 = region2;
}
/** {@inheritDoc} */
public boolean processRow(HRegionInfo info) {
if (meta1 == null && HRegion.rowIsInRange(info, region1)) {
meta1 = info;

View File

@ -20,7 +20,7 @@
package org.apache.hadoop.hbase.util;
import java.io.FileNotFoundException;
import java.io.FileNotFoundException; //TODO: remove
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;

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