Index: src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java =================================================================== --- src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java (revision 684990) +++ src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java (working copy) @@ -1,151 +1,151 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client; - -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.Map; - -import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Test batch updates - */ -public class TestBatchUpdate extends HBaseClusterTestCase { - private static final String CONTENTS_STR = "contents:"; - private static final byte [] CONTENTS = Bytes.toBytes(CONTENTS_STR); - private static final String SMALLFAM_STR = "smallfam:"; - private static final byte [] SMALLFAM = Bytes.toBytes(SMALLFAM_STR); - private static final int SMALL_LENGTH = 1; - private static final int NB_BATCH_ROWS = 10; - private byte[] value; - private byte[] smallValue; - - private HTableDescriptor desc = null; - private HTable table = null; - - /** - * @throws UnsupportedEncodingException - */ - public TestBatchUpdate() throws UnsupportedEncodingException { - super(); - value = "abcd".getBytes(HConstants.UTF8_ENCODING); - smallValue = "a".getBytes(HConstants.UTF8_ENCODING); - } - - /** - * {@inheritDoc} - */ - @Override - public void setUp() throws Exception { - super.setUp(); - this.desc = new HTableDescriptor("test"); - desc.addFamily(new HColumnDescriptor(CONTENTS_STR)); - desc.addFamily(new HColumnDescriptor(SMALLFAM, - HColumnDescriptor.DEFAULT_VERSIONS, - HColumnDescriptor.DEFAULT_COMPRESSION, - HColumnDescriptor.DEFAULT_IN_MEMORY, - HColumnDescriptor.DEFAULT_BLOCKCACHE, SMALL_LENGTH, - HColumnDescriptor.DEFAULT_TTL, HColumnDescriptor.DEFAULT_BLOOMFILTER)); - HBaseAdmin admin = new HBaseAdmin(conf); - admin.createTable(desc); - table = new HTable(conf, desc.getName()); - } - - /** - * @throws IOException - */ - public void testBatchUpdate() throws IOException { - BatchUpdate bu = new BatchUpdate("row1"); - bu.put(CONTENTS, value); - bu.delete(CONTENTS); - table.commit(bu); - - bu = new BatchUpdate("row2"); - bu.put(CONTENTS, value); - table.commit(bu); - - byte [][] columns = { CONTENTS }; - Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); - for (RowResult r : scanner) { - for(Map.Entry e: r.entrySet()) { - System.out.println(r.getRow() + ": row: " + e.getKey() + " value: " + - new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); - } - } - } - - public void testBatchUpdateMaxLength() { - // Test for a single good value - BatchUpdate batchUpdate = new BatchUpdate("row1"); - batchUpdate.put(SMALLFAM, value); - try { - table.commit(batchUpdate); - fail("Value is too long, should throw exception"); - } catch (IOException e) { - // This is expected - } - // Try to see if it's still inserted - try { - Cell cell = table.get("row1", SMALLFAM_STR); - assertNull(cell); - } catch (IOException e) { - e.printStackTrace(); - fail("This is unexpected"); - } - // Try to put a good value - batchUpdate = new BatchUpdate("row1"); - batchUpdate.put(SMALLFAM, smallValue); - try { - table.commit(batchUpdate); - } catch (IOException e) { - fail("Value is long enough, should not throw exception"); - } - } - - public void testRowsBatchUpdate() { - ArrayList rowsUpdate = new ArrayList(); - for(int i = 0; i < NB_BATCH_ROWS; i++) { - BatchUpdate batchUpdate = new BatchUpdate("row"+i); - batchUpdate.put(CONTENTS, value); - rowsUpdate.add(batchUpdate); - } - try { - table.commit(rowsUpdate); - - byte [][] columns = { CONTENTS }; - Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); - int nbRows = 0; - for(RowResult row : scanner) - nbRows++; - assertEquals(NB_BATCH_ROWS, nbRows); - } catch (IOException e) { - fail("This is unexpected : " + e); - } - } -} +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.Map; + +import org.apache.hadoop.hbase.HBaseClusterTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Test batch updates + */ +public class TestBatchUpdate extends HBaseClusterTestCase { + private static final String CONTENTS_STR = "contents:"; + private static final byte [] CONTENTS = Bytes.toBytes(CONTENTS_STR); + private static final String SMALLFAM_STR = "smallfam:"; + private static final byte [] SMALLFAM = Bytes.toBytes(SMALLFAM_STR); + private static final int SMALL_LENGTH = 1; + private static final int NB_BATCH_ROWS = 10; + private byte[] value; + private byte[] smallValue; + + private HTableDescriptor desc = null; + private HTable table = null; + + /** + * @throws UnsupportedEncodingException + */ + public TestBatchUpdate() throws UnsupportedEncodingException { + super(); + value = "abcd".getBytes(HConstants.UTF8_ENCODING); + smallValue = "a".getBytes(HConstants.UTF8_ENCODING); + } + + /** + * {@inheritDoc} + */ + @Override + public void setUp() throws Exception { + super.setUp(); + this.desc = new HTableDescriptor("test"); + desc.addFamily(new HColumnDescriptor(CONTENTS_STR)); + desc.addFamily(new HColumnDescriptor(SMALLFAM, + HColumnDescriptor.DEFAULT_VERSIONS, + HColumnDescriptor.DEFAULT_COMPRESSION, + HColumnDescriptor.DEFAULT_IN_MEMORY, + HColumnDescriptor.DEFAULT_BLOCKCACHE, SMALL_LENGTH, + HColumnDescriptor.DEFAULT_TTL, HColumnDescriptor.DEFAULT_BLOOMFILTER)); + HBaseAdmin admin = new HBaseAdmin(conf); + admin.createTable(desc); + table = new HTable(conf, desc.getName()); + } + + /** + * @throws IOException + */ + public void testBatchUpdate() throws IOException { + BatchUpdate bu = new BatchUpdate("row1"); + bu.put(CONTENTS, value); + bu.delete(CONTENTS); + table.commit(bu); + + bu = new BatchUpdate("row2"); + bu.put(CONTENTS, value); + table.commit(bu); + + byte [][] columns = { CONTENTS }; + Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); + for (RowResult r : scanner) { + for(Map.Entry e: r.entrySet()) { + System.out.println(r.getRow() + ": row: " + e.getKey() + " value: " + + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); + } + } + } + + public void testBatchUpdateMaxLength() { + // Test for a single good value + BatchUpdate batchUpdate = new BatchUpdate("row1"); + batchUpdate.put(SMALLFAM, value); + try { + table.commit(batchUpdate); + fail("Value is too long, should throw exception"); + } catch (IOException e) { + // This is expected + } + // Try to see if it's still inserted + try { + Cell cell = table.get("row1", SMALLFAM_STR); + assertNull(cell); + } catch (IOException e) { + e.printStackTrace(); + fail("This is unexpected"); + } + // Try to put a good value + batchUpdate = new BatchUpdate("row1"); + batchUpdate.put(SMALLFAM, smallValue); + try { + table.commit(batchUpdate); + } catch (IOException e) { + fail("Value is long enough, should not throw exception"); + } + } + + public void testRowsBatchUpdate() { + ArrayList rowsUpdate = new ArrayList(); + for(int i = 0; i < NB_BATCH_ROWS; i++) { + BatchUpdate batchUpdate = new BatchUpdate("row"+i); + batchUpdate.put(CONTENTS, value); + rowsUpdate.add(batchUpdate); + } + try { + table.commit(rowsUpdate); + + byte [][] columns = { CONTENTS }; + Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); + int nbRows = 0; + for(RowResult row : scanner) + nbRows++; + assertEquals(NB_BATCH_ROWS, nbRows); + } catch (IOException e) { + fail("This is unexpected : " + e); + } + } +} Index: src/java/org/apache/hadoop/hbase/HConstants.java =================================================================== --- src/java/org/apache/hadoop/hbase/HConstants.java (revision 684990) +++ src/java/org/apache/hadoop/hbase/HConstants.java (working copy) @@ -1,235 +1,235 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase; - -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * HConstants holds a bunch of HBase-related constants - */ -public interface HConstants { - - /** long constant for zero */ - static final Long ZERO_L = Long.valueOf(0L); - - static final String NINES = "99999999999999"; - static final String ZEROES = "00000000000000"; - - // For migration - - /** name of version file */ - static final String VERSION_FILE_NAME = "hbase.version"; - - /** - * Current version of file system - * Version 4 supports only one kind of bloom filter - */ - public static final String FILE_SYSTEM_VERSION = "4"; - - // Configuration parameters - - // TODO: URL for hbase master like hdfs URLs with host and port. - // Like jdbc URLs? URLs could be used to refer to table cells? - // jdbc:mysql://[host][,failoverhost...][:port]/[database] - // jdbc:mysql://[host][,failoverhost...][:port]/[database][?propertyName1][=propertyValue1][&propertyName2][=propertyValue2]... - - // Key into HBaseConfiguration for the hbase.master address. - // TODO: Support 'local': i.e. default of all running in single - // process. Same for regionserver. TODO: Is having HBase homed - // on port 60k OK? - - /** Parameter name for master address */ - static final String MASTER_ADDRESS = "hbase.master"; - - /** default host address */ - static final String DEFAULT_HOST = "0.0.0.0"; - - /** default port that the master listens on */ - static final int DEFAULT_MASTER_PORT = 60000; - - /** Default master address */ - static final String DEFAULT_MASTER_ADDRESS = DEFAULT_HOST + ":" + - DEFAULT_MASTER_PORT; - - /** default port for master web api */ - static final int DEFAULT_MASTER_INFOPORT = 60010; - - /** Parameter name for hbase.regionserver address. */ - static final String REGIONSERVER_ADDRESS = "hbase.regionserver"; - - /** Default region server address */ - static final String DEFAULT_REGIONSERVER_ADDRESS = DEFAULT_HOST + ":60020"; - - /** default port for region server web api */ - static final int DEFAULT_REGIONSERVER_INFOPORT = 60030; - - /** Parameter name for what region server interface to use. */ - static final String REGION_SERVER_CLASS = "hbase.regionserver.class"; - - /** Parameter name for what region server implementation to use. */ - static final String REGION_SERVER_IMPL= "hbase.regionserver.impl"; - - /** Default region server interface class name. */ - static final String DEFAULT_REGION_SERVER_CLASS = HRegionInterface.class.getName(); - - /** Parameter name for how often threads should wake up */ - static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency"; - - /** Parameter name for HBase instance root directory */ - static final String HBASE_DIR = "hbase.rootdir"; - - /** Used to construct the name of the log directory for a region server */ - static final String HREGION_LOGDIR_NAME = "log"; - - /** Name of old log file for reconstruction */ - static final String HREGION_OLDLOGFILE_NAME = "oldlogfile.log"; - - /** Default maximum file size */ - static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024; - - /** Default size of a reservation block */ - static final int DEFAULT_SIZE_RESERVATION_BLOCK = 1024 * 1024 * 5; - - // Always store the location of the root table's HRegion. - // This HRegion is never split. - - // region name = table + startkey + regionid. This is the row key. - // each row in the root and meta tables describes exactly 1 region - // Do we ever need to know all the information that we are storing? - - // Note that the name of the root table starts with "-" and the name of the - // meta table starts with "." Why? it's a trick. It turns out that when we - // store region names in memory, we use a SortedMap. Since "-" sorts before - // "." (and since no other table name can start with either of these - // characters, the root region will always be the first entry in such a Map, - // followed by all the meta regions (which will be ordered by their starting - // row key as well), followed by all user tables. So when the Master is - // choosing regions to assign, it will always choose the root region first, - // followed by the meta regions, followed by user regions. Since the root - // and meta regions always need to be on-line, this ensures that they will - // be the first to be reassigned if the server(s) they are being served by - // should go down. - - /** The root table's name.*/ - static final byte [] ROOT_TABLE_NAME = Bytes.toBytes("-ROOT-"); - - /** The META table's name. */ - static final byte [] META_TABLE_NAME = Bytes.toBytes(".META."); - - // Defines for the column names used in both ROOT and META HBase 'meta' tables. - - /** The ROOT and META column family (string) */ - static final String COLUMN_FAMILY_STR = "info:"; - - /** The META historian column family (string) */ - static final String COLUMN_FAMILY_HISTORIAN_STR = "historian:"; - - /** The ROOT and META column family */ - static final byte [] COLUMN_FAMILY = Bytes.toBytes(COLUMN_FAMILY_STR); - - /** The META historian column family */ - static final byte [] COLUMN_FAMILY_HISTORIAN = Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR); - - /** Array of meta column names */ - static final byte[][] COLUMN_FAMILY_ARRAY = new byte[][] {COLUMN_FAMILY}; - - /** ROOT/META column family member - contains HRegionInfo */ - static final byte [] COL_REGIONINFO = - Bytes.toBytes(COLUMN_FAMILY_STR + "regioninfo"); - - /** Array of column - contains HRegionInfo */ - static final byte[][] COL_REGIONINFO_ARRAY = new byte[][] {COL_REGIONINFO}; - - /** ROOT/META column family member - contains HServerAddress.toString() */ - static final byte[] COL_SERVER = Bytes.toBytes(COLUMN_FAMILY_STR + "server"); - - /** ROOT/META column family member - contains server start code (a long) */ - static final byte [] COL_STARTCODE = - Bytes.toBytes(COLUMN_FAMILY_STR + "serverstartcode"); - - /** the lower half of a split region */ - static final byte [] COL_SPLITA = Bytes.toBytes(COLUMN_FAMILY_STR + "splitA"); - - /** the upper half of a split region */ - static final byte [] COL_SPLITB = Bytes.toBytes(COLUMN_FAMILY_STR + "splitB"); - - /** All the columns in the catalog -ROOT- and .META. tables. - */ - static final byte[][] ALL_META_COLUMNS = {COL_REGIONINFO, COL_SERVER, - COL_STARTCODE, COL_SPLITA, COL_SPLITB}; - - // Other constants - - /** - * An empty instance. - */ - static final byte [] EMPTY_BYTE_ARRAY = new byte [0]; - - /** - * Used by scanners, etc when they want to start at the beginning of a region - */ - static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY; - - /** - * Last row in a table. - */ - static final byte [] EMPTY_END_ROW = EMPTY_START_ROW; - - /** - * Used by scanners and others when they're trying to detect the end of a - * table - */ - static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY; - - /** When we encode strings, we always specify UTF8 encoding */ - static final String UTF8_ENCODING = "UTF-8"; - - /** - * Timestamp to use when we want to refer to the latest cell. - * This is the timestamp sent by clients when no timestamp is specified on - * commit. - */ - static final long LATEST_TIMESTAMP = Long.MAX_VALUE; - - /** - * Define for 'return-all-versions'. - */ - static final int ALL_VERSIONS = Integer.MAX_VALUE; - - /** - * Unlimited time-to-live. - */ - static final int FOREVER = -1; - - public static final String HBASE_CLIENT_RETRIES_NUMBER_KEY = - "hbase.client.retries.number"; - public static final int DEFAULT_CLIENT_RETRIES = 5; - - public static final String NAME = "NAME"; - public static final String VERSIONS = "VERSIONS"; - public static final String IN_MEMORY = "IN_MEMORY"; - - /** - * This is a retry backoff multiplier table similar to the BSD TCP syn - * backoff table, a bit more aggressive than simple exponential backoff. - */ - public static int RETRY_BACKOFF[] = { 1, 1, 1, 1, 2, 4, 8, 16, 32, 64 }; +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase; + +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * HConstants holds a bunch of HBase-related constants + */ +public interface HConstants { + + /** long constant for zero */ + static final Long ZERO_L = Long.valueOf(0L); + + static final String NINES = "99999999999999"; + static final String ZEROES = "00000000000000"; + + // For migration + + /** name of version file */ + static final String VERSION_FILE_NAME = "hbase.version"; + + /** + * Current version of file system + * Version 4 supports only one kind of bloom filter + */ + public static final String FILE_SYSTEM_VERSION = "4"; + + // Configuration parameters + + // TODO: URL for hbase master like hdfs URLs with host and port. + // Like jdbc URLs? URLs could be used to refer to table cells? + // jdbc:mysql://[host][,failoverhost...][:port]/[database] + // jdbc:mysql://[host][,failoverhost...][:port]/[database][?propertyName1][=propertyValue1][&propertyName2][=propertyValue2]... + + // Key into HBaseConfiguration for the hbase.master address. + // TODO: Support 'local': i.e. default of all running in single + // process. Same for regionserver. TODO: Is having HBase homed + // on port 60k OK? + + /** Parameter name for master address */ + static final String MASTER_ADDRESS = "hbase.master"; + + /** default host address */ + static final String DEFAULT_HOST = "0.0.0.0"; + + /** default port that the master listens on */ + static final int DEFAULT_MASTER_PORT = 60000; + + /** Default master address */ + static final String DEFAULT_MASTER_ADDRESS = DEFAULT_HOST + ":" + + DEFAULT_MASTER_PORT; + + /** default port for master web api */ + static final int DEFAULT_MASTER_INFOPORT = 60010; + + /** Parameter name for hbase.regionserver address. */ + static final String REGIONSERVER_ADDRESS = "hbase.regionserver"; + + /** Default region server address */ + static final String DEFAULT_REGIONSERVER_ADDRESS = DEFAULT_HOST + ":60020"; + + /** default port for region server web api */ + static final int DEFAULT_REGIONSERVER_INFOPORT = 60030; + + /** Parameter name for what region server interface to use. */ + static final String REGION_SERVER_CLASS = "hbase.regionserver.class"; + + /** Parameter name for what region server implementation to use. */ + static final String REGION_SERVER_IMPL= "hbase.regionserver.impl"; + + /** Default region server interface class name. */ + static final String DEFAULT_REGION_SERVER_CLASS = HRegionInterface.class.getName(); + + /** Parameter name for how often threads should wake up */ + static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency"; + + /** Parameter name for HBase instance root directory */ + static final String HBASE_DIR = "hbase.rootdir"; + + /** Used to construct the name of the log directory for a region server */ + static final String HREGION_LOGDIR_NAME = "log"; + + /** Name of old log file for reconstruction */ + static final String HREGION_OLDLOGFILE_NAME = "oldlogfile.log"; + + /** Default maximum file size */ + static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024; + + /** Default size of a reservation block */ + static final int DEFAULT_SIZE_RESERVATION_BLOCK = 1024 * 1024 * 5; + + // Always store the location of the root table's HRegion. + // This HRegion is never split. + + // region name = table + startkey + regionid. This is the row key. + // each row in the root and meta tables describes exactly 1 region + // Do we ever need to know all the information that we are storing? + + // Note that the name of the root table starts with "-" and the name of the + // meta table starts with "." Why? it's a trick. It turns out that when we + // store region names in memory, we use a SortedMap. Since "-" sorts before + // "." (and since no other table name can start with either of these + // characters, the root region will always be the first entry in such a Map, + // followed by all the meta regions (which will be ordered by their starting + // row key as well), followed by all user tables. So when the Master is + // choosing regions to assign, it will always choose the root region first, + // followed by the meta regions, followed by user regions. Since the root + // and meta regions always need to be on-line, this ensures that they will + // be the first to be reassigned if the server(s) they are being served by + // should go down. + + /** The root table's name.*/ + static final byte [] ROOT_TABLE_NAME = Bytes.toBytes("-ROOT-"); + + /** The META table's name. */ + static final byte [] META_TABLE_NAME = Bytes.toBytes(".META."); + + // Defines for the column names used in both ROOT and META HBase 'meta' tables. + + /** The ROOT and META column family (string) */ + static final String COLUMN_FAMILY_STR = "info:"; + + /** The META historian column family (string) */ + static final String COLUMN_FAMILY_HISTORIAN_STR = "historian:"; + + /** The ROOT and META column family */ + static final byte [] COLUMN_FAMILY = Bytes.toBytes(COLUMN_FAMILY_STR); + + /** The META historian column family */ + static final byte [] COLUMN_FAMILY_HISTORIAN = Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR); + + /** Array of meta column names */ + static final byte[][] COLUMN_FAMILY_ARRAY = new byte[][] {COLUMN_FAMILY}; + + /** ROOT/META column family member - contains HRegionInfo */ + static final byte [] COL_REGIONINFO = + Bytes.toBytes(COLUMN_FAMILY_STR + "regioninfo"); + + /** Array of column - contains HRegionInfo */ + static final byte[][] COL_REGIONINFO_ARRAY = new byte[][] {COL_REGIONINFO}; + + /** ROOT/META column family member - contains HServerAddress.toString() */ + static final byte[] COL_SERVER = Bytes.toBytes(COLUMN_FAMILY_STR + "server"); + + /** ROOT/META column family member - contains server start code (a long) */ + static final byte [] COL_STARTCODE = + Bytes.toBytes(COLUMN_FAMILY_STR + "serverstartcode"); + + /** the lower half of a split region */ + static final byte [] COL_SPLITA = Bytes.toBytes(COLUMN_FAMILY_STR + "splitA"); + + /** the upper half of a split region */ + static final byte [] COL_SPLITB = Bytes.toBytes(COLUMN_FAMILY_STR + "splitB"); + + /** All the columns in the catalog -ROOT- and .META. tables. + */ + static final byte[][] ALL_META_COLUMNS = {COL_REGIONINFO, COL_SERVER, + COL_STARTCODE, COL_SPLITA, COL_SPLITB}; + + // Other constants + + /** + * An empty instance. + */ + static final byte [] EMPTY_BYTE_ARRAY = new byte [0]; + + /** + * Used by scanners, etc when they want to start at the beginning of a region + */ + static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY; + + /** + * Last row in a table. + */ + static final byte [] EMPTY_END_ROW = EMPTY_START_ROW; + + /** + * Used by scanners and others when they're trying to detect the end of a + * table + */ + static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY; + + /** When we encode strings, we always specify UTF8 encoding */ + static final String UTF8_ENCODING = "UTF-8"; + + /** + * Timestamp to use when we want to refer to the latest cell. + * This is the timestamp sent by clients when no timestamp is specified on + * commit. + */ + static final long LATEST_TIMESTAMP = Long.MAX_VALUE; + + /** + * Define for 'return-all-versions'. + */ + static final int ALL_VERSIONS = Integer.MAX_VALUE; + + /** + * Unlimited time-to-live. + */ + static final int FOREVER = -1; + + public static final String HBASE_CLIENT_RETRIES_NUMBER_KEY = + "hbase.client.retries.number"; + public static final int DEFAULT_CLIENT_RETRIES = 5; + + public static final String NAME = "NAME"; + public static final String VERSIONS = "VERSIONS"; + public static final String IN_MEMORY = "IN_MEMORY"; + + /** + * This is a retry backoff multiplier table similar to the BSD TCP syn + * backoff table, a bit more aggressive than simple exponential backoff. + */ + public static int RETRY_BACKOFF[] = { 1, 1, 1, 1, 2, 4, 8, 16, 32, 64 }; } \ No newline at end of file Index: src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java =================================================================== --- src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java (revision 684990) +++ src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java (working copy) @@ -1,210 +1,210 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import java.io.IOException; - -import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.RowResult; - -import org.apache.hadoop.ipc.VersionedProtocol; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.NotServingRegionException; - -/** - * Clients interact with HRegionServers using a handle to the HRegionInterface. - */ -public interface HRegionInterface extends VersionedProtocol { - /** - * Protocol version. - * Upped to 3 when we went from Text to byte arrays for row and column names. - */ - public static final long versionID = 3L; - - /** - * Get metainfo about an HRegion - * - * @param regionName name of the region - * @return HRegionInfo object for region - * @throws NotServingRegionException - */ - public HRegionInfo getRegionInfo(final byte [] regionName) - throws NotServingRegionException; - - /** - * Retrieve a single value from the specified region for the specified row - * and column keys - * - * @param regionName name of region - * @param row row key - * @param column column key - * @return alue for that region/row/column - * @throws IOException - */ - public Cell get(final byte [] regionName, final byte [] row, final byte [] column) - throws IOException; - - /** - * Get the specified number of versions of the specified row and column - * - * @param regionName region name - * @param row row key - * @param column column key - * @param numVersions number of versions to return - * @return array of values - * @throws IOException - */ - public Cell[] get(final byte [] regionName, final byte [] row, - final byte [] column, final int numVersions) - throws IOException; - - /** - * Get the specified number of versions of the specified row and column with - * the specified timestamp. - * - * @param regionName region name - * @param row row key - * @param column column key - * @param timestamp timestamp - * @param numVersions number of versions to return - * @return array of values - * @throws IOException - */ - public Cell[] get(final byte [] regionName, final byte [] row, - final byte [] column, final long timestamp, final int numVersions) - throws IOException; - - /** - * Return all the data for the row that matches row exactly, - * or the one that immediately preceeds it. - * - * @param regionName region name - * @param row row key - * @return map of values - * @throws IOException - */ - public RowResult getClosestRowBefore(final byte [] regionName, - final byte [] row) - throws IOException; - - /** - * Get selected columns for the specified row at a given timestamp. - * - * @param regionName region name - * @param row row key - * @return map of values - * @throws IOException - */ - public RowResult getRow(final byte [] regionName, final byte [] row, - final byte[][] columns, final long ts) - throws IOException; - - /** - * Applies a batch of updates via one RPC - * - * @param regionName name of the region to update - * @param b BatchUpdate - * @throws IOException - */ - public void batchUpdate(final byte [] regionName, final BatchUpdate b) - throws IOException; - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp. - * - * @param regionName region name - * @param row row key - * @param column column key - * @param timestamp Delete all entries that have this timestamp or older - * @throws IOException - */ - public void deleteAll(byte [] regionName, byte [] row, byte [] column, - long timestamp) - throws IOException; - - /** - * Delete all cells that match the passed row and whose - * timestamp is equal-to or older than the passed timestamp. - * - * @param regionName region name - * @param row row key - * @param timestamp Delete all entries that have this timestamp or older - * @throws IOException - */ - public void deleteAll(byte [] regionName, byte [] row, long timestamp) - throws IOException; - - /** - * Delete all cells for a row with matching column family with timestamps - * less than or equal to timestamp. - * - * @param regionName The name of the region to operate on - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - */ - public void deleteFamily(byte [] regionName, byte [] row, byte [] family, - long timestamp) - throws IOException; - - - // - // remote scanner interface - // - - /** - * Opens a remote scanner with a RowFilter. - * - * @param regionName name of region to scan - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex for column family name. A column name is judged to be - * regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row to scan - * @param timestamp only return values whose timestamp is <= this value - * @param filter RowFilter for filtering results at the row-level. - * - * @return scannerId scanner identifier used in other calls - * @throws IOException - */ - public long openScanner(final byte [] regionName, final byte [][] columns, - final byte [] startRow, long timestamp, RowFilterInterface filter) - throws IOException; - - /** - * Get the next set of values - * @param scannerId clientId passed to openScanner - * @return map of values - * @throws IOException - */ - public RowResult next(long scannerId) throws IOException; - - /** - * Close a scanner - * - * @param scannerId the scanner id returned by openScanner - * @throws IOException - */ - public void close(long scannerId) throws IOException; +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import java.io.IOException; + +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; + +import org.apache.hadoop.ipc.VersionedProtocol; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.NotServingRegionException; + +/** + * Clients interact with HRegionServers using a handle to the HRegionInterface. + */ +public interface HRegionInterface extends VersionedProtocol { + /** + * Protocol version. + * Upped to 3 when we went from Text to byte arrays for row and column names. + */ + public static final long versionID = 3L; + + /** + * Get metainfo about an HRegion + * + * @param regionName name of the region + * @return HRegionInfo object for region + * @throws NotServingRegionException + */ + public HRegionInfo getRegionInfo(final byte [] regionName) + throws NotServingRegionException; + + /** + * Retrieve a single value from the specified region for the specified row + * and column keys + * + * @param regionName name of region + * @param row row key + * @param column column key + * @return alue for that region/row/column + * @throws IOException + */ + public Cell get(final byte [] regionName, final byte [] row, final byte [] column) + throws IOException; + + /** + * Get the specified number of versions of the specified row and column + * + * @param regionName region name + * @param row row key + * @param column column key + * @param numVersions number of versions to return + * @return array of values + * @throws IOException + */ + public Cell[] get(final byte [] regionName, final byte [] row, + final byte [] column, final int numVersions) + throws IOException; + + /** + * Get the specified number of versions of the specified row and column with + * the specified timestamp. + * + * @param regionName region name + * @param row row key + * @param column column key + * @param timestamp timestamp + * @param numVersions number of versions to return + * @return array of values + * @throws IOException + */ + public Cell[] get(final byte [] regionName, final byte [] row, + final byte [] column, final long timestamp, final int numVersions) + throws IOException; + + /** + * Return all the data for the row that matches row exactly, + * or the one that immediately preceeds it. + * + * @param regionName region name + * @param row row key + * @return map of values + * @throws IOException + */ + public RowResult getClosestRowBefore(final byte [] regionName, + final byte [] row) + throws IOException; + + /** + * Get selected columns for the specified row at a given timestamp. + * + * @param regionName region name + * @param row row key + * @return map of values + * @throws IOException + */ + public RowResult getRow(final byte [] regionName, final byte [] row, + final byte[][] columns, final long ts) + throws IOException; + + /** + * Applies a batch of updates via one RPC + * + * @param regionName name of the region to update + * @param b BatchUpdate + * @throws IOException + */ + public void batchUpdate(final byte [] regionName, final BatchUpdate b) + throws IOException; + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp. + * + * @param regionName region name + * @param row row key + * @param column column key + * @param timestamp Delete all entries that have this timestamp or older + * @throws IOException + */ + public void deleteAll(byte [] regionName, byte [] row, byte [] column, + long timestamp) + throws IOException; + + /** + * Delete all cells that match the passed row and whose + * timestamp is equal-to or older than the passed timestamp. + * + * @param regionName region name + * @param row row key + * @param timestamp Delete all entries that have this timestamp or older + * @throws IOException + */ + public void deleteAll(byte [] regionName, byte [] row, long timestamp) + throws IOException; + + /** + * Delete all cells for a row with matching column family with timestamps + * less than or equal to timestamp. + * + * @param regionName The name of the region to operate on + * @param row The row to operate on + * @param family The column family to match + * @param timestamp Timestamp to match + */ + public void deleteFamily(byte [] regionName, byte [] row, byte [] family, + long timestamp) + throws IOException; + + + // + // remote scanner interface + // + + /** + * Opens a remote scanner with a RowFilter. + * + * @param regionName name of region to scan + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex for column family name. A column name is judged to be + * regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row to scan + * @param timestamp only return values whose timestamp is <= this value + * @param filter RowFilter for filtering results at the row-level. + * + * @return scannerId scanner identifier used in other calls + * @throws IOException + */ + public long openScanner(final byte [] regionName, final byte [][] columns, + final byte [] startRow, long timestamp, RowFilterInterface filter) + throws IOException; + + /** + * Get the next set of values + * @param scannerId clientId passed to openScanner + * @return map of values + * @throws IOException + */ + public RowResult next(long scannerId) throws IOException; + + /** + * Close a scanner + * + * @param scannerId the scanner id returned by openScanner + * @throws IOException + */ + public void close(long scannerId) throws IOException; } \ No newline at end of file Index: src/java/org/apache/hadoop/hbase/client/HTable.java =================================================================== --- src/java/org/apache/hadoop/hbase/client/HTable.java (revision 684990) +++ src/java/org/apache/hadoop/hbase/client/HTable.java (working copy) @@ -1,1408 +1,1408 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; -import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.filter.StopRowFilter; -import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; - -/** - * Used to communicate with a single HBase table - */ -public class HTable { - private final HConnection connection; - private final byte [] tableName; - private HBaseConfiguration configuration; - - /** - * Creates an object to access a HBase table - * - * @param tableName name of the table - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public HTable(final Text tableName) - throws IOException { - this(new HBaseConfiguration(), tableName.getBytes()); - } - - /** - * Creates an object to access a HBase table - * - * @param tableName name of the table - * @throws IOException - */ - public HTable(final String tableName) - throws IOException { - this(new HBaseConfiguration(), Bytes.toBytes(tableName)); - } - - /** - * Creates an object to access a HBase table - * - * @param tableName name of the table - * @throws IOException - */ - public HTable(final byte [] tableName) - throws IOException { - this(new HBaseConfiguration(), tableName); - } - - /** - * Creates an object to access a HBase table - * - * @param conf configuration object - * @param tableName name of the table - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public HTable(HBaseConfiguration conf, final Text tableName) - throws IOException { - this(conf, tableName.getBytes()); - } - - /** - * Creates an object to access a HBase table - * - * @param conf configuration object - * @param tableName name of the table - * @throws IOException - */ - public HTable(HBaseConfiguration conf, final String tableName) - throws IOException { - this(conf, Bytes.toBytes(tableName)); - } - - /** - * Creates an object to access a HBase table - * - * @param conf configuration object - * @param tableName name of the table - * @throws IOException - */ - public HTable(HBaseConfiguration conf, final byte [] tableName) - throws IOException { - this.connection = HConnectionManager.getConnection(conf); - this.configuration = conf; - this.tableName = tableName; - this.connection.locateRegion(tableName, HConstants.EMPTY_START_ROW); - } - - /** - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public static boolean isTableEnabled(Text tableName) throws IOException { - return isTableEnabled(tableName.getBytes()); - } - /** - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException - */ - public static boolean isTableEnabled(String tableName) throws IOException { - return isTableEnabled(Bytes.toBytes(tableName)); - } - /** - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException - */ - public static boolean isTableEnabled(byte[] tableName) throws IOException { - return isTableEnabled(new HBaseConfiguration(), tableName); - } - - /** - * @param conf HBaseConfiguration object - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public static boolean isTableEnabled(HBaseConfiguration conf, Text tableName) - throws IOException { - return isTableEnabled(conf, tableName.getBytes()); - } - - /** - * @param conf HBaseConfiguration object - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException - */ - public static boolean isTableEnabled(HBaseConfiguration conf, String tableName) - throws IOException { - return isTableEnabled(conf, Bytes.toBytes(tableName)); - } - - /** - * @param conf HBaseConfiguration object - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException - */ - public static boolean isTableEnabled(HBaseConfiguration conf, byte[] tableName) - throws IOException { - return HConnectionManager.getConnection(conf).isTableEnabled(tableName); - } - - /** - * Find region location hosting passed row using cached info - * @param row Row to find. - * @return Location of row. - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public HRegionLocation getRegionLocation(final Text row) - throws IOException { - return connection.getRegionLocation(tableName, row.getBytes(), false); - } - - /** - * Find region location hosting passed row using cached info - * @param row Row to find. - * @return Location of row. - * @throws IOException - */ - public HRegionLocation getRegionLocation(final String row) - throws IOException { - return connection.getRegionLocation(tableName, Bytes.toBytes(row), false); - } - - /** - * Find region location hosting passed row using cached info - * @param row Row to find. - * @return Location of row. - * @throws IOException - */ - public HRegionLocation getRegionLocation(final byte [] row) - throws IOException { - return connection.getRegionLocation(tableName, row, false); - } - - /** @return the table name */ - public byte [] getTableName() { - return this.tableName; - } - - /** - * Used by unit tests and tools to do low-level manipulations. Not for - * general use. - * @return An HConnection instance. - */ - public HConnection getConnection() { - return this.connection; - } - - /** - * @return table metadata - * @throws IOException - */ - @Deprecated - public HTableDescriptor getMetadata() throws IOException { - return getTableDescriptor(); - } - - /** - * @return table metadata - * @throws IOException - */ - public HTableDescriptor getTableDescriptor() throws IOException { - return new UnmodifyableHTableDescriptor( - this.connection.getHTableDescriptor(this.tableName)); - } - - /** - * Gets the starting row key for every region in the currently open table - * - * @return Array of region starting row keys - * @throws IOException - */ - @SuppressWarnings("null") - public byte[][] getStartKeys() throws IOException { - final List keyList = new ArrayList(); - - MetaScannerVisitor visitor = new MetaScannerVisitor() { - public boolean processRow(RowResult rowResult) throws IOException { - HRegionInfo info = Writables.getHRegionInfo( - rowResult.get(HConstants.COL_REGIONINFO)); - if (Bytes.equals(info.getTableDesc().getName(), getTableName())) { - if (!(info.isOffline() || info.isSplit())) { - keyList.add(info.getStartKey()); - } - } - return true; - } - - }; - MetaScanner.metaScan(configuration, visitor, this.tableName); - return keyList.toArray(new byte[keyList.size()][]); - } - - /** - * Get all the regions and their address for this table - * - * @return A map of HRegionInfo with it's server address - * @throws IOException - */ - public Map getRegionsInfo() throws IOException { - final Map regionMap = - new TreeMap(); - - MetaScannerVisitor visitor = new MetaScannerVisitor() { - public boolean processRow(RowResult rowResult) throws IOException { - HRegionInfo info = Writables.getHRegionInfo( - rowResult.get(HConstants.COL_REGIONINFO)); - - if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) { - return false; - } - - HServerAddress server = new HServerAddress(); - Cell c = rowResult.get(HConstants.COL_SERVER); - if (c != null && c.getValue() != null && c.getValue().length > 0) { - String address = Bytes.toString(c.getValue()); - server = new HServerAddress(address); - } - - if (!(info.isOffline() || info.isSplit())) { - regionMap.put(new UnmodifyableHRegionInfo(info), server); - } - return true; - } - - }; - MetaScanner.metaScan(configuration, visitor, tableName); - return regionMap; - } - - /** - * Get a single value for the specified row and column - * - * @param row row key - * @param column column name - * @return value for specified row/column - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public Cell get(final Text row, final Text column) - throws IOException { - return get(row.getBytes(), column.getBytes()); - } - - /** - * Get a single value for the specified row and column - * - * @param row row key - * @param column column name - * @param numVersions - number of versions to retrieve - * @return value for specified row/column - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public Cell[] get(final Text row, final Text column, int numVersions) - throws IOException { - return get(row.getBytes(), column.getBytes(), numVersions); - } - - /** - * Get a single value for the specified row and column - * - * @param row row key - * @param column column name - * @return value for specified row/column - * @throws IOException - */ - public Cell get(final String row, final String column) - throws IOException { - return get(Bytes.toBytes(row), Bytes.toBytes(column)); - } - - /** - * Get a single value for the specified row and column - * - * @param row row key - * @param column column name - * @param numVersions - number of versions to retrieve - * @return value for specified row/column - * @throws IOException - */ - public Cell[] get(final String row, final String column, int numVersions) - throws IOException { - return get(Bytes.toBytes(row), Bytes.toBytes(column), numVersions); - } - - /** - * Get a single value for the specified row and column - * - * @param row row key - * @param column column name - * @return value for specified row/column - * @throws IOException - */ - public Cell get(final byte [] row, final byte [] column) - throws IOException { - return connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Cell call() throws IOException { - return server.get(location.getRegionInfo().getRegionName(), row, - column); - } - } - ); - } - - /** - * Get the specified number of versions of the specified row and column - * @param row row key - * @param column column name - * @param numVersions number of versions to retrieve - * @return Array of Cells. - * @throws IOException - */ - public Cell[] get(final byte [] row, final byte [] column, - final int numVersions) - throws IOException { - return connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Cell[] call() throws IOException { - return server.get(location.getRegionInfo().getRegionName(), row, - column, numVersions); - } - } - ); - } - - /** - * Get the specified number of versions of the specified row and column with - * the specified timestamp. - * - * @param row - row key - * @param column - column name - * @param timestamp - timestamp - * @param numVersions - number of versions to retrieve - * @return - array of values that match the above criteria - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public Cell[] get(final Text row, final Text column, - final long timestamp, final int numVersions) - throws IOException { - return get(row.getBytes(), column.getBytes(), timestamp, numVersions); - } - - /** - * Get the specified number of versions of the specified row and column with - * the specified timestamp. - * - * @param row - row key - * @param column - column name - * @param timestamp - timestamp - * @param numVersions - number of versions to retrieve - * @return - array of values that match the above criteria - * @throws IOException - */ - public Cell[] get(final String row, final String column, - final long timestamp, final int numVersions) - throws IOException { - return get(Bytes.toBytes(row), Bytes.toBytes(column), timestamp, numVersions); - } - - /** - * Get the specified number of versions of the specified row and column with - * the specified timestamp. - * - * @param row - row key - * @param column - column name - * @param timestamp - timestamp - * @param numVersions - number of versions to retrieve - * @return - array of values that match the above criteria - * @throws IOException - */ - public Cell[] get(final byte [] row, final byte [] column, - final long timestamp, final int numVersions) - throws IOException { - Cell[] values = null; - values = connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Cell[] call() throws IOException { - return server.get(location.getRegionInfo().getRegionName(), row, - column, timestamp, numVersions); - } - } - ); - - if (values != null) { - ArrayList cellValues = new ArrayList(); - for (int i = 0 ; i < values.length; i++) { - cellValues.add(values[i]); - } - return cellValues.toArray(new Cell[values.length]); - } - return null; - } - - /** - * Get all the data for the specified row at the latest timestamp - * - * @param row row key - * @return RowResult is empty if row does not exist. - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public RowResult getRow(final Text row) throws IOException { - return getRow(row.getBytes()); - } - - /** - * Get all the data for the specified row at the latest timestamp - * - * @param row row key - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row) throws IOException { - return getRow(Bytes.toBytes(row)); - } - - /** - * Get all the data for the specified row at the latest timestamp - * - * @param row row key - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte [] row) throws IOException { - return getRow(row, HConstants.LATEST_TIMESTAMP); - } - - /** - * Get all the data for the specified row at a specified timestamp - * - * @param row row key - * @param ts timestamp - * @return RowResult is empty if row does not exist. - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public RowResult getRow(final Text row, final long ts) - throws IOException { - return getRow(row.getBytes(), ts); - } - - /** - * Get all the data for the specified row at a specified timestamp - * - * @param row row key - * @param ts timestamp - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row, final long ts) - throws IOException { - return getRow(Bytes.toBytes(row), ts); - } - - /** - * Get all the data for the specified row at a specified timestamp - * - * @param row row key - * @param ts timestamp - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte [] row, final long ts) - throws IOException { - return getRow(row,null,ts); - } - - /** - * Get selected columns for the specified row at the latest timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @return RowResult is empty if row does not exist. - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public RowResult getRow(final Text row, final Text[] columns) - throws IOException { - return getRow(row.getBytes(), Bytes.toByteArrays(columns)); - } - - /** - * Get selected columns for the specified row at the latest timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row, final String [] columns) - throws IOException { - return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns)); - } - - /** - * Get selected columns for the specified row at the latest timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte [] row, final byte [][] columns) - throws IOException { - return getRow(row, columns, HConstants.LATEST_TIMESTAMP); - } - - /** - * Get selected columns for the specified row at a specified timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @param ts timestamp - * @return RowResult is empty if row does not exist. - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public RowResult getRow(final Text row, final Text [] columns, - final long ts) - throws IOException { - return getRow(row.getBytes(), Bytes.toByteArrays(columns), ts); - } - - /** - * Get selected columns for the specified row at a specified timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @param ts timestamp - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row, final String [] columns, - final long ts) - throws IOException { - return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), ts); - } - - /** - * Get selected columns for the specified row at a specified timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @param ts timestamp - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte [] row, final byte [][] columns, - final long ts) - throws IOException { - return connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public RowResult call() throws IOException { - return server.getRow(location.getRegionInfo().getRegionName(), row, - columns, ts); - } - } - ); - } - - /** - * Get a scanner on the current table starting at first row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @return scanner - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public Scanner getScanner(final Text [] columns) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), HConstants.EMPTY_START_ROW); - } - - /** - * Get a scanner on the current table starting at first row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final String [] columns) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), HConstants.EMPTY_START_ROW); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @return scanner - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public Scanner getScanner(final Text [] columns, final Text startRow) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), startRow.getBytes()); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final String [] columns, final String startRow) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow)); - } - - /** - * Get a scanner on the current table starting at first row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte[][] columns) - throws IOException { - return getScanner(columns, HConstants.EMPTY_START_ROW, - HConstants.LATEST_TIMESTAMP, null); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte[][] columns, final byte [] startRow) - throws IOException { - return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, null); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param timestamp only return results whose timestamp <= this value - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte[][] columns, final byte [] startRow, - long timestamp) - throws IOException { - return getScanner(columns, startRow, timestamp, null); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param filter a row filter using row-key regexp and/or column data filter. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte[][] columns, final byte [] startRow, - RowFilterInterface filter) - throws IOException { - return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, filter); - } - - /** - * Get a scanner on the current table starting at the specified row and - * ending just before stopRow. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param stopRow Row to stop scanning on. Once we hit this row we stop - * returning values; i.e. we return the row before this one but not the - * stopRow itself. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte [][] columns, - final byte [] startRow, final byte [] stopRow) - throws IOException { - return getScanner(columns, startRow, stopRow, - HConstants.LATEST_TIMESTAMP); - } - - /** - * Get a scanner on the current table starting at the specified row and - * ending just before stopRow. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param stopRow Row to stop scanning on. Once we hit this row we stop - * returning values; i.e. we return the row before this one but not the - * stopRow itself. - * @param timestamp only return results whose timestamp <= this value - * @return scanner - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public Scanner getScanner(final Text[] columns, - final Text startRow, final Text stopRow, final long timestamp) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), startRow.getBytes(), - stopRow.getBytes(), timestamp); - } - - /** - * Get a scanner on the current table starting at the specified row and - * ending just before stopRow. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param stopRow Row to stop scanning on. Once we hit this row we stop - * returning values; i.e. we return the row before this one but not the - * stopRow itself. - * @param timestamp only return results whose timestamp <= this value - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final String [] columns, - final String startRow, final String stopRow, final long timestamp) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow), - Bytes.toBytes(stopRow), timestamp); - } - - /** - * Get a scanner on the current table starting at the specified row and - * ending just before stopRow. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param stopRow Row to stop scanning on. Once we hit this row we stop - * returning values; i.e. we return the row before this one but not the - * stopRow itself. - * @param timestamp only return results whose timestamp <= this value - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte [][] columns, - final byte [] startRow, final byte [] stopRow, final long timestamp) - throws IOException { - return getScanner(columns, startRow, timestamp, - new WhileMatchRowFilter(new StopRowFilter(stopRow))); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param timestamp only return results whose timestamp <= this value - * @param filter a row filter using row-key regexp and/or column data filter. - * @return scanner - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public Scanner getScanner(Text[] columns, - Text startRow, long timestamp, RowFilterInterface filter) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), startRow.getBytes(), - timestamp, filter); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param timestamp only return results whose timestamp <= this value - * @param filter a row filter using row-key regexp and/or column data filter. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(String[] columns, - String startRow, long timestamp, RowFilterInterface filter) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow), - timestamp, filter); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param timestamp only return results whose timestamp <= this value - * @param filter a row filter using row-key regexp and/or column data filter. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte [][] columns, - final byte [] startRow, long timestamp, RowFilterInterface filter) - throws IOException { - return new ClientScanner(columns, startRow, timestamp, filter); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @throws IOException - */ - public void deleteAll(final byte [] row) throws IOException { - deleteAll(row, null); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public void deleteAll(final Text row) throws IOException { - deleteAll(row, null); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @throws IOException - */ - public void deleteAll(final String row) throws IOException { - deleteAll(row, null); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @param column column to be deleted - * @throws IOException - */ - public void deleteAll(final byte [] row, final byte [] column) - throws IOException { - deleteAll(row, column, HConstants.LATEST_TIMESTAMP); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAll(final byte [] row, final long ts) - throws IOException { - deleteAll(row, null, ts); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAll(final String row, final long ts) - throws IOException { - deleteAll(row, null, ts); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public void deleteAll(final Text row, final long ts) - throws IOException { - deleteAll(row, null, ts); - } - - /** - * Delete all cells that match the passed row and column. - * @param row Row to update - * @param column name of column whose value is to be deleted - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public void deleteAll(final Text row, final Text column) throws IOException { - deleteAll(row, column, HConstants.LATEST_TIMESTAMP); - } - - /** - * Delete all cells that match the passed row and column. - * @param row Row to update - * @param column name of column whose value is to be deleted - * @throws IOException - */ - public void deleteAll(final String row, final String column) - throws IOException { - deleteAll(row, column, HConstants.LATEST_TIMESTAMP); - } - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp. - * @param row Row to update - * @param column name of column whose value is to be deleted - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public void deleteAll(final Text row, final Text column, final long ts) - throws IOException { - deleteAll(row.getBytes(), column.getBytes(), ts); - } - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp. - * @param row Row to update - * @param column name of column whose value is to be deleted - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAll(final String row, final String column, final long ts) - throws IOException { - deleteAll(Bytes.toBytes(row), - column != null? Bytes.toBytes(column): null, ts); - } - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp. - * @param row Row to update - * @param column name of column whose value is to be deleted - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAll(final byte [] row, final byte [] column, final long ts) - throws IOException { - connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Boolean call() throws IOException { - if (column != null) { - this.server.deleteAll(location.getRegionInfo().getRegionName(), - row, column, ts); - } else { - this.server.deleteAll(location.getRegionInfo().getRegionName(), - row, ts); - } - return null; - } - } - ); - } - - /** - * Delete all cells for a row with matching column family at all timestamps. - * - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - * @throws IOException - * @deprecated Use String or byte [] overload instead - */ - public void deleteFamily(final Text row, final Text family, - final long timestamp) - throws IOException{ - deleteFamily(row.getBytes(), family.getBytes(), timestamp); - } - - /** - * Delete all cells for a row with matching column family at all timestamps. - * - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - * @throws IOException - */ - public void deleteFamily(final String row, final String family, - final long timestamp) - throws IOException{ - deleteFamily(Bytes.toBytes(row), Bytes.toBytes(family), timestamp); - } - - /** - * Delete all cells for a row with matching column family with timestamps - * less than or equal to timestamp. - * - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - * @throws IOException - */ - public void deleteFamily(final byte [] row, final byte [] family, - final long timestamp) - throws IOException { - connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Boolean call() throws IOException { - server.deleteFamily(location.getRegionInfo().getRegionName(), row, - family, timestamp); - return null; - } - } - ); - } - - /** - * Commit a BatchUpdate to the table. - * @param batchUpdate - * @throws IOException - */ - public synchronized void commit(final BatchUpdate batchUpdate) - throws IOException { - connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, batchUpdate.getRow()) { - public Boolean call() throws IOException { - server.batchUpdate(location.getRegionInfo().getRegionName(), - batchUpdate); - return null; - } - } - ); - } - - /** - * Commit a RowsBatchUpdate to the table. - * @param batchUpdates - * @throws IOException - */ - public synchronized void commit(final List batchUpdates) - throws IOException { - for (BatchUpdate batchUpdate : batchUpdates) - commit(batchUpdate); - } - - /** - * Implements the scanner interface for the HBase client. - * If there are multiple regions in a table, this scanner will iterate - * through them all. - */ - protected class ClientScanner implements Scanner { - private final Log CLIENT_LOG = LogFactory.getLog(this.getClass()); - private byte[][] columns; - private byte [] startRow; - protected long scanTime; - @SuppressWarnings("hiding") - private boolean closed = false; - private HRegionInfo currentRegion = null; - private ScannerCallable callable = null; - protected RowFilterInterface filter; - - protected ClientScanner(final Text [] columns, final Text startRow, - long timestamp, RowFilterInterface filter) - throws IOException { - this(Bytes.toByteArrays(columns), startRow.getBytes(), timestamp, - filter); - } - - protected ClientScanner(final byte[][] columns, final byte [] startRow, - final long timestamp, final RowFilterInterface filter) - throws IOException { - if (CLIENT_LOG.isDebugEnabled()) { - CLIENT_LOG.debug("Creating scanner over " + Bytes.toString(getTableName()) + - " starting at key '" + Bytes.toString(startRow) + "'"); - } - // save off the simple parameters - this.columns = columns; - this.startRow = startRow; - this.scanTime = timestamp; - - // save the filter, and make sure that the filter applies to the data - // we're expecting to pull back - this.filter = filter; - if (filter != null) { - filter.validate(columns); - } - nextScanner(); - } - - protected byte[][] getColumns() { - return columns; - } - - protected long getTimestamp() { - return scanTime; - } - - protected RowFilterInterface getFilter() { - return filter; - } - - /* - * Gets a scanner for the next region. - * Returns false if there are no more scanners. - */ - private boolean nextScanner() throws IOException { - // Close the previous scanner if it's open - if (this.callable != null) { - this.callable.setClose(); - getConnection().getRegionServerWithRetries(callable); - this.callable = null; - } - - // if we're at the end of the table, then close and return false - // to stop iterating - if (currentRegion != null) { - if (CLIENT_LOG.isDebugEnabled()) { - CLIENT_LOG.debug("Advancing forward from region " + currentRegion); - } - - byte [] endKey = currentRegion.getEndKey(); - if (endKey == null || - Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) || - filterSaysStop(endKey)) { - close(); - return false; - } - } - - HRegionInfo oldRegion = this.currentRegion; - byte [] localStartKey = oldRegion == null? startRow: oldRegion.getEndKey(); - - if (CLIENT_LOG.isDebugEnabled()) { - CLIENT_LOG.debug("Advancing internal scanner to startKey at '" + - Bytes.toString(localStartKey) + "'"); - } - - try { - callable = getScannerCallable(localStartKey); - // open a scanner on the region server starting at the - // beginning of the region - getConnection().getRegionServerWithRetries(callable); - currentRegion = callable.getHRegionInfo(); - } catch (IOException e) { - close(); - throw e; - } - return true; - } - - protected ScannerCallable getScannerCallable(byte [] localStartKey) { - return new ScannerCallable(getConnection(), getTableName(), columns, - localStartKey, scanTime, filter); - } - - /** - * @param endKey - * @return Returns true if the passed region endkey is judged beyond - * filter. - */ - private boolean filterSaysStop(final byte [] endKey) { - if (this.filter == null) { - return false; - } - // Let the filter see current row. - this.filter.filterRowKey(endKey); - return this.filter.filterAllRemaining(); - } - - /** {@inheritDoc} */ - public RowResult next() throws IOException { - if (this.closed) { - return null; - } - RowResult values = null; - do { - values = getConnection().getRegionServerWithRetries(callable); - } while ((values == null || values.size() == 0) && nextScanner()); - - if (values != null && values.size() != 0) { - return values; - } - return null; - } - - /** - * {@inheritDoc} - */ - public void close() { - if (callable != null) { - callable.setClose(); - try { - getConnection().getRegionServerWithRetries(callable); - } catch (IOException e) { - // We used to catch this error, interpret, and rethrow. However, we - // have since decided that it's not nice for a scanner's close to - // throw exceptions. Chances are it was just an UnknownScanner - // exception due to lease time out. - } - callable = null; - } - closed = true; - } - - /** {@inheritDoc} */ - public Iterator iterator() { - return new Iterator() { - // The next RowResult, possibly pre-read - RowResult next = null; - - // return true if there is another item pending, false if there isn't. - // this method is where the actual advancing takes place, but you need - // to call next() to consume it. hasNext() will only advance if there - // isn't a pending next(). - public boolean hasNext() { - if (next == null) { - try { - next = ClientScanner.this.next(); - return next != null; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - return true; - } - - // get the pending next item and advance the iterator. returns null if - // there is no next item. - public RowResult next() { - // since hasNext() does the real advancing, we call this to determine - // if there is a next before proceeding. - if (!hasNext()) { - return null; - } - - // if we get to here, then hasNext() has given us an item to return. - // we want to return the item and then null out the next pointer, so - // we use a temporary variable. - RowResult temp = next; - next = null; - return temp; - } - - public void remove() { - throw new UnsupportedOperationException(); - } - }; - } - } -} +/** + * Copyright 2008 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.filter.StopRowFilter; +import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.io.Text; + +/** + * Used to communicate with a single HBase table + */ +public class HTable { + private final HConnection connection; + private final byte [] tableName; + private HBaseConfiguration configuration; + + /** + * Creates an object to access a HBase table + * + * @param tableName name of the table + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public HTable(final Text tableName) + throws IOException { + this(new HBaseConfiguration(), tableName.getBytes()); + } + + /** + * Creates an object to access a HBase table + * + * @param tableName name of the table + * @throws IOException + */ + public HTable(final String tableName) + throws IOException { + this(new HBaseConfiguration(), Bytes.toBytes(tableName)); + } + + /** + * Creates an object to access a HBase table + * + * @param tableName name of the table + * @throws IOException + */ + public HTable(final byte [] tableName) + throws IOException { + this(new HBaseConfiguration(), tableName); + } + + /** + * Creates an object to access a HBase table + * + * @param conf configuration object + * @param tableName name of the table + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public HTable(HBaseConfiguration conf, final Text tableName) + throws IOException { + this(conf, tableName.getBytes()); + } + + /** + * Creates an object to access a HBase table + * + * @param conf configuration object + * @param tableName name of the table + * @throws IOException + */ + public HTable(HBaseConfiguration conf, final String tableName) + throws IOException { + this(conf, Bytes.toBytes(tableName)); + } + + /** + * Creates an object to access a HBase table + * + * @param conf configuration object + * @param tableName name of the table + * @throws IOException + */ + public HTable(HBaseConfiguration conf, final byte [] tableName) + throws IOException { + this.connection = HConnectionManager.getConnection(conf); + this.configuration = conf; + this.tableName = tableName; + this.connection.locateRegion(tableName, HConstants.EMPTY_START_ROW); + } + + /** + * @param tableName name of table to check + * @return true if table is on-line + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public static boolean isTableEnabled(Text tableName) throws IOException { + return isTableEnabled(tableName.getBytes()); + } + /** + * @param tableName name of table to check + * @return true if table is on-line + * @throws IOException + */ + public static boolean isTableEnabled(String tableName) throws IOException { + return isTableEnabled(Bytes.toBytes(tableName)); + } + /** + * @param tableName name of table to check + * @return true if table is on-line + * @throws IOException + */ + public static boolean isTableEnabled(byte[] tableName) throws IOException { + return isTableEnabled(new HBaseConfiguration(), tableName); + } + + /** + * @param conf HBaseConfiguration object + * @param tableName name of table to check + * @return true if table is on-line + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public static boolean isTableEnabled(HBaseConfiguration conf, Text tableName) + throws IOException { + return isTableEnabled(conf, tableName.getBytes()); + } + + /** + * @param conf HBaseConfiguration object + * @param tableName name of table to check + * @return true if table is on-line + * @throws IOException + */ + public static boolean isTableEnabled(HBaseConfiguration conf, String tableName) + throws IOException { + return isTableEnabled(conf, Bytes.toBytes(tableName)); + } + + /** + * @param conf HBaseConfiguration object + * @param tableName name of table to check + * @return true if table is on-line + * @throws IOException + */ + public static boolean isTableEnabled(HBaseConfiguration conf, byte[] tableName) + throws IOException { + return HConnectionManager.getConnection(conf).isTableEnabled(tableName); + } + + /** + * Find region location hosting passed row using cached info + * @param row Row to find. + * @return Location of row. + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public HRegionLocation getRegionLocation(final Text row) + throws IOException { + return connection.getRegionLocation(tableName, row.getBytes(), false); + } + + /** + * Find region location hosting passed row using cached info + * @param row Row to find. + * @return Location of row. + * @throws IOException + */ + public HRegionLocation getRegionLocation(final String row) + throws IOException { + return connection.getRegionLocation(tableName, Bytes.toBytes(row), false); + } + + /** + * Find region location hosting passed row using cached info + * @param row Row to find. + * @return Location of row. + * @throws IOException + */ + public HRegionLocation getRegionLocation(final byte [] row) + throws IOException { + return connection.getRegionLocation(tableName, row, false); + } + + /** @return the table name */ + public byte [] getTableName() { + return this.tableName; + } + + /** + * Used by unit tests and tools to do low-level manipulations. Not for + * general use. + * @return An HConnection instance. + */ + public HConnection getConnection() { + return this.connection; + } + + /** + * @return table metadata + * @throws IOException + */ + @Deprecated + public HTableDescriptor getMetadata() throws IOException { + return getTableDescriptor(); + } + + /** + * @return table metadata + * @throws IOException + */ + public HTableDescriptor getTableDescriptor() throws IOException { + return new UnmodifyableHTableDescriptor( + this.connection.getHTableDescriptor(this.tableName)); + } + + /** + * Gets the starting row key for every region in the currently open table + * + * @return Array of region starting row keys + * @throws IOException + */ + @SuppressWarnings("null") + public byte[][] getStartKeys() throws IOException { + final List keyList = new ArrayList(); + + MetaScannerVisitor visitor = new MetaScannerVisitor() { + public boolean processRow(RowResult rowResult) throws IOException { + HRegionInfo info = Writables.getHRegionInfo( + rowResult.get(HConstants.COL_REGIONINFO)); + if (Bytes.equals(info.getTableDesc().getName(), getTableName())) { + if (!(info.isOffline() || info.isSplit())) { + keyList.add(info.getStartKey()); + } + } + return true; + } + + }; + MetaScanner.metaScan(configuration, visitor, this.tableName); + return keyList.toArray(new byte[keyList.size()][]); + } + + /** + * Get all the regions and their address for this table + * + * @return A map of HRegionInfo with it's server address + * @throws IOException + */ + public Map getRegionsInfo() throws IOException { + final Map regionMap = + new TreeMap(); + + MetaScannerVisitor visitor = new MetaScannerVisitor() { + public boolean processRow(RowResult rowResult) throws IOException { + HRegionInfo info = Writables.getHRegionInfo( + rowResult.get(HConstants.COL_REGIONINFO)); + + if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) { + return false; + } + + HServerAddress server = new HServerAddress(); + Cell c = rowResult.get(HConstants.COL_SERVER); + if (c != null && c.getValue() != null && c.getValue().length > 0) { + String address = Bytes.toString(c.getValue()); + server = new HServerAddress(address); + } + + if (!(info.isOffline() || info.isSplit())) { + regionMap.put(new UnmodifyableHRegionInfo(info), server); + } + return true; + } + + }; + MetaScanner.metaScan(configuration, visitor, tableName); + return regionMap; + } + + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @return value for specified row/column + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public Cell get(final Text row, final Text column) + throws IOException { + return get(row.getBytes(), column.getBytes()); + } + + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @param numVersions - number of versions to retrieve + * @return value for specified row/column + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public Cell[] get(final Text row, final Text column, int numVersions) + throws IOException { + return get(row.getBytes(), column.getBytes(), numVersions); + } + + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @return value for specified row/column + * @throws IOException + */ + public Cell get(final String row, final String column) + throws IOException { + return get(Bytes.toBytes(row), Bytes.toBytes(column)); + } + + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @param numVersions - number of versions to retrieve + * @return value for specified row/column + * @throws IOException + */ + public Cell[] get(final String row, final String column, int numVersions) + throws IOException { + return get(Bytes.toBytes(row), Bytes.toBytes(column), numVersions); + } + + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @return value for specified row/column + * @throws IOException + */ + public Cell get(final byte [] row, final byte [] column) + throws IOException { + return connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public Cell call() throws IOException { + return server.get(location.getRegionInfo().getRegionName(), row, + column); + } + } + ); + } + + /** + * Get the specified number of versions of the specified row and column + * @param row row key + * @param column column name + * @param numVersions number of versions to retrieve + * @return Array of Cells. + * @throws IOException + */ + public Cell[] get(final byte [] row, final byte [] column, + final int numVersions) + throws IOException { + return connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public Cell[] call() throws IOException { + return server.get(location.getRegionInfo().getRegionName(), row, + column, numVersions); + } + } + ); + } + + /** + * Get the specified number of versions of the specified row and column with + * the specified timestamp. + * + * @param row - row key + * @param column - column name + * @param timestamp - timestamp + * @param numVersions - number of versions to retrieve + * @return - array of values that match the above criteria + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public Cell[] get(final Text row, final Text column, + final long timestamp, final int numVersions) + throws IOException { + return get(row.getBytes(), column.getBytes(), timestamp, numVersions); + } + + /** + * Get the specified number of versions of the specified row and column with + * the specified timestamp. + * + * @param row - row key + * @param column - column name + * @param timestamp - timestamp + * @param numVersions - number of versions to retrieve + * @return - array of values that match the above criteria + * @throws IOException + */ + public Cell[] get(final String row, final String column, + final long timestamp, final int numVersions) + throws IOException { + return get(Bytes.toBytes(row), Bytes.toBytes(column), timestamp, numVersions); + } + + /** + * Get the specified number of versions of the specified row and column with + * the specified timestamp. + * + * @param row - row key + * @param column - column name + * @param timestamp - timestamp + * @param numVersions - number of versions to retrieve + * @return - array of values that match the above criteria + * @throws IOException + */ + public Cell[] get(final byte [] row, final byte [] column, + final long timestamp, final int numVersions) + throws IOException { + Cell[] values = null; + values = connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public Cell[] call() throws IOException { + return server.get(location.getRegionInfo().getRegionName(), row, + column, timestamp, numVersions); + } + } + ); + + if (values != null) { + ArrayList cellValues = new ArrayList(); + for (int i = 0 ; i < values.length; i++) { + cellValues.add(values[i]); + } + return cellValues.toArray(new Cell[values.length]); + } + return null; + } + + /** + * Get all the data for the specified row at the latest timestamp + * + * @param row row key + * @return RowResult is empty if row does not exist. + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public RowResult getRow(final Text row) throws IOException { + return getRow(row.getBytes()); + } + + /** + * Get all the data for the specified row at the latest timestamp + * + * @param row row key + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final String row) throws IOException { + return getRow(Bytes.toBytes(row)); + } + + /** + * Get all the data for the specified row at the latest timestamp + * + * @param row row key + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final byte [] row) throws IOException { + return getRow(row, HConstants.LATEST_TIMESTAMP); + } + + /** + * Get all the data for the specified row at a specified timestamp + * + * @param row row key + * @param ts timestamp + * @return RowResult is empty if row does not exist. + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public RowResult getRow(final Text row, final long ts) + throws IOException { + return getRow(row.getBytes(), ts); + } + + /** + * Get all the data for the specified row at a specified timestamp + * + * @param row row key + * @param ts timestamp + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final String row, final long ts) + throws IOException { + return getRow(Bytes.toBytes(row), ts); + } + + /** + * Get all the data for the specified row at a specified timestamp + * + * @param row row key + * @param ts timestamp + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final byte [] row, final long ts) + throws IOException { + return getRow(row,null,ts); + } + + /** + * Get selected columns for the specified row at the latest timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @return RowResult is empty if row does not exist. + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public RowResult getRow(final Text row, final Text[] columns) + throws IOException { + return getRow(row.getBytes(), Bytes.toByteArrays(columns)); + } + + /** + * Get selected columns for the specified row at the latest timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final String row, final String [] columns) + throws IOException { + return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns)); + } + + /** + * Get selected columns for the specified row at the latest timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final byte [] row, final byte [][] columns) + throws IOException { + return getRow(row, columns, HConstants.LATEST_TIMESTAMP); + } + + /** + * Get selected columns for the specified row at a specified timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @param ts timestamp + * @return RowResult is empty if row does not exist. + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public RowResult getRow(final Text row, final Text [] columns, + final long ts) + throws IOException { + return getRow(row.getBytes(), Bytes.toByteArrays(columns), ts); + } + + /** + * Get selected columns for the specified row at a specified timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @param ts timestamp + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final String row, final String [] columns, + final long ts) + throws IOException { + return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), ts); + } + + /** + * Get selected columns for the specified row at a specified timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @param ts timestamp + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final byte [] row, final byte [][] columns, + final long ts) + throws IOException { + return connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public RowResult call() throws IOException { + return server.getRow(location.getRegionInfo().getRegionName(), row, + columns, ts); + } + } + ); + } + + /** + * Get a scanner on the current table starting at first row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @return scanner + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public Scanner getScanner(final Text [] columns) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), HConstants.EMPTY_START_ROW); + } + + /** + * Get a scanner on the current table starting at first row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final String [] columns) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), HConstants.EMPTY_START_ROW); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @return scanner + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public Scanner getScanner(final Text [] columns, final Text startRow) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), startRow.getBytes()); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final String [] columns, final String startRow) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow)); + } + + /** + * Get a scanner on the current table starting at first row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte[][] columns) + throws IOException { + return getScanner(columns, HConstants.EMPTY_START_ROW, + HConstants.LATEST_TIMESTAMP, null); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte[][] columns, final byte [] startRow) + throws IOException { + return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, null); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param timestamp only return results whose timestamp <= this value + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte[][] columns, final byte [] startRow, + long timestamp) + throws IOException { + return getScanner(columns, startRow, timestamp, null); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param filter a row filter using row-key regexp and/or column data filter. + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte[][] columns, final byte [] startRow, + RowFilterInterface filter) + throws IOException { + return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, filter); + } + + /** + * Get a scanner on the current table starting at the specified row and + * ending just before stopRow. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param stopRow Row to stop scanning on. Once we hit this row we stop + * returning values; i.e. we return the row before this one but not the + * stopRow itself. + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte [][] columns, + final byte [] startRow, final byte [] stopRow) + throws IOException { + return getScanner(columns, startRow, stopRow, + HConstants.LATEST_TIMESTAMP); + } + + /** + * Get a scanner on the current table starting at the specified row and + * ending just before stopRow. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param stopRow Row to stop scanning on. Once we hit this row we stop + * returning values; i.e. we return the row before this one but not the + * stopRow itself. + * @param timestamp only return results whose timestamp <= this value + * @return scanner + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public Scanner getScanner(final Text[] columns, + final Text startRow, final Text stopRow, final long timestamp) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), startRow.getBytes(), + stopRow.getBytes(), timestamp); + } + + /** + * Get a scanner on the current table starting at the specified row and + * ending just before stopRow. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param stopRow Row to stop scanning on. Once we hit this row we stop + * returning values; i.e. we return the row before this one but not the + * stopRow itself. + * @param timestamp only return results whose timestamp <= this value + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final String [] columns, + final String startRow, final String stopRow, final long timestamp) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow), + Bytes.toBytes(stopRow), timestamp); + } + + /** + * Get a scanner on the current table starting at the specified row and + * ending just before stopRow. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param stopRow Row to stop scanning on. Once we hit this row we stop + * returning values; i.e. we return the row before this one but not the + * stopRow itself. + * @param timestamp only return results whose timestamp <= this value + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte [][] columns, + final byte [] startRow, final byte [] stopRow, final long timestamp) + throws IOException { + return getScanner(columns, startRow, timestamp, + new WhileMatchRowFilter(new StopRowFilter(stopRow))); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param timestamp only return results whose timestamp <= this value + * @param filter a row filter using row-key regexp and/or column data filter. + * @return scanner + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public Scanner getScanner(Text[] columns, + Text startRow, long timestamp, RowFilterInterface filter) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), startRow.getBytes(), + timestamp, filter); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param timestamp only return results whose timestamp <= this value + * @param filter a row filter using row-key regexp and/or column data filter. + * @return scanner + * @throws IOException + */ + public Scanner getScanner(String[] columns, + String startRow, long timestamp, RowFilterInterface filter) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow), + timestamp, filter); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param timestamp only return results whose timestamp <= this value + * @param filter a row filter using row-key regexp and/or column data filter. + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte [][] columns, + final byte [] startRow, long timestamp, RowFilterInterface filter) + throws IOException { + return new ClientScanner(columns, startRow, timestamp, filter); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @throws IOException + */ + public void deleteAll(final byte [] row) throws IOException { + deleteAll(row, null); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public void deleteAll(final Text row) throws IOException { + deleteAll(row, null); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @throws IOException + */ + public void deleteAll(final String row) throws IOException { + deleteAll(row, null); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @param column column to be deleted + * @throws IOException + */ + public void deleteAll(final byte [] row, final byte [] column) + throws IOException { + deleteAll(row, column, HConstants.LATEST_TIMESTAMP); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + */ + public void deleteAll(final byte [] row, final long ts) + throws IOException { + deleteAll(row, null, ts); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + */ + public void deleteAll(final String row, final long ts) + throws IOException { + deleteAll(row, null, ts); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public void deleteAll(final Text row, final long ts) + throws IOException { + deleteAll(row, null, ts); + } + + /** + * Delete all cells that match the passed row and column. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public void deleteAll(final Text row, final Text column) throws IOException { + deleteAll(row, column, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all cells that match the passed row and column. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @throws IOException + */ + public void deleteAll(final String row, final String column) + throws IOException { + deleteAll(row, column, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public void deleteAll(final Text row, final Text column, final long ts) + throws IOException { + deleteAll(row.getBytes(), column.getBytes(), ts); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + */ + public void deleteAll(final String row, final String column, final long ts) + throws IOException { + deleteAll(Bytes.toBytes(row), + column != null? Bytes.toBytes(column): null, ts); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + */ + public void deleteAll(final byte [] row, final byte [] column, final long ts) + throws IOException { + connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public Boolean call() throws IOException { + if (column != null) { + this.server.deleteAll(location.getRegionInfo().getRegionName(), + row, column, ts); + } else { + this.server.deleteAll(location.getRegionInfo().getRegionName(), + row, ts); + } + return null; + } + } + ); + } + + /** + * Delete all cells for a row with matching column family at all timestamps. + * + * @param row The row to operate on + * @param family The column family to match + * @param timestamp Timestamp to match + * @throws IOException + * @deprecated Use String or byte [] overload instead + */ + public void deleteFamily(final Text row, final Text family, + final long timestamp) + throws IOException{ + deleteFamily(row.getBytes(), family.getBytes(), timestamp); + } + + /** + * Delete all cells for a row with matching column family at all timestamps. + * + * @param row The row to operate on + * @param family The column family to match + * @param timestamp Timestamp to match + * @throws IOException + */ + public void deleteFamily(final String row, final String family, + final long timestamp) + throws IOException{ + deleteFamily(Bytes.toBytes(row), Bytes.toBytes(family), timestamp); + } + + /** + * Delete all cells for a row with matching column family with timestamps + * less than or equal to timestamp. + * + * @param row The row to operate on + * @param family The column family to match + * @param timestamp Timestamp to match + * @throws IOException + */ + public void deleteFamily(final byte [] row, final byte [] family, + final long timestamp) + throws IOException { + connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public Boolean call() throws IOException { + server.deleteFamily(location.getRegionInfo().getRegionName(), row, + family, timestamp); + return null; + } + } + ); + } + + /** + * Commit a BatchUpdate to the table. + * @param batchUpdate + * @throws IOException + */ + public synchronized void commit(final BatchUpdate batchUpdate) + throws IOException { + connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, batchUpdate.getRow()) { + public Boolean call() throws IOException { + server.batchUpdate(location.getRegionInfo().getRegionName(), + batchUpdate); + return null; + } + } + ); + } + + /** + * Commit a RowsBatchUpdate to the table. + * @param batchUpdates + * @throws IOException + */ + public synchronized void commit(final List batchUpdates) + throws IOException { + for (BatchUpdate batchUpdate : batchUpdates) + commit(batchUpdate); + } + + /** + * Implements the scanner interface for the HBase client. + * If there are multiple regions in a table, this scanner will iterate + * through them all. + */ + protected class ClientScanner implements Scanner { + private final Log CLIENT_LOG = LogFactory.getLog(this.getClass()); + private byte[][] columns; + private byte [] startRow; + protected long scanTime; + @SuppressWarnings("hiding") + private boolean closed = false; + private HRegionInfo currentRegion = null; + private ScannerCallable callable = null; + protected RowFilterInterface filter; + + protected ClientScanner(final Text [] columns, final Text startRow, + long timestamp, RowFilterInterface filter) + throws IOException { + this(Bytes.toByteArrays(columns), startRow.getBytes(), timestamp, + filter); + } + + protected ClientScanner(final byte[][] columns, final byte [] startRow, + final long timestamp, final RowFilterInterface filter) + throws IOException { + if (CLIENT_LOG.isDebugEnabled()) { + CLIENT_LOG.debug("Creating scanner over " + Bytes.toString(getTableName()) + + " starting at key '" + Bytes.toString(startRow) + "'"); + } + // save off the simple parameters + this.columns = columns; + this.startRow = startRow; + this.scanTime = timestamp; + + // save the filter, and make sure that the filter applies to the data + // we're expecting to pull back + this.filter = filter; + if (filter != null) { + filter.validate(columns); + } + nextScanner(); + } + + protected byte[][] getColumns() { + return columns; + } + + protected long getTimestamp() { + return scanTime; + } + + protected RowFilterInterface getFilter() { + return filter; + } + + /* + * Gets a scanner for the next region. + * Returns false if there are no more scanners. + */ + private boolean nextScanner() throws IOException { + // Close the previous scanner if it's open + if (this.callable != null) { + this.callable.setClose(); + getConnection().getRegionServerWithRetries(callable); + this.callable = null; + } + + // if we're at the end of the table, then close and return false + // to stop iterating + if (currentRegion != null) { + if (CLIENT_LOG.isDebugEnabled()) { + CLIENT_LOG.debug("Advancing forward from region " + currentRegion); + } + + byte [] endKey = currentRegion.getEndKey(); + if (endKey == null || + Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) || + filterSaysStop(endKey)) { + close(); + return false; + } + } + + HRegionInfo oldRegion = this.currentRegion; + byte [] localStartKey = oldRegion == null? startRow: oldRegion.getEndKey(); + + if (CLIENT_LOG.isDebugEnabled()) { + CLIENT_LOG.debug("Advancing internal scanner to startKey at '" + + Bytes.toString(localStartKey) + "'"); + } + + try { + callable = getScannerCallable(localStartKey); + // open a scanner on the region server starting at the + // beginning of the region + getConnection().getRegionServerWithRetries(callable); + currentRegion = callable.getHRegionInfo(); + } catch (IOException e) { + close(); + throw e; + } + return true; + } + + protected ScannerCallable getScannerCallable(byte [] localStartKey) { + return new ScannerCallable(getConnection(), getTableName(), columns, + localStartKey, scanTime, filter); + } + + /** + * @param endKey + * @return Returns true if the passed region endkey is judged beyond + * filter. + */ + private boolean filterSaysStop(final byte [] endKey) { + if (this.filter == null) { + return false; + } + // Let the filter see current row. + this.filter.filterRowKey(endKey); + return this.filter.filterAllRemaining(); + } + + /** {@inheritDoc} */ + public RowResult next() throws IOException { + if (this.closed) { + return null; + } + RowResult values = null; + do { + values = getConnection().getRegionServerWithRetries(callable); + } while ((values == null || values.size() == 0) && nextScanner()); + + if (values != null && values.size() != 0) { + return values; + } + return null; + } + + /** + * {@inheritDoc} + */ + public void close() { + if (callable != null) { + callable.setClose(); + try { + getConnection().getRegionServerWithRetries(callable); + } catch (IOException e) { + // We used to catch this error, interpret, and rethrow. However, we + // have since decided that it's not nice for a scanner's close to + // throw exceptions. Chances are it was just an UnknownScanner + // exception due to lease time out. + } + callable = null; + } + closed = true; + } + + /** {@inheritDoc} */ + public Iterator iterator() { + return new Iterator() { + // The next RowResult, possibly pre-read + RowResult next = null; + + // return true if there is another item pending, false if there isn't. + // this method is where the actual advancing takes place, but you need + // to call next() to consume it. hasNext() will only advance if there + // isn't a pending next(). + public boolean hasNext() { + if (next == null) { + try { + next = ClientScanner.this.next(); + return next != null; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return true; + } + + // get the pending next item and advance the iterator. returns null if + // there is no next item. + public RowResult next() { + // since hasNext() does the real advancing, we call this to determine + // if there is a next before proceeding. + if (!hasNext()) { + return null; + } + + // if we get to here, then hasNext() has given us an item to return. + // we want to return the item and then null out the next pointer, so + // we use a temporary variable. + RowResult temp = next; + next = null; + return temp; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + } +} Index: src/java/org/apache/hadoop/hbase/client/HConnectionManager.java =================================================================== --- src/java/org/apache/hadoop/hbase/client/HConnectionManager.java (revision 684990) +++ src/java/org/apache/hadoop/hbase/client/HConnectionManager.java (working copy) @@ -1,889 +1,889 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client; - -import java.io.IOException; -import java.lang.reflect.UndeclaredThrowableException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.LocalHBaseCluster; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.ipc.HMasterInterface; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.ipc.HbaseRPC; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.SoftSortedMap; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.ipc.RemoteException; - -/** - * A non-instantiable class that manages connections to multiple tables in - * multiple HBase instances. - * - * Used by {@link HTable} and {@link HBaseAdmin} - */ -public class HConnectionManager implements HConstants { - /* - * Private. Not instantiable. - */ - private HConnectionManager() { - super(); - } - - // A Map of master HServerAddress -> connection information for that instance - // Note that although the Map is synchronized, the objects it contains - // are mutable and hence require synchronized access to them - private static final Map HBASE_INSTANCES = - new ConcurrentHashMap(); - - /** - * Get the connection object for the instance specified by the configuration - * If no current connection exists, create a new connection for that instance - * @param conf - * @return HConnection object for the instance specified by the configuration - */ - public static HConnection getConnection(HBaseConfiguration conf) { - TableServers connection; - synchronized (HBASE_INSTANCES) { - String instanceName = conf.get(HBASE_DIR); - connection = HBASE_INSTANCES.get(instanceName); - if (connection == null) { - connection = new TableServers(conf); - HBASE_INSTANCES.put(instanceName, connection); - } - } - return connection; - } - - /** - * Delete connection information for the instance specified by the configuration - * @param conf - */ - public static void deleteConnectionInfo(HBaseConfiguration conf) { - synchronized (HBASE_INSTANCES) { - HBASE_INSTANCES.remove(conf.get(HBASE_DIR)); - } - } - - /** - * Clear the static map of connection info. - */ - public static void deleteConnectionInfo() { - synchronized (HBASE_INSTANCES) { - HBASE_INSTANCES.clear(); - } - } - - - /* Encapsulates finding the servers for an HBase instance */ - private static class TableServers implements HConnection, HConstants { - private static final Log LOG = LogFactory.getLog(TableServers.class); - private final Class serverInterfaceClass; - private final long pause; - private final int numRetries; - private final int maxRPCAttempts; - - private final Integer masterLock = new Integer(0); - private volatile boolean closed; - private volatile HMasterInterface master; - private volatile boolean masterChecked; - - private final Integer rootRegionLock = new Integer(0); - private final Integer metaRegionLock = new Integer(0); - private final Integer userRegionLock = new Integer(0); - - private volatile HBaseConfiguration conf; - - // Known region HServerAddress.toString() -> HRegionInterface - private final Map servers = - new ConcurrentHashMap(); - - private HRegionLocation rootRegionLocation; - - private final Map> - cachedRegionLocations = Collections.synchronizedMap( - new HashMap>()); - - /** - * constructor - * @param conf Configuration object - */ - @SuppressWarnings("unchecked") - public TableServers(HBaseConfiguration conf) { - this.conf = LocalHBaseCluster.doLocal(new HBaseConfiguration(conf)); - - String serverClassName = - conf.get(REGION_SERVER_CLASS, DEFAULT_REGION_SERVER_CLASS); - - this.closed = false; - - try { - this.serverInterfaceClass = - (Class) Class.forName(serverClassName); - - } catch (ClassNotFoundException e) { - throw new UnsupportedOperationException( - "Unable to find region server interface " + serverClassName, e); - } - - this.pause = conf.getLong("hbase.client.pause", 10 * 1000); - this.numRetries = conf.getInt("hbase.client.retries.number", 10); - this.maxRPCAttempts = conf.getInt("hbase.client.rpc.maxattempts", 1); - - this.master = null; - this.masterChecked = false; - } - - private long getPauseTime(int tries) { - if (tries >= HConstants.RETRY_BACKOFF.length) - tries = HConstants.RETRY_BACKOFF.length - 1; - return this.pause * HConstants.RETRY_BACKOFF[tries]; - } - - /** {@inheritDoc} */ - public HMasterInterface getMaster() throws MasterNotRunningException { - HServerAddress masterLocation = null; - synchronized (this.masterLock) { - for (int tries = 0; - !this.closed && - !this.masterChecked && this.master == null && - tries < numRetries; - tries++) { - - masterLocation = new HServerAddress(this.conf.get(MASTER_ADDRESS, - DEFAULT_MASTER_ADDRESS)); - try { - HMasterInterface tryMaster = (HMasterInterface)HbaseRPC.getProxy( - HMasterInterface.class, HMasterInterface.versionID, - masterLocation.getInetSocketAddress(), this.conf); - - if (tryMaster.isMasterRunning()) { - this.master = tryMaster; - break; - } - - } catch (IOException e) { - if(tries == numRetries - 1) { - // This was our last chance - don't bother sleeping - break; - } - LOG.info("Attempt " + tries + " of " + this.numRetries + - " failed with <" + e + ">. Retrying after sleep of " + - getPauseTime(tries)); - } - - // We either cannot connect to master or it is not running. Sleep & retry - - try { - Thread.sleep(getPauseTime(tries)); - } catch (InterruptedException e) { - // continue - } - } - this.masterChecked = true; - } - if (this.master == null) { - if (masterLocation == null) { - throw new MasterNotRunningException(); - } - throw new MasterNotRunningException(masterLocation.toString()); - } - return this.master; - } - - /** {@inheritDoc} */ - public boolean isMasterRunning() { - if (this.master == null) { - try { - getMaster(); - - } catch (MasterNotRunningException e) { - return false; - } - } - return true; - } - - /** {@inheritDoc} */ - public boolean tableExists(final byte [] tableName) - throws MasterNotRunningException { - getMaster(); - if (tableName == null) { - throw new IllegalArgumentException("Table name cannot be null"); - } - if (isMetaTableName(tableName)) { - return true; - } - boolean exists = false; - try { - HTableDescriptor[] tables = listTables(); - for (int i = 0; i < tables.length; i++) { - if (Bytes.equals(tables[i].getName(), tableName)) { - exists = true; - } - } - } catch (IOException e) { - LOG.warn("Testing for table existence threw exception", e); - } - return exists; - } - - /* - * @param n - * @return Truen if passed tablename n is equal to the name - * of a catalog table. - */ - private static boolean isMetaTableName(final byte [] n) { - return Bytes.equals(n, ROOT_TABLE_NAME) || - Bytes.equals(n, META_TABLE_NAME); - } - - /** {@inheritDoc} */ - public HRegionLocation getRegionLocation(final byte [] name, - final byte [] row, boolean reload) - throws IOException { - getMaster(); - return reload? relocateRegion(name, row): locateRegion(name, row); - } - - /** {@inheritDoc} */ - public HTableDescriptor[] listTables() throws IOException { - getMaster(); - final HashSet uniqueTables = - new HashSet(); - - MetaScannerVisitor visitor = new MetaScannerVisitor() { - - /** {@inheritDoc} */ - public boolean processRow(RowResult rowResult) throws IOException { - HRegionInfo info = Writables.getHRegionInfo( - rowResult.get(COL_REGIONINFO)); - - // Only examine the rows where the startKey is zero length - if (info.getStartKey().length == 0) { - uniqueTables.add(info.getTableDesc()); - } - return true; - } - - }; - MetaScanner.metaScan(conf, visitor); - - return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]); - } - - /** {@inheritDoc} */ - public boolean isTableEnabled(byte[] tableName) throws IOException { - if (!tableExists(tableName)) { - throw new TableNotFoundException(Bytes.toString(tableName)); - } - if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) { - // The root region is always enabled - return true; - } - - boolean result = true; - int rowsScanned = 0; - byte[] startKey = - HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES); - HRegionInfo currentRegion = null; - do { - if (currentRegion != null) { - byte[] endKey = currentRegion.getEndKey(); - if (endKey == null || - Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)) { - // We have reached the end of the table and we're done - break; - } - } - HRegionInfo oldRegion = currentRegion; - if (oldRegion != null) { - startKey = oldRegion.getEndKey(); - } - ScannerCallable s = new ScannerCallable(this, - (Bytes.equals(tableName, HConstants.META_TABLE_NAME) ? - HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME), - HConstants.COL_REGIONINFO_ARRAY, startKey, - HConstants.LATEST_TIMESTAMP, null - ); - // Open scanner - getRegionServerWithRetries(s); - currentRegion = s.getHRegionInfo(); - try { - RowResult r = null; - while (result && (r = getRegionServerWithRetries(s)) != null) { - Cell c = r.get(HConstants.COL_REGIONINFO); - if (c != null) { - byte[] value = c.getValue(); - if (value != null) { - HRegionInfo info = Writables.getHRegionInfoOrNull(value); - if (info != null) { - if (Bytes.equals(info.getTableDesc().getName(), tableName)) { - rowsScanned += 1; - result = !info.isOffline(); - } - } - } - } - } - } finally { - s.setClose(); - getRegionServerWithRetries(s); - } - } while (result); - return rowsScanned > 0 && result; - } - - private class HTableDescriptorFinder - implements MetaScanner.MetaScannerVisitor { - byte[] tableName; - HTableDescriptor result; - public HTableDescriptorFinder(byte[] tableName) { - this.tableName = tableName; - } - public boolean processRow(RowResult rowResult) throws IOException { - HRegionInfo info = Writables.getHRegionInfo( - rowResult.get(HConstants.COL_REGIONINFO)); - HTableDescriptor desc = info.getTableDesc(); - if (Bytes.compareTo(desc.getName(), tableName) == 0) { - result = desc; - return false; - } - return true; - } - HTableDescriptor getResult() { - return result; - } - } - - /** {@inheritDoc} */ - public HTableDescriptor getHTableDescriptor(final byte[] tableName) - throws IOException { - if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) { - return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC); - } - if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) { - return new UnmodifyableHTableDescriptor(HTableDescriptor.META_TABLEDESC); - } - HTableDescriptorFinder finder = new HTableDescriptorFinder(tableName); - MetaScanner.metaScan(conf, finder); - HTableDescriptor result = finder.getResult(); - if (result == null) { - throw new TableNotFoundException(Bytes.toString(tableName)); - } - return result; - } - - /** {@inheritDoc} */ - public HRegionLocation locateRegion(final byte [] tableName, - final byte [] row) - throws IOException{ - getMaster(); - return locateRegion(tableName, row, true); - } - - /** {@inheritDoc} */ - public HRegionLocation relocateRegion(final byte [] tableName, - final byte [] row) - throws IOException{ - getMaster(); - return locateRegion(tableName, row, false); - } - - private HRegionLocation locateRegion(final byte [] tableName, - final byte [] row, boolean useCache) - throws IOException{ - if (tableName == null || tableName.length == 0) { - throw new IllegalArgumentException( - "table name cannot be null or zero length"); - } - - if (Bytes.equals(tableName, ROOT_TABLE_NAME)) { - synchronized (rootRegionLock) { - // This block guards against two threads trying to find the root - // region at the same time. One will go do the find while the - // second waits. The second thread will not do find. - - if (!useCache || rootRegionLocation == null) { - return locateRootRegion(); - } - return rootRegionLocation; - } - } else if (Bytes.equals(tableName, META_TABLE_NAME)) { - synchronized (metaRegionLock) { - // This block guards against two threads trying to load the meta - // region at the same time. The first will load the meta region and - // the second will use the value that the first one found. - - return locateRegionInMeta(ROOT_TABLE_NAME, tableName, row, useCache); - } - } else { - synchronized(userRegionLock){ - return locateRegionInMeta(META_TABLE_NAME, tableName, row, useCache); - } - } - } - - /** - * Search one of the meta tables (-ROOT- or .META.) for the HRegionLocation - * info that contains the table and row we're seeking. - */ - private HRegionLocation locateRegionInMeta(final byte [] parentTable, - final byte [] tableName, final byte [] row, boolean useCache) - throws IOException{ - HRegionLocation location = null; - - // if we're supposed to be using the cache, then check it for a possible - // hit. otherwise, delete any existing cached location so it won't - // interfere. - if (useCache) { - location = getCachedLocation(tableName, row); - if (location != null) { - return location; - } - } else { - deleteCachedLocation(tableName, row); - } - - // build the key of the meta region we should be looking for. - // the extra 9's on the end are necessary to allow "exact" matches - // without knowing the precise region names. - byte [] metaKey = HRegionInfo.createRegionName(tableName, row, - HConstants.NINES); - for (int tries = 0; true; tries++) { - if (tries >= numRetries) { - throw new NoServerForRegionException("Unable to find region for " - + Bytes.toString(row) + " after " + numRetries + " tries."); - } - - try{ - // locate the root region - HRegionLocation metaLocation = locateRegion(parentTable, metaKey); - HRegionInterface server = - getHRegionConnection(metaLocation.getServerAddress()); - - // query the root region for the location of the meta region - RowResult regionInfoRow = server.getClosestRowBefore( - metaLocation.getRegionInfo().getRegionName(), metaKey); - - if (regionInfoRow == null) { - throw new TableNotFoundException(Bytes.toString(tableName)); - } - - Cell value = regionInfoRow.get(COL_REGIONINFO); - - if (value == null || value.getValue().length == 0) { - throw new IOException("HRegionInfo was null or empty in " + - Bytes.toString(parentTable)); - } - - // convert the row result into the HRegionLocation we need! - HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable( - value.getValue(), 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) + "' was not found."); - } - - if (regionInfo.isOffline()) { - throw new RegionOfflineException("region offline: " + - regionInfo.getRegionNameAsString()); - } - - String serverAddress = - Writables.cellToString(regionInfoRow.get(COL_SERVER)); - - if (serverAddress.equals("")) { - throw new NoServerForRegionException("No server address listed " + - "in " + Bytes.toString(parentTable) + " for region " + - regionInfo.getRegionNameAsString()); - } - - // instantiate the location - location = new HRegionLocation(regionInfo, - new HServerAddress(serverAddress)); - - cacheLocation(tableName, location); - - return location; - } catch (TableNotFoundException e) { - // if we got this error, probably means the table just plain doesn't - // exist. rethrow the error immediately. this should always be coming - // from the HTable constructor. - throw e; - } catch (IOException e) { - if (e instanceof RemoteException) { - e = RemoteExceptionHandler.decodeRemoteException( - (RemoteException) e); - } - if (tries < numRetries - 1) { - if (LOG.isDebugEnabled()) { - LOG.debug("reloading table servers because: " + e.getMessage()); - } - relocateRegion(parentTable, metaKey); - } else { - throw e; - } - } - - try{ - Thread.sleep(getPauseTime(tries)); - } catch (InterruptedException e){ - // continue - } - } - } - - /* - * Search the cache for a location that fits our table and row key. - * Return null if no suitable region is located. TODO: synchronization note - * - *

TODO: This method during writing consumes 15% of CPU doing lookup - * into the Soft Reference SortedMap. Improve. - * - * @param tableName - * @param row - * @return Null or region location found in cache. - */ - private HRegionLocation getCachedLocation(final byte [] tableName, - final byte [] row) { - // find the map of cached locations for this table - Integer key = Bytes.mapKey(tableName); - SoftSortedMap tableLocations = - cachedRegionLocations.get(key); - - // if tableLocations for this table isn't built yet, make one - if (tableLocations == null) { - tableLocations = new SoftSortedMap(Bytes.BYTES_COMPARATOR); - cachedRegionLocations.put(key, tableLocations); - } - - // start to examine the cache. we can only do cache actions - // if there's something in the cache for this table. - if (tableLocations.isEmpty()) { - return null; - } - - HRegionLocation rl = tableLocations.get(row); - if (rl != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Cache hit in table locations for row <" + - Bytes.toString(row) + - "> and tableName " + Bytes.toString(tableName) + - ": location server " + rl.getServerAddress() + - ", location region name " + - rl.getRegionInfo().getRegionNameAsString()); - } - return rl; - } - - // Cut the cache so that we only get the part that could contain - // regions that match our key - SoftSortedMap matchingRegions = - tableLocations.headMap(row); - - // if that portion of the map is empty, then we're done. otherwise, - // we need to examine the cached location to verify that it is - // a match by end key as well. - if (!matchingRegions.isEmpty()) { - HRegionLocation possibleRegion = - matchingRegions.get(matchingRegions.lastKey()); - - // there is a possibility that the reference was garbage collected - // in the instant since we checked isEmpty(). - if (possibleRegion != null) { - byte[] endKey = possibleRegion.getRegionInfo().getEndKey(); - - // make sure that the end key is greater than the row we're looking - // for, otherwise the row actually belongs in the next region, not - // this one. the exception case is when the endkey is EMPTY_START_ROW, - // signifying that the region we're checking is actually the last - // region in the table. - if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) || - Bytes.compareTo(endKey, row) > 0) { - return possibleRegion; - } - } - } - - // Passed all the way through, so we got nothin - complete cache miss - return null; - } - - /** - * Delete a cached location, if it satisfies the table name and row - * requirements. - */ - private void deleteCachedLocation(final byte [] tableName, - final byte [] row) { - // find the map of cached locations for this table - Integer key = Bytes.mapKey(tableName); - SoftSortedMap tableLocations = - cachedRegionLocations.get(key); - - // if tableLocations for this table isn't built yet, make one - if (tableLocations == null) { - tableLocations = - new SoftSortedMap(Bytes.BYTES_COMPARATOR); - cachedRegionLocations.put(key, tableLocations); - } - - // start to examine the cache. we can only do cache actions - // if there's something in the cache for this table. - if (!tableLocations.isEmpty()) { - // cut the cache so that we only get the part that could contain - // regions that match our key - SoftSortedMap matchingRegions = - tableLocations.headMap(row); - - // if that portion of the map is empty, then we're done. otherwise, - // we need to examine the cached location to verify that it is - // a match by end key as well. - if (!matchingRegions.isEmpty()) { - HRegionLocation possibleRegion = - matchingRegions.get(matchingRegions.lastKey()); - - byte [] endKey = possibleRegion.getRegionInfo().getEndKey(); - - // by nature of the map, we know that the start key has to be < - // otherwise it wouldn't be in the headMap. - if (Bytes.compareTo(endKey, row) <= 0) { - // delete any matching entry - HRegionLocation rl = - tableLocations.remove(matchingRegions.lastKey()); - if (rl != null && LOG.isDebugEnabled()) { - LOG.debug("Removed " + rl.getRegionInfo().getRegionNameAsString() + - " from cache because of " + Bytes.toString(row)); - } - } - } - } - } - - /** - * Put a newly discovered HRegionLocation into the cache. - */ - private void cacheLocation(final byte [] tableName, - final HRegionLocation location){ - byte [] startKey = location.getRegionInfo().getStartKey(); - - // find the map of cached locations for this table - Integer key = Bytes.mapKey(tableName); - SoftSortedMap tableLocations = - cachedRegionLocations.get(key); - - // if tableLocations for this table isn't built yet, make one - if (tableLocations == null) { - tableLocations = - new SoftSortedMap(Bytes.BYTES_COMPARATOR); - cachedRegionLocations.put(key, tableLocations); - } - - // save the HRegionLocation under the startKey - tableLocations.put(startKey, location); - } - - /** {@inheritDoc} */ - public HRegionInterface getHRegionConnection(HServerAddress regionServer) - throws IOException { - getMaster(); - HRegionInterface server; - synchronized (this.servers) { - // See if we already have a connection - server = this.servers.get(regionServer.toString()); - if (server == null) { // Get a connection - long versionId = 0; - try { - versionId = - serverInterfaceClass.getDeclaredField("versionID").getLong(server); - } catch (IllegalAccessException e) { - // Should never happen unless visibility of versionID changes - throw new UnsupportedOperationException( - "Unable to open a connection to a " + - serverInterfaceClass.getName() + " server.", e); - } catch (NoSuchFieldException e) { - // Should never happen unless versionID field name changes in HRegionInterface - throw new UnsupportedOperationException( - "Unable to open a connection to a " + - serverInterfaceClass.getName() + " server.", e); - } - - try { - server = (HRegionInterface)HbaseRPC.waitForProxy(serverInterfaceClass, - versionId, regionServer.getInetSocketAddress(), this.conf, - this.maxRPCAttempts); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } - this.servers.put(regionServer.toString(), server); - } - } - return server; - } - - /* - * Repeatedly try to find the root region by asking the master for where it is - * @return HRegionLocation for root region if found - * @throws NoServerForRegionException - if the root region can not be located - * after retrying - * @throws IOException - */ - private HRegionLocation locateRootRegion() - throws IOException { - getMaster(); - HServerAddress rootRegionAddress = null; - for (int tries = 0; tries < numRetries; tries++) { - int localTimeouts = 0; - - // ask the master which server has the root region - while (rootRegionAddress == null && localTimeouts < numRetries) { - rootRegionAddress = master.findRootRegion(); - if (rootRegionAddress == null) { - try { - if (LOG.isDebugEnabled()) { - LOG.debug("Sleeping. Waiting for root region."); - } - Thread.sleep(getPauseTime(tries)); - if (LOG.isDebugEnabled()) { - LOG.debug("Wake. Retry finding root region."); - } - } catch (InterruptedException iex) { - // continue - } - localTimeouts++; - } - } - - if (rootRegionAddress == null) { - throw new NoServerForRegionException( - "Timed out trying to locate root region"); - } - - // get a connection to the region server - HRegionInterface server = getHRegionConnection(rootRegionAddress); - - try { - // if this works, then we're good, and we have an acceptable address, - // so we can stop doing retries and return the result. - server.getRegionInfo(HRegionInfo.ROOT_REGIONINFO.getRegionName()); - if (LOG.isDebugEnabled()) { - LOG.debug("Found ROOT " + HRegionInfo.ROOT_REGIONINFO); - } - break; - } catch (IOException e) { - if (tries == numRetries - 1) { - // Don't bother sleeping. We've run out of retries. - if (e instanceof RemoteException) { - e = RemoteExceptionHandler.decodeRemoteException( - (RemoteException) e); - } - throw e; - } - - // Sleep and retry finding root region. - try { - if (LOG.isDebugEnabled()) { - LOG.debug("Root region location changed. Sleeping."); - } - Thread.sleep(getPauseTime(tries)); - if (LOG.isDebugEnabled()) { - LOG.debug("Wake. Retry finding root region."); - } - } catch (InterruptedException iex) { - // continue - } - } - - rootRegionAddress = null; - } - - // if the address is null by this point, then the retries have failed, - // and we're sort of sunk - if (rootRegionAddress == null) { - throw new NoServerForRegionException( - "unable to locate root region server"); - } - - // return the region location - return new HRegionLocation( - HRegionInfo.ROOT_REGIONINFO, rootRegionAddress); - } - - /** {@inheritDoc} */ - public T getRegionServerWithRetries(ServerCallable callable) - throws IOException, RuntimeException { - getMaster(); - List exceptions = new ArrayList(); - for(int tries = 0; tries < numRetries; tries++) { - try { - callable.instantiateServer(tries != 0); - return callable.call(); - } catch (Throwable t) { - if (t instanceof UndeclaredThrowableException) { - t = t.getCause(); - } - if (t instanceof RemoteException) { - t = RemoteExceptionHandler.decodeRemoteException((RemoteException) t); - } - if (t instanceof DoNotRetryIOException) { - throw (DoNotRetryIOException)t; - } - exceptions.add(t); - if (tries == numRetries - 1) { - throw new RetriesExhaustedException(callable.getServerName(), - callable.getRegionName(), callable.getRow(), tries, exceptions); - } - if (LOG.isDebugEnabled()) { - LOG.debug("reloading table servers because: " + t.getMessage()); - } - } - try { - Thread.sleep(getPauseTime(tries)); - } catch (InterruptedException e) { - // continue - } - } - return null; - } - } -} +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.LocalHBaseCluster; +import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.ipc.HMasterInterface; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.ipc.HbaseRPC; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.SoftSortedMap; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.ipc.RemoteException; + +/** + * A non-instantiable class that manages connections to multiple tables in + * multiple HBase instances. + * + * Used by {@link HTable} and {@link HBaseAdmin} + */ +public class HConnectionManager implements HConstants { + /* + * Private. Not instantiable. + */ + private HConnectionManager() { + super(); + } + + // A Map of master HServerAddress -> connection information for that instance + // Note that although the Map is synchronized, the objects it contains + // are mutable and hence require synchronized access to them + private static final Map HBASE_INSTANCES = + new ConcurrentHashMap(); + + /** + * Get the connection object for the instance specified by the configuration + * If no current connection exists, create a new connection for that instance + * @param conf + * @return HConnection object for the instance specified by the configuration + */ + public static HConnection getConnection(HBaseConfiguration conf) { + TableServers connection; + synchronized (HBASE_INSTANCES) { + String instanceName = conf.get(HBASE_DIR); + connection = HBASE_INSTANCES.get(instanceName); + if (connection == null) { + connection = new TableServers(conf); + HBASE_INSTANCES.put(instanceName, connection); + } + } + return connection; + } + + /** + * Delete connection information for the instance specified by the configuration + * @param conf + */ + public static void deleteConnectionInfo(HBaseConfiguration conf) { + synchronized (HBASE_INSTANCES) { + HBASE_INSTANCES.remove(conf.get(HBASE_DIR)); + } + } + + /** + * Clear the static map of connection info. + */ + public static void deleteConnectionInfo() { + synchronized (HBASE_INSTANCES) { + HBASE_INSTANCES.clear(); + } + } + + + /* Encapsulates finding the servers for an HBase instance */ + private static class TableServers implements HConnection, HConstants { + private static final Log LOG = LogFactory.getLog(TableServers.class); + private final Class serverInterfaceClass; + private final long pause; + private final int numRetries; + private final int maxRPCAttempts; + + private final Integer masterLock = new Integer(0); + private volatile boolean closed; + private volatile HMasterInterface master; + private volatile boolean masterChecked; + + private final Integer rootRegionLock = new Integer(0); + private final Integer metaRegionLock = new Integer(0); + private final Integer userRegionLock = new Integer(0); + + private volatile HBaseConfiguration conf; + + // Known region HServerAddress.toString() -> HRegionInterface + private final Map servers = + new ConcurrentHashMap(); + + private HRegionLocation rootRegionLocation; + + private final Map> + cachedRegionLocations = Collections.synchronizedMap( + new HashMap>()); + + /** + * constructor + * @param conf Configuration object + */ + @SuppressWarnings("unchecked") + public TableServers(HBaseConfiguration conf) { + this.conf = LocalHBaseCluster.doLocal(new HBaseConfiguration(conf)); + + String serverClassName = + conf.get(REGION_SERVER_CLASS, DEFAULT_REGION_SERVER_CLASS); + + this.closed = false; + + try { + this.serverInterfaceClass = + (Class) Class.forName(serverClassName); + + } catch (ClassNotFoundException e) { + throw new UnsupportedOperationException( + "Unable to find region server interface " + serverClassName, e); + } + + this.pause = conf.getLong("hbase.client.pause", 10 * 1000); + this.numRetries = conf.getInt("hbase.client.retries.number", 10); + this.maxRPCAttempts = conf.getInt("hbase.client.rpc.maxattempts", 1); + + this.master = null; + this.masterChecked = false; + } + + private long getPauseTime(int tries) { + if (tries >= HConstants.RETRY_BACKOFF.length) + tries = HConstants.RETRY_BACKOFF.length - 1; + return this.pause * HConstants.RETRY_BACKOFF[tries]; + } + + /** {@inheritDoc} */ + public HMasterInterface getMaster() throws MasterNotRunningException { + HServerAddress masterLocation = null; + synchronized (this.masterLock) { + for (int tries = 0; + !this.closed && + !this.masterChecked && this.master == null && + tries < numRetries; + tries++) { + + masterLocation = new HServerAddress(this.conf.get(MASTER_ADDRESS, + DEFAULT_MASTER_ADDRESS)); + try { + HMasterInterface tryMaster = (HMasterInterface)HbaseRPC.getProxy( + HMasterInterface.class, HMasterInterface.versionID, + masterLocation.getInetSocketAddress(), this.conf); + + if (tryMaster.isMasterRunning()) { + this.master = tryMaster; + break; + } + + } catch (IOException e) { + if(tries == numRetries - 1) { + // This was our last chance - don't bother sleeping + break; + } + LOG.info("Attempt " + tries + " of " + this.numRetries + + " failed with <" + e + ">. Retrying after sleep of " + + getPauseTime(tries)); + } + + // We either cannot connect to master or it is not running. Sleep & retry + + try { + Thread.sleep(getPauseTime(tries)); + } catch (InterruptedException e) { + // continue + } + } + this.masterChecked = true; + } + if (this.master == null) { + if (masterLocation == null) { + throw new MasterNotRunningException(); + } + throw new MasterNotRunningException(masterLocation.toString()); + } + return this.master; + } + + /** {@inheritDoc} */ + public boolean isMasterRunning() { + if (this.master == null) { + try { + getMaster(); + + } catch (MasterNotRunningException e) { + return false; + } + } + return true; + } + + /** {@inheritDoc} */ + public boolean tableExists(final byte [] tableName) + throws MasterNotRunningException { + getMaster(); + if (tableName == null) { + throw new IllegalArgumentException("Table name cannot be null"); + } + if (isMetaTableName(tableName)) { + return true; + } + boolean exists = false; + try { + HTableDescriptor[] tables = listTables(); + for (int i = 0; i < tables.length; i++) { + if (Bytes.equals(tables[i].getName(), tableName)) { + exists = true; + } + } + } catch (IOException e) { + LOG.warn("Testing for table existence threw exception", e); + } + return exists; + } + + /* + * @param n + * @return Truen if passed tablename n is equal to the name + * of a catalog table. + */ + private static boolean isMetaTableName(final byte [] n) { + return Bytes.equals(n, ROOT_TABLE_NAME) || + Bytes.equals(n, META_TABLE_NAME); + } + + /** {@inheritDoc} */ + public HRegionLocation getRegionLocation(final byte [] name, + final byte [] row, boolean reload) + throws IOException { + getMaster(); + return reload? relocateRegion(name, row): locateRegion(name, row); + } + + /** {@inheritDoc} */ + public HTableDescriptor[] listTables() throws IOException { + getMaster(); + final HashSet uniqueTables = + new HashSet(); + + MetaScannerVisitor visitor = new MetaScannerVisitor() { + + /** {@inheritDoc} */ + public boolean processRow(RowResult rowResult) throws IOException { + HRegionInfo info = Writables.getHRegionInfo( + rowResult.get(COL_REGIONINFO)); + + // Only examine the rows where the startKey is zero length + if (info.getStartKey().length == 0) { + uniqueTables.add(info.getTableDesc()); + } + return true; + } + + }; + MetaScanner.metaScan(conf, visitor); + + return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]); + } + + /** {@inheritDoc} */ + public boolean isTableEnabled(byte[] tableName) throws IOException { + if (!tableExists(tableName)) { + throw new TableNotFoundException(Bytes.toString(tableName)); + } + if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) { + // The root region is always enabled + return true; + } + + boolean result = true; + int rowsScanned = 0; + byte[] startKey = + HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES); + HRegionInfo currentRegion = null; + do { + if (currentRegion != null) { + byte[] endKey = currentRegion.getEndKey(); + if (endKey == null || + Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)) { + // We have reached the end of the table and we're done + break; + } + } + HRegionInfo oldRegion = currentRegion; + if (oldRegion != null) { + startKey = oldRegion.getEndKey(); + } + ScannerCallable s = new ScannerCallable(this, + (Bytes.equals(tableName, HConstants.META_TABLE_NAME) ? + HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME), + HConstants.COL_REGIONINFO_ARRAY, startKey, + HConstants.LATEST_TIMESTAMP, null + ); + // Open scanner + getRegionServerWithRetries(s); + currentRegion = s.getHRegionInfo(); + try { + RowResult r = null; + while (result && (r = getRegionServerWithRetries(s)) != null) { + Cell c = r.get(HConstants.COL_REGIONINFO); + if (c != null) { + byte[] value = c.getValue(); + if (value != null) { + HRegionInfo info = Writables.getHRegionInfoOrNull(value); + if (info != null) { + if (Bytes.equals(info.getTableDesc().getName(), tableName)) { + rowsScanned += 1; + result = !info.isOffline(); + } + } + } + } + } + } finally { + s.setClose(); + getRegionServerWithRetries(s); + } + } while (result); + return rowsScanned > 0 && result; + } + + private class HTableDescriptorFinder + implements MetaScanner.MetaScannerVisitor { + byte[] tableName; + HTableDescriptor result; + public HTableDescriptorFinder(byte[] tableName) { + this.tableName = tableName; + } + public boolean processRow(RowResult rowResult) throws IOException { + HRegionInfo info = Writables.getHRegionInfo( + rowResult.get(HConstants.COL_REGIONINFO)); + HTableDescriptor desc = info.getTableDesc(); + if (Bytes.compareTo(desc.getName(), tableName) == 0) { + result = desc; + return false; + } + return true; + } + HTableDescriptor getResult() { + return result; + } + } + + /** {@inheritDoc} */ + public HTableDescriptor getHTableDescriptor(final byte[] tableName) + throws IOException { + if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) { + return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC); + } + if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) { + return new UnmodifyableHTableDescriptor(HTableDescriptor.META_TABLEDESC); + } + HTableDescriptorFinder finder = new HTableDescriptorFinder(tableName); + MetaScanner.metaScan(conf, finder); + HTableDescriptor result = finder.getResult(); + if (result == null) { + throw new TableNotFoundException(Bytes.toString(tableName)); + } + return result; + } + + /** {@inheritDoc} */ + public HRegionLocation locateRegion(final byte [] tableName, + final byte [] row) + throws IOException{ + getMaster(); + return locateRegion(tableName, row, true); + } + + /** {@inheritDoc} */ + public HRegionLocation relocateRegion(final byte [] tableName, + final byte [] row) + throws IOException{ + getMaster(); + return locateRegion(tableName, row, false); + } + + private HRegionLocation locateRegion(final byte [] tableName, + final byte [] row, boolean useCache) + throws IOException{ + if (tableName == null || tableName.length == 0) { + throw new IllegalArgumentException( + "table name cannot be null or zero length"); + } + + if (Bytes.equals(tableName, ROOT_TABLE_NAME)) { + synchronized (rootRegionLock) { + // This block guards against two threads trying to find the root + // region at the same time. One will go do the find while the + // second waits. The second thread will not do find. + + if (!useCache || rootRegionLocation == null) { + return locateRootRegion(); + } + return rootRegionLocation; + } + } else if (Bytes.equals(tableName, META_TABLE_NAME)) { + synchronized (metaRegionLock) { + // This block guards against two threads trying to load the meta + // region at the same time. The first will load the meta region and + // the second will use the value that the first one found. + + return locateRegionInMeta(ROOT_TABLE_NAME, tableName, row, useCache); + } + } else { + synchronized(userRegionLock){ + return locateRegionInMeta(META_TABLE_NAME, tableName, row, useCache); + } + } + } + + /** + * Search one of the meta tables (-ROOT- or .META.) for the HRegionLocation + * info that contains the table and row we're seeking. + */ + private HRegionLocation locateRegionInMeta(final byte [] parentTable, + final byte [] tableName, final byte [] row, boolean useCache) + throws IOException{ + HRegionLocation location = null; + + // if we're supposed to be using the cache, then check it for a possible + // hit. otherwise, delete any existing cached location so it won't + // interfere. + if (useCache) { + location = getCachedLocation(tableName, row); + if (location != null) { + return location; + } + } else { + deleteCachedLocation(tableName, row); + } + + // build the key of the meta region we should be looking for. + // the extra 9's on the end are necessary to allow "exact" matches + // without knowing the precise region names. + byte [] metaKey = HRegionInfo.createRegionName(tableName, row, + HConstants.NINES); + for (int tries = 0; true; tries++) { + if (tries >= numRetries) { + throw new NoServerForRegionException("Unable to find region for " + + Bytes.toString(row) + " after " + numRetries + " tries."); + } + + try{ + // locate the root region + HRegionLocation metaLocation = locateRegion(parentTable, metaKey); + HRegionInterface server = + getHRegionConnection(metaLocation.getServerAddress()); + + // query the root region for the location of the meta region + RowResult regionInfoRow = server.getClosestRowBefore( + metaLocation.getRegionInfo().getRegionName(), metaKey); + + if (regionInfoRow == null) { + throw new TableNotFoundException(Bytes.toString(tableName)); + } + + Cell value = regionInfoRow.get(COL_REGIONINFO); + + if (value == null || value.getValue().length == 0) { + throw new IOException("HRegionInfo was null or empty in " + + Bytes.toString(parentTable)); + } + + // convert the row result into the HRegionLocation we need! + HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable( + value.getValue(), 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) + "' was not found."); + } + + if (regionInfo.isOffline()) { + throw new RegionOfflineException("region offline: " + + regionInfo.getRegionNameAsString()); + } + + String serverAddress = + Writables.cellToString(regionInfoRow.get(COL_SERVER)); + + if (serverAddress.equals("")) { + throw new NoServerForRegionException("No server address listed " + + "in " + Bytes.toString(parentTable) + " for region " + + regionInfo.getRegionNameAsString()); + } + + // instantiate the location + location = new HRegionLocation(regionInfo, + new HServerAddress(serverAddress)); + + cacheLocation(tableName, location); + + return location; + } catch (TableNotFoundException e) { + // if we got this error, probably means the table just plain doesn't + // exist. rethrow the error immediately. this should always be coming + // from the HTable constructor. + throw e; + } catch (IOException e) { + if (e instanceof RemoteException) { + e = RemoteExceptionHandler.decodeRemoteException( + (RemoteException) e); + } + if (tries < numRetries - 1) { + if (LOG.isDebugEnabled()) { + LOG.debug("reloading table servers because: " + e.getMessage()); + } + relocateRegion(parentTable, metaKey); + } else { + throw e; + } + } + + try{ + Thread.sleep(getPauseTime(tries)); + } catch (InterruptedException e){ + // continue + } + } + } + + /* + * Search the cache for a location that fits our table and row key. + * Return null if no suitable region is located. TODO: synchronization note + * + *

TODO: This method during writing consumes 15% of CPU doing lookup + * into the Soft Reference SortedMap. Improve. + * + * @param tableName + * @param row + * @return Null or region location found in cache. + */ + private HRegionLocation getCachedLocation(final byte [] tableName, + final byte [] row) { + // find the map of cached locations for this table + Integer key = Bytes.mapKey(tableName); + SoftSortedMap tableLocations = + cachedRegionLocations.get(key); + + // if tableLocations for this table isn't built yet, make one + if (tableLocations == null) { + tableLocations = new SoftSortedMap(Bytes.BYTES_COMPARATOR); + cachedRegionLocations.put(key, tableLocations); + } + + // start to examine the cache. we can only do cache actions + // if there's something in the cache for this table. + if (tableLocations.isEmpty()) { + return null; + } + + HRegionLocation rl = tableLocations.get(row); + if (rl != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Cache hit in table locations for row <" + + Bytes.toString(row) + + "> and tableName " + Bytes.toString(tableName) + + ": location server " + rl.getServerAddress() + + ", location region name " + + rl.getRegionInfo().getRegionNameAsString()); + } + return rl; + } + + // Cut the cache so that we only get the part that could contain + // regions that match our key + SoftSortedMap matchingRegions = + tableLocations.headMap(row); + + // if that portion of the map is empty, then we're done. otherwise, + // we need to examine the cached location to verify that it is + // a match by end key as well. + if (!matchingRegions.isEmpty()) { + HRegionLocation possibleRegion = + matchingRegions.get(matchingRegions.lastKey()); + + // there is a possibility that the reference was garbage collected + // in the instant since we checked isEmpty(). + if (possibleRegion != null) { + byte[] endKey = possibleRegion.getRegionInfo().getEndKey(); + + // make sure that the end key is greater than the row we're looking + // for, otherwise the row actually belongs in the next region, not + // this one. the exception case is when the endkey is EMPTY_START_ROW, + // signifying that the region we're checking is actually the last + // region in the table. + if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) || + Bytes.compareTo(endKey, row) > 0) { + return possibleRegion; + } + } + } + + // Passed all the way through, so we got nothin - complete cache miss + return null; + } + + /** + * Delete a cached location, if it satisfies the table name and row + * requirements. + */ + private void deleteCachedLocation(final byte [] tableName, + final byte [] row) { + // find the map of cached locations for this table + Integer key = Bytes.mapKey(tableName); + SoftSortedMap tableLocations = + cachedRegionLocations.get(key); + + // if tableLocations for this table isn't built yet, make one + if (tableLocations == null) { + tableLocations = + new SoftSortedMap(Bytes.BYTES_COMPARATOR); + cachedRegionLocations.put(key, tableLocations); + } + + // start to examine the cache. we can only do cache actions + // if there's something in the cache for this table. + if (!tableLocations.isEmpty()) { + // cut the cache so that we only get the part that could contain + // regions that match our key + SoftSortedMap matchingRegions = + tableLocations.headMap(row); + + // if that portion of the map is empty, then we're done. otherwise, + // we need to examine the cached location to verify that it is + // a match by end key as well. + if (!matchingRegions.isEmpty()) { + HRegionLocation possibleRegion = + matchingRegions.get(matchingRegions.lastKey()); + + byte [] endKey = possibleRegion.getRegionInfo().getEndKey(); + + // by nature of the map, we know that the start key has to be < + // otherwise it wouldn't be in the headMap. + if (Bytes.compareTo(endKey, row) <= 0) { + // delete any matching entry + HRegionLocation rl = + tableLocations.remove(matchingRegions.lastKey()); + if (rl != null && LOG.isDebugEnabled()) { + LOG.debug("Removed " + rl.getRegionInfo().getRegionNameAsString() + + " from cache because of " + Bytes.toString(row)); + } + } + } + } + } + + /** + * Put a newly discovered HRegionLocation into the cache. + */ + private void cacheLocation(final byte [] tableName, + final HRegionLocation location){ + byte [] startKey = location.getRegionInfo().getStartKey(); + + // find the map of cached locations for this table + Integer key = Bytes.mapKey(tableName); + SoftSortedMap tableLocations = + cachedRegionLocations.get(key); + + // if tableLocations for this table isn't built yet, make one + if (tableLocations == null) { + tableLocations = + new SoftSortedMap(Bytes.BYTES_COMPARATOR); + cachedRegionLocations.put(key, tableLocations); + } + + // save the HRegionLocation under the startKey + tableLocations.put(startKey, location); + } + + /** {@inheritDoc} */ + public HRegionInterface getHRegionConnection(HServerAddress regionServer) + throws IOException { + getMaster(); + HRegionInterface server; + synchronized (this.servers) { + // See if we already have a connection + server = this.servers.get(regionServer.toString()); + if (server == null) { // Get a connection + long versionId = 0; + try { + versionId = + serverInterfaceClass.getDeclaredField("versionID").getLong(server); + } catch (IllegalAccessException e) { + // Should never happen unless visibility of versionID changes + throw new UnsupportedOperationException( + "Unable to open a connection to a " + + serverInterfaceClass.getName() + " server.", e); + } catch (NoSuchFieldException e) { + // Should never happen unless versionID field name changes in HRegionInterface + throw new UnsupportedOperationException( + "Unable to open a connection to a " + + serverInterfaceClass.getName() + " server.", e); + } + + try { + server = (HRegionInterface)HbaseRPC.waitForProxy(serverInterfaceClass, + versionId, regionServer.getInetSocketAddress(), this.conf, + this.maxRPCAttempts); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + this.servers.put(regionServer.toString(), server); + } + } + return server; + } + + /* + * Repeatedly try to find the root region by asking the master for where it is + * @return HRegionLocation for root region if found + * @throws NoServerForRegionException - if the root region can not be located + * after retrying + * @throws IOException + */ + private HRegionLocation locateRootRegion() + throws IOException { + getMaster(); + HServerAddress rootRegionAddress = null; + for (int tries = 0; tries < numRetries; tries++) { + int localTimeouts = 0; + + // ask the master which server has the root region + while (rootRegionAddress == null && localTimeouts < numRetries) { + rootRegionAddress = master.findRootRegion(); + if (rootRegionAddress == null) { + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Sleeping. Waiting for root region."); + } + Thread.sleep(getPauseTime(tries)); + if (LOG.isDebugEnabled()) { + LOG.debug("Wake. Retry finding root region."); + } + } catch (InterruptedException iex) { + // continue + } + localTimeouts++; + } + } + + if (rootRegionAddress == null) { + throw new NoServerForRegionException( + "Timed out trying to locate root region"); + } + + // get a connection to the region server + HRegionInterface server = getHRegionConnection(rootRegionAddress); + + try { + // if this works, then we're good, and we have an acceptable address, + // so we can stop doing retries and return the result. + server.getRegionInfo(HRegionInfo.ROOT_REGIONINFO.getRegionName()); + if (LOG.isDebugEnabled()) { + LOG.debug("Found ROOT " + HRegionInfo.ROOT_REGIONINFO); + } + break; + } catch (IOException e) { + if (tries == numRetries - 1) { + // Don't bother sleeping. We've run out of retries. + if (e instanceof RemoteException) { + e = RemoteExceptionHandler.decodeRemoteException( + (RemoteException) e); + } + throw e; + } + + // Sleep and retry finding root region. + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Root region location changed. Sleeping."); + } + Thread.sleep(getPauseTime(tries)); + if (LOG.isDebugEnabled()) { + LOG.debug("Wake. Retry finding root region."); + } + } catch (InterruptedException iex) { + // continue + } + } + + rootRegionAddress = null; + } + + // if the address is null by this point, then the retries have failed, + // and we're sort of sunk + if (rootRegionAddress == null) { + throw new NoServerForRegionException( + "unable to locate root region server"); + } + + // return the region location + return new HRegionLocation( + HRegionInfo.ROOT_REGIONINFO, rootRegionAddress); + } + + /** {@inheritDoc} */ + public T getRegionServerWithRetries(ServerCallable callable) + throws IOException, RuntimeException { + getMaster(); + List exceptions = new ArrayList(); + for(int tries = 0; tries < numRetries; tries++) { + try { + callable.instantiateServer(tries != 0); + return callable.call(); + } catch (Throwable t) { + if (t instanceof UndeclaredThrowableException) { + t = t.getCause(); + } + if (t instanceof RemoteException) { + t = RemoteExceptionHandler.decodeRemoteException((RemoteException) t); + } + if (t instanceof DoNotRetryIOException) { + throw (DoNotRetryIOException)t; + } + exceptions.add(t); + if (tries == numRetries - 1) { + throw new RetriesExhaustedException(callable.getServerName(), + callable.getRegionName(), callable.getRow(), tries, exceptions); + } + if (LOG.isDebugEnabled()) { + LOG.debug("reloading table servers because: " + t.getMessage()); + } + } + try { + Thread.sleep(getPauseTime(tries)); + } catch (InterruptedException e) { + // continue + } + } + return null; + } + } +} Index: src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java =================================================================== --- src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (revision 684990) +++ src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (working copy) @@ -1,634 +1,634 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client; - -import java.io.IOException; -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.TableExistsException; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.ipc.HMasterInterface; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.ipc.RemoteException; - -/** - * Provides administrative functions for HBase - */ -public class HBaseAdmin { - private final Log LOG = LogFactory.getLog(this.getClass().getName()); - private final HConnection connection; - private final long pause; - private final int numRetries; - private volatile HMasterInterface master; - - /** - * Constructor - * - * @param conf Configuration object - * @throws MasterNotRunningException - */ - public HBaseAdmin(HBaseConfiguration conf) throws MasterNotRunningException { - this.connection = HConnectionManager.getConnection(conf); - this.pause = conf.getLong("hbase.client.pause", 30 * 1000); - this.numRetries = conf.getInt("hbase.client.retries.number", 5); - this.master = connection.getMaster(); - } - - /** - * @return proxy connection to master server for this instance - * @throws MasterNotRunningException - */ - public HMasterInterface getMaster() throws MasterNotRunningException{ - return this.connection.getMaster(); - } - - /** @return - true if the master server is running */ - public boolean isMasterRunning() { - return this.connection.isMasterRunning(); - } - - /** - * @param tableName Table to check. - * @return True if table exists already. - * @throws MasterNotRunningException - */ - public boolean tableExists(final Text tableName) - throws MasterNotRunningException { - return tableExists(tableName.getBytes()); - } - - /** - * @param tableName Table to check. - * @return True if table exists already. - * @throws MasterNotRunningException - */ - public boolean tableExists(final String tableName) - throws MasterNotRunningException { - return tableExists(Bytes.toBytes(tableName)); - } - - /** - * @param tableName Table to check. - * @return True if table exists already. - * @throws MasterNotRunningException - */ - public boolean tableExists(final byte [] tableName) - throws MasterNotRunningException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - return connection.tableExists(tableName); - } - - /** - * List all the userspace tables. In other words, scan the META table. - * - * If we wanted this to be really fast, we could implement a special - * catalog table that just contains table names and their descriptors. - * Right now, it only exists as part of the META table's region info. - * - * @return - returns an array of HTableDescriptors - * @throws IOException - */ - public HTableDescriptor[] listTables() throws IOException { - return this.connection.listTables(); - } - - private long getPauseTime(int tries) { - if (tries >= HConstants.RETRY_BACKOFF.length) - tries = HConstants.RETRY_BACKOFF.length - 1; - return this.pause * HConstants.RETRY_BACKOFF[tries]; - } - - /** - * Creates a new table - * - * @param desc table descriptor for table - * - * @throws IllegalArgumentException if the table name is reserved - * @throws MasterNotRunningException if master is not running - * @throws TableExistsException if table already exists (If concurrent - * threads, the table may have been created between test-for-existence - * and attempt-at-creation). - * @throws IOException - */ - public void createTable(HTableDescriptor desc) - throws IOException { - HTableDescriptor.isLegalTableName(desc.getName()); - createTableAsync(desc); - for (int tries = 0; tries < numRetries; tries++) { - try { - // Wait for new table to come on-line - connection.locateRegion(desc.getName(), HConstants.EMPTY_START_ROW); - break; - - } catch (TableNotFoundException e) { - if (tries == numRetries - 1) { - // Ran out of tries - throw e; - } - } - try { - Thread.sleep(getPauseTime(tries)); - } catch (InterruptedException e) { - // continue - } - } - } - - /** - * Creates a new table but does not block and wait for it to come online. - * - * @param desc table descriptor for table - * - * @throws IllegalArgumentException Bad table name. - * @throws MasterNotRunningException if master is not running - * @throws TableExistsException if table already exists (If concurrent - * threads, the table may have been created between test-for-existence - * and attempt-at-creation). - * @throws IOException - */ - public void createTableAsync(HTableDescriptor desc) - throws IOException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - HTableDescriptor.isLegalTableName(desc.getName()); - try { - this.master.createTable(desc); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } - } - - /** - * Deletes a table - * - * @param tableName name of table to delete - * @throws IOException - */ - public void deleteTable(final Text tableName) throws IOException { - deleteTable(tableName.getBytes()); - } - - /** - * Deletes a table - * - * @param tableName name of table to delete - * @throws IOException - */ - public void deleteTable(final String tableName) throws IOException { - deleteTable(Bytes.toBytes(tableName)); - } - - /** - * Deletes a table - * - * @param tableName name of table to delete - * @throws IOException - */ - public void deleteTable(final byte [] tableName) throws IOException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - HTableDescriptor.isLegalTableName(tableName); - HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName); - try { - this.master.deleteTable(tableName); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } - - // Wait until first region is deleted - HRegionInterface server = - connection.getHRegionConnection(firstMetaServer.getServerAddress()); - HRegionInfo info = new HRegionInfo(); - for (int tries = 0; tries < numRetries; tries++) { - long scannerId = -1L; - try { - scannerId = - server.openScanner(firstMetaServer.getRegionInfo().getRegionName(), - HConstants.COL_REGIONINFO_ARRAY, tableName, - HConstants.LATEST_TIMESTAMP, null); - RowResult values = server.next(scannerId); - if (values == null || values.size() == 0) { - break; - } - boolean found = false; - for (Map.Entry e: values.entrySet()) { - if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) { - info = (HRegionInfo) Writables.getWritable( - e.getValue().getValue(), info); - - if (Bytes.equals(info.getTableDesc().getName(), tableName)) { - found = true; - } - } - } - if (!found) { - break; - } - - } catch (IOException ex) { - if(tries == numRetries - 1) { // no more tries left - if (ex instanceof RemoteException) { - ex = RemoteExceptionHandler.decodeRemoteException((RemoteException) ex); - } - throw ex; - } - - } finally { - if (scannerId != -1L) { - try { - server.close(scannerId); - } catch (Exception ex) { - LOG.warn(ex); - } - } - } - - try { - Thread.sleep(getPauseTime(tries)); - } catch (InterruptedException e) { - // continue - } - } - LOG.info("Deleted " + Bytes.toString(tableName)); - } - - /** - * Brings a table on-line (enables it) - * - * @param tableName name of the table - * @throws IOException - */ - public void enableTable(final Text tableName) throws IOException { - enableTable(tableName.getBytes()); - } - - /** - * Brings a table on-line (enables it) - * - * @param tableName name of the table - * @throws IOException - */ - public void enableTable(final String tableName) throws IOException { - enableTable(Bytes.toBytes(tableName)); - } - - /** - * Brings a table on-line (enables it) - * - * @param tableName name of the table - * @throws IOException - */ - public void enableTable(final byte [] tableName) throws IOException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - try { - this.master.enableTable(tableName); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } - - // Wait until all regions are enabled - - for (int tries = 0; - (tries < numRetries) && (!isTableEnabled(tableName)); - tries++) { - if (LOG.isDebugEnabled()) { - LOG.debug("Sleep. Waiting for all regions to be enabled from " + - Bytes.toString(tableName)); - } - try { - Thread.sleep(getPauseTime(tries)); - } catch (InterruptedException e) { - // continue - } - if (LOG.isDebugEnabled()) { - LOG.debug("Wake. Waiting for all regions to be enabled from " + - Bytes.toString(tableName)); - } - } - if (!isTableEnabled(tableName)) - throw new IOException("unable to enable table " + - Bytes.toString(tableName)); - LOG.info("Enabled table " + Bytes.toString(tableName)); - } - - /** - * Disables a table (takes it off-line) If it is being served, the master - * will tell the servers to stop serving it. - * - * @param tableName name of table - * @throws IOException - */ - public void disableTable(final Text tableName) throws IOException { - disableTable(tableName.getBytes()); - } - - /** - * Disables a table (takes it off-line) If it is being served, the master - * will tell the servers to stop serving it. - * - * @param tableName name of table - * @throws IOException - */ - public void disableTable(final String tableName) throws IOException { - disableTable(Bytes.toBytes(tableName)); - } - - /** - * Disables a table (takes it off-line) If it is being served, the master - * will tell the servers to stop serving it. - * - * @param tableName name of table - * @throws IOException - */ - public void disableTable(final byte [] tableName) throws IOException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - try { - this.master.disableTable(tableName); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } - - // Wait until all regions are disabled - for (int tries = 0; - (tries < numRetries) && (isTableEnabled(tableName)); - tries++) { - if (LOG.isDebugEnabled()) { - LOG.debug("Sleep. Waiting for all regions to be disabled from " + - Bytes.toString(tableName)); - } - try { - Thread.sleep(getPauseTime(tries)); - } catch (InterruptedException e) { - // continue - } - if (LOG.isDebugEnabled()) { - LOG.debug("Wake. Waiting for all regions to be disabled from " + - Bytes.toString(tableName)); - } - } - if (isTableEnabled(tableName)) - throw new IOException("unable to disable table " + - Bytes.toString(tableName)); - LOG.info("Disabled " + Bytes.toString(tableName)); - } - - /** - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException - */ - public boolean isTableEnabled(Text tableName) throws IOException { - return isTableEnabled(tableName.getBytes()); - } - /** - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException - */ - public boolean isTableEnabled(String tableName) throws IOException { - return isTableEnabled(Bytes.toBytes(tableName)); - } - /** - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException - */ - public boolean isTableEnabled(byte[] tableName) throws IOException { - return connection.isTableEnabled(tableName); - } - - /** - * Add a column to an existing table - * - * @param tableName name of the table to add column to - * @param column column descriptor of column to be added - * @throws IOException - */ - public void addColumn(final Text tableName, HColumnDescriptor column) - throws IOException { - addColumn(tableName.getBytes(), column); - } - - /** - * Add a column to an existing table - * - * @param tableName name of the table to add column to - * @param column column descriptor of column to be added - * @throws IOException - */ - public void addColumn(final String tableName, HColumnDescriptor column) - throws IOException { - addColumn(Bytes.toBytes(tableName), column); - } - - /** - * Add a column to an existing table - * - * @param tableName name of the table to add column to - * @param column column descriptor of column to be added - * @throws IOException - */ - public void addColumn(final byte [] tableName, HColumnDescriptor column) - throws IOException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - HTableDescriptor.isLegalTableName(tableName); - try { - this.master.addColumn(tableName, column); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } - } - - /** - * Delete a column from a table - * - * @param tableName name of table - * @param columnName name of column to be deleted - * @throws IOException - */ - public void deleteColumn(final Text tableName, final Text columnName) - throws IOException { - deleteColumn(tableName.getBytes(), columnName.getBytes()); - } - - /** - * Delete a column from a table - * - * @param tableName name of table - * @param columnName name of column to be deleted - * @throws IOException - */ - public void deleteColumn(final String tableName, final String columnName) - throws IOException { - deleteColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName)); - } - - /** - * Delete a column from a table - * - * @param tableName name of table - * @param columnName name of column to be deleted - * @throws IOException - */ - public void deleteColumn(final byte [] tableName, final byte [] columnName) - throws IOException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - HTableDescriptor.isLegalTableName(tableName); - try { - this.master.deleteColumn(tableName, columnName); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } - } - - /** - * Modify an existing column family on a table - * - * @param tableName name of table - * @param columnName name of column to be modified - * @param descriptor new column descriptor to use - * @throws IOException - */ - public void modifyColumn(final Text tableName, final Text columnName, - HColumnDescriptor descriptor) - throws IOException { - modifyColumn(tableName.getBytes(), columnName.getBytes(), descriptor); - } - - /** - * Modify an existing column family on a table - * - * @param tableName name of table - * @param columnName name of column to be modified - * @param descriptor new column descriptor to use - * @throws IOException - */ - public void modifyColumn(final String tableName, final String columnName, - HColumnDescriptor descriptor) - throws IOException { - modifyColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName), - descriptor); - } - - /** - * Modify an existing column family on a table - * - * @param tableName name of table - * @param columnName name of column to be modified - * @param descriptor new column descriptor to use - * @throws IOException - */ - public void modifyColumn(final byte [] tableName, final byte [] columnName, - HColumnDescriptor descriptor) - throws IOException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - HTableDescriptor.isLegalTableName(tableName); - try { - this.master.modifyColumn(tableName, columnName, descriptor); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } - } - - /** - * Modify a table's HTableDescriptor - * - * @param tableName name of table - * @param desc the updated descriptor - * @throws IOException - */ - public void modifyTableMeta(final byte [] tableName, HTableDescriptor desc) - throws IOException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - HTableDescriptor.isLegalTableName(tableName); - try { - this.master.modifyTableMeta(tableName, desc); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } - } - - /** - * Shuts down the HBase instance - * @throws IOException - */ - public synchronized void shutdown() throws IOException { - if (this.master == null) { - throw new MasterNotRunningException("master has been shut down"); - } - try { - this.master.shutdown(); - } catch (RemoteException e) { - throw RemoteExceptionHandler.decodeRemoteException(e); - } finally { - this.master = null; - } - } - - private HRegionLocation getFirstMetaServerForTable(final byte [] tableName) - throws IOException { - return connection.locateRegion(HConstants.META_TABLE_NAME, - HRegionInfo.createRegionName(tableName, null, HConstants.NINES)); - } - - /** - * Check to see if HBase is running. Throw an exception if not. - * - * @param conf - * @throws MasterNotRunningException - */ - public static void checkHBaseAvailable(HBaseConfiguration conf) - throws MasterNotRunningException { - HBaseConfiguration copyOfConf = new HBaseConfiguration(conf); - copyOfConf.setInt("hbase.client.retries.number", 1); - new HBaseAdmin(copyOfConf); - } +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import java.io.IOException; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.ipc.HMasterInterface; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.ipc.RemoteException; + +/** + * Provides administrative functions for HBase + */ +public class HBaseAdmin { + private final Log LOG = LogFactory.getLog(this.getClass().getName()); + private final HConnection connection; + private final long pause; + private final int numRetries; + private volatile HMasterInterface master; + + /** + * Constructor + * + * @param conf Configuration object + * @throws MasterNotRunningException + */ + public HBaseAdmin(HBaseConfiguration conf) throws MasterNotRunningException { + this.connection = HConnectionManager.getConnection(conf); + this.pause = conf.getLong("hbase.client.pause", 30 * 1000); + this.numRetries = conf.getInt("hbase.client.retries.number", 5); + this.master = connection.getMaster(); + } + + /** + * @return proxy connection to master server for this instance + * @throws MasterNotRunningException + */ + public HMasterInterface getMaster() throws MasterNotRunningException{ + return this.connection.getMaster(); + } + + /** @return - true if the master server is running */ + public boolean isMasterRunning() { + return this.connection.isMasterRunning(); + } + + /** + * @param tableName Table to check. + * @return True if table exists already. + * @throws MasterNotRunningException + */ + public boolean tableExists(final Text tableName) + throws MasterNotRunningException { + return tableExists(tableName.getBytes()); + } + + /** + * @param tableName Table to check. + * @return True if table exists already. + * @throws MasterNotRunningException + */ + public boolean tableExists(final String tableName) + throws MasterNotRunningException { + return tableExists(Bytes.toBytes(tableName)); + } + + /** + * @param tableName Table to check. + * @return True if table exists already. + * @throws MasterNotRunningException + */ + public boolean tableExists(final byte [] tableName) + throws MasterNotRunningException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + return connection.tableExists(tableName); + } + + /** + * List all the userspace tables. In other words, scan the META table. + * + * If we wanted this to be really fast, we could implement a special + * catalog table that just contains table names and their descriptors. + * Right now, it only exists as part of the META table's region info. + * + * @return - returns an array of HTableDescriptors + * @throws IOException + */ + public HTableDescriptor[] listTables() throws IOException { + return this.connection.listTables(); + } + + private long getPauseTime(int tries) { + if (tries >= HConstants.RETRY_BACKOFF.length) + tries = HConstants.RETRY_BACKOFF.length - 1; + return this.pause * HConstants.RETRY_BACKOFF[tries]; + } + + /** + * Creates a new table + * + * @param desc table descriptor for table + * + * @throws IllegalArgumentException if the table name is reserved + * @throws MasterNotRunningException if master is not running + * @throws TableExistsException if table already exists (If concurrent + * threads, the table may have been created between test-for-existence + * and attempt-at-creation). + * @throws IOException + */ + public void createTable(HTableDescriptor desc) + throws IOException { + HTableDescriptor.isLegalTableName(desc.getName()); + createTableAsync(desc); + for (int tries = 0; tries < numRetries; tries++) { + try { + // Wait for new table to come on-line + connection.locateRegion(desc.getName(), HConstants.EMPTY_START_ROW); + break; + + } catch (TableNotFoundException e) { + if (tries == numRetries - 1) { + // Ran out of tries + throw e; + } + } + try { + Thread.sleep(getPauseTime(tries)); + } catch (InterruptedException e) { + // continue + } + } + } + + /** + * Creates a new table but does not block and wait for it to come online. + * + * @param desc table descriptor for table + * + * @throws IllegalArgumentException Bad table name. + * @throws MasterNotRunningException if master is not running + * @throws TableExistsException if table already exists (If concurrent + * threads, the table may have been created between test-for-existence + * and attempt-at-creation). + * @throws IOException + */ + public void createTableAsync(HTableDescriptor desc) + throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + HTableDescriptor.isLegalTableName(desc.getName()); + try { + this.master.createTable(desc); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + } + + /** + * Deletes a table + * + * @param tableName name of table to delete + * @throws IOException + */ + public void deleteTable(final Text tableName) throws IOException { + deleteTable(tableName.getBytes()); + } + + /** + * Deletes a table + * + * @param tableName name of table to delete + * @throws IOException + */ + public void deleteTable(final String tableName) throws IOException { + deleteTable(Bytes.toBytes(tableName)); + } + + /** + * Deletes a table + * + * @param tableName name of table to delete + * @throws IOException + */ + public void deleteTable(final byte [] tableName) throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + HTableDescriptor.isLegalTableName(tableName); + HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName); + try { + this.master.deleteTable(tableName); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + + // Wait until first region is deleted + HRegionInterface server = + connection.getHRegionConnection(firstMetaServer.getServerAddress()); + HRegionInfo info = new HRegionInfo(); + for (int tries = 0; tries < numRetries; tries++) { + long scannerId = -1L; + try { + scannerId = + server.openScanner(firstMetaServer.getRegionInfo().getRegionName(), + HConstants.COL_REGIONINFO_ARRAY, tableName, + HConstants.LATEST_TIMESTAMP, null); + RowResult values = server.next(scannerId); + if (values == null || values.size() == 0) { + break; + } + boolean found = false; + for (Map.Entry e: values.entrySet()) { + if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) { + info = (HRegionInfo) Writables.getWritable( + e.getValue().getValue(), info); + + if (Bytes.equals(info.getTableDesc().getName(), tableName)) { + found = true; + } + } + } + if (!found) { + break; + } + + } catch (IOException ex) { + if(tries == numRetries - 1) { // no more tries left + if (ex instanceof RemoteException) { + ex = RemoteExceptionHandler.decodeRemoteException((RemoteException) ex); + } + throw ex; + } + + } finally { + if (scannerId != -1L) { + try { + server.close(scannerId); + } catch (Exception ex) { + LOG.warn(ex); + } + } + } + + try { + Thread.sleep(getPauseTime(tries)); + } catch (InterruptedException e) { + // continue + } + } + LOG.info("Deleted " + Bytes.toString(tableName)); + } + + /** + * Brings a table on-line (enables it) + * + * @param tableName name of the table + * @throws IOException + */ + public void enableTable(final Text tableName) throws IOException { + enableTable(tableName.getBytes()); + } + + /** + * Brings a table on-line (enables it) + * + * @param tableName name of the table + * @throws IOException + */ + public void enableTable(final String tableName) throws IOException { + enableTable(Bytes.toBytes(tableName)); + } + + /** + * Brings a table on-line (enables it) + * + * @param tableName name of the table + * @throws IOException + */ + public void enableTable(final byte [] tableName) throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + try { + this.master.enableTable(tableName); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + + // Wait until all regions are enabled + + for (int tries = 0; + (tries < numRetries) && (!isTableEnabled(tableName)); + tries++) { + if (LOG.isDebugEnabled()) { + LOG.debug("Sleep. Waiting for all regions to be enabled from " + + Bytes.toString(tableName)); + } + try { + Thread.sleep(getPauseTime(tries)); + } catch (InterruptedException e) { + // continue + } + if (LOG.isDebugEnabled()) { + LOG.debug("Wake. Waiting for all regions to be enabled from " + + Bytes.toString(tableName)); + } + } + if (!isTableEnabled(tableName)) + throw new IOException("unable to enable table " + + Bytes.toString(tableName)); + LOG.info("Enabled table " + Bytes.toString(tableName)); + } + + /** + * Disables a table (takes it off-line) If it is being served, the master + * will tell the servers to stop serving it. + * + * @param tableName name of table + * @throws IOException + */ + public void disableTable(final Text tableName) throws IOException { + disableTable(tableName.getBytes()); + } + + /** + * Disables a table (takes it off-line) If it is being served, the master + * will tell the servers to stop serving it. + * + * @param tableName name of table + * @throws IOException + */ + public void disableTable(final String tableName) throws IOException { + disableTable(Bytes.toBytes(tableName)); + } + + /** + * Disables a table (takes it off-line) If it is being served, the master + * will tell the servers to stop serving it. + * + * @param tableName name of table + * @throws IOException + */ + public void disableTable(final byte [] tableName) throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + try { + this.master.disableTable(tableName); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + + // Wait until all regions are disabled + for (int tries = 0; + (tries < numRetries) && (isTableEnabled(tableName)); + tries++) { + if (LOG.isDebugEnabled()) { + LOG.debug("Sleep. Waiting for all regions to be disabled from " + + Bytes.toString(tableName)); + } + try { + Thread.sleep(getPauseTime(tries)); + } catch (InterruptedException e) { + // continue + } + if (LOG.isDebugEnabled()) { + LOG.debug("Wake. Waiting for all regions to be disabled from " + + Bytes.toString(tableName)); + } + } + if (isTableEnabled(tableName)) + throw new IOException("unable to disable table " + + Bytes.toString(tableName)); + LOG.info("Disabled " + Bytes.toString(tableName)); + } + + /** + * @param tableName name of table to check + * @return true if table is on-line + * @throws IOException + */ + public boolean isTableEnabled(Text tableName) throws IOException { + return isTableEnabled(tableName.getBytes()); + } + /** + * @param tableName name of table to check + * @return true if table is on-line + * @throws IOException + */ + public boolean isTableEnabled(String tableName) throws IOException { + return isTableEnabled(Bytes.toBytes(tableName)); + } + /** + * @param tableName name of table to check + * @return true if table is on-line + * @throws IOException + */ + public boolean isTableEnabled(byte[] tableName) throws IOException { + return connection.isTableEnabled(tableName); + } + + /** + * Add a column to an existing table + * + * @param tableName name of the table to add column to + * @param column column descriptor of column to be added + * @throws IOException + */ + public void addColumn(final Text tableName, HColumnDescriptor column) + throws IOException { + addColumn(tableName.getBytes(), column); + } + + /** + * Add a column to an existing table + * + * @param tableName name of the table to add column to + * @param column column descriptor of column to be added + * @throws IOException + */ + public void addColumn(final String tableName, HColumnDescriptor column) + throws IOException { + addColumn(Bytes.toBytes(tableName), column); + } + + /** + * Add a column to an existing table + * + * @param tableName name of the table to add column to + * @param column column descriptor of column to be added + * @throws IOException + */ + public void addColumn(final byte [] tableName, HColumnDescriptor column) + throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + HTableDescriptor.isLegalTableName(tableName); + try { + this.master.addColumn(tableName, column); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + } + + /** + * Delete a column from a table + * + * @param tableName name of table + * @param columnName name of column to be deleted + * @throws IOException + */ + public void deleteColumn(final Text tableName, final Text columnName) + throws IOException { + deleteColumn(tableName.getBytes(), columnName.getBytes()); + } + + /** + * Delete a column from a table + * + * @param tableName name of table + * @param columnName name of column to be deleted + * @throws IOException + */ + public void deleteColumn(final String tableName, final String columnName) + throws IOException { + deleteColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName)); + } + + /** + * Delete a column from a table + * + * @param tableName name of table + * @param columnName name of column to be deleted + * @throws IOException + */ + public void deleteColumn(final byte [] tableName, final byte [] columnName) + throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + HTableDescriptor.isLegalTableName(tableName); + try { + this.master.deleteColumn(tableName, columnName); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + } + + /** + * Modify an existing column family on a table + * + * @param tableName name of table + * @param columnName name of column to be modified + * @param descriptor new column descriptor to use + * @throws IOException + */ + public void modifyColumn(final Text tableName, final Text columnName, + HColumnDescriptor descriptor) + throws IOException { + modifyColumn(tableName.getBytes(), columnName.getBytes(), descriptor); + } + + /** + * Modify an existing column family on a table + * + * @param tableName name of table + * @param columnName name of column to be modified + * @param descriptor new column descriptor to use + * @throws IOException + */ + public void modifyColumn(final String tableName, final String columnName, + HColumnDescriptor descriptor) + throws IOException { + modifyColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName), + descriptor); + } + + /** + * Modify an existing column family on a table + * + * @param tableName name of table + * @param columnName name of column to be modified + * @param descriptor new column descriptor to use + * @throws IOException + */ + public void modifyColumn(final byte [] tableName, final byte [] columnName, + HColumnDescriptor descriptor) + throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + HTableDescriptor.isLegalTableName(tableName); + try { + this.master.modifyColumn(tableName, columnName, descriptor); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + } + + /** + * Modify a table's HTableDescriptor + * + * @param tableName name of table + * @param desc the updated descriptor + * @throws IOException + */ + public void modifyTableMeta(final byte [] tableName, HTableDescriptor desc) + throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + HTableDescriptor.isLegalTableName(tableName); + try { + this.master.modifyTableMeta(tableName, desc); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + } + + /** + * Shuts down the HBase instance + * @throws IOException + */ + public synchronized void shutdown() throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + try { + this.master.shutdown(); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } finally { + this.master = null; + } + } + + private HRegionLocation getFirstMetaServerForTable(final byte [] tableName) + throws IOException { + return connection.locateRegion(HConstants.META_TABLE_NAME, + HRegionInfo.createRegionName(tableName, null, HConstants.NINES)); + } + + /** + * Check to see if HBase is running. Throw an exception if not. + * + * @param conf + * @throws MasterNotRunningException + */ + public static void checkHBaseAvailable(HBaseConfiguration conf) + throws MasterNotRunningException { + HBaseConfiguration copyOfConf = new HBaseConfiguration(conf); + copyOfConf.setInt("hbase.client.retries.number", 1); + new HBaseAdmin(copyOfConf); + } } \ No newline at end of file