Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 544CD10A49 for ; Wed, 2 Apr 2014 20:50:22 +0000 (UTC) Received: (qmail 94303 invoked by uid 500); 2 Apr 2014 20:50:21 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 94183 invoked by uid 500); 2 Apr 2014 20:50:19 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 94152 invoked by uid 99); 2 Apr 2014 20:50:18 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 02 Apr 2014 20:50:17 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED 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; Wed, 02 Apr 2014 20:50:07 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id B94B52388C27; Wed, 2 Apr 2014 20:49:39 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1584172 [1/2] - in /hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase: ./ client/ io/ master/ regionserver/ regionserver/wal/ thrift/ util/ Date: Wed, 02 Apr 2014 20:49:38 -0000 To: commits@hbase.apache.org From: liyin@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20140402204939.B94B52388C27@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: liyin Date: Wed Apr 2 20:49:37 2014 New Revision: 1584172 URL: http://svn.apache.org/r1584172 Log: [master] Fix the problem of sunfire xml file Author: daviddeng Summary: Some log was output by calling `Bytes.toString` instead of `Bytes.toStringBinary`, the former generates unpritable chars, which failed the Jenkins. The diff change many (not all) usage of `toString` in `LOG` and `Exception` message composing. Test Plan: `TestHTable` and `TestFastFail` should pass after submitting. Other testcase. Reviewers: liyintang, manukranthk, fan Reviewed By: manukranthk CC: hbase-eng@, andrewcox Differential Revision: https://phabricator.fb.com/D1236212 Task ID: 3789093 Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HMerge.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/KeyValue.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/ResultScannerImpl.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/TableServers.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RetryableMetaOperation.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableDelete.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableOperation.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/Bytes.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HMerge.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HMerge.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HMerge.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HMerge.java Wed Apr 2 20:49:37 2014 @@ -19,6 +19,12 @@ */ package org.apache.hadoop.hbase; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Random; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -38,12 +44,6 @@ import org.apache.hadoop.hbase.regionser import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Random; - /** * A non-instantiable class that has a static method capable of compacting * a table by merging adjacent regions. @@ -164,16 +164,18 @@ class HMerge { if ((currentSize + nextSize) <= (maxFilesize / 2)) { // We merge two adjacent regions if their total size is less than // one half of the desired maximum size - LOG.info("merging regions " + Bytes.toString(currentRegion.getRegionName()) - + " and " + Bytes.toString(nextRegion.getRegionName())); + LOG.info("merging regions " + + Bytes.toStringBinary(currentRegion.getRegionName()) + " and " + + Bytes.toStringBinary(nextRegion.getRegionName())); HRegion mergedRegion = HRegion.mergeAdjacent(currentRegion, nextRegion); updateMeta(currentRegion.getRegionName(), nextRegion.getRegionName(), mergedRegion); break; } - LOG.info("not merging regions " + Bytes.toString(currentRegion.getRegionName()) - + " and " + Bytes.toString(nextRegion.getRegionName())); + LOG.info("not merging regions " + + Bytes.toStringBinary(currentRegion.getRegionName()) + " and " + + Bytes.toStringBinary(nextRegion.getRegionName())); currentRegion.close(); currentRegion = nextRegion; currentSize = nextSize; @@ -219,9 +221,9 @@ class HMerge { byte[] regionInfoValue = results.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); if (regionInfoValue == null || regionInfoValue.length == 0) { - throw new NoSuchElementException("meta region entry missing " + - Bytes.toString(HConstants.CATALOG_FAMILY) + ":" + - Bytes.toString(HConstants.REGIONINFO_QUALIFIER)); + throw new NoSuchElementException("meta region entry missing " + + Bytes.toStringBinary(HConstants.CATALOG_FAMILY) + ":" + + Bytes.toStringBinary(HConstants.REGIONINFO_QUALIFIER)); } HRegionInfo region = Writables.getHRegionInfo(regionInfoValue); if (!Bytes.equals(region.getTableDesc().getName(), this.tableName)) { @@ -254,7 +256,7 @@ class HMerge { Result currentRow = metaScanner.next(); boolean foundResult = false; while (currentRow != null) { - LOG.info("Row: <" + Bytes.toString(currentRow.getRow()) + ">"); + LOG.info("Row: <" + Bytes.toStringBinary(currentRow.getRow()) + ">"); byte[] regionInfoValue = currentRow.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); if (regionInfoValue == null || regionInfoValue.length == 0) { @@ -296,7 +298,8 @@ class HMerge { Delete delete = new Delete(regionsToDelete[r]); table.delete(delete); if(LOG.isDebugEnabled()) { - LOG.debug("updated columns in row: " + Bytes.toString(regionsToDelete[r])); + LOG.debug("updated columns in row: " + + Bytes.toStringBinary(regionsToDelete[r])); } } newRegion.getRegionInfo().setOffline(true); @@ -308,7 +311,7 @@ class HMerge { if(LOG.isDebugEnabled()) { LOG.debug("updated columns in row: " - + Bytes.toString(newRegion.getRegionName())); + + Bytes.toStringBinary(newRegion.getRegionName())); } } } @@ -389,7 +392,8 @@ class HMerge { root.delete(delete, null, true); if(LOG.isDebugEnabled()) { - LOG.debug("updated columns in row: " + Bytes.toString(regionsToDelete[r])); + LOG.debug("updated columns in row: " + + Bytes.toStringBinary(regionsToDelete[r])); } } HRegionInfo newInfo = newRegion.getRegionInfo(); @@ -399,7 +403,8 @@ class HMerge { Writables.getBytes(newInfo)); root.put(put); if(LOG.isDebugEnabled()) { - LOG.debug("updated columns in row: " + Bytes.toString(newRegion.getRegionName())); + LOG.debug("updated columns in row: " + + Bytes.toStringBinary(newRegion.getRegionName())); } } } Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java Wed Apr 2 20:49:37 2014 @@ -56,7 +56,7 @@ public class HTableDescriptor implements // Version 4 adds indexes // Version 5 removed transactional pollution -- e.g. indexes public static final byte TABLE_DESCRIPTOR_VERSION = 5; - + private byte [] name = HConstants.EMPTY_BYTE_ARRAY; private String nameAsString = ""; @@ -328,7 +328,7 @@ public class HTableDescriptor implements if (b[0] == '.' || b[0] == '-') { throw new IllegalArgumentException("Illegal first character <" + b[0] + "> at 0. User-space table names can only start with 'word " + - "characters': i.e. [a-zA-Z_0-9]: " + Bytes.toString(b)); + "characters': i.e. [a-zA-Z_0-9]: " + Bytes.toStringBinary(b)); } for (int i = 0; i < b.length; i++) { if (Character.isLetterOrDigit(b[i]) || b[i] == '_' || b[i] == '-' || @@ -337,7 +337,7 @@ public class HTableDescriptor implements } throw new IllegalArgumentException("Illegal character <" + b[i] + "> at " + i + ". User-space table names can only contain " + - "'word characters': i.e. [a-zA-Z_0-9-.]: " + Bytes.toString(b)); + "'word characters': i.e. [a-zA-Z_0-9-.]: " + Bytes.toStringBinary(b)); } return b; } @@ -469,8 +469,9 @@ public class HTableDescriptor implements /** @return max hregion size for table */ public long getMaxFileSize() { byte [] value = getValue(MAX_FILESIZE_KEY); - if (value != null) - return Long.valueOf(Bytes.toString(value)).longValue(); + if (value != null) { + return Long.parseLong(Bytes.toString(value)); + } return HConstants.DEFAULT_MAX_FILE_SIZE; } @@ -492,8 +493,9 @@ public class HTableDescriptor implements */ public long getMemStoreFlushSize() { byte [] value = getValue(MEMSTORE_FLUSHSIZE_KEY); - if (value != null) - return Long.valueOf(Bytes.toString(value)).longValue(); + if (value != null) { + return Long.parseLong(Bytes.toString(value)); + } return DEFAULT_MEMSTORE_FLUSH_SIZE; } @@ -507,7 +509,8 @@ public class HTableDescriptor implements /** * Adds a column family. - * @param family HColumnDescriptor of familyto add. + * + * @param family HColumnDescriptor of family to add. */ public void addFamily(final HColumnDescriptor family) { if (family.getName() == null || family.getName().length <= 0) { @@ -653,6 +656,7 @@ public class HTableDescriptor implements // Writable + @Override public void readFields(DataInput in) throws IOException { int version = in.readInt(); if (version < 3) @@ -683,6 +687,7 @@ public class HTableDescriptor implements } } + @Override public void write(DataOutput out) throws IOException { out.writeInt(TABLE_DESCRIPTOR_VERSION); Bytes.writeByteArray(out, name); @@ -704,6 +709,7 @@ public class HTableDescriptor implements // Comparable + @Override public int compareTo(final HTableDescriptor other) { int result = Bytes.compareTo(this.name, other.name); if (result == 0) { Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/KeyValue.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/KeyValue.java Wed Apr 2 20:49:37 2014 @@ -19,10 +19,14 @@ */ package org.apache.hadoop.hbase; -import com.facebook.swift.codec.ThriftConstructor; -import com.facebook.swift.codec.ThriftField; -import com.facebook.swift.codec.ThriftStruct; -import com.google.common.primitives.Longs; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.filter.KeyOnlyFilter; @@ -33,13 +37,10 @@ import org.apache.hadoop.hbase.util.Clas import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.Writable; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Map; +import com.facebook.swift.codec.ThriftConstructor; +import com.facebook.swift.codec.ThriftField; +import com.facebook.swift.codec.ThriftStruct; +import com.google.common.primitives.Longs; /** * An HBase Key/Value. @@ -554,6 +555,7 @@ public class KeyValue implements Writabl * Needed doing 'contains' on List. Only compares the key portion, not the * value. */ + @Override public boolean equals(Object other) { if (!(other instanceof KeyValue)) { return false; @@ -567,6 +569,7 @@ public class KeyValue implements Writabl return result; } + @Override public int hashCode() { byte[] b = getBuffer(); int start = getOffset(), end = getOffset() + getLength(); @@ -587,6 +590,7 @@ public class KeyValue implements Writabl * Clones a KeyValue. This creates a copy, re-allocating the buffer. * @return Fully copied clone of this KeyValue */ + @Override public KeyValue clone() { byte [] b = new byte[this.length]; System.arraycopy(this.bytes, this.offset, b, 0, this.length); @@ -615,6 +619,7 @@ public class KeyValue implements Writabl // //--------------------------------------------------------------------------- + @Override public String toString() { if (this.bytes == null || this.bytes.length == 0) { return "empty"; @@ -1361,7 +1366,8 @@ public class KeyValue implements Writabl int index = getDelimiter(b, offset, length, delimiter); if (index < 0) { throw new IllegalArgumentException("No '" + (char)delimiter + "' in <" + - Bytes.toString(b) + ">" + ", length=" + length + ", offset=" + offset); + Bytes.toStringBinary(b) + ">" + ", length=" + length + ", offset=" + + offset); } return index; } @@ -1374,8 +1380,9 @@ public class KeyValue implements Writabl final int offset, final int length, final int delimiter) { int index = getDelimiterInReverse(b, offset, length, delimiter); if (index < 0) { - throw new IllegalArgumentException("No '" + ((char) delimiter) + "' in <" + - Bytes.toString(b) + ">" + ", length=" + length + ", offset=" + offset); + throw new IllegalArgumentException("No '" + ((char) delimiter) + "' in <" + + Bytes.toStringBinary(b) + ">" + ", length=" + length + ", offset=" + + offset); } return index; } @@ -1429,6 +1436,7 @@ public class KeyValue implements Writabl public static class RootComparator extends MetaComparator { private final KeyComparator rawcomparator = new RootKeyComparator(); + @Override public KeyComparator getRawComparator() { return this.rawcomparator; } @@ -1446,6 +1454,7 @@ public class KeyValue implements Writabl public static class MetaComparator extends KVComparator { private final KeyComparator rawcomparator = new MetaKeyComparator(); + @Override public KeyComparator getRawComparator() { return this.rawcomparator; } @@ -1473,6 +1482,7 @@ public class KeyValue implements Writabl return this.rawcomparator; } + @Override public int compare(final KeyValue left, final KeyValue right) { int ret = getRawComparator().compare(left.getBuffer(), left.getOffset() + ROW_OFFSET, left.getKeyLength(), @@ -1714,6 +1724,7 @@ public class KeyValue implements Writabl * @return First possible key on passed row, column and timestamp * @deprecated */ + @Deprecated public static KeyValue createFirstOnRow(final byte [] row, final byte [] c, final long ts) { byte [][] split = parseColumn(c); @@ -1917,6 +1928,7 @@ public class KeyValue implements Writabl * table. */ public static class RootKeyComparator extends MetaKeyComparator { + @Override public int compareRows(byte [] left, int loffset, int llength, byte [] right, int roffset, int rlength) { // Rows look like this: .META.,ROW_FROM_META,RID @@ -1966,6 +1978,7 @@ public class KeyValue implements Writabl this.comparator = c; } + @Override public int compare(KeyValue left, KeyValue right) { return comparator.compareRows(left, right); } @@ -1976,6 +1989,7 @@ public class KeyValue implements Writabl * table. */ public static class MetaKeyComparator extends KeyComparator { + @Override public int compareRows(byte [] left, int loffset, int llength, byte [] right, int roffset, int rlength) { // LOG.info("META " + Bytes.toString(left, loffset, llength) + @@ -2050,6 +2064,7 @@ public class KeyValue implements Writabl volatile boolean ignoreTimestamp = false; volatile boolean ignoreType = false; + @Override public int compare(byte[] left, int loffset, int llength, byte[] right, int roffset, int rlength) { // Compare row @@ -2195,6 +2210,7 @@ public class KeyValue implements Writabl return 0; } + @Override public int compare(byte[] left, byte[] right) { return compare(left, 0, left.length, right, 0, right.length); } @@ -2226,6 +2242,7 @@ public class KeyValue implements Writabl } // HeapSize + @Override public long heapSize() { return ClassSize.align(ClassSize.OBJECT + ClassSize.REFERENCE + ClassSize.align(ClassSize.ARRAY + length) + @@ -2245,11 +2262,13 @@ public class KeyValue implements Writabl } // Writable + @Override public void readFields(final DataInput in) throws IOException { int length = in.readInt(); readFields(length, in); } + @Override public void write(final DataOutput out) throws IOException { this.verify(); out.writeInt(this.length); Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java Wed Apr 2 20:49:37 2014 @@ -48,6 +48,6 @@ public class NotServingRegionException e * @param s message */ public NotServingRegionException(final byte [] s) { - super(Bytes.toString(s)); + super(Bytes.toStringBinary(s)); } -} \ No newline at end of file +} Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java Wed Apr 2 20:49:37 2014 @@ -45,6 +45,6 @@ public class TableNotDisabledException e * @param tableName Name of table that is not disabled */ public TableNotDisabledException(byte[] tableName) { - this(Bytes.toString(tableName)); + this(Bytes.toStringBinary(tableName)); } -} \ No newline at end of file +} Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Wed Apr 2 20:49:37 2014 @@ -62,10 +62,10 @@ import org.apache.hadoop.ipc.RemoteExcep /** - * Provides an interface to manage HBase database table metadata + general - * administrative functions. Use HBaseAdmin to create, drop, list, enable and - * disable tables. Use it also to add and drop table column families. - * + * Provides an interface to manage HBase database table metadata + general + * administrative functions. Use HBaseAdmin to create, drop, list, enable and + * disable tables. Use it also to add and drop table column families. + * * See {@link HTable} to add, update, and delete data from an individual table. */ public class HBaseAdmin { @@ -98,21 +98,21 @@ public class HBaseAdmin { } master.enableLoadBalancer(); } - + public void disableLoadBalancer() throws MasterNotRunningException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } this.master.disableLoadBalancer(); } - + public boolean isLoadBalancerDisabled() throws MasterNotRunningException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } return this.master.isLoadBalancerDisabled(); } - + /** @return HConnection used by this object. */ public HConnection getConnection() { return connection; @@ -316,14 +316,14 @@ public class HBaseAdmin { MetaScanner.metaScan(conf, visitor, desc.getName()); if (actualRegCount.get() != numRegs) { if (tries == numRetries - 1) { - throw new RegionOfflineException("Only " + actualRegCount.get() + + throw new RegionOfflineException("Only " + actualRegCount.get() + " of " + numRegs + " regions are online; retries exhausted."); } try { // Sleep Thread.sleep(getPauseTime(tries)); } catch (InterruptedException e) { throw new InterruptedIOException("Interrupted when opening" + - " regions; " + actualRegCount.get() + " of " + numRegs + + " regions; " + actualRegCount.get() + " of " + numRegs + " regions processed so far"); } if (actualRegCount.get() > prevRegCount) { // Making progress @@ -497,7 +497,7 @@ public class HBaseAdmin { } // Delete cached information to prevent clients from using old locations HConnectionManager.deleteConnectionInfo(conf, false); - LOG.info("Deleted " + Bytes.toString(tableName)); + LOG.info("Deleted " + Bytes.toStringBinary(tableName)); } @@ -539,7 +539,7 @@ public class HBaseAdmin { long sleep = getPauseTime(tries); if (LOG.isDebugEnabled()) { LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " + - "enabled in " + Bytes.toString(tableName)); + "enabled in " + Bytes.toStringBinary(tableName)); } try { Thread.sleep(sleep); @@ -548,13 +548,13 @@ public class HBaseAdmin { } if (LOG.isDebugEnabled()) { LOG.debug("Wake. Waiting for all regions to be enabled from " + - Bytes.toString(tableName)); + Bytes.toStringBinary(tableName)); } } if (!enabled) throw new IOException("Unable to enable table " + - Bytes.toString(tableName)); - LOG.info("Enabled table " + Bytes.toString(tableName)); + Bytes.toStringBinary(tableName)); + LOG.info("Enabled table " + Bytes.toStringBinary(tableName)); } /** @@ -594,7 +594,7 @@ public class HBaseAdmin { if (disabled) break; if (LOG.isDebugEnabled()) { LOG.debug("Sleep. Waiting for all regions to be disabled from " + - Bytes.toString(tableName)); + Bytes.toStringBinary(tableName)); } try { Thread.sleep(getPauseTime(tries)); @@ -603,14 +603,14 @@ public class HBaseAdmin { } if (LOG.isDebugEnabled()) { LOG.debug("Wake. Waiting for all regions to be disabled from " + - Bytes.toString(tableName)); + Bytes.toStringBinary(tableName)); } } if (!disabled) { throw new RegionException("Retries exhausted, it took too long to wait"+ - " for the table " + Bytes.toString(tableName) + " to be disabled."); + " for the table " + Bytes.toStringBinary(tableName) + " to be disabled."); } - LOG.info("Disabled " + Bytes.toString(tableName)); + LOG.info("Disabled " + Bytes.toStringBinary(tableName)); } /** @@ -727,7 +727,7 @@ public class HBaseAdmin { /** * Batch alter a table. Only takes regions offline once and performs a single * update to .META. - * Any of the three lists can be null, in which case those types of + * Any of the three lists can be null, in which case those types of * alterations will be ignored. * Asynchronous operation. * @@ -791,7 +791,7 @@ public class HBaseAdmin { /** * Get the status of alter command - indicates how many regions have received * the updated schema Asynchronous operation. - * + * * @param tableName * name of the table to get the status of * @return List indicating the number of regions updated List.get(0) is the @@ -831,7 +831,7 @@ public class HBaseAdmin { * @param column column descriptor of column to be added * @throws IOException if a remote or network exception occurs */ - public void addColumn(final byte [] tableName, HColumnDescriptor column) + public void addColumn(final byte [] tableName, HColumnDescriptor column) throws IOException { alterTable(tableName, Arrays.asList(column), null, null); } @@ -858,7 +858,7 @@ public class HBaseAdmin { * @param columnName name of column to be deleted * @throws IOException if a remote or network exception occurs */ - public void deleteColumn(final byte [] tableName, + public void deleteColumn(final byte [] tableName, final byte [] columnName) throws IOException { alterTable(tableName, null, null, Arrays.asList(columnName)); @@ -876,7 +876,7 @@ public class HBaseAdmin { HColumnDescriptor descriptor) throws IOException { alterTable(Bytes.toBytes(tableName), null, Arrays.asList( - new Pair(Bytes.toBytes(columnName), + new Pair(Bytes.toBytes(columnName), descriptor)), null); } @@ -1126,7 +1126,7 @@ public class HBaseAdmin { throws IOException { modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_MAJOR_COMPACT); } - + /** * Major compacts a column family within a region or table. * Asynchronous operation. @@ -1181,7 +1181,7 @@ public class HBaseAdmin { } /** - * Split a table or an individual region. Implicitly finds an optimal split + * Split a table or an individual region. Implicitly finds an optimal split * point. Asynchronous operation. * * @param tableNameOrRegionName table to region to split @@ -1190,7 +1190,7 @@ public class HBaseAdmin { public void split(final byte [] tableNameOrRegionName) throws IOException { modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_SPLIT); } - + public void split(final String tableNameOrRegionName, final String splitPoint) throws IOException { split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint)); @@ -1204,7 +1204,7 @@ public class HBaseAdmin { * @param splitPoint the explicit position to split on * @throws IOException if a remote or network exception occurs */ - public void split(final byte [] tableNameOrRegionName, + public void split(final byte [] tableNameOrRegionName, final byte [] splitPoint) throws IOException { if (tableNameOrRegionName == null) { throw new IllegalArgumentException("Pass a table name or region name"); @@ -1212,7 +1212,7 @@ public class HBaseAdmin { byte [] tableName = tableExists(tableNameOrRegionName)? tableNameOrRegionName: null; byte [] regionName = tableName == null? tableNameOrRegionName: null; - Object [] args = regionName == null? + Object [] args = regionName == null? new byte [][] {splitPoint}: new byte [][] {regionName, splitPoint}; modifyTable(tableName, HConstants.Modify.TABLE_EXPLICIT_SPLIT, args); } @@ -1321,7 +1321,7 @@ public class HBaseAdmin { throw RemoteExceptionHandler.decodeRemoteException(e); } } - + private static Writable toWritable(Object o) { if (o == null) { return null; @@ -1360,7 +1360,7 @@ public class HBaseAdmin { /** * Stop the designated RegionServer for a restart. - * + * * @param hsa * the address of the RegionServer to stop * @throws IOException @@ -1372,10 +1372,10 @@ public class HBaseAdmin { LOG.info("Restarting RegionServer" + hsa.toString()); rs.stopForRestart(); } - + /** * Stop the designated RegionServer for a stop. - * + * * @param hsa * the address of the RegionServer to stop * @para message Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java Wed Apr 2 20:49:37 2014 @@ -20,21 +20,21 @@ package org.apache.hadoop.hbase.client; +import java.io.IOException; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.ipc.HBaseRPCOptions; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.hbase.ipc.HBaseRPCOptions; - -import java.io.IOException; /** * Scanner class that contains the .META. table scanning logic * and uses a Retryable scanner. Provided visitors will be called * for each row. - * + * * Although public visibility, this is not a public-facing API and may evolve in * minor releases. */ @@ -126,13 +126,15 @@ public class MetaScanner { HConstants.CATALOG_FAMILY); if (startRowResult == null) { throw new TableNotFoundException("Cannot find row in .META. for table: " - + Bytes.toString(tableName) + ", row=" + Bytes.toString(searchRow)); + + Bytes.toStringBinary(tableName) + ", row=" + + Bytes.toStringBinary(searchRow)); } byte[] value = startRowResult.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); if (value == null || value.length == 0) { - throw new IOException("HRegionInfo was null or empty in Meta for " + - Bytes.toString(tableName) + ", row=" + Bytes.toString(searchRow)); + throw new IOException("HRegionInfo was null or empty in Meta for " + + Bytes.toStringBinary(tableName) + ", row=" + + Bytes.toStringBinary(searchRow)); } HRegionInfo regionInfo = Writables.getHRegionInfo(value); Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/ResultScannerImpl.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/ResultScannerImpl.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/ResultScannerImpl.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/ResultScannerImpl.java Wed Apr 2 20:49:37 2014 @@ -61,7 +61,7 @@ public abstract class ResultScannerImpl } if (CLIENT_LOG.isDebugEnabled()) { CLIENT_LOG.debug("Creating scanner over " - + Bytes.toString(htable.getTableName()) + + Bytes.toStringBinary(htable.getTableName()) + " starting at key '" + Bytes.toStringBinary(scan.getStartRow()) + "'"); } } Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/TableServers.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/TableServers.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/TableServers.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/TableServers.java Wed Apr 2 20:49:37 2014 @@ -485,7 +485,7 @@ public class TableServers implements Ser private boolean testTableOnlineState(byte[] tableName, boolean online) throws IOException { if (!tableExists(tableName)) { - throw new TableNotFoundException(Bytes.toString(tableName)); + throw new TableNotFoundException(Bytes.toStringBinary(tableName)); } if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) { // The root region is always enabled @@ -580,7 +580,7 @@ public class TableServers implements Ser MetaScanner.metaScan(conf, finder, tableName); HTableDescriptor result = finder.getResult(); if (result == null) { - throw new TableNotFoundException(Bytes.toString(tableName)); + throw new TableNotFoundException(Bytes.toStringBinary(tableName)); } return result; } @@ -857,20 +857,20 @@ public class TableServers implements Ser private HRegionLocation getLocationFromRow(Result regionInfoRow, byte[] tableName, byte[] parentTable, byte[] row) throws IOException { if (regionInfoRow == null) { - throw new TableNotFoundException(Bytes.toString(tableName)); + throw new TableNotFoundException(Bytes.toStringBinary(tableName)); } byte[] value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); if (value == null || value.length == 0) { throw new IOException("HRegionInfo was null or empty in " - + Bytes.toString(parentTable) + ", row=" + regionInfoRow); + + Bytes.toStringBinary(parentTable) + ", row=" + regionInfoRow); } // convert the row result into the HRegionLocation we need! HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(value, new HRegionInfo()); // possible we got a region of a different table... if (!Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) { - throw new TableNotFoundException("Table '" + Bytes.toString(tableName) + throw new TableNotFoundException("Table '" + Bytes.toStringBinary(tableName) + "' was not found."); } if (regionInfo.isOffline()) { @@ -886,7 +886,7 @@ private HRegionLocation getLocationFromR } if (serverAddress.equals("")) { throw new NoServerForRegionException("No server address listed " - + "in " + Bytes.toString(parentTable) + " for region " + + "in " + Bytes.toStringBinary(parentTable) + " for region " + regionInfo.getRegionNameAsString() + " containing row " + Bytes.toStringBinary(row)); } @@ -1127,7 +1127,7 @@ private HRegionLocation locateMetaInRoot if (rl != null) { if (LOG.isTraceEnabled()) { LOG.trace("Cache hit for row <" + Bytes.toStringBinary(row) - + "> in tableName " + Bytes.toString(tableName) + + "> in tableName " + Bytes.toStringBinary(tableName) + ": location server " + rl.getServerAddress() + ", location region name " + rl.getRegionInfo().getRegionNameAsString()); @@ -1184,7 +1184,7 @@ private HRegionLocation locateMetaInRoot tableLocations.remove(rl.getRegionInfo().getStartKey()); if (LOG.isDebugEnabled()) { LOG.debug("Removed " + rl.getRegionInfo().getRegionNameAsString() - + " for tableName=" + Bytes.toString(tableName) + + " for tableName=" + Bytes.toStringBinary(tableName) + " from cache " + "because of " + Bytes.toStringBinary(row)); } } @@ -2014,7 +2014,7 @@ private HRegionLocation locateMetaInRoot if (location == null) { throw new RetriesExhaustedException( " -- nothing found, no 'location' returned," + " tableName=" - + Bytes.toString(tableName) + ", reload=" + reload + " --", + + Bytes.toStringBinary(tableName) + ", reload=" + reload + " --", HConstants.EMPTY_BYTE_ARRAY, rowKey, tries, exceptions); } return location; Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java Wed Apr 2 20:49:37 2014 @@ -26,7 +26,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -88,35 +87,41 @@ public class HalfStoreFileReader extends final HFileScanner delegate = s; public boolean atEnd = false; + @Override public ByteBuffer getKey() { if (atEnd) return null; return delegate.getKey(); } + @Override public String getKeyString() { if (atEnd) return null; return delegate.getKeyString(); } + @Override public ByteBuffer getValue() { if (atEnd) return null; return delegate.getValue(); } + @Override public String getValueString() { if (atEnd) return null; return delegate.getValueString(); } + @Override public KeyValue getKeyValue() { if (atEnd) return null; return delegate.getKeyValue(); } + @Override public boolean next() throws IOException { if (atEnd) return false; @@ -136,10 +141,12 @@ public class HalfStoreFileReader extends return true; } + @Override public boolean seekBefore(byte[] key) throws IOException { return seekBefore(key, 0, key.length); } + @Override public boolean seekBefore(byte [] key, int offset, int length) throws IOException { if (top) { @@ -156,6 +163,7 @@ public class HalfStoreFileReader extends return this.delegate.seekBefore(key, offset, length); } + @Override public boolean seekTo() throws IOException { if (top) { int r = this.delegate.seekTo(splitkey); @@ -180,10 +188,12 @@ public class HalfStoreFileReader extends splitkey, 0, splitkey.length) < 0; } + @Override public int seekTo(byte[] key) throws IOException { return seekTo(key, 0, key.length); } + @Override public int seekTo(byte[] key, int offset, int length) throws IOException { if (top) { if (getComparator().compare(key, offset, length, splitkey, 0, @@ -240,10 +250,12 @@ public class HalfStoreFileReader extends return delegate.reseekTo(key, offset, length); } + @Override public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() { return this.delegate.getReader(); } + @Override public boolean isSeeked() { return this.delegate.isSeeked(); } @@ -272,7 +284,7 @@ public class HalfStoreFileReader extends return Bytes.toBytes(scanner.getKey()); } } catch (IOException e) { - LOG.warn("Failed seekBefore " + Bytes.toString(this.splitkey), e); + LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e); } return null; } Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java Wed Apr 2 20:49:37 2014 @@ -197,16 +197,16 @@ abstract class BaseScanner extends Chore this.master.getRegionManager().getAssignmentManager(); if (favoredNodes != null) { // compare the update TS - long updateTimeStamp = + long updateTimeStamp = values.getLastestTimeStamp(HConstants.CATALOG_FAMILY, HConstants.FAVOREDNODES_QUALIFIER); - long lastUpdate = + long lastUpdate = assignmentManager.getAssignmentPlanUpdateTimeStamp(region); if (lastUpdate < updateTimeStamp) { // need to update the persistent assignment List servers = RegionPlacement.getFavoredNodesList(favoredNodes); - assignmentManager.updateAssignmentPlan(region, + assignmentManager.updateAssignmentPlan(region, servers, updateTimeStamp); } } else { @@ -257,8 +257,9 @@ abstract class BaseScanner extends Chore // First clean up any meta region rows which had null HRegionInfos if (emptyRows.size() > 0) { - LOG.warn("Found " + emptyRows.size() + " rows with empty HRegionInfo " + - "while scanning meta region " + Bytes.toString(metaRegion.getRegionName())); + LOG.warn("Found " + emptyRows.size() + " rows with empty HRegionInfo " + + "while scanning meta region " + + Bytes.toStringBinary(metaRegion.getRegionName())); this.master.deleteEmptyMetaRows(regionServer, metaRegion.getRegionName(), emptyRows); } @@ -539,6 +540,7 @@ abstract class BaseScanner extends Chore // instance of PathFilter. FileStatus [] ps = this.master.getFileSystem().listStatus(p, new PathFilter () { + @Override public boolean accept(Path path) { return StoreFile.isReference(path); } Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Wed Apr 2 20:49:37 2014 @@ -1329,11 +1329,11 @@ public class HMaster extends HasThread i if (existingValue != null) { if (entry.getValue() < existingValue) { if (LOG.isDebugEnabled()) { - LOG.debug("RegionServer " + serverInfo + - " indicates a last flushed sequence id (" + entry.getValue() + - ") that is less than the previous last flushed sequence id (" + - existingValue + ") for region " + - Bytes.toString(entry.getKey()) + " Ignoring."); + LOG.debug("RegionServer " + serverInfo + + " indicates a last flushed sequence id (" + entry.getValue() + + ") that is less than the previous last flushed sequence id (" + + existingValue + ") for region " + + Bytes.toStringBinary(entry.getKey()) + " Ignoring."); } continue; // Don't let smaller sequence ids override greater // sequence ids. @@ -1660,7 +1660,7 @@ public class HMaster extends HasThread i } finally { unlockTable(tableName); } - LOG.info("deleted table: " + Bytes.toString(tableName)); + LOG.info("deleted table: " + Bytes.toStringBinary(tableName)); } @Override @@ -1672,8 +1672,9 @@ public class HMaster extends HasThread i int maxConcurrentRegionsClosed) throws IOException { //This lock will be released when the ThrottledRegionReopener is done. if (!tryLockTable(tableName, "alter", schemaChangeTryLockTimeoutMs)) { - throw new TableLockTimeoutException("Timed out acquiring " + - "lock for " + Bytes.toString(tableName) + " after " + schemaChangeTryLockTimeoutMs + " ms."); + throw new TableLockTimeoutException("Timed out acquiring lock for " + + Bytes.toStringBinary(tableName) + " after " + + schemaChangeTryLockTimeoutMs + " ms."); } InjectionHandler.processEvent(InjectionEvent.HMASTER_ALTER_TABLE); @@ -1946,7 +1947,8 @@ public class HMaster extends HasThread i byte [] rowKey = ((ImmutableBytesWritable)args[0]).get(); pair = getTableRegionForRow(tableName, rowKey); } - LOG.info("About to " + op.toString() + " on " + Bytes.toString(tableName) + " and pair is " + pair); + LOG.info("About to " + op.toString() + " on " + + Bytes.toStringBinary(tableName) + " and pair is " + pair); if (pair != null && pair.getSecond() != null) { // If the column family name is specified, we need to perform a // column family specific action instead of an action on the whole @@ -2000,10 +2002,10 @@ public class HMaster extends HasThread i } HRegionInfo r = pair.getFirst(); r.setSplitPoint(splitPoint); - LOG.info("About to " + op.toString() + " on " + - Bytes.toString(pair.getFirst().getTableDesc().getName()) + - " at " + Bytes.toString(splitPoint) + - " and pair is " + pair); + LOG.info("About to " + op.toString() + " on " + + Bytes.toStringBinary(pair.getFirst().getTableDesc().getName()) + + " at " + Bytes.toStringBinary(splitPoint) + " and pair is " + + pair); if (pair.getSecond() != null) { this.regionManager.startAction(pair.getFirst().getRegionName(), pair.getFirst(), pair.getSecond(), Modify.TABLE_SPLIT); @@ -2150,12 +2152,12 @@ public class HMaster extends HasThread i for (byte [] regionName: emptyRows) { try { HRegion.removeRegionFromMETA(s, metaRegionName, regionName); - LOG.warn("Removed region: " + Bytes.toString(regionName) + - " from meta region: " + - Bytes.toString(metaRegionName) + " because HRegionInfo was empty"); + LOG.warn("Removed region: " + Bytes.toStringBinary(regionName) + + " from meta region: " + Bytes.toStringBinary(metaRegionName) + + " because HRegionInfo was empty"); } catch (IOException e) { - LOG.error("deleting region: " + Bytes.toString(regionName) + - " from meta region: " + Bytes.toString(metaRegionName), e); + LOG.error("deleting region: " + Bytes.toStringBinary(regionName) + + " from meta region: " + Bytes.toStringBinary(metaRegionName), e); } } } Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java Wed Apr 2 20:49:37 2014 @@ -19,6 +19,8 @@ */ package org.apache.hadoop.hbase.master; +import java.io.IOException; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; @@ -30,9 +32,6 @@ import org.apache.hadoop.hbase.ipc.HRegi import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; -import java.io.IOException; -import java.util.concurrent.ExecutionException; - /** Instantiated to modify table descriptor metadata */ class ModifyTableMeta extends TableOperation { @@ -45,8 +44,7 @@ class ModifyTableMeta extends TableOpera throws IOException { super(master, tableName); this.desc = desc; - LOG.debug("modifying " + Bytes.toString(tableName) + ": " + - desc.toString()); + LOG.debug("modifying " + Bytes.toStringBinary(tableName) + ": " + desc); } protected void updateRegionInfo(HRegionInterface server, byte [] regionName, @@ -63,7 +61,7 @@ class ModifyTableMeta extends TableOpera protected void processScanItem(String serverName, final HRegionInfo info) throws IOException { if (isEnabled(info)) { - throw new TableNotDisabledException(Bytes.toString(tableName)); + throw new TableNotDisabledException(Bytes.toStringBinary(tableName)); } } Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java Wed Apr 2 20:49:37 2014 @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; -import java.util.concurrent.ExecutionException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -109,11 +108,12 @@ public class ProcessRegionOpen extends P Bytes.toBytes(serverInfo.getStartCode())); server.put(region.getRegionName(), p); LOG.info("Updated row " + regionInfo.getRegionNameAsString() + " in region " - + Bytes.toString(region.getRegionName()) + " with startcode=" + serverInfo.getStartCode() + + Bytes.toStringBinary(region.getRegionName()) + + " with startcode=" + serverInfo.getStartCode() + ", server=" + serverInfo.getHostnamePort()); this.master.getServerManager().getRegionChecker().becameOpened(regionInfo); } - + @Override protected int getPriority() { return 0; // highest priority Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java Wed Apr 2 20:49:37 2014 @@ -153,7 +153,7 @@ class ProcessServerShutdown extends Regi byte [] regionName) throws IOException { long scannerId = server.openScanner(regionName, scan); - + int rows = scan.getCaching(); // The default caching if not set for scans is -1. Handle it if (rows < 1) rows = 1; @@ -196,8 +196,8 @@ class ProcessServerShutdown extends Regi } if (LOG.isDebugEnabled() && row != null) { - LOG.debug("Shutdown scanner for " + serverName + " processing " + - Bytes.toString(row)); + LOG.debug("Shutdown scanner for " + serverName + " processing " + + Bytes.toStringBinary(row)); } HRegionInfo info = master.getHRegionInfo(row, value); @@ -226,9 +226,9 @@ class ProcessServerShutdown extends Regi // Scan complete. Remove any rows which had empty HRegionInfos if (emptyRows.size() > 0) { - LOG.warn("Found " + emptyRows.size() + - " rows with empty HRegionInfo while scanning meta region " + - Bytes.toString(regionName) + " for " + this); + LOG.warn("Found " + emptyRows.size() + + " rows with empty HRegionInfo while scanning meta region " + + Bytes.toStringBinary(regionName) + " for " + this); master.deleteEmptyMetaRows(server, regionName, emptyRows); } @@ -240,12 +240,13 @@ class ProcessServerShutdown extends Regi if (e.info.isMetaTable() && !this.metaRegionsUnassigned.contains( new MetaRegion(this.deadServerAddress, e.info))) { if (LOG.isDebugEnabled()) { - LOG.debug("removing meta region " + - Bytes.toString(e.info.getRegionName()) + - " from online meta regions"); + LOG.debug("removing meta region " + + Bytes.toStringBinary(e.info.getRegionName()) + + " from online meta regions"); } - LOG.debug(this.toString() + " setting RegionManager.offlineMetaReginWithStartKey : " - + Bytes.toString(e.info.getStartKey())); + LOG.debug(this + + " setting RegionManager.offlineMetaReginWithStartKey : " + + Bytes.toStringBinary(e.info.getStartKey())); master.getRegionManager().offlineMetaRegionWithStartKey(e.info.getStartKey()); } @@ -266,7 +267,7 @@ class ProcessServerShutdown extends Regi } t2 = System.currentTimeMillis(); if (LOG.isDebugEnabled()) - LOG.debug("Took " + this.toString() + " " + (t2 - t0) + LOG.debug("Took " + this.toString() + " " + (t2 - t0) + " ms. to update RegionManager. And," + (t1 - t0) + " ms. to get the lock."); @@ -290,7 +291,7 @@ class ProcessServerShutdown extends Regi LOG.debug(this.toString() + (skip? "skipping set " : "setting ") + " unassigned: " + info.toString()); - if (skip) + if (skip) continue; this.setRegionUnassigned(info, false); @@ -302,10 +303,10 @@ class ProcessServerShutdown extends Regi } } t2 = System.currentTimeMillis(); - + if (LOG.isDebugEnabled()) - LOG.debug("Took " + this.toString() + " " - + (t1 - t0 ) + " ms. to mark regions offlineInMeta" + LOG.debug("Took " + this.toString() + " " + + (t1 - t0 ) + " ms. to mark regions offlineInMeta" + (t2 - t1) + " ms. to set " + regions.size() + " regions unassigned"); return true; } @@ -315,6 +316,7 @@ class ProcessServerShutdown extends Regi super(m, master); } + @Override public Boolean call() throws IOException { if (LOG.isDebugEnabled()) { HServerAddress addr = master.getRegionManager().getRootRegionLocation(); @@ -355,10 +357,11 @@ class ProcessServerShutdown extends Regi super(m, master); } + @Override public Boolean call() throws IOException { if (LOG.isDebugEnabled()) { - LOG.debug(ProcessServerShutdown.this.toString() + " scanning " + - Bytes.toString(m.getRegionName()) + " on " + m.getServer()); + LOG.debug(ProcessServerShutdown.this.toString() + " scanning " + + Bytes.toStringBinary(m.getRegionName()) + " on " + m.getServer()); } Scan scan = new Scan(); scan.addFamily(HConstants.CATALOG_FAMILY); @@ -483,20 +486,20 @@ class ProcessServerShutdown extends Regi if (successfulMetaScans.contains(Bytes.toString(r.getRegionName()))) continue; if (LOG.isDebugEnabled()) { - LOG.debug(this.toString() + ". Begin scan meta region " + - Bytes.toString(r.getRegionName()) + " on " + r.getServer()); + LOG.debug(this.toString() + ". Begin scan meta region " + + Bytes.toStringBinary(r.getRegionName()) + " on " + r.getServer()); } Boolean result = new ScanMetaRegions(r, this.master).doWithRetries(); if (result == null || result.booleanValue() == false) { - LOG.debug("Meta scan failed " + - Bytes.toString(r.getRegionName()) + " on " + r.getServer()); + LOG.debug("Meta scan failed " + Bytes.toStringBinary(r.getRegionName()) + + " on " + r.getServer()); return RegionServerOperationResult.OPERATION_DELAYED; } successfulMetaScans.add(Bytes.toString(r.getRegionName())); if (LOG.isDebugEnabled()) { - LOG.debug(this.toString() + ". finished scanning " + - Bytes.toString(r.getRegionName()) + " on " + r.getServer()); + LOG.debug(this.toString() + ". finished scanning " + + Bytes.toStringBinary(r.getRegionName()) + " on " + r.getServer()); } } Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RetryableMetaOperation.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RetryableMetaOperation.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RetryableMetaOperation.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RetryableMetaOperation.java Wed Apr 2 20:49:37 2014 @@ -20,6 +20,11 @@ package org.apache.hadoop.hbase.master; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.RemoteExceptionHandler; @@ -30,11 +35,6 @@ import org.apache.hadoop.hbase.util.Byte import org.apache.hadoop.hbase.util.Sleeper; import org.apache.hadoop.ipc.RemoteException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.Callable; - /** * Uses Callable pattern so that operations against meta regions do not need * to duplicate retry logic. @@ -77,9 +77,9 @@ abstract class RetryableMetaOperation if (tries == this.master.getNumRetries() - 1) { if (LOG.isDebugEnabled()) { StringBuilder message = new StringBuilder( - "Trying to contact region server for regionName '" + - Bytes.toString(m.getRegionName()) + "', but failed after " + - (tries + 1) + " attempts.\n"); + "Trying to contact region server for regionName '" + + Bytes.toStringBinary(m.getRegionName()) + + "', but failed after " + (tries + 1) + " attempts.\n"); int i = 1; for (IOException e2 : exceptions) { message.append("Exception " + i + ":\n" + e2); @@ -100,4 +100,4 @@ abstract class RetryableMetaOperation } return null; } -} \ No newline at end of file +} Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java Wed Apr 2 20:49:37 2014 @@ -54,11 +54,11 @@ import org.apache.hadoop.hbase.ipc.HRegi import org.apache.hadoop.hbase.master.RegionManager.RegionState; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.HasThread; import org.apache.hadoop.hbase.util.Threads; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher.Event.EventType; -import org.apache.hadoop.hbase.util.HasThread; /** * The ServerManager class manages info about region servers - HServerInfo, @@ -89,10 +89,10 @@ public class ServerManager implements Co /* * Set of messages that we want to send to a server. - * - * Most of the messages that we wish to send to a server is generated and - * sent during processMsgs. The only exception is when we process a region - * open. processRegionOpen gets called from handleRegionOpenedEvent in + * + * Most of the messages that we wish to send to a server is generated and + * sent during processMsgs. The only exception is when we process a region + * open. processRegionOpen gets called from handleRegionOpenedEvent in * response to the ZK event. It stores the intended message (for example: * MSG_REGION_CLOSE_WITHOUT_REPORT, in the case of duplicate assignment) * to be piggybacked upon the next processMsgs; @@ -228,7 +228,7 @@ public class ServerManager implements Co rackManager = new RackManager(c); Threads.setDaemonThreadRunning(new ServerTimeoutMonitor(c), n + "ServerManager-Timeout-Monitor"); - + this.pendingMsgsToSvrsMap = new ConcurrentHashMap>(); this.regionChecker = new RegionChecker(master); @@ -353,7 +353,7 @@ public class ServerManager implements Co // Could not set a watch, undo the above changes and re-throw. serversToLoad.updateServerLoad(serverName, oldServerLoad); undoMapUpdate(serversToServerInfo, serverName, oldServerInfo); - LOG.error("Could not set watch on regionserver znode for " + serverName); + LOG.error("Could not set watch on regionserver znode for " + serverName); throw ex; } } @@ -396,7 +396,7 @@ public class ServerManager implements Co } else { LOG.info("Server " + serverInfo.getServerName() + " sent preparing to shutdown, " + - "but that server probably already exited"); + "but that server probably already exited"); } return HMsg.EMPTY_HMSG_ARRAY; } else if (msgs[0].isType(HMsg.Type.MSG_REPORT_EXITING)) { @@ -693,30 +693,30 @@ public class ServerManager implements Co // Send any pending table actions. this.master.getRegionManager().applyActions(serverInfo, returnMsgs); - // add any pending messages that we may be holding for the server + // add any pending messages that we may be holding for the server piggyBackPendingMessages(serverInfo, returnMsgs); } return returnMsgs.toArray(new HMsg[returnMsgs.size()]); } - + /* * Holds a set of messages that we want to send to a server. - * + * * Most of the messages that we wish to send to a server is generated and sent * during processMsgs. The only exception is when we process a region open. - * In this case, processRegionOpen gets called from handleRegionOpenedEvent in - * response to the ZK event. - * - * This method stores the intended message (for example: + * In this case, processRegionOpen gets called from handleRegionOpenedEvent in + * response to the ZK event. + * + * This method stores the intended message (for example: * MSG_REGION_CLOSE_WITHOUT_REPORT, in the case of duplicate assignment ) to be * piggybacked upon the next processMsgs; - * + * * @param serverInfo The server for whom the messages are intended * @param msgsToSend Messages to send */ public void holdMessages(HServerInfo serverInfo, ArrayList msgsToSend) { ArrayList msgsForServer = pendingMsgsToSvrsMap.get(serverInfo); - + if (msgsForServer == null) { msgsForServer = new ArrayList(); ArrayList newMsgsForServer = @@ -726,22 +726,22 @@ public class ServerManager implements Co msgsForServer = newMsgsForServer; } } - + synchronized(msgsForServer) { msgsForServer.addAll(msgsToSend); } } - + /* * Get the set of messages that we want to send to a server. - * + * * @param serverInfo The server whose messages we want to get * @param returnMsgs List to which pending messages are added. */ public void piggyBackPendingMessages(HServerInfo serverInfo, List returnMsgs) { ArrayList msgsForServer = pendingMsgsToSvrsMap.get(serverInfo); - + if (msgsForServer != null) { synchronized(msgsForServer) { returnMsgs.addAll(msgsForServer); @@ -865,8 +865,9 @@ public class ServerManager implements Co } if (duplicateAssignment) { - LOG.warn("region server " + serverInfo.getServerAddress().toString() + - " should not have opened region " + Bytes.toString(region.getRegionName())); + LOG.warn("region server " + serverInfo.getServerAddress().toString() + + " should not have opened region " + + Bytes.toStringBinary(region.getRegionName())); // This Region should not have been opened. // Ask the server to shut it down, but don't report it as closed. @@ -937,7 +938,7 @@ public class ServerManager implements Co // setClosed works for both CLOSING, and PENDING_CLOSE this.master.getRegionManager().setClosed(region.getRegionNameAsString()); RegionServerOperation op = - new ProcessRegionClose(master, serverInfo.getServerName(), + new ProcessRegionClose(master, serverInfo.getServerName(), region, offlineRegion, reassignRegion); this.master.getRegionServerOperationQueue().put(op); @@ -955,19 +956,19 @@ public class ServerManager implements Co private void processServerInfoOnShutdown(HServerInfo info) { String serverName = info.getServerName(); this.master.getRegionManager().offlineMetaServer(info.getServerAddress()); - + //HBASE-1928: Check whether this server has been transitioning the ROOT table if (this.master.getRegionManager().isRootInTransitionOnThisServer(serverName)) { this.master.getRegionManager().unsetRootRegion(); this.master.getRegionManager().reassignRootRegion(); } - + //HBASE-1928: Check whether this server has been transitioning the META table HRegionInfo metaServerRegionInfo = this.master.getRegionManager().getMetaServerRegionInfo (serverName); if (metaServerRegionInfo != null) { this.master.getRegionManager().setUnassigned(metaServerRegionInfo, true); } - + return; } @@ -1253,7 +1254,7 @@ public class ServerManager implements Co public void run() { try { while (true) { - boolean waitingForMoreServersInRackToTimeOut = + boolean waitingForMoreServersInRackToTimeOut = expireTimedOutServers(timeout, maxServersToExpirePerRack); if (waitingForMoreServersInRackToTimeOut) { sleep(shortTimeout/2); Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableDelete.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableDelete.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableDelete.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableDelete.java Wed Apr 2 20:49:37 2014 @@ -19,6 +19,8 @@ */ package org.apache.hadoop.hbase.master; +import java.io.IOException; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; @@ -29,8 +31,6 @@ import org.apache.hadoop.hbase.ipc.HRegi import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.util.Bytes; -import java.io.IOException; - /** * Instantiated to delete a table. Table must be offline. */ @@ -66,7 +66,8 @@ class TableDelete extends TableOperation this.master.getRootDir(), i); } catch (IOException e) { - LOG.error("failed to delete region " + Bytes.toString(i.getRegionName()), + LOG.error("failed to delete region " + + Bytes.toStringBinary(i.getRegionName()), RemoteExceptionHandler.checkIOException(e)); } } Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableOperation.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableOperation.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableOperation.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/TableOperation.java Wed Apr 2 20:49:37 2014 @@ -19,11 +19,15 @@ */ package org.apache.hadoop.hbase.master; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.ExecutionException; + import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RemoteExceptionHandler; @@ -33,13 +37,6 @@ import org.apache.hadoop.hbase.client.Sc import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.util.Bytes; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.ExecutionException; - /** * Abstract base class for operations that need to examine all HRegionInfo * objects in a table. (For a table, operate on each of its rows @@ -52,7 +49,7 @@ abstract class TableOperation { protected final Set unservedRegions = new TreeSet(); protected final Set regionsToProcess = new TreeSet(); protected HMaster master; - + protected TableOperation(final HMaster master, final byte [] tableName) throws IOException { this.master = master; @@ -82,12 +79,12 @@ abstract class TableOperation { tableOp = operation; } + @Override public Boolean call() throws IOException, InterruptedException, ExecutionException { boolean tableExists = false; // Open a scanner on the meta region - byte [] tableNameMetaStart = - Bytes.toBytes(Bytes.toString(tableName) + ",,"); + byte[] tableNameMetaStart = Bytes.add(tableName, Bytes.toBytes(",,")); final Scan scan = new Scan(tableNameMetaStart) .addFamily(HConstants.CATALOG_FAMILY); long scannerId = this.server.openScanner(m.getRegionName(), scan); @@ -106,8 +103,8 @@ abstract class TableOperation { this.master.getHRegionInfo(values.getRow(), values); if (info == null) { emptyRows.add(values.getRow()); - LOG.error(Bytes.toString(HConstants.CATALOG_FAMILY) + ":" - + Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + LOG.error(Bytes.toStringBinary(HConstants.CATALOG_FAMILY) + ":" + + Bytes.toStringBinary(HConstants.REGIONINFO_QUALIFIER) + " not found on " + Bytes.toStringBinary(values.getRow())); continue; @@ -124,7 +121,7 @@ abstract class TableOperation { tableExists = true; if(tableOp instanceof AddColumn || tableOp instanceof ModifyColumn || - tableOp instanceof DeleteColumn || + tableOp instanceof DeleteColumn || tableOp instanceof MultiColumnOperation) { regionsToProcess.add(info); } @@ -148,16 +145,16 @@ abstract class TableOperation { // Get rid of any rows that have a null HRegionInfo if (emptyRows.size() > 0) { - LOG.warn("Found " + emptyRows.size() + - " rows with empty HRegionInfo while scanning meta region " + - Bytes.toString(m.getRegionName())); + LOG.warn("Found " + emptyRows.size() + + " rows with empty HRegionInfo while scanning meta region " + + Bytes.toStringBinary(m.getRegionName())); master.deleteEmptyMetaRows(server, m.getRegionName(), emptyRows); } if (!tableExists) { - throw new TableNotFoundException(Bytes.toString(tableName)); + throw new TableNotFoundException(Bytes.toStringBinary(tableName)); } - + postProcessMeta(m, server); unservedRegions.clear(); return Boolean.TRUE; Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Wed Apr 2 20:49:37 2014 @@ -2618,13 +2618,15 @@ public class HRegionServer implements HR @Override public Object call() throws IOException { if (LOG.isDebugEnabled()) { - LOG.debug("closing region " + Bytes.toString(region.getRegionName())); + LOG.debug("closing region " + + Bytes.toStringBinary(region.getRegionName())); } try { region.close(abortRequested); removeFromOnlineRegions(region.getRegionInfo()); } catch (IOException e) { - cleanup(e, "Error closing " + Bytes.toString(region.getRegionName())); + cleanup(e, + "Error closing " + Bytes.toStringBinary(region.getRegionName())); throw e; } return null; Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java Wed Apr 2 20:49:37 2014 @@ -19,7 +19,8 @@ */ package org.apache.hadoop.hbase.regionserver.wal; -import static org.apache.hadoop.hbase.util.FSUtils.*; +import static org.apache.hadoop.hbase.util.FSUtils.recoverFileLease; +import static org.apache.hadoop.hbase.util.FSUtils.tryRecoverFileLease; import java.io.DataInput; import java.io.DataOutput; @@ -755,7 +756,7 @@ public class HLog implements Syncable { LOG.debug(hlogName + "Found " + logsToRemove + " hlogs to remove" + " out of total " + this.outputfiles.size() + ";" + " oldest outstanding sequenceid is " + oldestOutstandingSeqNum + - " from region " + Bytes.toString(oldestRegion)); + " from region " + Bytes.toStringBinary(oldestRegion)); } for (Long seq : sequenceNumbers) { archiveLogFile(this.outputfiles.get(seq), seq); @@ -1236,6 +1237,7 @@ public class HLog implements Syncable { } } + @Override public void sync() throws IOException { sync(false); } @@ -1429,9 +1431,10 @@ public class HLog implements Syncable { this.closeLock.readLock().lock(); synchronized (oldestSeqNumsLock) { if (this.firstSeqWrittenInSnapshotMemstore.containsKey(regionName)) { - LOG.warn("Requested a startCacheFlush while firstSeqWrittenInSnapshotMemstore still" - + " contains " + Bytes.toString(regionName) + " . Did the previous flush fail?" - + " Will try to complete it"); + LOG.warn("Requested a startCacheFlush while " + + "firstSeqWrittenInSnapshotMemstore still contains " + + Bytes.toStringBinary(regionName) + " . Did the previous flush " + + "fail? Will try to complete it"); } else { // If we are flushing the entire memstore, remove the entry from the @@ -1498,8 +1501,8 @@ public class HLog implements Syncable { * by the failure gets restored to the memstore. */ public void abortCacheFlush(byte[] regionName) { - LOG.debug(hlogName + "Aborting cache flush of region " + - Bytes.toString(regionName)); + LOG.debug(hlogName + "Aborting cache flush of region " + + Bytes.toStringBinary(regionName)); this.closeLock.readLock().unlock(); } @@ -1945,10 +1948,10 @@ public class HLog implements Syncable { } catch (ExecutionException e) { throw (new IOException(e.getCause())); } catch (InterruptedException e1) { - String errorMsgr = "Writer for region " + - Bytes.toString(entry.getKey()) + - " was interrupted, however the write process should have " + - "finished. Throwing up "; + String errorMsgr = + "Writer for region " + Bytes.toStringBinary(entry.getKey()) + + " was interrupted, however the write process should have " + + "finished. Throwing up "; LOG.info(errorMsgr, e1); throw (InterruptedIOException)new InterruptedIOException( errorMsgr).initCause(e1); Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java Wed Apr 2 20:49:37 2014 @@ -120,7 +120,7 @@ public class ThriftServerRunner implemen private static ImplType DEFAULT_SERVER_TYPE = ImplType.THREADED_SELECTOR; private static String NOT_SUPPORTED_BY_PROXY_MSG = "Not supported by Thrift proxy"; - + /** An enum of server implementation selections */ enum ImplType { HS_HA("hsha", true, THsHaServer.class, false), @@ -178,7 +178,7 @@ public class ThriftServerRunner implemen if (confType == null) { return DEFAULT_SERVER_TYPE; } - + for (ImplType t : values()) { if (confType.equals(t.option)) { return t; @@ -488,7 +488,7 @@ public class ThriftServerRunner implemen * hash-map. * * @param scanner - * @return integer scanner id + * @return integer scanner id */ protected synchronized int addScanner(ResultScanner scanner) { int id = nextScannerId++; @@ -812,7 +812,7 @@ public class ThriftServerRunner implemen if (metrics != null) { metrics.incNumBatchGetRowKeys(rows.size()); } - + // For now, don't support ragged gets, with different columns per row // Probably pretty sensible indefinitely anyways. for (ByteBuffer row : rows) { @@ -921,11 +921,11 @@ public class ThriftServerRunner implemen boolean writeToWAL = false; for (Mutation m : mutations) { byte[][] famAndQf = KeyValue.parseColumn(getBytes(m.column)); - + // If this mutation has timestamp set, it takes precedence, otherwise we use the // timestamp provided in the argument. long effectiveTimestamp = getMutationTimestamp(m, timestamp); - + if (m.isDelete) { if (delete == null) { delete = new Delete(rowBytes); @@ -949,7 +949,7 @@ public class ThriftServerRunner implemen } } } - + if (delete != null) { delete.setWriteToWAL(writeToWAL); if (deletes != null) { @@ -958,7 +958,7 @@ public class ThriftServerRunner implemen processDelete(tableName, regionName, delete); } } - + if (put != null) { put.setWriteToWAL(writeToWAL); if (puts != null) { @@ -1390,7 +1390,7 @@ public class ThriftServerRunner implemen if (startRowResult == null) { throw new IOException("Cannot find row in .META., row=" - + Bytes.toString(searchRowBytes)); + + Bytes.toStringBinary(searchRowBytes)); } // find region start and end keys @@ -1398,7 +1398,8 @@ public class ThriftServerRunner implemen HConstants.REGIONINFO_QUALIFIER); if (value == null || value.length == 0) { throw new IOException("HRegionInfo REGIONINFO was null or " - + " empty in Meta for row=" + Bytes.toString(searchRowBytes)); + + " empty in Meta for row=" + + Bytes.toStringBinary(searchRowBytes)); } HRegionInfo regionInfo = Writables.getHRegionInfo(value); TRegionInfo region = new TRegionInfo(); @@ -1516,7 +1517,7 @@ public class ThriftServerRunner implemen public void flushRegion(ByteBuffer regionName, long ifOlderThanTS) throws TException, IOError { throw new TException(NOT_SUPPORTED_BY_PROXY_MSG); } - + } public static void registerFilters(Configuration conf) { @@ -1550,16 +1551,16 @@ public class ThriftServerRunner implemen /** * Update the given delete object. - * + * * @param delete the delete object to update * @param famAndQf family and qualifier. null or empty family means "delete from all CFs". * @param timestamp Delete at this timestamp and older. */ - private static void updateDelete(Delete delete, byte[][] famAndQf, long timestamp) { + private static void updateDelete(Delete delete, byte[][] famAndQf, long timestamp) { if (famAndQf.length == 1) { // Column qualifier not specified. if (famAndQf[0].length == 0) { - // Delete from all column families in the row. + // Delete from all column families in the row. delete.deleteRow(timestamp); } else { // Delete from all columns in the given column family Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/Bytes.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/Bytes.java?rev=1584172&r1=1584171&r2=1584172&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/Bytes.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/Bytes.java Wed Apr 2 20:49:37 2014 @@ -310,7 +310,7 @@ public class Bytes { * @param b Presumed UTF-8 encoded byte array. * @return String made from b */ - public static String toString(final byte [] b) { + public static String toString(final byte[] b) { if (b == null) { return null; }