HADOOP-1746 Clean up findbugs warnings
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@568776 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
cf43495417
commit
1756a22f03
|
@ -19,6 +19,7 @@ Trunk (unreleased changes)
|
|||
|
||||
IMPROVEMENTS
|
||||
HADOOP-1737 Make HColumnDescriptor data publically members settable
|
||||
HADOOP-1746 Clean up findbugs warnings
|
||||
|
||||
|
||||
Below are the list of changes before 2007-08-18
|
||||
|
|
|
@ -185,9 +185,6 @@ public abstract class HAbstractScanner implements HInternalScannerInterface {
|
|||
/** Mechanism used by concrete implementation to shut down a particular scanner */
|
||||
abstract void closeSubScanner(int i);
|
||||
|
||||
/** Mechanism used to shut down the whole scan */
|
||||
public abstract void close();
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public boolean isWildcardScanner() {
|
||||
return this.wildcardMatch;
|
||||
|
|
|
@ -37,7 +37,7 @@ public class HColumnDescriptor implements WritableComparable {
|
|||
// For future backward compatibility
|
||||
private static final byte COLUMN_DESCRIPTOR_VERSION = (byte)1;
|
||||
|
||||
// Legal family names can only contain 'word characters' and end in a colon.
|
||||
/** Legal family names can only contain 'word characters' and end in a colon. */
|
||||
public static final Pattern LEGAL_FAMILY_NAME = Pattern.compile("\\w+:");
|
||||
|
||||
/**
|
||||
|
|
|
@ -83,7 +83,7 @@ public class HLog implements HConstants {
|
|||
long filenum = 0;
|
||||
AtomicInteger numEntries = new AtomicInteger(0);
|
||||
|
||||
Integer rollLock = Integer.valueOf(0);
|
||||
Integer rollLock = new Integer(0);
|
||||
|
||||
/**
|
||||
* Split up a bunch of log files, that are no longer being written to,
|
||||
|
|
|
@ -34,7 +34,7 @@ public class HLogEdit implements Writable {
|
|||
private Text column = new Text();
|
||||
private byte [] val;
|
||||
private long timestamp;
|
||||
private final int MAX_VALUE_LEN = 128;
|
||||
private static final int MAX_VALUE_LEN = 128;
|
||||
|
||||
/**
|
||||
* Default constructor used by Writable
|
||||
|
|
|
@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.io.BatchUpdate;
|
|||
public class HMaster implements HConstants, HMasterInterface,
|
||||
HMasterRegionInterface, Runnable {
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public long getProtocolVersion(String protocol,
|
||||
@SuppressWarnings("unused") long clientVersion) throws IOException {
|
||||
|
||||
|
@ -322,7 +323,7 @@ HMasterRegionInterface, Runnable {
|
|||
LOG.warn("Deletion of " + parent.getRegionName() + " failed");
|
||||
}
|
||||
|
||||
BatchUpdate b = new BatchUpdate();
|
||||
BatchUpdate b = new BatchUpdate(rand.nextLong());
|
||||
long lockid = b.startUpdate(parent.getRegionName());
|
||||
b.delete(lockid, COL_REGIONINFO);
|
||||
b.delete(lockid, COL_SERVER);
|
||||
|
@ -388,7 +389,7 @@ HMasterRegionInterface, Runnable {
|
|||
+" no longer has references to " + parent.toString());
|
||||
}
|
||||
|
||||
BatchUpdate b = new BatchUpdate();
|
||||
BatchUpdate b = new BatchUpdate(rand.nextLong());
|
||||
long lockid = b.startUpdate(parent);
|
||||
b.delete(lockid, splitColumn);
|
||||
srvr.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
|
||||
|
@ -542,7 +543,7 @@ HMasterRegionInterface, Runnable {
|
|||
|
||||
private RootScanner rootScanner;
|
||||
private Thread rootScannerThread;
|
||||
Integer rootScannerLock = Integer.valueOf(0);
|
||||
Integer rootScannerLock = new Integer(0);
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
static class MetaRegion implements Comparable {
|
||||
|
@ -731,7 +732,7 @@ HMasterRegionInterface, Runnable {
|
|||
|
||||
MetaScanner metaScanner;
|
||||
private Thread metaScannerThread;
|
||||
Integer metaScannerLock = Integer.valueOf(0);
|
||||
Integer metaScannerLock = new Integer(0);
|
||||
|
||||
/**
|
||||
* The 'unassignedRegions' table maps from a region name to a HRegionInfo
|
||||
|
@ -938,6 +939,7 @@ HMasterRegionInterface, Runnable {
|
|||
Thread.currentThread().setName("HMaster");
|
||||
try {
|
||||
// Start things up
|
||||
this.serverLeases.start();
|
||||
this.rootScannerThread.start();
|
||||
this.metaScannerThread.start();
|
||||
|
||||
|
@ -1824,7 +1826,7 @@ HMasterRegionInterface, Runnable {
|
|||
// Remove server from root/meta entries
|
||||
|
||||
for (ToDoEntry e: toDoList) {
|
||||
BatchUpdate b = new BatchUpdate();
|
||||
BatchUpdate b = new BatchUpdate(rand.nextLong());
|
||||
long lockid = b.startUpdate(e.row);
|
||||
|
||||
if (e.deleteRegion) {
|
||||
|
@ -2094,7 +2096,7 @@ HMasterRegionInterface, Runnable {
|
|||
}
|
||||
|
||||
try {
|
||||
BatchUpdate b = new BatchUpdate();
|
||||
BatchUpdate b = new BatchUpdate(rand.nextLong());
|
||||
long lockid = b.startUpdate(regionInfo.regionName);
|
||||
|
||||
if (deleteRegion) {
|
||||
|
@ -2239,7 +2241,7 @@ HMasterRegionInterface, Runnable {
|
|||
serverAddress.toString());
|
||||
|
||||
try {
|
||||
BatchUpdate b = new BatchUpdate();
|
||||
BatchUpdate b = new BatchUpdate(rand.nextLong());
|
||||
long lockid = b.startUpdate(region.getRegionName());
|
||||
b.put(lockid, COL_SERVER,
|
||||
Writables.stringToBytes(serverAddress.toString()));
|
||||
|
@ -2404,7 +2406,7 @@ HMasterRegionInterface, Runnable {
|
|||
|
||||
HRegionInfo info = region.getRegionInfo();
|
||||
Text regionName = region.getRegionName();
|
||||
BatchUpdate b = new BatchUpdate();
|
||||
BatchUpdate b = new BatchUpdate(rand.nextLong());
|
||||
long lockid = b.startUpdate(regionName);
|
||||
b.put(lockid, COL_REGIONINFO, Writables.getBytes(info));
|
||||
server.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
|
||||
|
@ -2689,7 +2691,7 @@ HMasterRegionInterface, Runnable {
|
|||
LOG.debug("updating columns in row: " + i.regionName);
|
||||
}
|
||||
|
||||
BatchUpdate b = new BatchUpdate();
|
||||
BatchUpdate b = new BatchUpdate(rand.nextLong());
|
||||
lockid = b.startUpdate(i.regionName);
|
||||
updateRegionInfo(b, i);
|
||||
b.delete(lockid, COL_SERVER);
|
||||
|
@ -2846,7 +2848,7 @@ HMasterRegionInterface, Runnable {
|
|||
protected void updateRegionInfo(HRegionInterface server, Text regionName,
|
||||
HRegionInfo i) throws IOException {
|
||||
|
||||
BatchUpdate b = new BatchUpdate();
|
||||
BatchUpdate b = new BatchUpdate(rand.nextLong());
|
||||
long lockid = b.startUpdate(i.regionName);
|
||||
b.put(lockid, COL_REGIONINFO, Writables.getBytes(i));
|
||||
|
||||
|
|
|
@ -380,7 +380,6 @@ public class HMemcache {
|
|||
}
|
||||
|
||||
/** Shut down map iterators, and release the lock */
|
||||
@Override
|
||||
public void close() {
|
||||
if(! scannerClosed) {
|
||||
try {
|
||||
|
|
|
@ -74,7 +74,6 @@ import org.apache.hadoop.util.StringUtils;
|
|||
public class HRegion implements HConstants {
|
||||
static String SPLITDIR = "splits";
|
||||
static String MERGEDIR = "merges";
|
||||
static String TMPREGION_PREFIX = "tmpregion_";
|
||||
static final Random rand = new Random();
|
||||
static final Log LOG = LogFactory.getLog(HRegion.class);
|
||||
final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
|
@ -991,7 +990,7 @@ public class HRegion implements HConstants {
|
|||
storelist.add(stores.get(family));
|
||||
}
|
||||
return new HScanner(cols, firstRow, timestamp, memcache,
|
||||
storelist.toArray(new HStore [] {}), filter);
|
||||
storelist.toArray(new HStore [storelist.size()]), filter);
|
||||
} finally {
|
||||
lock.releaseReadLock();
|
||||
}
|
||||
|
|
|
@ -577,6 +577,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
logRollerThread.start();
|
||||
leases = new Leases(conf.getLong("hbase.regionserver.lease.period",
|
||||
3 * 60 * 1000), threadWakeFrequency);
|
||||
leases.start();
|
||||
|
||||
// Server
|
||||
|
||||
|
@ -1310,7 +1311,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
s = scanners.remove(scannerName);
|
||||
}
|
||||
if(s == null) {
|
||||
throw new UnknownScannerException(scannerName.toString());
|
||||
throw new UnknownScannerException(scannerName);
|
||||
}
|
||||
s.close();
|
||||
leases.cancelLease(scannerId, scannerId);
|
||||
|
|
|
@ -141,14 +141,14 @@ class HRegiondirReader {
|
|||
Text family = new Text(d.getName() + ":");
|
||||
families.add(family);
|
||||
}
|
||||
return families.toArray(new Text [] {});
|
||||
return families.toArray(new Text [families.size()]);
|
||||
}
|
||||
|
||||
List <HRegionInfo> getRegions() {
|
||||
return this.infos;
|
||||
}
|
||||
|
||||
HRegionInfo getRegionInfo(final String tableName) {
|
||||
HRegionInfo getRegionInfo(final Text tableName) {
|
||||
HRegionInfo result = null;
|
||||
for(HRegionInfo i: getRegions()) {
|
||||
if(i.tableDesc.getName().equals(tableName)) {
|
||||
|
@ -163,14 +163,15 @@ class HRegiondirReader {
|
|||
return result;
|
||||
}
|
||||
|
||||
private void dump(final String tableName) throws IOException {
|
||||
private void dump(final Text tableName) throws IOException {
|
||||
dump(getRegionInfo(tableName));
|
||||
}
|
||||
|
||||
private void dump(final HRegionInfo info) throws IOException {
|
||||
HRegion r = new HRegion(this.parentdir, null,
|
||||
FileSystem.get(this.conf), conf, info, null);
|
||||
Text [] families = info.tableDesc.families().keySet().toArray(new Text [] {});
|
||||
Text [] families = info.tableDesc.families().keySet().toArray(
|
||||
new Text [info.tableDesc.families.size()]);
|
||||
HInternalScannerInterface scanner =
|
||||
r.getScanner(families, new Text(), System.currentTimeMillis(), null);
|
||||
|
||||
|
@ -224,7 +225,7 @@ class HRegiondirReader {
|
|||
}
|
||||
} else {
|
||||
for (int i = 1; i < args.length; i++) {
|
||||
reader.dump(args[i]);
|
||||
reader.dump(new Text(args[i]));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -742,7 +742,7 @@ class HStore implements HConstants {
|
|||
}
|
||||
|
||||
/** A compaction reader for MapFile */
|
||||
class MapFileCompactionReader implements CompactionReader {
|
||||
static class MapFileCompactionReader implements CompactionReader {
|
||||
final MapFile.Reader reader;
|
||||
|
||||
MapFileCompactionReader(final MapFile.Reader r) {
|
||||
|
@ -1114,7 +1114,7 @@ class HStore implements HConstants {
|
|||
/*
|
||||
* Data structure to hold result of a look at store file sizes.
|
||||
*/
|
||||
class HStoreSize {
|
||||
static class HStoreSize {
|
||||
final long aggregate;
|
||||
final long largest;
|
||||
boolean splitable;
|
||||
|
@ -1361,7 +1361,6 @@ class HStore implements HConstants {
|
|||
}
|
||||
|
||||
/** Shut it down! */
|
||||
@Override
|
||||
public void close() {
|
||||
if(! scannerClosed) {
|
||||
try {
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Map;
|
|||
import java.util.Random;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.atomic.AtomicReferenceArray;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -52,7 +53,7 @@ public class HTable implements HConstants {
|
|||
protected final int numRetries;
|
||||
protected Random rand;
|
||||
protected volatile SortedMap<Text, HRegionLocation> tableServers;
|
||||
protected BatchUpdate batch;
|
||||
protected AtomicReference<BatchUpdate> batch;
|
||||
|
||||
// For row mutation operations
|
||||
|
||||
|
@ -79,7 +80,7 @@ public class HTable implements HConstants {
|
|||
this.numRetries = conf.getInt("hbase.client.retries.number", 5);
|
||||
this.rand = new Random();
|
||||
tableServers = connection.getTableServers(tableName);
|
||||
this.batch = null;
|
||||
this.batch = new AtomicReference<BatchUpdate>();
|
||||
closed = false;
|
||||
}
|
||||
|
||||
|
@ -113,7 +114,7 @@ public class HTable implements HConstants {
|
|||
public synchronized void close() {
|
||||
closed = true;
|
||||
tableServers = null;
|
||||
batch = null;
|
||||
batch.set(null);
|
||||
connection.close(tableName);
|
||||
}
|
||||
|
||||
|
@ -136,11 +137,11 @@ public class HTable implements HConstants {
|
|||
*/
|
||||
private void updateInProgress(boolean updateMustBeInProgress) {
|
||||
if (updateMustBeInProgress) {
|
||||
if (batch == null) {
|
||||
if (batch.get() == null) {
|
||||
throw new IllegalStateException("no update in progress");
|
||||
}
|
||||
} else {
|
||||
if (batch != null) {
|
||||
if (batch.get() != null) {
|
||||
throw new IllegalStateException("update in progress");
|
||||
}
|
||||
}
|
||||
|
@ -508,8 +509,8 @@ public class HTable implements HConstants {
|
|||
public synchronized long startUpdate(final Text row) {
|
||||
checkClosed();
|
||||
updateInProgress(false);
|
||||
batch = new BatchUpdate();
|
||||
return batch.startUpdate(row);
|
||||
batch.set(new BatchUpdate(rand.nextLong()));
|
||||
return batch.get().startUpdate(row);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -526,7 +527,7 @@ public class HTable implements HConstants {
|
|||
throw new IllegalArgumentException("value cannot be null");
|
||||
}
|
||||
updateInProgress(true);
|
||||
batch.put(lockid, column, val);
|
||||
batch.get().put(lockid, column, val);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -538,7 +539,7 @@ public class HTable implements HConstants {
|
|||
public void delete(long lockid, Text column) {
|
||||
checkClosed();
|
||||
updateInProgress(true);
|
||||
batch.delete(lockid, column);
|
||||
batch.get().delete(lockid, column);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -549,10 +550,10 @@ public class HTable implements HConstants {
|
|||
public synchronized void abort(long lockid) {
|
||||
checkClosed();
|
||||
updateInProgress(true);
|
||||
if (batch.getLockid() != lockid) {
|
||||
if (batch.get().getLockid() != lockid) {
|
||||
throw new IllegalArgumentException("invalid lock id " + lockid);
|
||||
}
|
||||
batch = null;
|
||||
batch.set(null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -577,18 +578,18 @@ public class HTable implements HConstants {
|
|||
|
||||
checkClosed();
|
||||
updateInProgress(true);
|
||||
if (batch.getLockid() != lockid) {
|
||||
if (batch.get().getLockid() != lockid) {
|
||||
throw new IllegalArgumentException("invalid lock id " + lockid);
|
||||
}
|
||||
|
||||
try {
|
||||
for (int tries = 0; tries < numRetries; tries++) {
|
||||
HRegionLocation r = getRegionLocation(batch.getRow());
|
||||
HRegionLocation r = getRegionLocation(batch.get().getRow());
|
||||
HRegionInterface server =
|
||||
connection.getHRegionConnection(r.getServerAddress());
|
||||
try {
|
||||
server.batchUpdate(r.getRegionInfo().getRegionName(), timestamp,
|
||||
batch);
|
||||
batch.get());
|
||||
break;
|
||||
} catch (IOException e) {
|
||||
if (e instanceof RemoteException) {
|
||||
|
@ -612,7 +613,7 @@ public class HTable implements HConstants {
|
|||
}
|
||||
}
|
||||
} finally {
|
||||
batch = null;
|
||||
batch.set(null);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -63,6 +63,10 @@ public class Leases {
|
|||
this.leaseMonitor = new LeaseMonitor();
|
||||
this.leaseMonitorThread = new Thread(leaseMonitor);
|
||||
this.leaseMonitorThread.setName("Lease.monitor");
|
||||
}
|
||||
|
||||
/** Starts the lease monitor */
|
||||
public void start() {
|
||||
leaseMonitorThread.start();
|
||||
}
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.io.DataOutput;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
@ -38,26 +37,34 @@ import org.apache.hadoop.io.Writable;
|
|||
*/
|
||||
public class BatchUpdate implements Writable, Iterable<BatchOperation> {
|
||||
|
||||
// used to generate lock ids
|
||||
private Random rand;
|
||||
|
||||
// the row being updated
|
||||
private Text row;
|
||||
|
||||
// the lockid
|
||||
private long lockid;
|
||||
// the lockid - not used on server side
|
||||
private transient long lockid;
|
||||
|
||||
// the batched operations
|
||||
private ArrayList<BatchOperation> operations;
|
||||
|
||||
/** constructor */
|
||||
/** Default constructor - used by Writable. */
|
||||
public BatchUpdate() {
|
||||
this.rand = new Random();
|
||||
this.row = new Text();
|
||||
this.lockid = -1L;
|
||||
this.operations = new ArrayList<BatchOperation>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Client side constructor. Clients need to provide the lockid by some means
|
||||
* such as Random.nextLong()
|
||||
*
|
||||
* @param lockid
|
||||
*/
|
||||
public BatchUpdate(long lockid) {
|
||||
this.row = new Text();
|
||||
this.lockid = Long.valueOf(Math.abs(lockid));
|
||||
this.operations = new ArrayList<BatchOperation>();
|
||||
}
|
||||
|
||||
/** @return the lock id */
|
||||
public long getLockid() {
|
||||
return lockid;
|
||||
|
@ -84,7 +91,6 @@ public class BatchUpdate implements Writable, Iterable<BatchOperation> {
|
|||
*/
|
||||
public synchronized long startUpdate(final Text row) {
|
||||
this.row = row;
|
||||
this.lockid = Long.valueOf(Math.abs(rand.nextLong()));
|
||||
return this.lockid;
|
||||
}
|
||||
|
||||
|
|
|
@ -71,12 +71,6 @@ public abstract class TableMap extends MapReduceBase implements Mapper {
|
|||
job.set(TableInputFormat.COLUMN_LIST, columns);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public void configure(JobConf job) {
|
||||
super.configure(job);
|
||||
}
|
||||
|
||||
/**
|
||||
* Input:
|
||||
* @param key is of type HStoreKey
|
||||
|
|
Loading…
Reference in New Issue