HBASE-12586 Task 6 & 7 from HBASE-9117, delete all public HTable constructors and delete ConnectionManager#{delete,get}Connection (Mikhail Antonov)

This commit is contained in:
stack 2015-03-16 11:20:14 -07:00
parent 76ba53004b
commit c5aca1919d
20 changed files with 59 additions and 355 deletions

View File

@ -20,7 +20,7 @@ import java.util.concurrent.ExecutorService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Table;
/**
* Coprocessor environment state.
@ -50,14 +50,14 @@ public interface CoprocessorEnvironment {
* @return an interface for accessing the given table
* @throws IOException
*/
HTableInterface getTable(TableName tableName) throws IOException;
Table getTable(TableName tableName) throws IOException;
/**
* @return an interface for accessing the given table using the passed executor to run batch
* operations
* @throws IOException
*/
HTableInterface getTable(TableName tableName, ExecutorService service) throws IOException;
Table getTable(TableName tableName, ExecutorService service) throws IOException;
/**
* @return the classloader for the loaded coprocessor instance

View File

@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.NeedUnmanagedConnectionException;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@ -263,7 +263,7 @@ public class MetaTableAccessor {
// There should still be a way to use this method with an unmanaged connection.
if (connection instanceof ClusterConnection) {
if (((ClusterConnection) connection).isManaged()) {
return new HTable(TableName.META_TABLE_NAME, connection);
throw new NeedUnmanagedConnectionException();
}
}
return connection.getTable(TableName.META_TABLE_NAME);

View File

@ -286,11 +286,11 @@ final class ConnectionManager {
synchronized (CONNECTION_INSTANCES) {
HConnectionImplementation connection = CONNECTION_INSTANCES.get(connectionKey);
if (connection == null) {
connection = (HConnectionImplementation)createConnection(conf, true);
connection = (HConnectionImplementation) ConnectionFactory.createConnection(conf);
CONNECTION_INSTANCES.put(connectionKey, connection);
} else if (connection.isClosed()) {
ConnectionManager.deleteConnection(connectionKey, true);
connection = (HConnectionImplementation)createConnection(conf, true);
connection = (HConnectionImplementation) ConnectionFactory.createConnection(conf);
CONNECTION_INSTANCES.put(connectionKey, connection);
}
connection.incCount();

View File

@ -128,78 +128,6 @@ public class HTable implements HTableInterface {
private RpcRetryingCallerFactory rpcCallerFactory;
private RpcControllerFactory rpcControllerFactory;
/**
* Creates an object to access a HBase table.
* @param conf Configuration object to use.
* @param tableName Name of the table.
* @throws IOException if a remote or network exception occurs
* @deprecated Constructing HTable objects manually has been deprecated. Please use
* {@link Connection} to instantiate a {@link Table} instead.
*/
@Deprecated
public HTable(Configuration conf, final String tableName)
throws IOException {
this(conf, TableName.valueOf(tableName));
}
/**
* Creates an object to access a HBase table.
* @param conf Configuration object to use.
* @param tableName Name of the table.
* @throws IOException if a remote or network exception occurs
* @deprecated Constructing HTable objects manually has been deprecated. Please use
* {@link Connection} to instantiate a {@link Table} instead.
*/
@Deprecated
public HTable(Configuration conf, final byte[] tableName)
throws IOException {
this(conf, TableName.valueOf(tableName));
}
/**
* Creates an object to access a HBase table.
* @param conf Configuration object to use.
* @param tableName table name pojo
* @throws IOException if a remote or network exception occurs
* @deprecated Constructing HTable objects manually has been deprecated. Please use
* {@link Connection} to instantiate a {@link Table} instead.
*/
@Deprecated
public HTable(Configuration conf, final TableName tableName)
throws IOException {
this.tableName = tableName;
this.cleanupPoolOnClose = true;
this.cleanupConnectionOnClose = true;
if (conf == null) {
this.connection = null;
return;
}
this.connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
this.configuration = conf;
this.pool = getDefaultExecutor(conf);
this.finishSetup();
}
/**
* Creates an object to access a HBase table.
* @param tableName Name of the table.
* @param connection HConnection to be used.
* @throws IOException if a remote or network exception occurs
* @deprecated Do not use.
*/
@Deprecated
public HTable(TableName tableName, Connection connection) throws IOException {
this.tableName = tableName;
this.cleanupPoolOnClose = true;
this.cleanupConnectionOnClose = false;
this.connection = (ClusterConnection)connection;
this.configuration = connection.getConfiguration();
this.pool = getDefaultExecutor(this.configuration);
this.finishSetup();
}
// Marked Private @since 1.0
@InterfaceAudience.Private
public static ThreadPoolExecutor getDefaultExecutor(Configuration conf) {
@ -219,68 +147,6 @@ public class HTable implements HTableInterface {
return pool;
}
/**
* Creates an object to access a HBase table.
* @param conf Configuration object to use.
* @param tableName Name of the table.
* @param pool ExecutorService to be used.
* @throws IOException if a remote or network exception occurs
* @deprecated Constructing HTable objects manually has been deprecated. Please use
* {@link Connection} to instantiate a {@link Table} instead.
*/
@Deprecated
public HTable(Configuration conf, final byte[] tableName, final ExecutorService pool)
throws IOException {
this(conf, TableName.valueOf(tableName), pool);
}
/**
* Creates an object to access a HBase table.
* @param conf Configuration object to use.
* @param tableName Name of the table.
* @param pool ExecutorService to be used.
* @throws IOException if a remote or network exception occurs
* @deprecated Constructing HTable objects manually has been deprecated. Please use
* {@link Connection} to instantiate a {@link Table} instead.
*/
@Deprecated
public HTable(Configuration conf, final TableName tableName, final ExecutorService pool)
throws IOException {
this.connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
this.configuration = conf;
this.pool = pool;
if (pool == null) {
this.pool = getDefaultExecutor(conf);
this.cleanupPoolOnClose = true;
} else {
this.cleanupPoolOnClose = false;
}
this.tableName = tableName;
this.cleanupConnectionOnClose = true;
this.finishSetup();
}
/**
* Creates an object to access a HBase table.
* @param tableName Name of the table.
* @param connection HConnection to be used.
* @param pool ExecutorService to be used.
* @throws IOException if a remote or network exception occurs.
* @deprecated Do not use, internal ctor.
*/
@Deprecated
public HTable(final byte[] tableName, final Connection connection,
final ExecutorService pool) throws IOException {
this(TableName.valueOf(tableName), connection, pool);
}
/** @deprecated Do not use, internal ctor. */
@Deprecated
public HTable(TableName tableName, final Connection connection,
final ExecutorService pool) throws IOException {
this(tableName, (ClusterConnection)connection, null, null, null, pool);
}
/**
* Creates an object to access a HBase table.
* Used by HBase internally. DO NOT USE. See {@link ConnectionFactory} class comment for how to
@ -291,7 +157,7 @@ public class HTable implements HTableInterface {
* @throws IOException if a remote or network exception occurs
*/
@InterfaceAudience.Private
public HTable(TableName tableName, final ClusterConnection connection,
protected HTable(TableName tableName, final ClusterConnection connection,
final TableConfiguration tableConfig,
final RpcRetryingCallerFactory rpcCallerFactory,
final RpcControllerFactory rpcControllerFactory,

View File

@ -1,51 +0,0 @@
/**
*
* 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 org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Factory for creating HTable instances.
*
* @deprecated as of 0.98.1. See {@link HConnectionManager#createConnection(Configuration)}.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@Deprecated
public class HTableFactory implements HTableInterfaceFactory {
@Override
public HTableInterface createHTableInterface(Configuration config,
byte[] tableName) {
try {
return new HTable(config, TableName.valueOf(tableName));
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
@Override
public void releaseHTableInterface(HTableInterface table) throws IOException {
table.close();
}
}

View File

@ -1,54 +0,0 @@
/**
*
* 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 org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Defines methods to create new HTableInterface.
*
* @since 0.21.0
* @deprecated in favor of {@link ConnectionFactory} and {@link Connection}.
*/
@Deprecated
@InterfaceAudience.Public
@InterfaceStability.Stable
public interface HTableInterfaceFactory {
/**
* Creates a new HTableInterface.
*
* @param config HBaseConfiguration instance.
* @param tableName name of the HBase table.
* @return HTableInterface instance.
*/
HTableInterface createHTableInterface(Configuration config, byte[] tableName);
/**
* Release the HTable resource represented by the table.
* @param table
*/
void releaseHTableInterface(final HTableInterface table) throws IOException;
}

View File

@ -58,23 +58,23 @@ import com.google.protobuf.ServiceException;
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Stable
public final class HTableWrapper implements HTableInterface {
public final class HTableWrapper implements Table {
private final HTableInterface table;
private final Table table;
private ClusterConnection connection;
private final List<HTableInterface> openTables;
private final List<Table> openTables;
/**
* @param openTables External list of tables used for tracking wrappers.
* @throws IOException
*/
public static HTableInterface createWrapper(List<HTableInterface> openTables,
public static Table createWrapper(List<Table> openTables,
TableName tableName, Environment env, ExecutorService pool) throws IOException {
return new HTableWrapper(openTables, tableName,
CoprocessorHConnection.getConnectionForEnvironment(env), pool);
}
private HTableWrapper(List<HTableInterface> openTables, TableName tableName,
private HTableWrapper(List<Table> openTables, TableName tableName,
ClusterConnection connection, ExecutorService pool)
throws IOException {
this.table = connection.getTable(tableName, pool);
@ -116,8 +116,7 @@ public final class HTableWrapper implements HTableInterface {
}
/**
* @deprecated in 0.99 since setting clearBufferOnFail is deprecated. Use
* {@link #setAutoFlushTo(boolean)}} instead.
* @deprecated in 0.99 since setting clearBufferOnFail is deprecated.
*/
@Deprecated
public Result getRowOrBefore(byte[] row, byte[] family)
@ -215,14 +214,6 @@ public final class HTableWrapper implements HTableInterface {
return table.increment(increment);
}
public void flushCommits() throws IOException {
table.flushCommits();
}
public boolean isAutoFlush() {
return table.isAutoFlush();
}
public ResultScanner getScanner(Scan scan) throws IOException {
return table.getScanner(scan);
}
@ -240,11 +231,6 @@ public final class HTableWrapper implements HTableInterface {
return table.getTableDescriptor();
}
@Override
public byte[] getTableName() {
return table.getTableName();
}
@Override
public TableName getName() {
return table.getName();
@ -316,30 +302,6 @@ public final class HTableWrapper implements HTableInterface {
table.mutateRow(rm);
}
@Override
public void setAutoFlush(boolean autoFlush) {
table.setAutoFlush(autoFlush);
}
@Override
public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
setAutoFlush(autoFlush);
if (!autoFlush && !clearBufferOnFail) {
// We don't support his combination. In HTable, the implementation is this:
//
// this.clearBufferOnFail = autoFlush || clearBufferOnFail
//
// So if autoFlush == false and clearBufferOnFail is false, that is not supported in
// the new Table Interface so just throwing UnsupportedOperationException here.
throw new UnsupportedOperationException("Can't do this via wrapper");
}
}
@Override
public void setAutoFlushTo(boolean autoFlush) {
table.setAutoFlushTo(autoFlush);
}
@Override
public long getWriteBufferSize() {
return table.getWriteBufferSize();
@ -350,13 +312,6 @@ public final class HTableWrapper implements HTableInterface {
table.setWriteBufferSize(writeBufferSize);
}
@Override
public long incrementColumnValue(byte[] row, byte[] family,
byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
return table.incrementColumnValue(row, family, qualifier, amount,
writeToWAL? Durability.USE_DEFAULT: Durability.SKIP_WAL);
}
@Override
public <R extends Message> Map<byte[], R> batchCoprocessorService(
MethodDescriptor methodDescriptor, Message request, byte[] startKey, byte[] endKey,

View File

@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.HTableWrapper;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;
import org.apache.hadoop.hbase.util.VersionInfo;
@ -377,8 +377,8 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
/** Current coprocessor state */
Coprocessor.State state = Coprocessor.State.UNINSTALLED;
/** Accounting for tables opened by the coprocessor */
protected List<HTableInterface> openTables =
Collections.synchronizedList(new ArrayList<HTableInterface>());
protected List<Table> openTables =
Collections.synchronizedList(new ArrayList<Table>());
private int seq;
private Configuration conf;
private ClassLoader classLoader;
@ -438,7 +438,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
" because not active (state="+state.toString()+")");
}
// clean up any table references
for (HTableInterface table: openTables) {
for (Table table: openTables) {
try {
((HTableWrapper)table).internalClose();
} catch (IOException e) {
@ -493,7 +493,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
* @exception java.io.IOException Exception
*/
@Override
public HTableInterface getTable(TableName tableName) throws IOException {
public Table getTable(TableName tableName) throws IOException {
return this.getTable(tableName, HTable.getDefaultExecutor(getConfiguration()));
}
@ -504,7 +504,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
* @exception java.io.IOException Exception
*/
@Override
public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException {
public Table getTable(TableName tableName, ExecutorService pool) throws IOException {
return HTableWrapper.createWrapper(openTables, tableName, this, pool);
}
}

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter;
@ -83,7 +84,8 @@ public abstract class TableInputFormatBase
implements InputFormat<ImmutableBytesWritable, Result> {
private static final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
private byte [][] inputColumns;
private HTable table;
private Table table;
private RegionLocator regionLocator;
private Connection connection;
private TableRecordReader tableRecordReader;
private Filter rowFilter;
@ -197,7 +199,7 @@ implements InputFormat<ImmutableBytesWritable, Result> {
throw new IOException(INITIALIZATION_ERROR, exception);
}
byte [][] startKeys = this.table.getStartKeys();
byte [][] startKeys = this.regionLocator.getStartKeys();
if (startKeys == null || startKeys.length == 0) {
throw new IOException("Expecting at least one region");
}
@ -212,7 +214,7 @@ implements InputFormat<ImmutableBytesWritable, Result> {
for (int i = 0; i < realNumSplits; i++) {
int lastPos = startPos + middle;
lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos;
String regionLocation = table.getRegionLocation(startKeys[startPos]).
String regionLocation = regionLocator.getRegionLocation(startKeys[startPos]).
getHostname();
splits[i] = new TableSplit(this.table.getName(),
startKeys[startPos], ((i + 1) < realNumSplits) ? startKeys[lastPos]:
@ -235,7 +237,8 @@ implements InputFormat<ImmutableBytesWritable, Result> {
LOG.warn("initializeTable called multiple times. Overwriting connection and table " +
"reference; TableInputFormatBase will not close these old references when done.");
}
this.table = (HTable) connection.getTable(tableName);
this.table = connection.getTable(tableName);
this.regionLocator = connection.getRegionLocator(tableName);
this.connection = connection;
}

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.security.UserGroupInformation;
@ -146,7 +146,7 @@ public class ConnectionCache {
/**
* Caller closes the table afterwards.
*/
public HTableInterface getTable(String tableName) throws IOException {
public Table getTable(String tableName) throws IOException {
ConnectionInfo connInfo = getCurrentConnection();
return connInfo.connection.getTable(tableName);
}

View File

@ -1034,7 +1034,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
// Don't leave here till we've done a successful scan of the hbase:meta
Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
Connection conn = ConnectionFactory.createConnection(this.conf);
Table t = conn.getTable(TableName.META_TABLE_NAME);
ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) {
// do nothing
@ -1042,6 +1043,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
LOG.info("HBase has been restarted");
s.close();
t.close();
conn.close();
}
/**

View File

@ -161,7 +161,7 @@ public class HConnectionTestingUtility {
HConnectionImplementation connection =
ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
if (connection == null) {
connection = Mockito.spy(new HConnectionImplementation(conf, true));
connection = Mockito.spy(new HConnectionImplementation(conf, false));
ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
}
return connection;
@ -175,7 +175,7 @@ public class HConnectionTestingUtility {
HConnectionImplementation connection =
ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
if (connection == null) {
connection = Mockito.spy(new HConnectionImplementation(conf, true));
connection = Mockito.spy(new HConnectionImplementation(conf, false));
ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
}
return connection;

View File

@ -952,10 +952,8 @@ public class TestHCM {
c3.close();
// still a reference left
assertFalse(c3.isClosed());
c3.close();
assertTrue(c3.isClosed());
// c3 was removed from the cache
Connection c5 = HConnectionManager.getConnection(configuration);
assertTrue(c5 != c3);

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -86,7 +85,7 @@ public class TestHTableWrapper {
static class DummyRegionObserver extends BaseRegionObserver {
}
private HTableInterface hTableInterface;
private Table hTableInterface;
private Table table;
@BeforeClass
@ -144,10 +143,8 @@ public class TestHTableWrapper {
private void checkHTableInterfaceMethods() throws Exception {
checkConf();
checkNameAndDescriptor();
checkAutoFlush();
checkBufferSize();
checkExists();
checkGetRowOrBefore();
checkAppend();
checkPutsAndDeletes();
checkCheckAndPut();
@ -159,7 +156,6 @@ public class TestHTableWrapper {
checkMutateRow();
checkResultScanner();
hTableInterface.flushCommits();
hTableInterface.close();
}
@ -174,15 +170,6 @@ public class TestHTableWrapper {
assertEquals(table.getTableDescriptor(), hTableInterface.getTableDescriptor());
}
private void checkAutoFlush() {
boolean initialAutoFlush = hTableInterface.isAutoFlush();
hTableInterface.setAutoFlush(false);
assertFalse(hTableInterface.isAutoFlush());
hTableInterface.setAutoFlush(true);
assertTrue(hTableInterface.isAutoFlush());
hTableInterface.setAutoFlush(initialAutoFlush);
}
private void checkBufferSize() throws IOException {
long initialWriteBufferSize = hTableInterface.getWriteBufferSize();
hTableInterface.setWriteBufferSize(12345L);
@ -194,19 +181,12 @@ public class TestHTableWrapper {
boolean ex = hTableInterface.exists(new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1));
assertTrue(ex);
Boolean[] exArray = hTableInterface.exists(Arrays.asList(new Get[] {
new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1),
new Get(ROW_B).addColumn(TEST_FAMILY, qualifierCol1),
new Get(ROW_C).addColumn(TEST_FAMILY, qualifierCol1),
new Get(Bytes.toBytes("does not exist")).addColumn(TEST_FAMILY, qualifierCol1), }));
assertArrayEquals(new Boolean[] { Boolean.TRUE, Boolean.TRUE, Boolean.TRUE, Boolean.FALSE },
exArray);
}
@SuppressWarnings("deprecation")
private void checkGetRowOrBefore() throws IOException {
Result rowOrBeforeResult = hTableInterface.getRowOrBefore(ROW_A, TEST_FAMILY);
assertArrayEquals(ROW_A, rowOrBeforeResult.getRow());
boolean[] exArray = hTableInterface.existsAll(Arrays.asList(new Get[]{
new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1),
new Get(ROW_B).addColumn(TEST_FAMILY, qualifierCol1),
new Get(ROW_C).addColumn(TEST_FAMILY, qualifierCol1),
new Get(Bytes.toBytes("does not exist")).addColumn(TEST_FAMILY, qualifierCol1),}));
assertTrue(Arrays.equals(new boolean[]{true, true, true, false}, exArray));
}
private void checkAppend() throws IOException {

View File

@ -403,10 +403,10 @@ public class TestTableInputFormat {
@Override
public void configure(JobConf job) {
try {
HTable exampleTable = new HTable(HBaseConfiguration.create(job),
Bytes.toBytes("exampleDeprecatedTable"));
Connection connection = ConnectionFactory.createConnection(job);
Table exampleTable = connection.getTable(TableName.valueOf("exampleDeprecatedTable"));
// mandatory
setHTable(exampleTable);
initializeTable(connection, exampleTable.getName());
byte[][] inputColumns = new byte [][] { Bytes.toBytes("columnA"),
Bytes.toBytes("columnB") };
// mandatory

View File

@ -412,10 +412,10 @@ public class TestTableInputFormat {
@Override
public void configure(JobConf job) {
try {
HTable exampleTable = new HTable(HBaseConfiguration.create(job),
Bytes.toBytes("exampleDeprecatedTable"));
Connection connection = ConnectionFactory.createConnection(job);
Table exampleTable = connection.getTable(TableName.valueOf(("exampleDeprecatedTable")));
// mandatory
setHTable(exampleTable);
initializeTable(connection, exampleTable.getName());
byte[][] inputColumns = new byte [][] { Bytes.toBytes("columnA"),
Bytes.toBytes("columnB") };
// optional

View File

@ -57,6 +57,8 @@ import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@ -958,7 +960,8 @@ public class TestSplitTransactionOnCluster {
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
admin.createTable(desc);
HTable hTable = new HTable(cluster.getConfiguration(), desc.getTableName());
Connection connection = ConnectionFactory.createConnection(cluster.getConfiguration());
HTable hTable = (HTable) connection.getTable(desc.getTableName());
for(int i = 1; i < 5; i++) {
Put p1 = new Put(("r"+i).getBytes());
p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());

View File

@ -37,11 +37,11 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
@ -350,7 +350,7 @@ public class TestMasterReplication {
int numClusters = utilities.length;
Table[] htables = new Table[numClusters];
for (int i = 0; i < numClusters; i++) {
Table htable = new HTable(configurations[i], tableName);
Table htable = ConnectionFactory.createConnection(configurations[i]).getTable(tableName);
htable.setWriteBufferSize(1024);
htables[i] = htable;
}

View File

@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
@ -195,13 +194,16 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
List<Cell> cells = null;
if (labelsRegion == null) {
Table table = null;
Connection connection = null;
try {
table = new HTable(conf, VisibilityConstants.LABELS_TABLE_NAME);
connection = ConnectionFactory.createConnection(conf);
table = connection.getTable(VisibilityConstants.LABELS_TABLE_NAME);
Result result = table.get(get);
cells = result.listCells();
} finally {
if (table != null) {
table.close();
connection.close();
}
}
} else {

View File

@ -1219,7 +1219,7 @@ public class TestHBaseFsck {
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
admin.createTable(desc);
tbl = new HTable(cluster.getConfiguration(), desc.getTableName());
tbl = (HTable) connection.getTable(desc.getTableName());
for (int i = 0; i < 5; i++) {
Put p1 = new Put(("r" + i).getBytes());
p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());