Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E9A2317B7A for ; Mon, 23 Mar 2015 22:23:03 +0000 (UTC) Received: (qmail 45937 invoked by uid 500); 23 Mar 2015 22:22:46 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 45787 invoked by uid 500); 23 Mar 2015 22:22:46 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 45068 invoked by uid 99); 23 Mar 2015 22:22:46 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 23 Mar 2015 22:22:46 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 3F462E183C; Mon, 23 Mar 2015 22:22:46 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: mbertozzi@apache.org To: commits@hbase.apache.org Date: Mon, 23 Mar 2015 22:22:54 -0000 Message-Id: <7f45a0adb90f449aafdfb8f943938b6f@git.apache.org> In-Reply-To: <2251c49789274412b472e76f97c515ed@git.apache.org> References: <2251c49789274412b472e76f97c515ed@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [09/44] hbase git commit: HBASE-12586 Task 6 & 7 from HBASE-9117, delete all public HTable constructors and delete ConnectionManager#{delete, get}Connection (Mikhail Antonov) HBASE-12586 Task 6 & 7 from HBASE-9117, delete all public HTable constructors and delete ConnectionManager#{delete,get}Connection (Mikhail Antonov) Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c5aca191 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c5aca191 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c5aca191 Branch: refs/heads/hbase-12439 Commit: c5aca1919d774d2145005b748b3430a4b569922c Parents: 76ba530 Author: stack Authored: Mon Mar 16 11:20:14 2015 -0700 Committer: stack Committed: Mon Mar 16 11:20:14 2015 -0700 ---------------------------------------------------------------------- .../hadoop/hbase/CoprocessorEnvironment.java | 6 +- .../apache/hadoop/hbase/MetaTableAccessor.java | 4 +- .../hadoop/hbase/client/ConnectionManager.java | 4 +- .../org/apache/hadoop/hbase/client/HTable.java | 136 +------------------ .../hadoop/hbase/client/HTableFactory.java | 51 ------- .../hbase/client/HTableInterfaceFactory.java | 54 -------- .../hadoop/hbase/client/HTableWrapper.java | 57 +------- .../hbase/coprocessor/CoprocessorHost.java | 12 +- .../hbase/mapred/TableInputFormatBase.java | 11 +- .../hadoop/hbase/util/ConnectionCache.java | 4 +- .../hadoop/hbase/HBaseTestingUtility.java | 4 +- .../hbase/client/HConnectionTestingUtility.java | 4 +- .../org/apache/hadoop/hbase/client/TestHCM.java | 4 +- .../hbase/coprocessor/TestHTableWrapper.java | 34 +---- .../hbase/mapred/TestTableInputFormat.java | 6 +- .../hbase/mapreduce/TestTableInputFormat.java | 6 +- .../TestSplitTransactionOnCluster.java | 5 +- .../replication/TestMasterReplication.java | 4 +- .../ExpAsStringVisibilityLabelServiceImpl.java | 6 +- .../apache/hadoop/hbase/util/TestHBaseFsck.java | 2 +- 20 files changed, 59 insertions(+), 355 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java index 37f1a33..362439a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java @@ -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 http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index 15d325d..4df58a2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -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); http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java index e283895..50e2755 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java @@ -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(); http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 4ad9eac..2a8063a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -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) { @@ -221,68 +149,6 @@ public class HTable implements HTableInterface { /** * 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 * get a {@link Table} instance (use {@link Table} instead of {@link HTable}). * @param tableName Name of the table. @@ -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, http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java deleted file mode 100644 index 6970333..0000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java +++ /dev/null @@ -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(); - } -} http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java deleted file mode 100644 index b6349c2..0000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java +++ /dev/null @@ -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; -} http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java index c16b4c3..e3641c7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java @@ -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 openTables; + private final List openTables; /** * @param openTables External list of tables used for tracking wrappers. * @throws IOException */ - public static HTableInterface createWrapper(List openTables, + public static Table createWrapper(List
openTables, TableName tableName, Environment env, ExecutorService pool) throws IOException { return new HTableWrapper(openTables, tableName, CoprocessorHConnection.getConnectionForEnvironment(env), pool); } - private HTableWrapper(List openTables, TableName tableName, + private HTableWrapper(List
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); } @@ -241,11 +232,6 @@ public final class HTableWrapper implements HTableInterface { } @Override - public byte[] getTableName() { - return table.getTableName(); - } - - @Override public TableName getName() { return table.getName(); } @@ -317,30 +303,6 @@ public final class HTableWrapper implements HTableInterface { } @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(); } @@ -351,13 +313,6 @@ public final class HTableWrapper implements HTableInterface { } @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 Map batchCoprocessorService( MethodDescriptor methodDescriptor, Message request, byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable { http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java index 237f617..b047d33 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java @@ -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 { /** Current coprocessor state */ Coprocessor.State state = Coprocessor.State.UNINSTALLED; /** Accounting for tables opened by the coprocessor */ - protected List openTables = - Collections.synchronizedList(new ArrayList()); + protected List
openTables = + Collections.synchronizedList(new ArrayList
()); private int seq; private Configuration conf; private ClassLoader classLoader; @@ -438,7 +438,7 @@ public abstract class CoprocessorHost { " 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 { * @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 { * @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); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java index dd58d5c..f8ccea3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java @@ -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 { 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 { 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 { 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 { 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; } http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java index 21714af..8fa711c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java @@ -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); } http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 61d5ba9..0445cb0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -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(); } /** http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java index 998cdf0..0d05c68 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java @@ -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; http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index 219496f..5d284a2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -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); http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java index 317707a..8efbfbf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java @@ -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 { http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java index d7dd8ec..3497cdf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java @@ -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 http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java index 566a642..bc2d08f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java @@ -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 http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index 06f9eb8..9d14be6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -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()); http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index a501af9..0ec410e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -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; } http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java index 6f46fd3..3c71a8c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java @@ -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 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 { http://git-wip-us.apache.org/repos/asf/hbase/blob/c5aca191/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index 2b16dce..ba7ba9c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -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());