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:
Jim Kellerman 2007-08-22 23:59:30 +00:00
parent cf43495417
commit 1756a22f03
15 changed files with 62 additions and 58 deletions

View File

@ -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

View File

@ -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;

View File

@ -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+:");
/**

View File

@ -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,

View File

@ -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

View File

@ -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));

View File

@ -380,7 +380,6 @@ public class HMemcache {
}
/** Shut down map iterators, and release the lock */
@Override
public void close() {
if(! scannerClosed) {
try {

View File

@ -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();
}

View File

@ -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);

View File

@ -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]));
}
}
}

View File

@ -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 {

View File

@ -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);
}
}

View File

@ -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();
}

View File

@ -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,25 +37,33 @@ 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() {
@ -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;
}

View File

@ -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