diff --git a/src/java/org/apache/hadoop/hbase/HAbstractScanner.java b/src/java/org/apache/hadoop/hbase/HAbstractScanner.java index e04c9ac76c2..a3b59bf12ce 100644 --- a/src/java/org/apache/hadoop/hbase/HAbstractScanner.java +++ b/src/java/org/apache/hadoop/hbase/HAbstractScanner.java @@ -180,7 +180,7 @@ public abstract class HAbstractScanner implements HScannerInterface { * @see org.apache.hadoop.hbase.HScannerInterface#next(org.apache.hadoop.hbase.HStoreKey, java.util.TreeMap) */ public boolean next(HStoreKey key, TreeMap results) - throws IOException { + throws IOException { // Find the next row label (and timestamp) @@ -188,12 +188,12 @@ public abstract class HAbstractScanner implements HScannerInterface { long chosenTimestamp = -1; for(int i = 0; i < keys.length; i++) { while((keys[i] != null) - && (columnMatch(i)) - && (keys[i].getTimestamp() <= this.timestamp) - && ((chosenRow == null) - || (keys[i].getRow().compareTo(chosenRow) < 0) - || ((keys[i].getRow().compareTo(chosenRow) == 0) - && (keys[i].getTimestamp() > chosenTimestamp)))) { + && (columnMatch(i)) + && (keys[i].getTimestamp() <= this.timestamp) + && ((chosenRow == null) + || (keys[i].getRow().compareTo(chosenRow) < 0) + || ((keys[i].getRow().compareTo(chosenRow) == 0) + && (keys[i].getTimestamp() > chosenTimestamp)))) { chosenRow = new Text(keys[i].getRow()); chosenTimestamp = keys[i].getTimestamp(); @@ -212,8 +212,8 @@ public abstract class HAbstractScanner implements HScannerInterface { // Fetch the data while((keys[i] != null) - && (keys[i].getRow().compareTo(chosenRow) == 0) - && (keys[i].getTimestamp() == chosenTimestamp)) { + && (keys[i].getRow().compareTo(chosenRow) == 0) + && (keys[i].getTimestamp() == chosenTimestamp)) { if(columnMatch(i)) { outbuf.reset(); @@ -235,9 +235,9 @@ public abstract class HAbstractScanner implements HScannerInterface { // a valid timestamp, so we're ready next time. while((keys[i] != null) - && ((keys[i].getRow().compareTo(chosenRow) <= 0) - || (keys[i].getTimestamp() > this.timestamp) - || (! columnMatch(i)))) { + && ((keys[i].getRow().compareTo(chosenRow) <= 0) + || (keys[i].getTimestamp() > this.timestamp) + || (! columnMatch(i)))) { getNext(i); } diff --git a/src/java/org/apache/hadoop/hbase/HClient.java b/src/java/org/apache/hadoop/hbase/HClient.java index 51e9fc03d28..a78be96c4e1 100644 --- a/src/java/org/apache/hadoop/hbase/HClient.java +++ b/src/java/org/apache/hadoop/hbase/HClient.java @@ -146,8 +146,8 @@ public class HClient extends HGlobals implements HConstants { private void locateRootRegion() throws IOException { if(master == null) { master = (HMasterInterface)RPC.getProxy(HMasterInterface.class, - HMasterInterface.versionID, - masterLocation.getInetSocketAddress(), conf); + HMasterInterface.versionID, + masterLocation.getInetSocketAddress(), conf); } int tries = 0; @@ -229,7 +229,7 @@ public class HClient extends HGlobals implements HConstants { String serverName = new String(serverBytes, UTF8_ENCODING); tableServers.put(regionInfo.startKey, - new TableInfo(regionInfo, new HServerAddress(serverName))); + new TableInfo(regionInfo, new HServerAddress(serverName))); results.clear(); } @@ -239,16 +239,16 @@ public class HClient extends HGlobals implements HConstants { } public synchronized HRegionInterface getHRegionConnection(HServerAddress regionServer) - throws IOException { + throws IOException { - // See if we already have a connection + // See if we already have a connection HRegionInterface server = servers.get(regionServer.toString()); if(server == null) { // Get a connection server = (HRegionInterface)RPC.waitForProxy(HRegionInterface.class, - HRegionInterface.versionID, regionServer.getInetSocketAddress(), conf); + HRegionInterface.versionID, regionServer.getInetSocketAddress(), conf); servers.put(regionServer.toString(), server); } @@ -325,14 +325,14 @@ public class HClient extends HGlobals implements HConstants { public byte[] get(Text row, Text column) throws IOException { TableInfo info = getTableInfo(row); return getHRegionConnection(info.serverAddress).get( - info.regionInfo.regionName, row, column).get(); + info.regionInfo.regionName, row, column).get(); } /** Get the specified number of versions of the specified row and column */ public byte[][] get(Text row, Text column, int numVersions) throws IOException { TableInfo info = getTableInfo(row); BytesWritable[] values = getHRegionConnection(info.serverAddress).get( - info.regionInfo.regionName, row, column, numVersions); + info.regionInfo.regionName, row, column, numVersions); ArrayList bytes = new ArrayList(); for(int i = 0 ; i < values.length; i++) { @@ -348,7 +348,7 @@ public class HClient extends HGlobals implements HConstants { public byte[][] get(Text row, Text column, long timestamp, int numVersions) throws IOException { TableInfo info = getTableInfo(row); BytesWritable[] values = getHRegionConnection(info.serverAddress).get( - info.regionInfo.regionName, row, column, timestamp, numVersions); + info.regionInfo.regionName, row, column, timestamp, numVersions); ArrayList bytes = new ArrayList(); for(int i = 0 ; i < values.length; i++) { @@ -361,7 +361,7 @@ public class HClient extends HGlobals implements HConstants { public LabelledData[] getRow(Text row) throws IOException { TableInfo info = getTableInfo(row); return getHRegionConnection(info.serverAddress).getRow( - info.regionInfo.regionName, row); + info.regionInfo.regionName, row); } /** @@ -492,7 +492,7 @@ public class HClient extends HGlobals implements HConstants { try { server = getHRegionConnection(regions[currentRegion].serverAddress); scanner = server.openScanner(regions[currentRegion].regionInfo.regionName, - columns, startRow); + columns, startRow); } catch(IOException e) { close(); diff --git a/src/java/org/apache/hadoop/hbase/HLog.java b/src/java/org/apache/hadoop/hbase/HLog.java index 9576dde1786..25f89aeb389 100644 --- a/src/java/org/apache/hadoop/hbase/HLog.java +++ b/src/java/org/apache/hadoop/hbase/HLog.java @@ -342,7 +342,7 @@ public class HLog { } writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId), - new HLogEdit(HLog.METACOLUMN, HStoreKey.COMPLETE_CACHEFLUSH, System.currentTimeMillis())); + new HLogEdit(HLog.METACOLUMN, HStoreKey.COMPLETE_CACHEFLUSH, System.currentTimeMillis())); numEntries++; // Remember the most-recent flush for each region. diff --git a/src/java/org/apache/hadoop/hbase/HMaster.java b/src/java/org/apache/hadoop/hbase/HMaster.java index 165250b8f7a..13b8aa6e97f 100644 --- a/src/java/org/apache/hadoop/hbase/HMaster.java +++ b/src/java/org/apache/hadoop/hbase/HMaster.java @@ -104,7 +104,7 @@ public class HMaster extends HGlobals public void run() { Text cols[] = { - ROOT_COLUMN_FAMILY + ROOT_COLUMN_FAMILY }; Text firstRow = new Text(); @@ -156,7 +156,7 @@ public class HMaster extends HGlobals synchronized(serversToServerInfo) { storedInfo = serversToServerInfo.get(serverName); if(storedInfo == null - || storedInfo.getStartCode() != startCode) { + || storedInfo.getStartCode() != startCode) { // The current assignment is no good; load the region. @@ -216,7 +216,7 @@ public class HMaster extends HGlobals */ private class MetaScanner implements Runnable { private final Text cols[] = { - META_COLUMN_FAMILY + META_COLUMN_FAMILY }; private final Text firstRow = new Text(); @@ -262,7 +262,7 @@ public class HMaster extends HGlobals synchronized(serversToServerInfo) { storedInfo = serversToServerInfo.get(serverName); if(storedInfo == null - || storedInfo.getStartCode() != startCode) { + || storedInfo.getStartCode() != startCode) { // The current assignment is no good; load the region. @@ -370,8 +370,8 @@ public class HMaster extends HGlobals /** Build the HMaster out of a raw configuration item. */ public HMaster(Configuration conf) throws IOException { this(new Path(conf.get(HREGION_DIR, DEFAULT_HREGION_DIR)), - new HServerAddress(conf.get(MASTER_DEFAULT_NAME)), - conf); + new HServerAddress(conf.get(MASTER_DEFAULT_NAME)), + conf); } /** @@ -410,9 +410,9 @@ public class HMaster extends HGlobals this.maxRegionOpenTime = conf.getLong("hbase.hbasemaster.maxregionopen", 30 * 1000); this.msgQueue = new Vector(); this.serverLeases = new Leases(conf.getLong("hbase.master.lease.period", 15 * 1000), - conf.getLong("hbase.master.lease.thread.wakefrequency", 15 * 1000)); + conf.getLong("hbase.master.lease.thread.wakefrequency", 15 * 1000)); this.server = RPC.getServer(this, address.getBindAddress(), - address.getPort(), conf.getInt("hbase.hregionserver.handler.count", 10), false, conf); + address.getPort(), conf.getInt("hbase.hregionserver.handler.count", 10), false, conf); this.client = new HClient(conf); this.metaRescanInterval @@ -714,7 +714,7 @@ public class HMaster extends HGlobals default: throw new IOException("Impossible state during msg processing. Instruction: " - + incomingMsgs[i].getMsg()); + + incomingMsgs[i].getMsg()); } } @@ -725,13 +725,13 @@ public class HMaster extends HGlobals // Open new regions as necessary int targetForServer = (int) Math.ceil(unassignedRegions.size() - / (1.0 * serversToServerInfo.size())); + / (1.0 * serversToServerInfo.size())); int counter = 0; long now = System.currentTimeMillis(); for(Iterator it = unassignedRegions.keySet().iterator(); - it.hasNext(); ) { + it.hasNext(); ) { Text curRegionName = it.next(); HRegionInfo regionInfo = unassignedRegions.get(curRegionName); @@ -790,7 +790,7 @@ public class HMaster extends HGlobals abstract class PendingOperation { protected final Text[] columns = { - META_COLUMN_FAMILY + META_COLUMN_FAMILY }; protected final Text startRow = new Text(); protected long clientId; @@ -813,7 +813,7 @@ public class HMaster extends HGlobals } private void scanMetaRegion(HRegionInterface server, HScannerInterface scanner, - Text regionName) throws IOException { + Text regionName) throws IOException { Vector toDoList = new Vector(); TreeMap regions = new TreeMap(); @@ -899,7 +899,7 @@ public class HMaster extends HGlobals HRegionInterface server = client.getHRegionConnection(rootRegionLocation); HScannerInterface scanner = server.openScanner(rootRegionInfo.regionName, - columns, startRow); + columns, startRow); scanMetaRegion(server, scanner, rootRegionInfo.regionName); for(Iterator i = knownMetaRegions.values().iterator(); @@ -1003,10 +1003,10 @@ public class HMaster extends HGlobals try { this.serverAddress = new BytesWritable( - info.getServerAddress().toString().getBytes(UTF8_ENCODING)); + info.getServerAddress().toString().getBytes(UTF8_ENCODING)); this.startCode = new BytesWritable( - String.valueOf(info.getStartCode()).getBytes(UTF8_ENCODING)); + String.valueOf(info.getStartCode()).getBytes(UTF8_ENCODING)); } catch(UnsupportedEncodingException e) { } @@ -1100,7 +1100,7 @@ public class HMaster extends HGlobals long clientId = rand.nextLong(); long lockid = server.startUpdate(metaRegionName, clientId, regionName); server.put(metaRegionName, clientId, lockid, META_COL_REGIONINFO, - new BytesWritable(byteValue.toByteArray())); + new BytesWritable(byteValue.toByteArray())); server.commit(metaRegionName, clientId, lockid); // 4. Get it assigned to a server @@ -1122,14 +1122,14 @@ public class HMaster extends HGlobals * @throws IOException */ private HRegion createNewHRegion(HTableDescriptor desc, long regionId) - throws IOException { + throws IOException { HRegionInfo info = new HRegionInfo(regionId, desc, null, null); Path regionDir = HStoreFile.getHRegionDir(dir, info.regionName); fs.mkdirs(regionDir); return new HRegion(dir, new HLog(fs, new Path(regionDir, "log"), conf), fs, - conf, info, null, null); + conf, info, null, null); } /** @@ -1168,7 +1168,7 @@ public class HMaster extends HGlobals public void deleteTable(Text tableName) throws IOException { Text[] columns = { - META_COLUMN_FAMILY + META_COLUMN_FAMILY }; // We can not access any meta region if they have not already been assigned diff --git a/src/java/org/apache/hadoop/hbase/HMemcache.java b/src/java/org/apache/hadoop/hbase/HMemcache.java index 8766b924764..e0ce64c189b 100644 --- a/src/java/org/apache/hadoop/hbase/HMemcache.java +++ b/src/java/org/apache/hadoop/hbase/HMemcache.java @@ -31,10 +31,10 @@ public class HMemcache { private static final Log LOG = LogFactory.getLog(HMemcache.class); TreeMap memcache - = new TreeMap(); + = new TreeMap(); Vector> history - = new Vector>(); + = new Vector>(); TreeMap snapshot = null; @@ -199,7 +199,7 @@ public class HMemcache { } void internalGetFull(TreeMap map, HStoreKey key, - TreeMap results) { + TreeMap results) { SortedMap tailMap = map.tailMap(key); @@ -208,7 +208,7 @@ public class HMemcache { Text itCol = itKey.getColumn(); if(results.get(itCol) == null - && key.matchesWithoutColumn(itKey)) { + && key.matchesWithoutColumn(itKey)) { BytesWritable val = tailMap.get(itKey); results.put(itCol, val.get()); @@ -251,7 +251,7 @@ public class HMemcache { * Return a scanner over the keys in the HMemcache */ public HScannerInterface getScanner(long timestamp, Text targetCols[], Text firstRow) - throws IOException { + throws IOException { return new HMemcacheScanner(timestamp, targetCols, firstRow); } @@ -266,8 +266,8 @@ public class HMemcache { Iterator keyIterators[]; @SuppressWarnings("unchecked") - public HMemcacheScanner(long timestamp, Text targetCols[], Text firstRow) - throws IOException { + public HMemcacheScanner(long timestamp, Text targetCols[], Text firstRow) + throws IOException { super(timestamp, targetCols); diff --git a/src/java/org/apache/hadoop/hbase/HMsg.java b/src/java/org/apache/hadoop/hbase/HMsg.java index 044a85395fd..cc7e4a58dde 100644 --- a/src/java/org/apache/hadoop/hbase/HMsg.java +++ b/src/java/org/apache/hadoop/hbase/HMsg.java @@ -67,13 +67,13 @@ public class HMsg implements Writable { // Writable ////////////////////////////////////////////////////////////////////////////// - public void write(DataOutput out) throws IOException { - out.writeByte(msg); - info.write(out); - } + public void write(DataOutput out) throws IOException { + out.writeByte(msg); + info.write(out); + } - public void readFields(DataInput in) throws IOException { - this.msg = in.readByte(); - this.info.readFields(in); - } + public void readFields(DataInput in) throws IOException { + this.msg = in.readByte(); + this.info.readFields(in); + } } diff --git a/src/java/org/apache/hadoop/hbase/HRegionInfo.java b/src/java/org/apache/hadoop/hbase/HRegionInfo.java index bc0a22c17f6..78d0b2733d8 100644 --- a/src/java/org/apache/hadoop/hbase/HRegionInfo.java +++ b/src/java/org/apache/hadoop/hbase/HRegionInfo.java @@ -38,7 +38,7 @@ public class HRegionInfo implements Writable { } public HRegionInfo(long regionId, HTableDescriptor tableDesc, Text startKey, - Text endKey) throws IllegalArgumentException { + Text endKey) throws IllegalArgumentException { this.regionId = regionId; @@ -59,7 +59,7 @@ public class HRegionInfo implements Writable { } this.regionName = new Text(tableDesc.getName() + "_" - + (startKey == null ? "" : startKey.toString()) + "_" + regionId); + + (startKey == null ? "" : startKey.toString()) + "_" + regionId); } ////////////////////////////////////////////////////////////////////////////// diff --git a/src/java/org/apache/hadoop/hbase/HRegionServer.java b/src/java/org/apache/hadoop/hbase/HRegionServer.java index e9f9c830a2b..59c9143d146 100644 --- a/src/java/org/apache/hadoop/hbase/HRegionServer.java +++ b/src/java/org/apache/hadoop/hbase/HRegionServer.java @@ -111,7 +111,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { Text tableToUpdate = (oldRegion.find(META_TABLE_NAME.toString()) == 0) - ? ROOT_TABLE_NAME : META_TABLE_NAME; + ? ROOT_TABLE_NAME : META_TABLE_NAME; client.openTable(tableToUpdate); long lockid = client.startUpdate(oldRegion); @@ -249,13 +249,13 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { /** Start a HRegionServer at the default location */ public HRegionServer(Configuration conf) throws IOException { this(new Path(conf.get(HREGION_DIR, DEFAULT_HREGION_DIR)), - new HServerAddress(conf.get("hbase.regionserver.default.name")), - conf); + new HServerAddress(conf.get("hbase.regionserver.default.name")), + conf); } /** Start a HRegionServer at an indicated location */ public HRegionServer(Path regionDir, HServerAddress address, Configuration conf) - throws IOException { + throws IOException { // Basic setup @@ -302,10 +302,10 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { // Remote HMaster this.hbaseMaster = (HMasterRegionInterface) - RPC.waitForProxy(HMasterRegionInterface.class, - HMasterRegionInterface.versionId, - new HServerAddress(conf.get(MASTER_DEFAULT_NAME)).getInetSocketAddress(), - conf); + RPC.waitForProxy(HMasterRegionInterface.class, + HMasterRegionInterface.versionId, + new HServerAddress(conf.get(MASTER_DEFAULT_NAME)).getInetSocketAddress(), + conf); // Threads @@ -313,12 +313,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { this.splitCheckerThread.start(); this.logRollerThread.start(); this.leases = new Leases(conf.getLong("hbase.hregionserver.lease.period", - 3 * 60 * 1000), threadWakeFrequency); + 3 * 60 * 1000), threadWakeFrequency); // Server this.server = RPC.getServer(this, address.getBindAddress().toString(), - address.getPort(), conf.getInt("hbase.hregionserver.handler.count", 10), false, conf); + address.getPort(), conf.getInt("hbase.hregionserver.handler.count", 10), false, conf); this.server.start(); } catch(IOException e) { @@ -523,7 +523,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } private void closeRegion(HRegionInfo info, boolean reportWhenCompleted) - throws IOException { + throws IOException { locking.obtainWriteLock(); try { @@ -580,24 +580,24 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { * * For now, we do not do merging. Splits are driven by the HRegionServer. ****************************************************************************/ -/* - private void mergeRegions(Text regionNameA, Text regionNameB) throws IOException { + /* + private void mergeRegions(Text regionNameA, Text regionNameB) throws IOException { locking.obtainWriteLock(); try { - HRegion srcA = regions.remove(regionNameA); - HRegion srcB = regions.remove(regionNameB); - HRegion newRegion = HRegion.closeAndMerge(srcA, srcB); - regions.put(newRegion.getRegionName(), newRegion); + HRegion srcA = regions.remove(regionNameA); + HRegion srcB = regions.remove(regionNameB); + HRegion newRegion = HRegion.closeAndMerge(srcA, srcB); + regions.put(newRegion.getRegionName(), newRegion); - reportClose(srcA); - reportClose(srcB); - reportOpen(newRegion); + reportClose(srcA); + reportClose(srcB); + reportOpen(newRegion); } finally { - locking.releaseWriteLock(); + locking.releaseWriteLock(); } - } -*/ + } + */ ////////////////////////////////////////////////////////////////////////////// // HRegionInterface @@ -614,7 +614,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { /** Start a scanner for a given HRegion. */ public HScannerInterface openScanner(Text regionName, Text[] cols, - Text firstRow) throws IOException { + Text firstRow) throws IOException { HRegion r = getRegion(regionName); if(r == null) { @@ -639,7 +639,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { /** Get multiple versions of the indicated row/col */ public BytesWritable[] get(Text regionName, Text row, Text column, - int numVersions) throws IOException { + int numVersions) throws IOException { HRegion region = getRegion(regionName); if(region == null) { @@ -661,7 +661,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { /** Get multiple timestamped versions of the indicated row/col */ public BytesWritable[] get(Text regionName, Text row, Text column, - long timestamp, int numVersions) throws IOException { + long timestamp, int numVersions) throws IOException { HRegion region = getRegion(regionName); if(region == null) { @@ -723,7 +723,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } public long startUpdate(Text regionName, long clientid, Text row) - throws IOException { + throws IOException { HRegion region = getRegion(regionName); if(region == null) { @@ -732,15 +732,15 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { long lockid = region.startUpdate(row); leases.createLease(new Text(String.valueOf(clientid)), - new Text(String.valueOf(lockid)), - new RegionListener(region, lockid)); + new Text(String.valueOf(lockid)), + new RegionListener(region, lockid)); return lockid; } /** Add something to the HBase. */ public void put(Text regionName, long clientid, long lockid, Text column, - BytesWritable val) throws IOException { + BytesWritable val) throws IOException { HRegion region = getRegion(regionName); if(region == null) { @@ -748,14 +748,14 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } leases.renewLease(new Text(String.valueOf(clientid)), - new Text(String.valueOf(lockid))); + new Text(String.valueOf(lockid))); region.put(lockid, column, val.get()); } /** Remove a cell from the HBase. */ public void delete(Text regionName, long clientid, long lockid, Text column) - throws IOException { + throws IOException { HRegion region = getRegion(regionName); if(region == null) { @@ -763,14 +763,14 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } leases.renewLease(new Text(String.valueOf(clientid)), - new Text(String.valueOf(lockid))); + new Text(String.valueOf(lockid))); region.delete(lockid, column); } /** Abandon the transaction */ public void abort(Text regionName, long clientid, long lockid) - throws IOException { + throws IOException { HRegion region = getRegion(regionName); if(region == null) { @@ -778,14 +778,14 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } leases.cancelLease(new Text(String.valueOf(clientid)), - new Text(String.valueOf(lockid))); + new Text(String.valueOf(lockid))); region.abort(lockid); } /** Confirm the transaction */ public void commit(Text regionName, long clientid, long lockid) - throws IOException { + throws IOException { HRegion region = getRegion(regionName); if(region == null) { @@ -793,7 +793,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } leases.cancelLease(new Text(String.valueOf(clientid)), - new Text(String.valueOf(lockid))); + new Text(String.valueOf(lockid))); region.commit(lockid); } @@ -801,7 +801,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { /** Don't let the client's lease expire just yet... */ public void renewLease(long lockid, long clientid) throws IOException { leases.renewLease(new Text(String.valueOf(clientid)), - new Text(String.valueOf(lockid))); + new Text(String.valueOf(lockid))); } /** Private utility method for safely obtaining an HRegion handle. */ diff --git a/src/java/org/apache/hadoop/hbase/HStore.java b/src/java/org/apache/hadoop/hbase/HStore.java index cc064e8cbf3..9ba150d654f 100644 --- a/src/java/org/apache/hadoop/hbase/HStore.java +++ b/src/java/org/apache/hadoop/hbase/HStore.java @@ -88,7 +88,7 @@ public class HStore { * will be deleted (by whoever has instantiated the HStore). */ public HStore(Path dir, Text regionName, Text colFamily, int maxVersions, - FileSystem fs, Path reconstructionLog, Configuration conf) throws IOException { + FileSystem fs, Path reconstructionLog, Configuration conf) throws IOException { this.dir = dir; this.regionName = regionName; @@ -174,7 +174,7 @@ public class HStore { continue; } reconstructedCache.put(new HStoreKey(key.getRow(), val.getColumn(), - val.getTimestamp()), val.getVal()); + val.getTimestamp()), val.getVal()); } } finally { @@ -252,13 +252,13 @@ public class HStore { * Return the entire list of HStoreFiles currently used by the HStore. */ public Vector flushCache(TreeMap inputCache, - long logCacheFlushId) throws IOException { + long logCacheFlushId) throws IOException { return flushCacheHelper(inputCache, logCacheFlushId, true); } Vector flushCacheHelper(TreeMap inputCache, - long logCacheFlushId, boolean addToAvailableMaps) throws IOException { + long logCacheFlushId, boolean addToAvailableMaps) throws IOException { synchronized(flushLock) { LOG.debug("flushing HStore " + this.regionName + "/" + this.colFamily); @@ -270,7 +270,7 @@ public class HStore { Path mapfile = flushedFile.getMapFilePath(); MapFile.Writer out = new MapFile.Writer(conf, fs, mapfile.toString(), - HStoreKey.class, BytesWritable.class); + HStoreKey.class, BytesWritable.class); try { for(Iterator it = inputCache.keySet().iterator(); it.hasNext(); ) { @@ -392,8 +392,8 @@ public class HStore { // Step through them, writing to the brand-new TreeMap MapFile.Writer compactedOut = new MapFile.Writer(conf, fs, - compactedOutputFile.getMapFilePath().toString(), HStoreKey.class, - BytesWritable.class); + compactedOutputFile.getMapFilePath().toString(), HStoreKey.class, + BytesWritable.class); try { @@ -464,7 +464,7 @@ public class HStore { HStoreKey sk = keys[smallestKey]; if(lastRow.equals(sk.getRow()) - && lastColumn.equals(sk.getColumn())) { + && lastColumn.equals(sk.getColumn())) { timesSeen++; @@ -478,7 +478,7 @@ public class HStore { // Then just skip them. if(sk.getRow().getLength() != 0 - && sk.getColumn().getLength() != 0) { + && sk.getColumn().getLength() != 0) { // Only write out objects which have a non-zero length key and value @@ -683,7 +683,7 @@ public class HStore { mapFiles.put(orderVal, finalCompactedFile); maps.put(orderVal, new MapFile.Reader(fs, - finalCompactedFile.getMapFilePath().toString(), conf)); + finalCompactedFile.getMapFilePath().toString(), conf)); } finally { @@ -721,7 +721,7 @@ public class HStore { do { Text readcol = readkey.getColumn(); if(results.get(readcol) == null - && key.matchesWithoutColumn(readkey)) { + && key.matchesWithoutColumn(readkey)) { results.put(new Text(readcol), readval.get()); readval = new BytesWritable(); @@ -850,7 +850,7 @@ public class HStore { * These should be closed after the user is done with them. */ public HScannerInterface getScanner(long timestamp, Text targetCols[], - Text firstRow) throws IOException { + Text firstRow) throws IOException { return new HStoreScanner(timestamp, targetCols, firstRow); } diff --git a/src/java/org/apache/hadoop/hbase/HStoreFile.java b/src/java/org/apache/hadoop/hbase/HStoreFile.java index 4e462466309..6d5c865a3f2 100644 --- a/src/java/org/apache/hadoop/hbase/HStoreFile.java +++ b/src/java/org/apache/hadoop/hbase/HStoreFile.java @@ -61,7 +61,7 @@ public class HStoreFile implements HConstants, WritableComparable { } public HStoreFile(Configuration conf, Path dir, Text regionName, - Text colFamily, long fileId) { + Text colFamily, long fileId) { this.conf = conf; this.dir = dir; @@ -92,12 +92,12 @@ public class HStoreFile implements HConstants, WritableComparable { public Path getMapFilePath() { return new Path(HStoreFile.getMapDir(dir, regionName, colFamily), - HSTORE_DATFILE_PREFIX + fileId); + HSTORE_DATFILE_PREFIX + fileId); } public Path getInfoFilePath() { return new Path(HStoreFile.getInfoDir(dir, regionName, colFamily), - HSTORE_INFOFILE_PREFIX + fileId); + HSTORE_INFOFILE_PREFIX + fileId); } // Static methods to build partial paths to internal directories. Useful for @@ -105,17 +105,17 @@ public class HStoreFile implements HConstants, WritableComparable { public static Path getMapDir(Path dir, Text regionName, Text colFamily) { return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, - new Path(colFamily.toString(), HSTORE_DATFILE_DIR))); + new Path(colFamily.toString(), HSTORE_DATFILE_DIR))); } public static Path getInfoDir(Path dir, Text regionName, Text colFamily) { return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, - new Path(colFamily.toString(), HSTORE_INFO_DIR))); + new Path(colFamily.toString(), HSTORE_INFO_DIR))); } public static Path getHStoreDir(Path dir, Text regionName, Text colFamily) { return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, - colFamily.toString())); + colFamily.toString())); } public static Path getHRegionDir(Path dir, Text regionName) { @@ -127,7 +127,7 @@ public class HStoreFile implements HConstants, WritableComparable { * filesystem if the file already exists. */ static HStoreFile obtainNewHStoreFile(Configuration conf, Path dir, - Text regionName, Text colFamily, FileSystem fs) throws IOException { + Text regionName, Text colFamily, FileSystem fs) throws IOException { Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily); long fileId = Math.abs(rand.nextLong()); @@ -149,7 +149,7 @@ public class HStoreFile implements HConstants, WritableComparable { * If only one exists, we'll delete it. */ static Vector loadHStoreFiles(Configuration conf, Path dir, - Text regionName, Text colFamily, FileSystem fs) throws IOException { + Text regionName, Text colFamily, FileSystem fs) throws IOException { Vector results = new Vector(); Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily); @@ -200,18 +200,18 @@ public class HStoreFile implements HConstants, WritableComparable { * brand-new HRegions. */ public void splitStoreFile(Text midKey, HStoreFile dstA, HStoreFile dstB, - FileSystem fs, Configuration conf) throws IOException { + FileSystem fs, Configuration conf) throws IOException { // Copy the appropriate tuples to one MapFile or the other. MapFile.Reader in = new MapFile.Reader(fs, getMapFilePath().toString(), conf); try { MapFile.Writer outA = new MapFile.Writer(conf, fs, - dstA.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class); + dstA.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class); try { MapFile.Writer outB = new MapFile.Writer(conf, fs, - dstB.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class); + dstB.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class); try { HStoreKey readkey = new HStoreKey(); @@ -252,12 +252,12 @@ public class HStoreFile implements HConstants, WritableComparable { * We are merging multiple regions into a single new one. */ public void mergeStoreFiles(Vector srcFiles, FileSystem fs, - Configuration conf) throws IOException { + Configuration conf) throws IOException { // Copy all the source MapFile tuples into this HSF's MapFile MapFile.Writer out = new MapFile.Writer(conf, fs, getMapFilePath().toString(), - HStoreKey.class, BytesWritable.class); + HStoreKey.class, BytesWritable.class); try { for(Iterator it = srcFiles.iterator(); it.hasNext(); ) { diff --git a/src/java/org/apache/hadoop/hbase/HStoreKey.java b/src/java/org/apache/hadoop/hbase/HStoreKey.java index dbb63076c25..51460ed658c 100644 --- a/src/java/org/apache/hadoop/hbase/HStoreKey.java +++ b/src/java/org/apache/hadoop/hbase/HStoreKey.java @@ -95,7 +95,7 @@ public class HStoreKey implements WritableComparable { public boolean matchesRowCol(HStoreKey other) { if(this.row.compareTo(other.row) == 0 && - this.column.compareTo(other.column) == 0) { + this.column.compareTo(other.column) == 0) { return true; } else { @@ -105,7 +105,7 @@ public class HStoreKey implements WritableComparable { public boolean matchesWithoutColumn(HStoreKey other) { if((this.row.compareTo(other.row) == 0) && - (this.timestamp >= other.getTimestamp())) { + (this.timestamp >= other.getTimestamp())) { return true; } else { diff --git a/src/java/org/apache/hadoop/hbase/Leases.java b/src/java/org/apache/hadoop/hbase/Leases.java index 757fbbc39bf..02694d4776a 100644 --- a/src/java/org/apache/hadoop/hbase/Leases.java +++ b/src/java/org/apache/hadoop/hbase/Leases.java @@ -137,7 +137,7 @@ public class Leases { synchronized(sortedLeases) { Lease top; while((sortedLeases.size() > 0) - && ((top = sortedLeases.first()) != null)) { + && ((top = sortedLeases.first()) != null)) { if(top.shouldExpire()) { leases.remove(top.getLeaseId()); diff --git a/src/test/org/apache/hadoop/hbase/TestHRegion.java b/src/test/org/apache/hadoop/hbase/TestHRegion.java index 99b559b52c8..dee9f78d32a 100644 --- a/src/test/org/apache/hadoop/hbase/TestHRegion.java +++ b/src/test/org/apache/hadoop/hbase/TestHRegion.java @@ -103,7 +103,7 @@ public class TestHRegion extends TestCase { rootLogger.setLevel(Level.WARN); PatternLayout consoleLayout - = (PatternLayout)rootLogger.getAppender("console").getLayout(); + = (PatternLayout)rootLogger.getAppender("console").getLayout(); consoleLayout.setConversionPattern("%d %-5p [%t] %l: %m%n"); Logger.getLogger("org.apache.hadoop.hbase").setLevel(Environment.logLevel); @@ -121,7 +121,7 @@ public class TestHRegion extends TestCase { desc.addFamily(new Text("contents")); desc.addFamily(new Text("anchor")); region = new HRegion(parentdir, log, fs, conf, - new HRegionInfo(1, desc, null, null), null, oldlogfile); + new HRegionInfo(1, desc, null, null), null, oldlogfile); } catch(IOException e) { failures = true; @@ -160,27 +160,27 @@ public class TestHRegion extends TestCase { String bodystr = new String(bodydata).toString().trim(); String teststr = CONTENTSTR + k; assertEquals("Incorrect value for key: (" + rowlabel + "," + CONTENTS_BASIC - + "), expected: '" + teststr + "' got: '" + bodystr + "'", - bodystr, teststr); + + "), expected: '" + teststr + "' got: '" + bodystr + "'", + bodystr, teststr); collabel = new Text(ANCHORNUM + k); bodydata = region.get(rowlabel, collabel); bodystr = new String(bodydata).toString().trim(); teststr = ANCHORSTR + k; assertEquals("Incorrect value for key: (" + rowlabel + "," + collabel - + "), expected: '" + teststr + "' got: '" + bodystr + "'", - bodystr, teststr); -/* + + "), expected: '" + teststr + "' got: '" + bodystr + "'", + bodystr, teststr); + /* // Check to make sure that null values are actually null for (int j = 0; j < Math.min(15, NUM_VALS); j++) { - if (k != j) { - collabel = new Text(ANCHORNUM + j); - byte results[] = region.get(rowlabel, collabel); - if (results != null) { - throw new IOException("Found incorrect value at [" + rowlabel + ", " + collabel + "] == " + new String(results).toString().trim()); - } - } + if (k != j) { + collabel = new Text(ANCHORNUM + j); + byte results[] = region.get(rowlabel, collabel); + if (results != null) { + throw new IOException("Found incorrect value at [" + rowlabel + ", " + collabel + "] == " + new String(results).toString().trim()); } -*/ + } + } + */ } } catch(IOException e) { failures = true; @@ -196,8 +196,8 @@ public class TestHRegion extends TestCase { } Text cols[] = new Text[] { - CONTENTS_FIRSTCOL, - ANCHOR_SECONDCOL + CONTENTS_FIRSTCOL, + ANCHOR_SECONDCOL }; // Test the Scanner!!! @@ -233,8 +233,8 @@ public class TestHRegion extends TestCase { for(int j = 0; j < cols.length; j++) { if(col.compareTo(cols[j]) == 0) { assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() - + ", Value for " + col + " should be: " + k - + ", but was fetched as: " + curval, k, curval); + + ", Value for " + col + " should be: " + k + + ", but was fetched as: " + curval, k, curval); numFetched++; } } @@ -266,8 +266,8 @@ public class TestHRegion extends TestCase { for(int j = 0; j < cols.length; j++) { if(col.compareTo(cols[j]) == 0) { assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() - + ", Value for " + col + " should be: " + k - + ", but was fetched as: " + curval, k, curval); + + ", Value for " + col + " should be: " + k + + ", but was fetched as: " + curval, k, curval); numFetched++; } } @@ -307,8 +307,8 @@ public class TestHRegion extends TestCase { for(int j = 0; j < cols.length; j++) { if(col.compareTo(cols[j]) == 0) { assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() - + ", Value for " + col + " should be: " + k - + ", but was fetched as: " + curval, k, curval); + + ", Value for " + col + " should be: " + k + + ", but was fetched as: " + curval, k, curval); numFetched++; } } @@ -340,7 +340,7 @@ public class TestHRegion extends TestCase { for (int j = 0; j < cols.length; j++) { if (col.compareTo(cols[j]) == 0) { assertEquals("Value for " + col + " should be: " + k - + ", but was fetched as: " + curval, curval, k); + + ", but was fetched as: " + curval, curval, k); numFetched++; } } @@ -370,7 +370,7 @@ public class TestHRegion extends TestCase { for (int j = 0; j < cols.length; j++) { if (col.compareTo(cols[j]) == 0) { assertEquals("Value for " + col + " should be: " + k - + ", but was fetched as: " + curval, curval, k); + + ", but was fetched as: " + curval, curval, k); numFetched++; } } @@ -511,8 +511,8 @@ public class TestHRegion extends TestCase { // First verify the data written by testBasic() Text[] cols = new Text[] { - new Text(ANCHORNUM + "[0-9]+"), - new Text(CONTENTS_BASIC) + new Text(ANCHORNUM + "[0-9]+"), + new Text(CONTENTS_BASIC) }; HScannerInterface s = region.getScanner(cols, new Text()); @@ -532,16 +532,16 @@ public class TestHRegion extends TestCase { if(col.compareTo(CONTENTS_BASIC) == 0) { assertTrue("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() - + ", Value for " + col + " should start with: " + CONTENTSTR - + ", but was fetched as: " + curval, - curval.startsWith(CONTENTSTR)); + + ", Value for " + col + " should start with: " + CONTENTSTR + + ", but was fetched as: " + curval, + curval.startsWith(CONTENTSTR)); contentsFetched++; } else if(col.toString().startsWith(ANCHORNUM)) { assertTrue("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() - + ", Value for " + col + " should start with: " + ANCHORSTR - + ", but was fetched as: " + curval, - curval.startsWith(ANCHORSTR)); + + ", Value for " + col + " should start with: " + ANCHORSTR + + ", but was fetched as: " + curval, + curval.startsWith(ANCHORSTR)); anchorFetched++; } else { @@ -561,8 +561,8 @@ public class TestHRegion extends TestCase { // Verify testScan data cols = new Text[] { - CONTENTS_FIRSTCOL, - ANCHOR_SECONDCOL + CONTENTS_FIRSTCOL, + ANCHOR_SECONDCOL }; s = region.getScanner(cols, new Text()); @@ -580,7 +580,7 @@ public class TestHRegion extends TestCase { for (int j = 0; j < cols.length; j++) { if (col.compareTo(cols[j]) == 0) { assertEquals("Value for " + col + " should be: " + k - + ", but was fetched as: " + curval, curval, k); + + ", but was fetched as: " + curval, curval, k); numFetched++; } } @@ -625,7 +625,7 @@ public class TestHRegion extends TestCase { // Test a scanner which only specifies the column family name cols = new Text[] { - new Text("anchor:") + new Text("anchor:") }; s = region.getScanner(cols, new Text()); @@ -672,5 +672,5 @@ public class TestHRegion extends TestCase { deleteFile(new File(System.getProperty("test.build.data"), "dfs")); - } + } }