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.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Table;
/** /**
* Coprocessor environment state. * Coprocessor environment state.
@ -50,14 +50,14 @@ public interface CoprocessorEnvironment {
* @return an interface for accessing the given table * @return an interface for accessing the given table
* @throws IOException * @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 * @return an interface for accessing the given table using the passed executor to run batch
* operations * operations
* @throws IOException * @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 * @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.Consistency;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; 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.Mutation;
import org.apache.hadoop.hbase.client.NeedUnmanagedConnectionException;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; 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. // There should still be a way to use this method with an unmanaged connection.
if (connection instanceof ClusterConnection) { if (connection instanceof ClusterConnection) {
if (((ClusterConnection) connection).isManaged()) { if (((ClusterConnection) connection).isManaged()) {
return new HTable(TableName.META_TABLE_NAME, connection); throw new NeedUnmanagedConnectionException();
} }
} }
return connection.getTable(TableName.META_TABLE_NAME); return connection.getTable(TableName.META_TABLE_NAME);

View File

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

View File

@ -128,78 +128,6 @@ public class HTable implements HTableInterface {
private RpcRetryingCallerFactory rpcCallerFactory; private RpcRetryingCallerFactory rpcCallerFactory;
private RpcControllerFactory rpcControllerFactory; 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 // Marked Private @since 1.0
@InterfaceAudience.Private @InterfaceAudience.Private
public static ThreadPoolExecutor getDefaultExecutor(Configuration conf) { public static ThreadPoolExecutor getDefaultExecutor(Configuration conf) {
@ -219,68 +147,6 @@ public class HTable implements HTableInterface {
return pool; 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. * Creates an object to access a HBase table.
* Used by HBase internally. DO NOT USE. See {@link ConnectionFactory} class comment for how to * 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 * @throws IOException if a remote or network exception occurs
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public HTable(TableName tableName, final ClusterConnection connection, protected HTable(TableName tableName, final ClusterConnection connection,
final TableConfiguration tableConfig, final TableConfiguration tableConfig,
final RpcRetryingCallerFactory rpcCallerFactory, final RpcRetryingCallerFactory rpcCallerFactory,
final RpcControllerFactory rpcControllerFactory, 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) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Stable @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 ClusterConnection connection;
private final List<HTableInterface> openTables; private final List<Table> openTables;
/** /**
* @param openTables External list of tables used for tracking wrappers. * @param openTables External list of tables used for tracking wrappers.
* @throws IOException * @throws IOException
*/ */
public static HTableInterface createWrapper(List<HTableInterface> openTables, public static Table createWrapper(List<Table> openTables,
TableName tableName, Environment env, ExecutorService pool) throws IOException { TableName tableName, Environment env, ExecutorService pool) throws IOException {
return new HTableWrapper(openTables, tableName, return new HTableWrapper(openTables, tableName,
CoprocessorHConnection.getConnectionForEnvironment(env), pool); CoprocessorHConnection.getConnectionForEnvironment(env), pool);
} }
private HTableWrapper(List<HTableInterface> openTables, TableName tableName, private HTableWrapper(List<Table> openTables, TableName tableName,
ClusterConnection connection, ExecutorService pool) ClusterConnection connection, ExecutorService pool)
throws IOException { throws IOException {
this.table = connection.getTable(tableName, pool); 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 * @deprecated in 0.99 since setting clearBufferOnFail is deprecated.
* {@link #setAutoFlushTo(boolean)}} instead.
*/ */
@Deprecated @Deprecated
public Result getRowOrBefore(byte[] row, byte[] family) public Result getRowOrBefore(byte[] row, byte[] family)
@ -215,14 +214,6 @@ public final class HTableWrapper implements HTableInterface {
return table.increment(increment); return table.increment(increment);
} }
public void flushCommits() throws IOException {
table.flushCommits();
}
public boolean isAutoFlush() {
return table.isAutoFlush();
}
public ResultScanner getScanner(Scan scan) throws IOException { public ResultScanner getScanner(Scan scan) throws IOException {
return table.getScanner(scan); return table.getScanner(scan);
} }
@ -240,11 +231,6 @@ public final class HTableWrapper implements HTableInterface {
return table.getTableDescriptor(); return table.getTableDescriptor();
} }
@Override
public byte[] getTableName() {
return table.getTableName();
}
@Override @Override
public TableName getName() { public TableName getName() {
return table.getName(); return table.getName();
@ -316,30 +302,6 @@ public final class HTableWrapper implements HTableInterface {
table.mutateRow(rm); 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 @Override
public long getWriteBufferSize() { public long getWriteBufferSize() {
return table.getWriteBufferSize(); return table.getWriteBufferSize();
@ -350,13 +312,6 @@ public final class HTableWrapper implements HTableInterface {
table.setWriteBufferSize(writeBufferSize); 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 @Override
public <R extends Message> Map<byte[], R> batchCoprocessorService( public <R extends Message> Map<byte[], R> batchCoprocessorService(
MethodDescriptor methodDescriptor, Message request, byte[] startKey, byte[] endKey, 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.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable; 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.HTableWrapper;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.CoprocessorClassLoader; import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet; import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;
import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.hbase.util.VersionInfo;
@ -377,8 +377,8 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
/** Current coprocessor state */ /** Current coprocessor state */
Coprocessor.State state = Coprocessor.State.UNINSTALLED; Coprocessor.State state = Coprocessor.State.UNINSTALLED;
/** Accounting for tables opened by the coprocessor */ /** Accounting for tables opened by the coprocessor */
protected List<HTableInterface> openTables = protected List<Table> openTables =
Collections.synchronizedList(new ArrayList<HTableInterface>()); Collections.synchronizedList(new ArrayList<Table>());
private int seq; private int seq;
private Configuration conf; private Configuration conf;
private ClassLoader classLoader; private ClassLoader classLoader;
@ -438,7 +438,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
" because not active (state="+state.toString()+")"); " because not active (state="+state.toString()+")");
} }
// clean up any table references // clean up any table references
for (HTableInterface table: openTables) { for (Table table: openTables) {
try { try {
((HTableWrapper)table).internalClose(); ((HTableWrapper)table).internalClose();
} catch (IOException e) { } catch (IOException e) {
@ -493,7 +493,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
* @exception java.io.IOException Exception * @exception java.io.IOException Exception
*/ */
@Override @Override
public HTableInterface getTable(TableName tableName) throws IOException { public Table getTable(TableName tableName) throws IOException {
return this.getTable(tableName, HTable.getDefaultExecutor(getConfiguration())); return this.getTable(tableName, HTable.getDefaultExecutor(getConfiguration()));
} }
@ -504,7 +504,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
* @exception java.io.IOException Exception * @exception java.io.IOException Exception
*/ */
@Override @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); 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.TableName;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HTable; 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.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
@ -83,7 +84,8 @@ public abstract class TableInputFormatBase
implements InputFormat<ImmutableBytesWritable, Result> { implements InputFormat<ImmutableBytesWritable, Result> {
private static final Log LOG = LogFactory.getLog(TableInputFormatBase.class); private static final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
private byte [][] inputColumns; private byte [][] inputColumns;
private HTable table; private Table table;
private RegionLocator regionLocator;
private Connection connection; private Connection connection;
private TableRecordReader tableRecordReader; private TableRecordReader tableRecordReader;
private Filter rowFilter; private Filter rowFilter;
@ -197,7 +199,7 @@ implements InputFormat<ImmutableBytesWritable, Result> {
throw new IOException(INITIALIZATION_ERROR, exception); throw new IOException(INITIALIZATION_ERROR, exception);
} }
byte [][] startKeys = this.table.getStartKeys(); byte [][] startKeys = this.regionLocator.getStartKeys();
if (startKeys == null || startKeys.length == 0) { if (startKeys == null || startKeys.length == 0) {
throw new IOException("Expecting at least one region"); throw new IOException("Expecting at least one region");
} }
@ -212,7 +214,7 @@ implements InputFormat<ImmutableBytesWritable, Result> {
for (int i = 0; i < realNumSplits; i++) { for (int i = 0; i < realNumSplits; i++) {
int lastPos = startPos + middle; int lastPos = startPos + middle;
lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos; lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos;
String regionLocation = table.getRegionLocation(startKeys[startPos]). String regionLocation = regionLocator.getRegionLocation(startKeys[startPos]).
getHostname(); getHostname();
splits[i] = new TableSplit(this.table.getName(), splits[i] = new TableSplit(this.table.getName(),
startKeys[startPos], ((i + 1) < realNumSplits) ? startKeys[lastPos]: 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 " + LOG.warn("initializeTable called multiple times. Overwriting connection and table " +
"reference; TableInputFormatBase will not close these old references when done."); "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; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; 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.User;
import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
@ -146,7 +146,7 @@ public class ConnectionCache {
/** /**
* Caller closes the table afterwards. * Caller closes the table afterwards.
*/ */
public HTableInterface getTable(String tableName) throws IOException { public Table getTable(String tableName) throws IOException {
ConnectionInfo connInfo = getCurrentConnection(); ConnectionInfo connInfo = getCurrentConnection();
return connInfo.connection.getTable(tableName); return connInfo.connection.getTable(tableName);
} }

View File

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

View File

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

View File

@ -952,10 +952,8 @@ public class TestHCM {
c3.close(); c3.close();
// still a reference left // still a reference left
assertFalse(c3.isClosed());
c3.close();
assertTrue(c3.isClosed()); assertTrue(c3.isClosed());
// c3 was removed from the cache
Connection c5 = HConnectionManager.getConnection(configuration); Connection c5 = HConnectionManager.getConnection(configuration);
assertTrue(c5 != c3); 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.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; 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.Increment;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -86,7 +85,7 @@ public class TestHTableWrapper {
static class DummyRegionObserver extends BaseRegionObserver { static class DummyRegionObserver extends BaseRegionObserver {
} }
private HTableInterface hTableInterface; private Table hTableInterface;
private Table table; private Table table;
@BeforeClass @BeforeClass
@ -144,10 +143,8 @@ public class TestHTableWrapper {
private void checkHTableInterfaceMethods() throws Exception { private void checkHTableInterfaceMethods() throws Exception {
checkConf(); checkConf();
checkNameAndDescriptor(); checkNameAndDescriptor();
checkAutoFlush();
checkBufferSize(); checkBufferSize();
checkExists(); checkExists();
checkGetRowOrBefore();
checkAppend(); checkAppend();
checkPutsAndDeletes(); checkPutsAndDeletes();
checkCheckAndPut(); checkCheckAndPut();
@ -159,7 +156,6 @@ public class TestHTableWrapper {
checkMutateRow(); checkMutateRow();
checkResultScanner(); checkResultScanner();
hTableInterface.flushCommits();
hTableInterface.close(); hTableInterface.close();
} }
@ -174,15 +170,6 @@ public class TestHTableWrapper {
assertEquals(table.getTableDescriptor(), hTableInterface.getTableDescriptor()); 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 { private void checkBufferSize() throws IOException {
long initialWriteBufferSize = hTableInterface.getWriteBufferSize(); long initialWriteBufferSize = hTableInterface.getWriteBufferSize();
hTableInterface.setWriteBufferSize(12345L); hTableInterface.setWriteBufferSize(12345L);
@ -194,19 +181,12 @@ public class TestHTableWrapper {
boolean ex = hTableInterface.exists(new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1)); boolean ex = hTableInterface.exists(new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1));
assertTrue(ex); assertTrue(ex);
Boolean[] exArray = hTableInterface.exists(Arrays.asList(new Get[] { boolean[] exArray = hTableInterface.existsAll(Arrays.asList(new Get[]{
new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1), new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1),
new Get(ROW_B).addColumn(TEST_FAMILY, qualifierCol1), new Get(ROW_B).addColumn(TEST_FAMILY, qualifierCol1),
new Get(ROW_C).addColumn(TEST_FAMILY, qualifierCol1), new Get(ROW_C).addColumn(TEST_FAMILY, qualifierCol1),
new Get(Bytes.toBytes("does not exist")).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 }, assertTrue(Arrays.equals(new boolean[]{true, true, true, false}, exArray));
exArray);
}
@SuppressWarnings("deprecation")
private void checkGetRowOrBefore() throws IOException {
Result rowOrBeforeResult = hTableInterface.getRowOrBefore(ROW_A, TEST_FAMILY);
assertArrayEquals(ROW_A, rowOrBeforeResult.getRow());
} }
private void checkAppend() throws IOException { private void checkAppend() throws IOException {

View File

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

View File

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

View File

@ -57,6 +57,8 @@ import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Admin; 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.Consistency;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
@ -958,7 +960,8 @@ public class TestSplitTransactionOnCluster {
HTableDescriptor desc = new HTableDescriptor(table); HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f"))); desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
admin.createTable(desc); 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++) { for(int i = 1; i < 5; i++) {
Put p1 = new Put(("r"+i).getBytes()); Put p1 = new Put(("r"+i).getBytes());
p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".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.KeyValue;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; 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.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -350,7 +350,7 @@ public class TestMasterReplication {
int numClusters = utilities.length; int numClusters = utilities.length;
Table[] htables = new Table[numClusters]; Table[] htables = new Table[numClusters];
for (int i = 0; i < numClusters; i++) { 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); htable.setWriteBufferSize(1024);
htables[i] = htable; 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.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -195,13 +194,16 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
List<Cell> cells = null; List<Cell> cells = null;
if (labelsRegion == null) { if (labelsRegion == null) {
Table table = null; Table table = null;
Connection connection = null;
try { 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); Result result = table.get(get);
cells = result.listCells(); cells = result.listCells();
} finally { } finally {
if (table != null) { if (table != null) {
table.close(); table.close();
connection.close();
} }
} }
} else { } else {

View File

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