Return-Path: Delivered-To: apmail-hadoop-hbase-commits-archive@minotaur.apache.org Received: (qmail 31708 invoked from network); 7 May 2010 19:28:29 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 7 May 2010 19:28:29 -0000 Received: (qmail 53886 invoked by uid 500); 7 May 2010 19:28:29 -0000 Delivered-To: apmail-hadoop-hbase-commits-archive@hadoop.apache.org Received: (qmail 53863 invoked by uid 500); 7 May 2010 19:28:29 -0000 Mailing-List: contact hbase-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hbase-dev@hadoop.apache.org Delivered-To: mailing list hbase-commits@hadoop.apache.org Received: (qmail 53784 invoked by uid 99); 7 May 2010 19:28:29 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 07 May 2010 19:28:29 +0000 X-ASF-Spam-Status: No, hits=-1676.7 required=10.0 tests=ALL_TRUSTED,AWL X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 07 May 2010 19:28:19 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id CCE902388ACC; Fri, 7 May 2010 19:26:56 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r942186 [7/18] - in /hadoop/hbase/trunk: ./ contrib/stargate/core/src/test/java/org/apache/hadoop/hbase/stargate/ core/src/main/java/org/apache/hadoop/hbase/ core/src/main/java/org/apache/hadoop/hbase/client/ core/src/main/java/org/apache/h... Date: Fri, 07 May 2010 19:26:51 -0000 To: hbase-commits@hadoop.apache.org From: stack@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20100507192656.CCE902388ACC@eris.apache.org> Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java Fri May 7 19:26:45 2010 @@ -28,7 +28,7 @@ import org.apache.hadoop.metrics.jvm.Jvm import org.apache.hadoop.metrics.util.MetricsRegistry; -/** +/** * This class is for maintaining the various master statistics * and publishing them through the metrics interfaces. *

@@ -43,7 +43,7 @@ public class MasterMetrics implements Up /* * Count of requests to the cluster since last call to metrics update */ - private final MetricsRate cluster_requests = + private final MetricsRate cluster_requests = new MetricsRate("cluster_requests", registry); public MasterMetrics(final String name) { @@ -58,16 +58,16 @@ public class MasterMetrics implements Up LOG.info("Initialized"); } - + public void shutdown() { if (masterStatistics != null) masterStatistics.shutdown(); } - + /** * Since this object is a registered updater, this method will be called * periodically, e.g. every 5 seconds. - * @param unused + * @param unused */ public void doUpdates(MetricsContext unused) { synchronized (this) { @@ -75,7 +75,7 @@ public class MasterMetrics implements Up } this.metricsRecord.update(); } - + public void resetAllMinMax() { // Nothing to do } @@ -86,7 +86,7 @@ public class MasterMetrics implements Up public float getRequests() { return this.cluster_requests.getPreviousIntervalValue(); } - + /** * @param inc How much to add to requests. */ Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java Fri May 7 19:26:45 2010 @@ -32,7 +32,7 @@ public class MasterStatistics extends Me public MasterStatistics(MetricsRegistry registry) { super(registry, "MasterStatistics"); - mbeanName = MBeanUtil.registerMBean("Master", "MasterStatistics", this); + mbeanName = MBeanUtil.registerMBean("Master", "MasterStatistics", this); } public void shutdown() { Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/MetricsMBeanBase.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/MetricsMBeanBase.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/MetricsMBeanBase.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/MetricsMBeanBase.java Fri May 7 19:26:45 2010 @@ -38,8 +38,8 @@ import org.apache.hadoop.metrics.util.Me /** * Extends the Hadoop MetricsDynamicMBeanBase class to provide JMX support for - * custom HBase MetricsBase implementations. MetricsDynamicMBeanBase ignores - * registered MetricsBase instance that are not instances of one of the + * custom HBase MetricsBase implementations. MetricsDynamicMBeanBase ignores + * registered MetricsBase instance that are not instances of one of the * org.apache.hadoop.metrics.util implementations. * */ @@ -50,13 +50,13 @@ public class MetricsMBeanBase extends Me protected final MetricsRegistry registry; protected final String description; protected int registryLength; - /** HBase MetricsBase implementations that MetricsDynamicMBeanBase does - * not understand + /** HBase MetricsBase implementations that MetricsDynamicMBeanBase does + * not understand */ - protected Map extendedAttributes = + protected Map extendedAttributes = new HashMap(); protected MBeanInfo extendedInfo; - + protected MetricsMBeanBase( MetricsRegistry mr, String description ) { super(copyMinusHBaseMetrics(mr), description); this.registry = mr; @@ -87,45 +87,45 @@ public class MetricsMBeanBase extends Me attributes.add(attr); parentAttributes.add(attr.getName()); } - + this.registryLength = this.registry.getMetricsList().size(); - + for (MetricsBase metric : this.registry.getMetricsList()) { if (metric.getName() == null || parentAttributes.contains(metric.getName())) continue; - + // add on custom HBase metric types if (metric instanceof org.apache.hadoop.hbase.metrics.MetricsRate) { - attributes.add( new MBeanAttributeInfo(metric.getName(), + attributes.add( new MBeanAttributeInfo(metric.getName(), "java.lang.Float", metric.getDescription(), true, false, false) ); extendedAttributes.put(metric.getName(), metric); } // else, its probably a hadoop metric already registered. Skip it. } - this.extendedInfo = new MBeanInfo( this.getClass().getName(), - this.description, attributes.toArray( new MBeanAttributeInfo[0] ), - parentInfo.getConstructors(), parentInfo.getOperations(), + this.extendedInfo = new MBeanInfo( this.getClass().getName(), + this.description, attributes.toArray( new MBeanAttributeInfo[0] ), + parentInfo.getConstructors(), parentInfo.getOperations(), parentInfo.getNotifications() ); } private void checkAndUpdateAttributes() { - if (this.registryLength != this.registry.getMetricsList().size()) + if (this.registryLength != this.registry.getMetricsList().size()) this.init(); } - + @Override public Object getAttribute( String name ) throws AttributeNotFoundException, MBeanException, ReflectionException { - + if (name == null) { throw new IllegalArgumentException("Attribute name is NULL"); } /* * Ugly. Since MetricsDynamicMBeanBase implementation is private, - * we need to first check the parent class for the attribute. + * we need to first check the parent class for the attribute. * In case that the MetricsRegistry contents have changed, this will * allow the parent to update it's internal structures (which we rely on * to update our own. @@ -133,9 +133,9 @@ public class MetricsMBeanBase extends Me try { return super.getAttribute(name); } catch (AttributeNotFoundException ex) { - + checkAndUpdateAttributes(); - + MetricsBase metric = this.extendedAttributes.get(name); if (metric != null) { if (metric instanceof MetricsRate) { @@ -146,7 +146,7 @@ public class MetricsMBeanBase extends Me } } } - + throw new AttributeNotFoundException(); } Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/MetricsRate.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/MetricsRate.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/MetricsRate.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/MetricsRate.java Fri May 7 19:26:45 2010 @@ -30,12 +30,12 @@ import org.apache.hadoop.util.StringUtil */ public class MetricsRate extends MetricsBase { private static final Log LOG = LogFactory.getLog("org.apache.hadoop.hbase.metrics"); - + private int value; private float prevRate; private long ts; - - public MetricsRate(final String name, final MetricsRegistry registry, + + public MetricsRate(final String name, final MetricsRegistry registry, final String description) { super(name, description); this.value = 0; @@ -43,19 +43,19 @@ public class MetricsRate extends Metrics this.ts = System.currentTimeMillis(); registry.add(name, this); } - + public MetricsRate(final String name, final MetricsRegistry registry) { this(name, registry, NO_DESCRIPTION); } - + public synchronized void inc(final int incr) { value += incr; } - + public synchronized void inc() { value++; } - + private synchronized void intervalHeartBeat() { long now = System.currentTimeMillis(); long diff = (now-ts)/1000; @@ -64,18 +64,18 @@ public class MetricsRate extends Metrics this.value = 0; this.ts = now; } - + @Override public synchronized void pushMetric(final MetricsRecord mr) { intervalHeartBeat(); try { mr.setMetric(getName(), getPreviousIntervalValue()); } catch (Exception e) { - LOG.info("pushMetric failed for " + getName() + "\n" + + LOG.info("pushMetric failed for " + getName() + "\n" + StringUtils.stringifyException(e)); } } - + public synchronized float getPreviousIntervalValue() { return this.prevRate; } Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/file/TimeStampingFileContext.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/file/TimeStampingFileContext.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/file/TimeStampingFileContext.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/metrics/file/TimeStampingFileContext.java Fri May 7 19:26:45 2010 @@ -40,7 +40,7 @@ public class TimeStampingFileContext ext private File file = null; private PrintWriter writer = null; private final SimpleDateFormat sdf; - + public TimeStampingFileContext() { super(); this.sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss"); Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java Fri May 7 19:26:45 2010 @@ -29,7 +29,7 @@ public class ColumnCount { private final int offset; private final int length; private int count; - + /** * Constructor * @param column the qualifier to count the versions for @@ -37,7 +37,7 @@ public class ColumnCount { public ColumnCount(byte [] column) { this(column, 0); } - + /** * Constructor * @param column the qualifier to count the versions for @@ -46,7 +46,7 @@ public class ColumnCount { public ColumnCount(byte [] column, int count) { this(column, 0, column.length, count); } - + /** * Constuctor * @param column the qualifier to count the versions for @@ -60,28 +60,28 @@ public class ColumnCount { this.length = length; this.count = count; } - + /** * @return the buffer */ public byte [] getBuffer(){ return this.bytes; } - + /** * @return the offset */ public int getOffset(){ return this.offset; } - + /** * @return the length */ public int getLength(){ return this.length; - } - + } + /** * Decrement the current version count * @return current count @@ -104,9 +104,9 @@ public class ColumnCount { */ public void setCount(int count) { this.count = count; - } + } + - /** * Check to see if needed to fetch more versions * @param max Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java Fri May 7 19:26:45 2010 @@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.regionser /** * Implementing classes of this interface will be used for the tracking - * and enforcement of columns and numbers of versions during the course of a + * and enforcement of columns and numbers of versions during the course of a * Get or Scan operation. *

* Currently there are two different types of Store/Family-level queries. @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.regionser * what action should be taken. *

  • {@link #update} is called at the end of every StoreFile or memstore. *

    - * This class is NOT thread-safe as queries are never multi-threaded + * This class is NOT thread-safe as queries are never multi-threaded */ public interface ColumnTracker { /** @@ -49,19 +49,19 @@ public interface ColumnTracker { * @return The match code instance. */ public MatchCode checkColumn(byte [] bytes, int offset, int length); - + /** * Updates internal variables in between files */ public void update(); - + /** * Resets the Matcher */ public void reset(); - + /** - * + * * @return true when done. */ public boolean done(); Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Fri May 7 19:26:45 2010 @@ -37,23 +37,23 @@ import java.util.concurrent.LinkedBlocki import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantLock; -/** +/** * Compact region on request and then run split if appropriate */ class CompactSplitThread extends Thread implements HConstants { static final Log LOG = LogFactory.getLog(CompactSplitThread.class); - + private HTable root = null; private HTable meta = null; private final long frequency; private final ReentrantLock lock = new ReentrantLock(); - + private final HRegionServer server; private final Configuration conf; - + private final BlockingQueue compactionQueue = new LinkedBlockingQueue(); - + private final HashSet regionsInQueue = new HashSet(); /** @param server */ @@ -65,7 +65,7 @@ class CompactSplitThread extends Thread conf.getLong("hbase.regionserver.thread.splitcompactcheckfrequency", 20 * 1000); } - + @Override public void run() { int count = 0; @@ -144,7 +144,7 @@ class CompactSplitThread extends Thread } } } - + private void split(final HRegion region, final byte [] midKey) throws IOException { final HRegionInfo oldRegionInfo = region.getRegionInfo(); @@ -154,7 +154,7 @@ class CompactSplitThread extends Thread // Didn't need to be split return; } - + // When a region is split, the META table needs to updated if we're // splitting a 'normal' region, and the ROOT table needs to be // updated if we are splitting a META region. @@ -181,14 +181,14 @@ class CompactSplitThread extends Thread this.server.removeFromOnlineRegions(oldRegionInfo); Put put = new Put(oldRegionInfo.getRegionName()); - put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, + put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(oldRegionInfo)); put.add(CATALOG_FAMILY, SPLITA_QUALIFIER, Writables.getBytes(newRegions[0].getRegionInfo())); put.add(CATALOG_FAMILY, SPLITB_QUALIFIER, Writables.getBytes(newRegions[1].getRegionInfo())); t.put(put); - + // If we crash here, then the daughters will not be added and we'll have // and offlined parent but no daughters to take up the slack. hbase-2244 // adds fixup to the metascanners. @@ -200,7 +200,7 @@ class CompactSplitThread extends Thread newRegions[i].getRegionInfo())); t.put(put); } - + // If we crash here, the master will not know of the new daughters and they // will not be assigned. The metascanner when it runs will notice and take // care of assigning the new daughters. @@ -218,20 +218,20 @@ class CompactSplitThread extends Thread /** * Only interrupt once it's done with a run through the work loop. - */ + */ void interruptIfNecessary() { if (lock.tryLock()) { this.interrupt(); } } - + /** - * Returns the current size of the queue containing regions that are - * processed. + * Returns the current size of the queue containing regions that are + * processed. * * @return The current size of the regions queue. */ public int getCompactionQueueSize() { - return compactionQueue.size(); - } + return compactionQueue.size(); + } } \ No newline at end of file Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java Fri May 7 19:26:45 2010 @@ -25,10 +25,10 @@ import org.apache.hadoop.hbase.util.Byte /** - * Class that provides static method needed when putting deletes into memstore + * Class that provides static method needed when putting deletes into memstore */ public class DeleteCompare { - + /** * Return codes from deleteCompare. */ @@ -37,12 +37,12 @@ public class DeleteCompare { * Do nothing. Move to next KV in memstore */ SKIP, - + /** * Add to the list of deletes. */ DELETE, - + /** * Stop looking at KVs in memstore. Finalize. */ @@ -134,5 +134,5 @@ public class DeleteCompare { } else { return DeleteCode.SKIP; } - } + } } Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/DeleteTracker.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/DeleteTracker.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/DeleteTracker.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/DeleteTracker.java Fri May 7 19:26:45 2010 @@ -26,10 +26,10 @@ package org.apache.hadoop.hbase.regionse * This class is utilized through three methods: *

    • {@link #add} when encountering a Delete *
    • {@link #isDeleted} when checking if a Put KeyValue has been deleted - *
    • {@link #update} when reaching the end of a StoreFile + *
    • {@link #update} when reaching the end of a StoreFile */ public interface DeleteTracker { - + /** * Add the specified KeyValue to the list of deletes to check against for * this row operation. @@ -43,7 +43,7 @@ public interface DeleteTracker { */ public void add(byte [] buffer, int qualifierOffset, int qualifierLength, long timestamp, byte type); - + /** * Check if the specified KeyValue buffer has been deleted by a previously * seen delete. @@ -55,12 +55,12 @@ public interface DeleteTracker { */ public boolean isDeleted(byte [] buffer, int qualifierOffset, int qualifierLength, long timestamp); - + /** * @return true if there are no current delete, false otherwise */ public boolean isEmpty(); - + /** * Called at the end of every StoreFile. *

      @@ -68,14 +68,14 @@ public interface DeleteTracker { * when the end of each StoreFile is reached. */ public void update(); - + /** * Called between rows. *

      * This clears everything as if a new DeleteTracker was instantiated. */ public void reset(); - + /** * Return codes for comparison of two Deletes. @@ -85,7 +85,7 @@ public interface DeleteTracker { * INCLUDE means add the specified Delete to the merged list. * NEXT means move to the next element in the specified list(s). */ - enum DeleteCompare { + enum DeleteCompare { INCLUDE_OLD_NEXT_OLD, INCLUDE_OLD_NEXT_BOTH, INCLUDE_NEW_NEXT_NEW, @@ -93,5 +93,5 @@ public interface DeleteTracker { NEXT_OLD, NEXT_NEW } - + } Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java Fri May 7 19:26:45 2010 @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.regionser import org.apache.hadoop.hbase.util.Bytes; /** - * This class is used for the tracking and enforcement of columns and numbers + * This class is used for the tracking and enforcement of columns and numbers * of versions during the course of a Get or Scan operation, when explicit * column qualifiers have been asked for in the query. * @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.util.Byte * for both scans and gets. The main difference is 'next' and 'done' collapse * for the scan case (since we see all columns in order), and we only reset * between rows. - * + * *

      * This class is utilized by {@link QueryMatcher} through two methods: *

      • {@link #checkColumn} is called when a Put satisfies all other @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.util.Byte * what action should be taken. *
      • {@link #update} is called at the end of every StoreFile or memstore. *

        - * This class is NOT thread-safe as queries are never multi-threaded + * This class is NOT thread-safe as queries are never multi-threaded */ public class ExplicitColumnTracker implements ColumnTracker { @@ -51,7 +51,7 @@ public class ExplicitColumnTracker imple private final List columnsToReuse; private int index; private ColumnCount column; - + /** * Default constructor. * @param columns columns specified user in query @@ -66,7 +66,7 @@ public class ExplicitColumnTracker imple } reset(); } - + /** * Done when there are no more columns to match against. */ @@ -77,7 +77,7 @@ public class ExplicitColumnTracker imple public ColumnCount getColumnHint() { return this.column; } - + /** * Checks against the parameters of the query and the columns which have * already been processed by this query. @@ -135,7 +135,7 @@ public class ExplicitColumnTracker imple } } while(true); } - + /** * Called at the end of every StoreFile or memstore. */ Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java Fri May 7 19:26:45 2010 @@ -29,7 +29,7 @@ package org.apache.hadoop.hbase.regionse public interface FlushRequester { /** * Tell the listener the cache needs to be flushed. - * + * * @param region the HRegion requesting the cache flush */ void request(HRegion region); Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java Fri May 7 19:26:45 2010 @@ -129,7 +129,7 @@ class GetClosestRowBeforeTracker { return isDeleted(kv, rowdeletes); } - /** + /** * Check if the specified KeyValue buffer has been deleted by a previously * seen delete. * @param kv @@ -237,4 +237,4 @@ class GetClosestRowBeforeTracker { this.tablenamePlusDelimiterLength, kv.getBuffer(), kv.getRowOffset(), this.tablenamePlusDelimiterLength) == 0; } -} \ No newline at end of file +} \ No newline at end of file Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/GetDeleteTracker.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/GetDeleteTracker.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/GetDeleteTracker.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/GetDeleteTracker.java Fri May 7 19:26:45 2010 @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.util.Byte *

      • {@link #isDeleted} when checking if a Put KeyValue has been deleted *
      • {@link #update} when reaching the end of a StoreFile *

        - * This class is NOT thread-safe as queries are never multi-threaded + * This class is NOT thread-safe as queries are never multi-threaded */ public class GetDeleteTracker implements DeleteTracker { private static long UNSET = -1L; @@ -76,7 +76,7 @@ public class GetDeleteTracker implements } } - /** + /** * Check if the specified KeyValue buffer has been deleted by a previously * seen delete. * @param buffer KeyValue buffer @@ -100,7 +100,7 @@ public class GetDeleteTracker implements // Check column int ret = Bytes.compareTo(buffer, qualifierOffset, qualifierLength, - this.delete.buffer, this.delete.qualifierOffset, + this.delete.buffer, this.delete.qualifierOffset, this.delete.qualifierLength); while (ret != 0) { if (ret <= -1) { @@ -120,7 +120,7 @@ public class GetDeleteTracker implements } } - + // Check Timestamp if(timestamp > this.delete.timestamp) { return false; @@ -186,7 +186,7 @@ public class GetDeleteTracker implements } // Merge previous deletes with new deletes - List mergeDeletes = + List mergeDeletes = new ArrayList(this.newDeletes.size()); int oldIndex = 0; int newIndex = 0; @@ -295,7 +295,7 @@ public class GetDeleteTracker implements } } - private void mergeDown(List mergeDeletes, List srcDeletes, + private void mergeDown(List mergeDeletes, List srcDeletes, int srcIndex) { int index = srcIndex; while(index < srcDeletes.size()) { @@ -335,7 +335,7 @@ public class GetDeleteTracker implements } if(oldDelete.timestamp < newDelete.timestamp) { return DeleteCompare.INCLUDE_NEW_NEXT_BOTH; - } + } return DeleteCompare.INCLUDE_OLD_NEXT_BOTH; } Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=942186&r1=942185&r2=942186&view=diff ============================================================================== --- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original) +++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri May 7 19:26:45 2010 @@ -77,9 +77,9 @@ import java.util.concurrent.locks.Reentr * for each row. A given table consists of one or more HRegions. * *

        We maintain multiple HStores for a single HRegion. - * + * *

        An Store is a set of rows with some column data; together, - * they make up all the data for the rows. + * they make up all the data for the rows. * *

        Each HRegion has a 'startKey' and 'endKey'. *

        The first is inclusive, the second is exclusive (except for @@ -95,15 +95,15 @@ import java.util.concurrent.locks.Reentr * constructed, it holds a read lock until it is closed. A close takes out a * write lock and consequently will block for ongoing operations and will block * new operations from starting while the close is in progress. - * + * *

        An HRegion is defined by its table and its key extent. - * + * *

        It consists of at least one Store. The number of Stores should be * configurable, so that data which is accessed together is stored in the same - * Store. Right now, we approximate that by building a single Store for - * each column family. (This config info will be communicated via the + * Store. Right now, we approximate that by building a single Store for + * each column family. (This config info will be communicated via the * tabledesc.) - * + * *

        The HTableDescriptor contains metainfo about the HRegion's table. * regionName is a unique identifier for this HRegion. (startKey, endKey] * defines the keyspace for this HRegion. @@ -113,8 +113,8 @@ public class HRegion implements HConstan static final String SPLITDIR = "splits"; static final String MERGEDIR = "merges"; final AtomicBoolean closed = new AtomicBoolean(false); - /* Closing can take some time; use the closing flag if there is stuff we don't - * want to do while in closing state; e.g. like offer this region up to the + /* Closing can take some time; use the closing flag if there is stuff we don't + * want to do while in closing state; e.g. like offer this region up to the * master as a region to close if the carrying regionserver is overloaded. * Once set, it is never cleared. */ @@ -133,13 +133,13 @@ public class HRegion implements HConstan protected final Map stores = new ConcurrentSkipListMap(Bytes.BYTES_RAWCOMPARATOR); - + //These variable are just used for getting data out of the region, to test on //client side // private int numStores = 0; // private int [] storeSize = null; // private byte [] name = null; - + final AtomicLong memstoreSize = new AtomicLong(0); // This is the table subdirectory. @@ -181,7 +181,7 @@ public class HRegion implements HConstan this.writesEnabled = !onOff; this.readOnly = onOff; } - + boolean isReadOnly() { return this.readOnly; } @@ -201,7 +201,7 @@ public class HRegion implements HConstan // Used to guard splits and closes private final ReentrantReadWriteLock splitsAndClosesLock = new ReentrantReadWriteLock(); - private final ReentrantReadWriteLock newScannerLock = + private final ReentrantReadWriteLock newScannerLock = new ReentrantReadWriteLock(); // Stop updates lock @@ -210,7 +210,7 @@ public class HRegion implements HConstan private final Object splitLock = new Object(); private long minSequenceId; private boolean splitRequest; - + /** * Name of the region info file that resides just under the region directory. */ @@ -238,7 +238,7 @@ public class HRegion implements HConstan this.regionInfo = null; this.threadWakeFrequency = 0L; } - + /** * HRegion constructor. * @@ -251,7 +251,7 @@ public class HRegion implements HConstan * appropriate log info for this HRegion. If there is a previous log file * (implying that the HRegion has been written-to before), then read it from * the supplied path. - * @param fs is the filesystem. + * @param fs is the filesystem. * @param conf is global configuration settings. * @param regionInfo - HRegionInfo that describes the region * is new), then read them from the supplied path. @@ -259,7 +259,7 @@ public class HRegion implements HConstan * making progress to master -- otherwise master might think region deploy * failed. Can be null. */ - public HRegion(Path basedir, HLog log, FileSystem fs, Configuration conf, + public HRegion(Path basedir, HLog log, FileSystem fs, Configuration conf, HRegionInfo regionInfo, FlushRequester flushListener) { this.basedir = basedir; this.comparator = regionInfo.getComparator(); @@ -291,7 +291,7 @@ public class HRegion implements HConstan /** * Initialize this region and get it ready to roll. * Called after construction. - * + * * @param initialFiles * @param reporter * @throws IOException @@ -308,7 +308,7 @@ public class HRegion implements HConstan // Load in all the HStores. long maxSeqId = -1; long minSeqIdToRecover = Integer.MAX_VALUE; - + for (HColumnDescriptor c : this.regionInfo.getTableDesc().getFamilies()) { Store store = instantiateHStore(this.basedir, c, oldLogFile, reporter); this.stores.put(c.getName(), store); @@ -316,7 +316,7 @@ public class HRegion implements HConstan if (storeSeqId > maxSeqId) { maxSeqId = storeSeqId; } - + long storeSeqIdBeforeRecovery = store.getMaxSeqIdBeforeLogRecovery(); if (storeSeqIdBeforeRecovery < minSeqIdToRecover) { minSeqIdToRecover = storeSeqIdBeforeRecovery; @@ -331,7 +331,7 @@ public class HRegion implements HConstan } fs.delete(oldLogFile, false); } - + // Add one to the current maximum sequence id so new edits are beyond. this.minSequenceId = maxSeqId + 1; @@ -390,7 +390,7 @@ public class HRegion implements HConstan // Name of this file has two leading and trailing underscores so it doesn't // clash w/ a store/family name. There is possibility, but assumption is // that its slim (don't want to use control character in filename because - // + // Path regioninfo = new Path(this.regiondir, REGIONINFO_FILE); if (this.fs.exists(regioninfo) && this.fs.getFileStatus(regioninfo).getLen() > 0) { @@ -414,7 +414,7 @@ public class HRegion implements HConstan long getMinSequenceId() { return this.minSequenceId; } - + /** @return a HRegionInfo object for this region */ public HRegionInfo getRegionInfo() { return this.regionInfo; @@ -424,7 +424,7 @@ public class HRegion implements HConstan public boolean isClosed() { return this.closed.get(); } - + /** * @return True if closing process has started. */ @@ -433,16 +433,16 @@ public class HRegion implements HConstan } /** - * Close down this HRegion. Flush the cache, shut down each HStore, don't + * Close down this HRegion. Flush the cache, shut down each HStore, don't * service any more calls. * - *

        This method could take some time to execute, so don't call it from a + *

        This method could take some time to execute, so don't call it from a * time-sensitive thread. - * - * @return Vector of all the storage files that the HRegion's component + * + * @return Vector of all the storage files that the HRegion's component * HStores make use of. It's a list of all HStoreFile objects. Returns empty * vector if already closed and null if judged that it should not close. - * + * * @throws IOException */ public List close() throws IOException { @@ -453,14 +453,14 @@ public class HRegion implements HConstan * Close down this HRegion. Flush the cache unless abort parameter is true, * Shut down each HStore, don't service any more calls. * - * This method could take some time to execute, so don't call it from a + * This method could take some time to execute, so don't call it from a * time-sensitive thread. - * + * * @param abort true if server is aborting (only during testing) - * @return Vector of all the storage files that the HRegion's component + * @return Vector of all the storage files that the HRegion's component * HStores make use of. It's a list of HStoreFile objects. Can be null if * we are not to close at this time or we are already closed. - * + * * @throws IOException */ public List close(final boolean abort) throws IOException { @@ -508,12 +508,12 @@ public class HRegion implements HConstan // outstanding updates. waitOnRowLocks(); LOG.debug("No more row locks outstanding on region " + this); - + // Don't flush the cache if we are aborting if (!abort) { internalFlushcache(); } - + List result = new ArrayList(); for (Store store: stores.values()) { result.addAll(store.close()); @@ -596,11 +596,11 @@ public class HRegion implements HConstan public long getLastFlushTime() { return this.lastFlushTime; } - + ////////////////////////////////////////////////////////////////////////////// - // HRegion maintenance. + // HRegion maintenance. // - // These methods are meant to be called periodically by the HRegionServer for + // These methods are meant to be called periodically by the HRegionServer for // upkeep. ////////////////////////////////////////////////////////////////////////////// @@ -719,11 +719,11 @@ public class HRegion implements HConstan } return d; } - + protected void prepareToSplit() { // nothing } - + /* * @param dir * @return compaction directory for the passed in dir @@ -740,7 +740,7 @@ public class HRegion implements HConstan private void doRegionCompactionPrep() throws IOException { doRegionCompactionCleanup(); } - + /* * Removes the compaction directory for this Store. * @throws IOException @@ -761,13 +761,13 @@ public class HRegion implements HConstan * Called by compaction thread and after region is opened to compact the * HStores if necessary. * - *

        This operation could block for a long time, so don't call it from a + *

        This operation could block for a long time, so don't call it from a * time-sensitive thread. * * Note that no locking is necessary at this level because compaction only * conflicts with a region split, and that cannot happen because the region * server does them sequentially and not in parallel. - * + * * @return mid key if split is needed * @throws IOException */ @@ -781,13 +781,13 @@ public class HRegion implements HConstan * Called by compaction thread and after region is opened to compact the * HStores if necessary. * - *

        This operation could block for a long time, so don't call it from a + *

        This operation could block for a long time, so don't call it from a * time-sensitive thread. * * Note that no locking is necessary at this level because compaction only * conflicts with a region split, and that cannot happen because the region * server does them sequentially and not in parallel. - * + * * @param majorCompaction True to force a major compaction regardless of thresholds * @return split row if split is needed * @throws IOException @@ -815,7 +815,7 @@ public class HRegion implements HConstan return splitRow; } } - LOG.info("Starting" + (majorCompaction? " major " : " ") + + LOG.info("Starting" + (majorCompaction? " major " : " ") + "compaction on region " + this); long startTime = System.currentTimeMillis(); doRegionCompactionPrep(); @@ -828,7 +828,7 @@ public class HRegion implements HConstan } } doRegionCompactionCleanup(); - String timeTaken = StringUtils.formatTimeDiff(System.currentTimeMillis(), + String timeTaken = StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime); LOG.info("compaction completed on region " + this + " in " + timeTaken); } finally { @@ -845,7 +845,7 @@ public class HRegion implements HConstan /** * Flush the cache. - * + * * When this method is called the cache will be flushed unless: *

          *
        1. the cache is empty
        2. @@ -854,11 +854,11 @@ public class HRegion implements HConstan *
        3. writes are disabled
        4. *
        * - *

        This method may block for some time, so it should not be called from a + *

        This method may block for some time, so it should not be called from a * time-sensitive thread. - * + * * @return true if cache was flushed - * + * * @throws IOException * @throws DroppedSnapshotException Thrown when replay of hlog is required * because a Snapshot was not properly persisted. @@ -877,7 +877,7 @@ public class HRegion implements HConstan writestate.flushing + ", writesEnabled=" + writestate.writesEnabled); } - return false; + return false; } } try { @@ -906,25 +906,25 @@ public class HRegion implements HConstan * flushed. (That way, during recovery, we know when we can rely on the * on-disk flushed structures and when we have to recover the memstore from * the log.) - * + * *

        So, we have a three-step process: - * + * *

        • A. Flush the memstore to the on-disk stores, noting the current * sequence ID for the log.
        • - * + * *
        • B. Write a FLUSHCACHE-COMPLETE message to the log, using the sequence * ID that was current at the time of memstore-flush.
        • - * + * *
        • C. Get rid of the memstore structures that are now redundant, as * they've been flushed to the on-disk HStores.
        • *
        *

        This method is protected, but can be accessed via several public * routes. - * + * *

        This method may block for some time. - * + * * @return true if the region needs compacting - * + * * @throws IOException * @throws DroppedSnapshotException Thrown when replay of hlog is required * because a Snapshot was not properly persisted. @@ -1002,7 +1002,7 @@ public class HRegion implements HConstan // B. Write a FLUSHCACHE-COMPLETE message to the log. // This tells future readers that the HStores were emitted correctly, - // and that all updates to the log for this regionName that have lower + // and that all updates to the log for this regionName that have lower // log-sequence-ids can be safely ignored. this.log.completeCacheFlush(getRegionName(), regionInfo.getTableDesc().getName(), completeSequenceId, @@ -1013,7 +1013,7 @@ public class HRegion implements HConstan synchronized (this) { notifyAll(); // FindBugs NN_NAKED_NOTIFY } - + if (LOG.isDebugEnabled()) { long now = System.currentTimeMillis(); LOG.debug("Finished memstore flush of ~" + @@ -1023,27 +1023,27 @@ public class HRegion implements HConstan } return compactionRequested; } - + /** * Get the sequence number to be associated with this cache flush. Used by * TransactionalRegion to not complete pending transactions. - * - * + * + * * @param currentSequenceId * @return sequence id to complete the cache flush with - */ + */ protected long getCompleteCacheFlushSequenceId(long currentSequenceId) { return currentSequenceId; } - + ////////////////////////////////////////////////////////////////////////////// // get() methods for client use. ////////////////////////////////////////////////////////////////////////////// /** - * Return all the data for the row that matches row exactly, - * or the one that immediately preceeds it, at or immediately before + * Return all the data for the row that matches row exactly, + * or the one that immediately preceeds it, at or immediately before * ts. - * + * * @param row row key * @return map of values * @throws IOException @@ -1054,10 +1054,10 @@ public class HRegion implements HConstan } /** - * Return all the data for the row that matches row exactly, - * or the one that immediately preceeds it, at or immediately before + * Return all the data for the row that matches row exactly, + * or the one that immediately preceeds it, at or immediately before * ts. - * + * * @param row row key * @param family * @return map of values @@ -1089,7 +1089,7 @@ public class HRegion implements HConstan } /** - * Return an iterator that scans over the HRegion, returning the indicated + * Return an iterator that scans over the HRegion, returning the indicated * columns and rows specified by the {@link Scan}. *

        * This Iterator must be closed by the caller. @@ -1102,7 +1102,7 @@ public class HRegion implements HConstan throws IOException { return getScanner(scan, null); } - + protected InternalScanner getScanner(Scan scan, List additionalScanners) throws IOException { newScannerLock.readLock().lock(); try { @@ -1120,7 +1120,7 @@ public class HRegion implements HConstan } } return new RegionScanner(scan, additionalScanners); - + } finally { newScannerLock.readLock().unlock(); } @@ -1169,8 +1169,8 @@ public class HRegion implements HConstan splitsAndClosesLock.readLock().unlock(); } } - - + + /** * @param familyMap map of family to edits for the given family. * @param writeToWAL @@ -1187,10 +1187,10 @@ public class HRegion implements HConstan for (Map.Entry> e : familyMap.entrySet()) { - byte[] family = e.getKey(); + byte[] family = e.getKey(); List kvs = e.getValue(); Map kvCount = new TreeMap(Bytes.BYTES_COMPARATOR); - + Store store = getStore(family); for (KeyValue kv: kvs) { // Check if time is LATEST, change to time of most recent addition if so @@ -1262,9 +1262,9 @@ public class HRegion implements HConstan // for (Map.Entry> e : familyMap.entrySet()) { - byte[] family = e.getKey(); + byte[] family = e.getKey(); List kvs = e.getValue(); - + Store store = getStore(family); for (KeyValue kv: kvs) { size = this.memstoreSize.addAndGet(store.delete(kv)); @@ -1279,7 +1279,7 @@ public class HRegion implements HConstan requestFlush(); } } - + /** * @param put * @throws IOException @@ -1287,7 +1287,7 @@ public class HRegion implements HConstan public void put(Put put) throws IOException { this.put(put, null, put.getWriteToWAL()); } - + /** * @param put * @param writeToWAL @@ -1317,7 +1317,7 @@ public class HRegion implements HConstan checkReadOnly(); // Do a rough check that we have resources to accept a write. The check is - // 'rough' in that between the resource check and the call to obtain a + // 'rough' in that between the resource check and the call to obtain a // read lock, resources may run out. For now, the thought is that this // will be extremely rare; we'll deal with it when it happens. checkResources(); @@ -1343,13 +1343,13 @@ public class HRegion implements HConstan } } - - //TODO, Think that gets/puts and deletes should be refactored a bit so that + + //TODO, Think that gets/puts and deletes should be refactored a bit so that //the getting of the lock happens before, so that you would just pass it into - //the methods. So in the case of checkAndPut you could just do lockRow, + //the methods. So in the case of checkAndPut you could just do lockRow, //get, put, unlockRow or something /** - * + * * @param row * @param family * @param qualifier @@ -1361,10 +1361,10 @@ public class HRegion implements HConstan * @return true if the new put was execute, false otherwise */ public boolean checkAndPut(byte [] row, byte [] family, byte [] qualifier, - byte [] expectedValue, Put put, Integer lockId, boolean writeToWAL) + byte [] expectedValue, Put put, Integer lockId, boolean writeToWAL) throws IOException{ checkReadOnly(); - //TODO, add check for value length or maybe even better move this to the + //TODO, add check for value length or maybe even better move this to the //client if this becomes a global setting checkResources(); splitsAndClosesLock.readLock().lock(); @@ -1376,7 +1376,7 @@ public class HRegion implements HConstan byte [] now = Bytes.toBytes(System.currentTimeMillis()); // Lock row - Integer lid = getLock(lockId, get.getRow()); + Integer lid = getLock(lockId, get.getRow()); List result = new ArrayList(); try { //Getting data @@ -1396,7 +1396,7 @@ public class HRegion implements HConstan if (matches) { // All edits for the given row (across all column families) must happen atomically. put(put.getFamilyMap(), writeToWAL); - return true; + return true; } return false; } finally { @@ -1404,10 +1404,10 @@ public class HRegion implements HConstan } } finally { splitsAndClosesLock.readLock().unlock(); - } + } } - - + + /** * Checks if any stamps is Long.MAX_VALUE. If so, sets them to now. *

        @@ -1427,10 +1427,10 @@ public class HRegion implements HConstan } return true; } - + // /* -// * Utility method to verify values length. +// * Utility method to verify values length. // * @param batchUpdate The update to verify // * @throws IOException Thrown if a value is too long // */ @@ -1438,7 +1438,7 @@ public class HRegion implements HConstan // throws IOException { // Map> families = put.getFamilyMap(); // for(Map.Entry> entry : families.entrySet()) { -// HColumnDescriptor hcd = +// HColumnDescriptor hcd = // this.regionInfo.getTableDesc().getFamily(entry.getKey()); // int maxLen = hcd.getMaxValueLength(); // for(KeyValue kv : entry.getValue()) { @@ -1453,7 +1453,7 @@ public class HRegion implements HConstan /* * Check if resources to support an update. - * + * * Here we synchronize on HRegion, a broad scoped lock. Its appropriate * given we're figuring in here whether this region is able to take on * writes. This is only method with a synchronize (at time of writing), @@ -1500,7 +1500,7 @@ public class HRegion implements HConstan } } - /** + /** * Add updates first to the hlog and then add values to memstore. * Warning: Assumption is caller has lock on passed in row. * @param edits Cell updates by column @@ -1514,7 +1514,7 @@ public class HRegion implements HConstan this.put(familyMap, true); } - /** + /** * Add updates first to the hlog (if writeToWal) and then add values to memstore. * Warning: Assumption is caller has lock on passed in row. * @param familyMap map of family to edits for the given family. @@ -1530,7 +1530,7 @@ public class HRegion implements HConstan this.updatesLock.readLock().lock(); try { WALEdit walEdit = new WALEdit(); - + // check if column families are valid; // check if any timestampupdates are needed; // and if writeToWAL is set, then also collapse edits into a single list. @@ -1544,7 +1544,7 @@ public class HRegion implements HConstan // update timestamp on keys if required. if (updateKeys(edits, byteNow)) { if (writeToWAL) { - // bunch up all edits across all column families into a + // bunch up all edits across all column families into a // single WALEdit. for (KeyValue kv : edits) { walEdit.add(kv); @@ -1570,7 +1570,7 @@ public class HRegion implements HConstan long size = 0; // now make changes to the memstore for (Map.Entry> e : familyMap.entrySet()) { - byte[] family = e.getKey(); + byte[] family = e.getKey(); List edits = e.getValue(); Store store = getStore(family); @@ -1621,7 +1621,7 @@ public class HRegion implements HConstan // Used by subclasses; e.g. THBase. } - protected Store instantiateHStore(Path baseDir, + protected Store instantiateHStore(Path baseDir, HColumnDescriptor c, Path oldLogFile, Progressable reporter) throws IOException { return new Store(baseDir, this, c, this.fs, oldLogFile, @@ -1636,7 +1636,7 @@ public class HRegion implements HConstan * TODO: Make this lookup faster. */ public Store getStore(final byte [] column) { - return this.stores.get(column); + return this.stores.get(column); } ////////////////////////////////////////////////////////////////////////////// @@ -1669,10 +1669,10 @@ public class HRegion implements HConstan * But it acts as a guard on the client; a miswritten client just can't * submit the name of a row and start writing to it; it must know the correct * lockid, which matches the lock list in memory. - * - *

        It would be more memory-efficient to assume a correctly-written client, + * + *

        It would be more memory-efficient to assume a correctly-written client, * which maybe we'll do in the future. - * + * * @param row Name of row to lock. * @throws IOException * @return The id of the held lock. @@ -1695,7 +1695,7 @@ public class HRegion implements HConstan // generate a new lockid. Attempt to insert the new [lockid, row]. // if this lockid already exists in the map then revert and retry // We could have first done a lockIds.get, and if it does not exist only - // then do a lockIds.put, but the hope is that the lockIds.put will + // then do a lockIds.put, but the hope is that the lockIds.put will // mostly return null the first time itself because there won't be // too many lockId collisions. byte [] prev = null; @@ -1717,7 +1717,7 @@ public class HRegion implements HConstan splitsAndClosesLock.readLock().unlock(); } } - + /** * Used by unit tests. * @param lockid @@ -1728,8 +1728,8 @@ public class HRegion implements HConstan return lockIds.get(lockid); } } - - /** + + /** * Release the row lock! * @param lockid The lock ID to release. */ @@ -1740,7 +1740,7 @@ public class HRegion implements HConstan lockedRows.notifyAll(); } } - + /** * See if row is currently locked. * @param lockid @@ -1754,14 +1754,14 @@ public class HRegion implements HConstan return false; } } - + /** * Returns existing row lock if found, otherwise * obtains a new row lock and returns it. * @param lockid * @return lockid */ - private Integer getLock(Integer lockid, byte [] row) + private Integer getLock(Integer lockid, byte [] row) throws IOException { Integer lid = null; if (lockid == null) { @@ -1774,7 +1774,7 @@ public class HRegion implements HConstan } return lid; } - + private void waitOnRowLocks() { synchronized (lockedRows) { while (!this.lockedRows.isEmpty()) { @@ -1789,7 +1789,7 @@ public class HRegion implements HConstan } } } - + @Override public boolean equals(Object o) { if (!(o instanceof HRegion)) { @@ -1797,12 +1797,12 @@ public class HRegion implements HConstan } return this.hashCode() == ((HRegion)o).hashCode(); } - + @Override public int hashCode() { return Bytes.hashCode(this.regionInfo.getRegionName()); } - + @Override public String toString() { return this.regionInfo.getRegionNameAsString(); @@ -1834,20 +1834,20 @@ public class HRegion implements HConstan } else { this.stopRow = scan.getStopRow(); } - + List scanners = new ArrayList(); if (additionalScanners != null) { scanners.addAll(additionalScanners); } - for (Map.Entry> entry : + for (Map.Entry> entry : scan.getFamilyMap().entrySet()) { Store store = stores.get(entry.getKey()); scanners.add(store.getScanner(scan, entry.getValue())); } - this.storeHeap = + this.storeHeap = new KeyValueHeap(scanners.toArray(new KeyValueScanner[0]), comparator); } - + RegionScanner(Scan scan) { this(scan, null); } @@ -1912,7 +1912,7 @@ public class HRegion implements HConstan filterCurrentRow = false; // See if we passed stopRow if (this.stopRow != null && - comparator.compareRows(this.stopRow, 0, this.stopRow.length, + comparator.compareRows(this.stopRow, 0, this.stopRow.length, currentRow, 0, currentRow.length) <= 0) { return false; } @@ -1960,7 +1960,7 @@ public class HRegion implements HConstan } /** - * + * * @param scanner to be closed */ public void close(KeyValueScanner scanner) { @@ -1968,7 +1968,7 @@ public class HRegion implements HConstan scanner.close(); } catch(NullPointerException npe) {} } - + /** * @return the current storeHeap */ @@ -1976,7 +1976,7 @@ public class HRegion implements HConstan return this.storeHeap; } } - + // Utility methods /** @@ -1989,7 +1989,7 @@ public class HRegion implements HConstan * @param rootDir Root directory for HBase instance * @param conf * @return new HRegion - * + * * @throws IOException */ public static HRegion createHRegion(final HRegionInfo info, final Path rootDir, @@ -2007,7 +2007,7 @@ public class HRegion implements HConstan region.initialize(null, null); return region; } - + /** * Convenience method to open a HRegion outside of an HRegionServer context. * @param info Info for region to be opened. @@ -2018,7 +2018,7 @@ public class HRegion implements HConstan * up. HRegionStore does this every time it opens a new region. * @param conf * @return new HRegion - * + * * @throws IOException */ public static HRegion openHRegion(final HRegionInfo info, final Path rootDir, @@ -2039,18 +2039,18 @@ public class HRegion implements HConstan } return r; } - + /** * Inserts a new region's meta information into the passed * meta region. Used by the HMaster bootstrap code adding * new table to ROOT table. - * + * * @param meta META HRegion to be updated * @param r HRegion to add to meta * * @throws IOException */ - public static void addRegionToMETA(HRegion meta, HRegion r) + public static void addRegionToMETA(HRegion meta, HRegion r) throws IOException { meta.checkResources(); // The row key is the region name @@ -2103,7 +2103,7 @@ public class HRegion implements HConstan srvr.put(metaRegionName, put); cleanRegionInMETA(srvr, metaRegionName, info); } - + /** * Clean COL_SERVER and COL_STARTCODE for passed info in * .META. @@ -2123,7 +2123,7 @@ public class HRegion implements HConstan /** * Deletes all the files for a HRegion - * + * * @param fs the file system object * @param rootdir qualified path of HBase root directory * @param info HRegionInfo for region to be deleted @@ -2146,7 +2146,7 @@ public class HRegion implements HConstan /** * Computes the Path of the HRegion - * + * * @param tabledir qualified path for table * @param name ENCODED region name * @return Path of HRegion directory @@ -2154,10 +2154,10 @@ public class HRegion implements HConstan public static Path getRegionDir(final Path tabledir, final int name) { return new Path(tabledir, Integer.toString(name)); } - + /** * Computes the Path of the HRegion - * + * * @param rootdir qualified path of HBase root directory * @param info HRegionInfo for the region * @return qualified path of region directory @@ -2171,7 +2171,7 @@ public class HRegion implements HConstan /** * Determines if the specified row is within the row range specified by the * specified HRegionInfo - * + * * @param info HRegionInfo that specifies the row range * @param row row to be checked * @return true if the row is within the range specified by the HRegionInfo @@ -2185,7 +2185,7 @@ public class HRegion implements HConstan /** * Make the directories for a specific column family - * + * * @param fs the file system * @param tabledir base directory where region will live (usually the table dir) * @param hri @@ -2203,7 +2203,7 @@ public class HRegion implements HConstan /** * Merge two HRegions. The regions must be adjacent and must not overlap. - * + * * @param srcA * @param srcB * @return new merged HRegion @@ -2235,7 +2235,7 @@ public class HRegion implements HConstan /** * Merge two regions whether they are adjacent or not. - * + * * @param a region a * @param b region b * @return new merged region @@ -2250,12 +2250,12 @@ public class HRegion implements HConstan FileSystem fs = a.getFilesystem(); // Make sure each region's cache is empty - + a.flushcache(); b.flushcache(); - + // Compact each region so we only have one store file per family - + a.compactStores(true); if (LOG.isDebugEnabled()) { LOG.debug("Files for region: " + a); @@ -2266,12 +2266,12 @@ public class HRegion implements HConstan LOG.debug("Files for region: " + b); listPaths(fs, b.getRegionDir()); } - + Configuration conf = a.getConf(); HTableDescriptor tabledesc = a.getTableDesc(); HLog log = a.getLog(); Path basedir = a.getBaseDir(); - // Presume both are of same region type -- i.e. both user or catalog + // Presume both are of same region type -- i.e. both user or catalog // table regions. This way can use comparator. final byte [] startKey = a.comparator.matchingRows(a.getStartKey(), 0, a.getStartKey().length, @@ -2279,7 +2279,7 @@ public class HRegion implements HConstan b.comparator.matchingRows(b.getStartKey(), 0, b.getStartKey().length, EMPTY_BYTE_ARRAY, 0, EMPTY_BYTE_ARRAY.length)? EMPTY_BYTE_ARRAY: - a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length, + a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length, b.getStartKey(), 0, b.getStartKey().length) <= 0? a.getStartKey(): b.getStartKey(); final byte [] endKey = a.comparator.matchingRows(a.getEndKey(), 0, @@ -2293,7 +2293,7 @@ public class HRegion implements HConstan HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey); LOG.info("Creating new region " + newRegionInfo.toString()); - int encodedName = newRegionInfo.getEncodedName(); + int encodedName = newRegionInfo.getEncodedName(); Path newRegionDir = HRegion.getRegionDir(a.getBaseDir(), encodedName); if(fs.exists(newRegionDir)) { throw new IOException("Cannot merge; target file collision at " + @@ -2353,7 +2353,7 @@ public class HRegion implements HConstan } /* - * Fills a map with a vector of store files keyed by column family. + * Fills a map with a vector of store files keyed by column family. * @param byFamily Map to fill. * @param storeFiles Store files to process. * @param family @@ -2375,7 +2375,7 @@ public class HRegion implements HConstan /** * @return True if needs a mojor compaction. - * @throws IOException + * @throws IOException */ boolean isMajorCompaction() throws IOException { for (Store store: this.stores.values()) { @@ -2388,7 +2388,7 @@ public class HRegion implements HConstan /* * List the files under the specified directory - * + * * @param fs * @param dir * @throws IOException @@ -2411,7 +2411,7 @@ public class HRegion implements HConstan } } - + // // HBASE-880 // @@ -2433,7 +2433,7 @@ public class HRegion implements HConstan } } // Lock row - Integer lid = getLock(lockid, get.getRow()); + Integer lid = getLock(lockid, get.getRow()); List result = new ArrayList(); try { for (Map.Entry> entry: @@ -2453,7 +2453,7 @@ public class HRegion implements HConstan } /** - * + * * @param row * @param family * @param qualifier @@ -2519,13 +2519,13 @@ public class HRegion implements HConstan return result; } - - + + // // New HBASE-880 Helpers // - - private void checkFamily(final byte [] family) + + private void checkFamily(final byte [] family) throws NoSuchColumnFamilyException { if(!regionInfo.getTableDesc().hasFamily(family)) { throw new NoSuchColumnFamilyException("Column family " + @@ -2537,9 +2537,9 @@ public class HRegion implements HConstan public static final long FIXED_OVERHEAD = ClassSize.align( (5 * Bytes.SIZEOF_LONG) + Bytes.SIZEOF_BOOLEAN + (20 * ClassSize.REFERENCE) + ClassSize.OBJECT + Bytes.SIZEOF_INT); - + public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD + - ClassSize.OBJECT + (2 * ClassSize.ATOMIC_BOOLEAN) + + ClassSize.OBJECT + (2 * ClassSize.ATOMIC_BOOLEAN) + ClassSize.ATOMIC_LONG + ClassSize.ATOMIC_INTEGER + // Using TreeMap for TreeSet @@ -2552,7 +2552,7 @@ public class HRegion implements HConstan ClassSize.align(ClassSize.OBJECT + (5 * Bytes.SIZEOF_BOOLEAN)) + (3 * ClassSize.REENTRANT_LOCK)); - + public long heapSize() { long heapSize = DEEP_OVERHEAD; for(Store store : this.stores.values()) { @@ -2661,7 +2661,7 @@ public class HRegion implements HConstan * ./bin/hbase org.apache.hadoop.hbase.regionserver.HRegion * * @param args - * @throws IOException + * @throws IOException */ public static void main(String[] args) throws IOException { if (args.length < 1) {