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 2324610A72 for ; Wed, 10 Jul 2013 17:57:16 +0000 (UTC) Received: (qmail 16292 invoked by uid 500); 10 Jul 2013 17:56:50 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 16223 invoked by uid 500); 10 Jul 2013 17:56:47 -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 15931 invoked by uid 99); 10 Jul 2013 17:56:41 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 10 Jul 2013 17:56:41 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 10 Jul 2013 17:55:06 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 78DF523889EA; Wed, 10 Jul 2013 17:54:42 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1501881 [1/4] - in /hbase/branches/0.95: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/... Date: Wed, 10 Jul 2013 17:54:37 -0000 To: commits@hbase.apache.org From: stack@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20130710175442.78DF523889EA@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: stack Date: Wed Jul 10 17:54:35 2013 New Revision: 1501881 URL: http://svn.apache.org/r1501881 Log: HBASE-8918 Removes redundant identifiers from interfaces Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Server.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Stoppable.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAdminKeepAliveConnection.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CompoundConfiguration.java hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/HeapSize.java hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDecodingContext.java hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockEncodingContext.java hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/ClassFinder.java hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSource.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceFactory.java hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/MetricHistogram.java hbase/branches/0.95/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/HadoopShims.java hbase/branches/0.95/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java hbase/branches/0.95/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTokenizerData.java hbase/branches/0.95/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/TestColumnData.java hbase/branches/0.95/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestRowData.java hbase/branches/0.95/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/timestamp/TestTimestampData.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/InterProcessLock.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/InterProcessReadWriteLock.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraint.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorService.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionListener.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionSnare.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WritableWithSize.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheableDeserializer.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoder.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/Delayable.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRPCErrorHandler.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotSentinel.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TotesHRegionInfo.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureCoordinatorRpcs.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureMemberRpcs.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/SubprocedureFactory.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DeleteTracker.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LastSequenceId.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseListener.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSinkService.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreConfigInformation.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/Dictionary.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/Constants.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ProtobufMessageHandler.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CodeToClassAndBackFor96Migration.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescerMBean.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CancelableProgressable.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/KeyRange.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomDistribution.java hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java Wed Jul 10 17:54:35 2013 @@ -35,11 +35,11 @@ public interface Abortable { * @param why Why we're aborting. * @param e Throwable that caused abort. Can be null. */ - public void abort(String why, Throwable e); + void abort(String why, Throwable e); /** * Check if the server or client was aborted. * @return true if the server or client was aborted, false otherwise */ - public boolean isAborted(); + boolean isAborted(); } Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java Wed Jul 10 17:54:35 2013 @@ -26,21 +26,21 @@ import java.io.IOException; @InterfaceAudience.Public @InterfaceStability.Evolving public interface Coprocessor { - static final int VERSION = 1; + int VERSION = 1; /** Highest installation priority */ - static final int PRIORITY_HIGHEST = 0; + int PRIORITY_HIGHEST = 0; /** High (system) installation priority */ - static final int PRIORITY_SYSTEM = Integer.MAX_VALUE / 4; + int PRIORITY_SYSTEM = Integer.MAX_VALUE / 4; /** Default installation priority for user coprocessors */ - static final int PRIORITY_USER = Integer.MAX_VALUE / 2; + int PRIORITY_USER = Integer.MAX_VALUE / 2; /** Lowest installation priority */ - static final int PRIORITY_LOWEST = Integer.MAX_VALUE; + int PRIORITY_LOWEST = Integer.MAX_VALUE; /** * Lifecycle state of a given coprocessor instance. */ - public enum State { + enum State { UNINSTALLED, INSTALLED, STARTING, Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java Wed Jul 10 17:54:35 2013 @@ -30,26 +30,26 @@ import java.io.IOException; public interface CoprocessorEnvironment { /** @return the Coprocessor interface version */ - public int getVersion(); + int getVersion(); /** @return the HBase version as a string (e.g. "0.21.0") */ - public String getHBaseVersion(); + String getHBaseVersion(); /** @return the loaded coprocessor instance */ - public Coprocessor getInstance(); + Coprocessor getInstance(); /** @return the priority assigned to the loaded coprocessor */ - public int getPriority(); + int getPriority(); /** @return the load sequence number */ - public int getLoadSequence(); + int getLoadSequence(); /** @return the configuration */ - public Configuration getConfiguration(); + Configuration getConfiguration(); /** * @return an interface for accessing the given table * @throws IOException */ - public HTableInterface getTable(byte[] tableName) throws IOException; + HTableInterface getTable(byte[] tableName) throws IOException; } Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Server.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Server.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Server.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Server.java Wed Jul 10 17:54:35 2013 @@ -32,20 +32,20 @@ public interface Server extends Abortabl /** * Gets the configuration object for this server. */ - public Configuration getConfiguration(); + Configuration getConfiguration(); /** * Gets the ZooKeeper instance for this server. */ - public ZooKeeperWatcher getZooKeeper(); + ZooKeeperWatcher getZooKeeper(); /** * @return Master's instance of {@link CatalogTracker} */ - public CatalogTracker getCatalogTracker(); + CatalogTracker getCatalogTracker(); /** * @return The unique server name for this server. */ - public ServerName getServerName(); + ServerName getServerName(); } Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Stoppable.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Stoppable.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Stoppable.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Stoppable.java Wed Jul 10 17:54:35 2013 @@ -29,10 +29,10 @@ public interface Stoppable { * Stop this service. * @param why Why we're stopping. */ - public void stop(String why); + void stop(String why); /** * @return True if {@link #stop(String)} has been closed. */ - public boolean isStopped(); + boolean isStopped(); } Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java Wed Jul 10 17:54:35 2013 @@ -558,7 +558,7 @@ public class MetaReader { * @return True if we are to proceed scanning the table, else false if * we are to stop now. */ - public boolean visit(final Result r) throws IOException; + boolean visit(final Result r) throws IOException; } /** Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java Wed Jul 10 17:54:35 2013 @@ -117,7 +117,7 @@ class AsyncProcess { * * */ - static interface AsyncProcessCallback { + interface AsyncProcessCallback { /** * Called on success. originalIndex holds the index in the action list. Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java Wed Jul 10 17:54:35 2013 @@ -34,18 +34,18 @@ public interface Attributes { * @param name attribute name * @param value attribute value */ - public void setAttribute(String name, byte[] value); + void setAttribute(String name, byte[] value); /** * Gets an attribute * @param name attribute name * @return attribute value if attribute is set, null otherwise */ - public byte[] getAttribute(String name); + byte[] getAttribute(String name); /** * Gets all attributes * @return unmodifiable map of all attributes */ - public Map getAttributesMap(); + Map getAttributesMap(); } Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java Wed Jul 10 17:54:35 2013 @@ -82,19 +82,19 @@ class ClusterStatusListener implements C * * @param sn - the server name */ - public void newDead(ServerName sn); + void newDead(ServerName sn); } /** * The interface to be implented by a listener of a cluster status event. */ - static interface Listener extends Closeable { + interface Listener extends Closeable { /** * Called to close the resources, if any. Cannot throw an exception. */ @Override - public void close(); + void close(); /** * Called to connect. @@ -102,7 +102,7 @@ class ClusterStatusListener implements C * @param conf Configuration to use. * @throws IOException */ - public void connect(Configuration conf) throws IOException; + void connect(Configuration conf) throws IOException; } public ClusterStatusListener(DeadServerHandler dsh, Configuration conf, Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java Wed Jul 10 17:54:35 2013 @@ -62,10 +62,10 @@ public interface HConnection extends Abo /** * @return Configuration instance being used by this HConnection instance. */ - public Configuration getConfiguration(); + Configuration getConfiguration(); /** @return - true if the master server is running */ - public boolean isMasterRunning() + boolean isMasterRunning() throws MasterNotRunningException, ZooKeeperConnectionException; /** @@ -76,21 +76,21 @@ public interface HConnection extends Abo * @return true if the table is enabled, false otherwise * @throws IOException if a remote or network exception occurs */ - public boolean isTableEnabled(byte[] tableName) throws IOException; + boolean isTableEnabled(byte[] tableName) throws IOException; /** * @param tableName table name * @return true if the table is disabled, false otherwise * @throws IOException if a remote or network exception occurs */ - public boolean isTableDisabled(byte[] tableName) throws IOException; + boolean isTableDisabled(byte[] tableName) throws IOException; /** * @param tableName table name * @return true if all regions of the table are available, false otherwise * @throws IOException if a remote or network exception occurs */ - public boolean isTableAvailable(byte[] tableName) throws IOException; + boolean isTableAvailable(byte[] tableName) throws IOException; /** * Use this api to check if the table has been created with the specified number of @@ -104,7 +104,7 @@ public interface HConnection extends Abo * @throws IOException * if a remote or network exception occurs */ - public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException; + boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException; /** * List all the userspace tables. In other words, scan the META table. @@ -116,14 +116,14 @@ public interface HConnection extends Abo * @return - returns an array of HTableDescriptors * @throws IOException if a remote or network exception occurs */ - public HTableDescriptor[] listTables() throws IOException; + HTableDescriptor[] listTables() throws IOException; /** * @param tableName table name * @return table metadata * @throws IOException if a remote or network exception occurs */ - public HTableDescriptor getHTableDescriptor(byte[] tableName) + HTableDescriptor getHTableDescriptor(byte[] tableName) throws IOException; /** @@ -135,14 +135,15 @@ public interface HConnection extends Abo * question * @throws IOException if a remote or network exception occurs */ - public HRegionLocation locateRegion(final byte [] tableName, - final byte [] row) + HRegionLocation locateRegion( + final byte[] tableName, final byte[] row + ) throws IOException; /** * Allows flushing the region cache. */ - public void clearRegionCache(); + void clearRegionCache(); /** * Allows flushing the region cache of all locations that pertain to @@ -150,13 +151,13 @@ public interface HConnection extends Abo * @param tableName Name of the table whose regions we are to remove from * cache. */ - public void clearRegionCache(final byte [] tableName); + void clearRegionCache(final byte[] tableName); /** * Deletes cached locations for the specific region. * @param location The location object for the region, to be purged from cache. */ - public void deleteCachedRegionLocation(final HRegionLocation location); + void deleteCachedRegionLocation(final HRegionLocation location); /** * Find the location of the region of tableName that row @@ -167,8 +168,9 @@ public interface HConnection extends Abo * question * @throws IOException if a remote or network exception occurs */ - public HRegionLocation relocateRegion(final byte [] tableName, - final byte [] row) + HRegionLocation relocateRegion( + final byte[] tableName, final byte[] row + ) throws IOException; /** @@ -179,8 +181,9 @@ public interface HConnection extends Abo * @param exception the exception if any. Can be null. * @param source the previous location */ - public void updateCachedLocations(byte[] tableName, byte[] rowkey, - Object exception, HRegionLocation source); + void updateCachedLocations( + byte[] tableName, byte[] rowkey, Object exception, HRegionLocation source + ); /** * Gets the location of the region of regionName. @@ -189,7 +192,7 @@ public interface HConnection extends Abo * question * @throws IOException if a remote or network exception occurs */ - public HRegionLocation locateRegion(final byte [] regionName) + HRegionLocation locateRegion(final byte[] regionName) throws IOException; /** @@ -198,7 +201,7 @@ public interface HConnection extends Abo * @return list of region locations for all regions of table * @throws IOException */ - public List locateRegions(final byte[] tableName) + List locateRegions(final byte[] tableName) throws IOException; /** @@ -210,18 +213,19 @@ public interface HConnection extends Abo * @return list of region locations for all regions of table * @throws IOException */ - public List locateRegions(final byte[] tableName, final boolean useCache, - final boolean offlined) throws IOException; + List locateRegions( + final byte[] tableName, final boolean useCache, final boolean offlined + ) throws IOException; /** * Returns a {@link MasterAdminKeepAliveConnection} to the active master */ - public MasterAdminService.BlockingInterface getMasterAdmin() throws IOException; + MasterAdminService.BlockingInterface getMasterAdmin() throws IOException; /** * Returns an {@link MasterMonitorKeepAliveConnection} to the active master */ - public MasterMonitorService.BlockingInterface getMasterMonitor() throws IOException; + MasterMonitorService.BlockingInterface getMasterMonitor() throws IOException; /** * Establishes a connection to the region server at the specified address. @@ -229,7 +233,7 @@ public interface HConnection extends Abo * @return proxy for HRegionServer * @throws IOException if a remote or network exception occurs */ - public AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException; + AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException; /** * Establishes a connection to the region server at the specified address, and returns @@ -240,7 +244,7 @@ public interface HConnection extends Abo * @throws IOException if a remote or network exception occurs * */ - public ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException; + ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException; /** * Establishes a connection to the region server at the specified address. @@ -250,7 +254,7 @@ public interface HConnection extends Abo * @throws IOException if a remote or network exception occurs * @deprecated You can pass master flag but nothing special is done. */ - public AdminService.BlockingInterface getAdmin(final ServerName serverName, boolean getMaster) + AdminService.BlockingInterface getAdmin(final ServerName serverName, boolean getMaster) throws IOException; /** @@ -277,7 +281,7 @@ public interface HConnection extends Abo * @throws RuntimeException other unspecified error */ @Deprecated - public T getRegionServerWithRetries(ServerCallable callable) + T getRegionServerWithRetries(ServerCallable callable) throws IOException, RuntimeException; /** @@ -290,7 +294,7 @@ public interface HConnection extends Abo * @throws RuntimeException other unspecified error */ @Deprecated - public T getRegionServerWithoutRetries(ServerCallable callable) + T getRegionServerWithoutRetries(ServerCallable callable) throws IOException, RuntimeException; /** @@ -309,8 +313,9 @@ public interface HConnection extends Abo * @deprecated since 0.96 - Use {@link HTableInterface#batch} instead */ @Deprecated - public void processBatch(List actions, final byte[] tableName, - ExecutorService pool, Object[] results) + void processBatch( + List actions, final byte[] tableName, ExecutorService pool, Object[] results + ) throws IOException, InterruptedException; /** @@ -319,11 +324,13 @@ public interface HConnection extends Abo * @deprecated since 0.96 - Use {@link HTableInterface#batchCallback} instead */ @Deprecated - public void processBatchCallback(List list, - byte[] tableName, - ExecutorService pool, - Object[] results, - Batch.Callback callback) throws IOException, InterruptedException; + void processBatchCallback( + List list, + byte[] tableName, + ExecutorService pool, + Object[] results, + Batch.Callback callback + ) throws IOException, InterruptedException; /** * Enable or disable region cache prefetch for the table. It will be @@ -332,8 +339,9 @@ public interface HConnection extends Abo * @param tableName name of table to configure. * @param enable Set to true to enable region cache prefetch. */ - public void setRegionCachePrefetch(final byte[] tableName, - final boolean enable); + void setRegionCachePrefetch( + final byte[] tableName, final boolean enable + ); /** * Check whether region cache prefetch is enabled or not. @@ -341,34 +349,34 @@ public interface HConnection extends Abo * @return true if table's region cache prefetch is enabled. Otherwise * it is disabled. */ - public boolean getRegionCachePrefetch(final byte[] tableName); + boolean getRegionCachePrefetch(final byte[] tableName); /** * @return the number of region servers that are currently running * @throws IOException if a remote or network exception occurs * @deprecated This method will be changed from public to package protected. */ - public int getCurrentNrHRS() throws IOException; + int getCurrentNrHRS() throws IOException; /** * @param tableNames List of table names * @return HTD[] table metadata * @throws IOException if a remote or network exception occurs */ - public HTableDescriptor[] getHTableDescriptors(List tableNames) + HTableDescriptor[] getHTableDescriptors(List tableNames) throws IOException; /** * @return true if this connection is closed */ - public boolean isClosed(); + boolean isClosed(); /** * Clear any caches that pertain to server name sn. * @param sn A server name */ - public void clearCaches(final ServerName sn); + void clearCaches(final ServerName sn); /** * This function allows HBaseAdmin and potentially others to get a shared MasterMonitor @@ -377,7 +385,7 @@ public interface HConnection extends Abo * @throws MasterNotRunningException */ // TODO: Why is this in the public interface when the returned type is shutdown package access? - public MasterMonitorKeepAliveConnection getKeepAliveMasterMonitorService() + MasterMonitorKeepAliveConnection getKeepAliveMasterMonitorService() throws MasterNotRunningException; /** @@ -387,11 +395,11 @@ public interface HConnection extends Abo * @throws MasterNotRunningException */ // TODO: Why is this in the public interface when the returned type is shutdown package access? - public MasterAdminKeepAliveConnection getKeepAliveMasterAdminService() throws MasterNotRunningException; + MasterAdminKeepAliveConnection getKeepAliveMasterAdminService() throws MasterNotRunningException; /** * @param serverName * @return true if the server is known as dead, false otherwise. */ - public boolean isDeadServer(ServerName serverName); + boolean isDeadServer(ServerName serverName); } Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java Wed Jul 10 17:54:35 2013 @@ -127,8 +127,9 @@ public interface HTableInterface extends * Same as {@link #batch(List, Object[])}, but with a callback. * @since 0.96.0 */ - public void batchCallback( - final List actions, final Object[] results, final Batch.Callback callback) + void batchCallback( + final List actions, final Object[] results, final Batch.Callback callback + ) throws IOException, InterruptedException; @@ -136,8 +137,9 @@ public interface HTableInterface extends * Same as {@link #batch(List)}, but with a callback. * @since 0.96.0 */ - public Object[] batchCallback( - List actions, Batch.Callback callback) throws IOException, + Object[] batchCallback( + List actions, Batch.Callback callback + ) throws IOException, InterruptedException; /** @@ -309,7 +311,7 @@ public interface HTableInterface extends * @param rm object that specifies the set of mutations to perform atomically * @throws IOException */ - public void mutateRow(final RowMutations rm) throws IOException; + void mutateRow(final RowMutations rm) throws IOException; /** * Appends values to one or more columns within a single row. @@ -324,7 +326,7 @@ public interface HTableInterface extends * @throws IOException e * @return values of columns after the append operation (maybe null) */ - public Result append(final Append append) throws IOException; + Result append(final Append append) throws IOException; /** * Increments one or more columns within a single row. @@ -339,7 +341,7 @@ public interface HTableInterface extends * @throws IOException e * @return values of columns after the increment */ - public Result increment(final Increment increment) throws IOException; + Result increment(final Increment increment) throws IOException; /** * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)} @@ -493,7 +495,7 @@ public interface HTableInterface extends * @param autoFlush * Whether or not to enable 'auto-flush'. */ - public void setAutoFlush(boolean autoFlush); + void setAutoFlush(boolean autoFlush); /** * Turns 'auto-flush' on or off. @@ -522,7 +524,7 @@ public interface HTableInterface extends * Whether to keep Put failures in the writeBuffer * @see #flushCommits */ - public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail); + void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail); /** * Returns the maximum size in bytes of the write buffer for this HTable. @@ -531,7 +533,7 @@ public interface HTableInterface extends * {@code hbase.client.write.buffer}. * @return The size of the write buffer in bytes. */ - public long getWriteBufferSize(); + long getWriteBufferSize(); /** * Sets the size of the buffer in bytes. @@ -541,5 +543,5 @@ public interface HTableInterface extends * @param writeBufferSize The new write buffer size, in bytes. * @throws IOException if a remote or network exception occurs. */ - public void setWriteBufferSize(long writeBufferSize) throws IOException; + void setWriteBufferSize(long writeBufferSize) throws IOException; } Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAdminKeepAliveConnection.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAdminKeepAliveConnection.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAdminKeepAliveConnection.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAdminKeepAliveConnection.java Wed Jul 10 17:54:35 2013 @@ -40,5 +40,5 @@ extends MasterAdminProtos.MasterAdminSer */ // The Closeable Interface wants to throw an IOE out of a close. // Thats a PITA. Do this below instead of Closeable. - public void close(); -} \ No newline at end of file + void close(); +} Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java Wed Jul 10 17:54:35 2013 @@ -297,7 +297,7 @@ public class MetaScanner { * @return A boolean to know if it should continue to loop in the region * @throws IOException e */ - public boolean processRow(Result rowResult) throws IOException; + boolean processRow(Result rowResult) throws IOException; } public static abstract class MetaScannerVisitorBase implements MetaScannerVisitor { Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java Wed Jul 10 17:54:35 2013 @@ -38,17 +38,17 @@ public interface ResultScanner extends C * exhausted. * @throws IOException e */ - public Result next() throws IOException; + Result next() throws IOException; /** * @param nbRows number of rows to return * @return Between zero and nbRows Results * @throws IOException e */ - public Result [] next(int nbRows) throws IOException; + Result [] next(int nbRows) throws IOException; /** * Closes the scanner and releases any resources it has allocated */ - public void close(); + void close(); } Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java Wed Jul 10 17:54:35 2013 @@ -30,5 +30,5 @@ public interface Row extends Comparable< /** * @return The row. */ - public byte [] getRow(); -} \ No newline at end of file + byte [] getRow(); +} Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java Wed Jul 10 17:54:35 2013 @@ -50,8 +50,8 @@ public abstract class Batch { * {@link Batch.Call#call(Object)} * @param the return type from {@link Batch.Call#call(Object)} */ - public static interface Call { - public R call(T instance) throws IOException; + public interface Call { + R call(T instance) throws IOException; } /** @@ -68,7 +68,7 @@ public abstract class Batch { * @param the return type from the associated {@link Batch.Call#call(Object)} * @see org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call) */ - public static interface Callback { - public void update(byte[] region, byte[] row, R result); + public interface Callback { + void update(byte[] region, byte[] row, R result); } -} \ No newline at end of file +} Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java Wed Jul 10 17:54:35 2013 @@ -46,7 +46,7 @@ public interface ReplicationPeers { * Initialize the ReplicationPeers interface. * @throws KeeperException */ - public void init() throws IOException, KeeperException; + void init() throws IOException, KeeperException; /** * Add a new remote slave cluster for replication. @@ -54,65 +54,65 @@ public interface ReplicationPeers { * @param clusterKey the concatenation of the slave cluster's: * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent */ - public void addPeer(String peerId, String clusterKey) throws IOException; + void addPeer(String peerId, String clusterKey) throws IOException; /** * Removes a remote slave cluster and stops the replication to it. * @param peerId a short that identifies the cluster */ - public void removePeer(String peerId) throws IOException; + void removePeer(String peerId) throws IOException; /** * Restart the replication to the specified remote slave cluster. * @param peerId a short that identifies the cluster */ - public void enablePeer(String peerId) throws IOException; + void enablePeer(String peerId) throws IOException; /** * Stop the replication to the specified remote slave cluster. * @param peerId a short that identifies the cluster */ - public void disablePeer(String peerId) throws IOException; + void disablePeer(String peerId) throws IOException; /** * Get the replication status for the specified connected remote slave cluster. * @param peerId a short that identifies the cluster * @return true if replication is enabled, false otherwise. */ - public boolean getStatusOfConnectedPeer(String peerId); + boolean getStatusOfConnectedPeer(String peerId); /** * Get a set of all connected remote slave clusters. * @return set of peer ids */ - public Set getConnectedPeers(); + Set getConnectedPeers(); /** * List the cluster keys of all remote slave clusters (whether they are enabled/disabled or * connected/disconnected). * @return A map of peer ids to peer cluster keys */ - public Map getAllPeerClusterKeys(); + Map getAllPeerClusterKeys(); /** * List the peer ids of all remote slave clusters (whether they are enabled/disabled or * connected/disconnected). * @return A list of peer ids */ - public List getAllPeerIds(); + List getAllPeerIds(); /** * Attempt to connect to a new remote slave cluster. * @param peerId a short that identifies the cluster * @return true if a new connection was made, false if no new connection was made. */ - public boolean connectToPeer(String peerId) throws IOException, KeeperException; + boolean connectToPeer(String peerId) throws IOException, KeeperException; /** * Disconnect from a remote slave cluster. * @param peerId a short that identifies the cluster */ - public void disconnectFromPeer(String peerId); + void disconnectFromPeer(String peerId); /** * Returns all region servers from given connected remote slave cluster. @@ -120,19 +120,19 @@ public interface ReplicationPeers { * @return addresses of all region servers in the peer cluster. Returns an empty list if the peer * cluster is unavailable or there are no region servers in the cluster. */ - public List getRegionServersOfConnectedPeer(String peerId); + List getRegionServersOfConnectedPeer(String peerId); /** * Returns the UUID of the provided peer id. * @param peerId the peer's ID that will be converted into a UUID * @return a UUID or null if the peer cluster does not exist or is not connected. */ - public UUID getPeerUUID(String peerId); + UUID getPeerUUID(String peerId); /** * Returns the configuration needed to talk to the remote slave cluster. * @param peerId a short that identifies the cluster * @return the configuration for the peer cluster, null if it was unable to get the configuration */ - public Configuration getPeerConf(String peerId) throws KeeperException; -} \ No newline at end of file + Configuration getPeerConf(String peerId) throws KeeperException; +} Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java Wed Jul 10 17:54:35 2013 @@ -37,13 +37,13 @@ public interface ReplicationQueues { * @param serverName The server name of the region server that owns the replication queues this * interface manages. */ - public void init(String serverName) throws KeeperException; + void init(String serverName) throws KeeperException; /** * Remove a replication queue. * @param queueId a String that identifies the queue. */ - public void removeQueue(String queueId); + void removeQueue(String queueId); /** * Add a new HLog file to the given queue. If the queue does not exist it is created. @@ -51,14 +51,14 @@ public interface ReplicationQueues { * @param filename name of the HLog * @throws KeeperException */ - public void addLog(String queueId, String filename) throws KeeperException; + void addLog(String queueId, String filename) throws KeeperException; /** * Remove an HLog file from the given queue. * @param queueId a String that identifies the queue. * @param filename name of the HLog */ - public void removeLog(String queueId, String filename); + void removeLog(String queueId, String filename); /** * Set the current position for a specific HLog in a given queue. @@ -66,7 +66,7 @@ public interface ReplicationQueues { * @param filename name of the HLog * @param position the current position in the file */ - public void setLogPosition(String queueId, String filename, long position); + void setLogPosition(String queueId, String filename, long position); /** * Get the current position for a specific HLog in a given queue. @@ -74,25 +74,25 @@ public interface ReplicationQueues { * @param filename name of the HLog * @return the current position in the file */ - public long getLogPosition(String queueId, String filename) throws KeeperException; + long getLogPosition(String queueId, String filename) throws KeeperException; /** * Remove all replication queues for this region server. */ - public void removeAllQueues(); + void removeAllQueues(); /** * Get a list of all HLogs in the given queue. * @param queueId a String that identifies the queue * @return a list of HLogs, null if this region server is dead and has no outstanding queues */ - public List getLogsInQueue(String queueId); + List getLogsInQueue(String queueId); /** * Get a list of all queues for this region server. * @return a list of queueIds, null if this region server is dead and has no outstanding queues */ - public List getAllQueues(); + List getAllQueues(); /** * Take ownership for the set of queues belonging to a dead region server. @@ -100,12 +100,12 @@ public interface ReplicationQueues { * @return A SortedMap of the queues that have been claimed, including a SortedSet of HLogs in * each queue. Returns an empty map if no queues were failed-over. */ - public SortedMap> claimQueues(String regionserver); + SortedMap> claimQueues(String regionserver); /** * Get a list of all region servers that have outstanding replication queues. These servers could * be alive, dead or from a previous run of the cluster. * @return a list of server names */ - public List getListOfReplicators(); -} \ No newline at end of file + List getListOfReplicators(); +} Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java Wed Jul 10 17:54:35 2013 @@ -31,7 +31,7 @@ public interface ReplicationQueuesClient * be alive, dead or from a previous run of the cluster. * @return a list of server names */ - public List getListOfReplicators(); + List getListOfReplicators(); /** * Get a list of all HLogs in the given queue on the given region server. @@ -39,12 +39,12 @@ public interface ReplicationQueuesClient * @param queueId a String that identifies the queue * @return a list of HLogs, null if this region server is dead and has no outstanding queues */ - public List getLogsInQueue(String serverName, String queueId); + List getLogsInQueue(String serverName, String queueId); /** * Get a list of all queues for the specified region server. * @param serverName the server name of the region server that owns the set of queues * @return a list of queueIds, null if this region server is not a replicator. */ - public List getAllQueues(String serverName); -} \ No newline at end of file + List getAllQueues(String serverName); +} Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java (original) +++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java Wed Jul 10 17:54:35 2013 @@ -216,17 +216,17 @@ public class PoolMap implements Ma } protected interface Pool { - public R get(); + R get(); - public R put(R resource); + R put(R resource); - public boolean remove(R resource); + boolean remove(R resource); - public void clear(); + void clear(); - public Collection values(); + Collection values(); - public int size(); + int size(); } public enum PoolType { Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CompoundConfiguration.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CompoundConfiguration.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CompoundConfiguration.java (original) +++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CompoundConfiguration.java Wed Jul 10 17:54:35 2013 @@ -66,7 +66,7 @@ public class CompoundConfiguration exten // Devs: these APIs are the same contract as their counterparts in // Configuration.java - private static interface ImmutableConfigMap extends Iterable> { + private interface ImmutableConfigMap extends Iterable> { String get(String key); String getRaw(String key); Class getClassByName(String name) throws ClassNotFoundException; Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original) +++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java Wed Jul 10 17:54:35 2013 @@ -2514,14 +2514,14 @@ public class KeyValue implements Cell, H /** * Avoids redundant comparisons for better performance. */ - public static interface SamePrefixComparator { + public interface SamePrefixComparator { /** * Compare two keys assuming that the first n bytes are the same. * @param commonPrefix How many bytes are the same. */ - public int compareIgnoringPrefix(int commonPrefix, - T left, int loffset, int llength, - T right, int roffset, int rlength); + int compareIgnoringPrefix( + int commonPrefix, T left, int loffset, int llength, T right, int roffset, int rlength + ); } /** Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java (original) +++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java Wed Jul 10 17:54:35 2013 @@ -37,14 +37,14 @@ public interface Codec { * Call flush when done. Some encoders may not put anything on the stream until flush is called. * On flush, let go of any resources used by the encoder. */ - public interface Encoder extends CellOutputStream {} + interface Encoder extends CellOutputStream {} /** * Implementations should implicitly clean up any resources allocated when the * Decoder/CellScanner runs off the end of the cell block. Do this rather than require the user * call close explicitly. */ - public interface Decoder extends CellScanner {}; + interface Decoder extends CellScanner {}; Decoder getDecoder(InputStream is); Encoder getEncoder(OutputStream os); Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/HeapSize.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/HeapSize.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/HeapSize.java (original) +++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/HeapSize.java Wed Jul 10 17:54:35 2013 @@ -45,5 +45,5 @@ public interface HeapSize { * @return Approximate 'exclusive deep size' of implementing object. Includes * count of payload and hosting object sizings. */ - public long heapSize(); -} \ No newline at end of file + long heapSize(); +} Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java (original) +++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java Wed Jul 10 17:54:35 2013 @@ -57,9 +57,9 @@ public interface DataBlockEncoder { * @throws IOException * If there is an error writing to output stream. */ - public void encodeKeyValues( - ByteBuffer in, boolean includesMemstoreTS, - HFileBlockEncodingContext encodingContext) throws IOException; + void encodeKeyValues( + ByteBuffer in, boolean includesMemstoreTS, HFileBlockEncodingContext encodingContext + ) throws IOException; /** * Decode. @@ -69,8 +69,9 @@ public interface DataBlockEncoder { * @return Uncompressed block of KeyValues. * @throws IOException If there is an error in source. */ - public ByteBuffer decodeKeyValues(DataInputStream source, - boolean includesMemstoreTS) throws IOException; + ByteBuffer decodeKeyValues( + DataInputStream source, boolean includesMemstoreTS + ) throws IOException; /** * Uncompress. @@ -82,8 +83,9 @@ public interface DataBlockEncoder { * @return Uncompressed block of KeyValues. * @throws IOException If there is an error in source. */ - public ByteBuffer decodeKeyValues(DataInputStream source, - int allocateHeaderLength, int skipLastBytes, boolean includesMemstoreTS) + ByteBuffer decodeKeyValues( + DataInputStream source, int allocateHeaderLength, int skipLastBytes, boolean includesMemstoreTS + ) throws IOException; /** @@ -94,7 +96,7 @@ public interface DataBlockEncoder { * @param block encoded block we want index, the position will not change * @return First key in block. */ - public ByteBuffer getFirstKeyInBlock(ByteBuffer block); + ByteBuffer getFirstKeyInBlock(ByteBuffer block); /** * Create a HFileBlock seeker which find KeyValues within a block. @@ -103,8 +105,9 @@ public interface DataBlockEncoder { * key-value pair * @return A newly created seeker. */ - public EncodedSeeker createSeeker(RawComparator comparator, - boolean includesMemstoreTS); + EncodedSeeker createSeeker( + RawComparator comparator, boolean includesMemstoreTS + ); /** * Creates a encoder specific encoding context @@ -119,9 +122,9 @@ public interface DataBlockEncoder { * is unknown * @return a newly created encoding context */ - public HFileBlockEncodingContext newDataBlockEncodingContext( - Algorithm compressionAlgorithm, DataBlockEncoding encoding, - byte[] headerBytes); + HFileBlockEncodingContext newDataBlockEncodingContext( + Algorithm compressionAlgorithm, DataBlockEncoding encoding, byte[] headerBytes + ); /** * Creates an encoder specific decoding context, which will prepare the data @@ -131,8 +134,9 @@ public interface DataBlockEncoder { * compression algorithm used if the data needs to be decompressed * @return a newly created decoding context */ - public HFileBlockDecodingContext newDataBlockDecodingContext( - Algorithm compressionAlgorithm); + HFileBlockDecodingContext newDataBlockDecodingContext( + Algorithm compressionAlgorithm + ); /** * An interface which enable to seek while underlying data is encoded. @@ -140,19 +144,19 @@ public interface DataBlockEncoder { * It works on one HFileBlock, but it is reusable. See * {@link #setCurrentBuffer(ByteBuffer)}. */ - public static interface EncodedSeeker { + interface EncodedSeeker { /** * Set on which buffer there will be done seeking. * @param buffer Used for seeking. */ - public void setCurrentBuffer(ByteBuffer buffer); + void setCurrentBuffer(ByteBuffer buffer); /** * Does a deep copy of the key at the current position. A deep copy is * necessary because buffers are reused in the decoder. * @return key at current position */ - public ByteBuffer getKeyDeepCopy(); + ByteBuffer getKeyDeepCopy(); /** * Does a shallow copy of the value at the current position. A shallow @@ -160,25 +164,25 @@ public interface DataBlockEncoder { * of the original encoded buffer. * @return value at current position */ - public ByteBuffer getValueShallowCopy(); + ByteBuffer getValueShallowCopy(); /** @return key value at current position with position set to limit */ - public ByteBuffer getKeyValueBuffer(); + ByteBuffer getKeyValueBuffer(); /** * @return the KeyValue object at the current position. Includes memstore * timestamp. */ - public KeyValue getKeyValue(); + KeyValue getKeyValue(); /** Set position to beginning of given block */ - public void rewind(); + void rewind(); /** * Move to next position * @return true on success, false if there is no more positions. */ - public boolean next(); + boolean next(); /** * Moves the seeker position within the current block to: @@ -197,7 +201,8 @@ public interface DataBlockEncoder { * of an exact match. Does not matter in case of an inexact match. * @return 0 on exact match, 1 on inexact match. */ - public int seekToKeyInBlock(byte[] key, int offset, int length, - boolean seekBefore); + int seekToKeyInBlock( + byte[] key, int offset, int length, boolean seekBefore + ); } } Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDecodingContext.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDecodingContext.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDecodingContext.java (original) +++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDecodingContext.java Wed Jul 10 17:54:35 2013 @@ -32,7 +32,7 @@ public interface HFileBlockDecodingConte /** * @return the compression algorithm used by this decoding context */ - public Compression.Algorithm getCompression(); + Compression.Algorithm getCompression(); /** * Perform all actions that need to be done before the encoder's real decoding process. @@ -47,7 +47,12 @@ public interface HFileBlockDecodingConte * @param offset data start offset in onDiskBlock * @throws IOException */ - public void prepareDecoding(int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader, - ByteBuffer blockBufferWithoutHeader, byte[] onDiskBlock, int offset) throws IOException; + void prepareDecoding( + int onDiskSizeWithoutHeader, + int uncompressedSizeWithoutHeader, + ByteBuffer blockBufferWithoutHeader, + byte[] onDiskBlock, + int offset + ) throws IOException; } Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockEncodingContext.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockEncodingContext.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockEncodingContext.java (original) +++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockEncodingContext.java Wed Jul 10 17:54:35 2013 @@ -34,39 +34,39 @@ public interface HFileBlockEncodingConte /** * @return OutputStream to which encoded data is written */ - public OutputStream getOutputStreamForEncoder(); + OutputStream getOutputStreamForEncoder(); /** * @return encoded and compressed bytes with header which are ready to write * out to disk */ - public byte[] getOnDiskBytesWithHeader(); + byte[] getOnDiskBytesWithHeader(); /** * @return encoded but not heavily compressed bytes with header which can be * cached in block cache */ - public byte[] getUncompressedBytesWithHeader(); + byte[] getUncompressedBytesWithHeader(); /** * @return the block type after encoding */ - public BlockType getBlockType(); + BlockType getBlockType(); /** * @return the compression algorithm used by this encoding context */ - public Compression.Algorithm getCompression(); + Compression.Algorithm getCompression(); /** * sets the dummy header bytes */ - public void setDummyHeader(byte[] headerBytes); + void setDummyHeader(byte[] headerBytes); /** * @return the {@link DataBlockEncoding} encoding used */ - public DataBlockEncoding getDataBlockEncoding(); + DataBlockEncoding getDataBlockEncoding(); /** * Do any action that needs to be performed after the encoding. @@ -76,11 +76,11 @@ public interface HFileBlockEncodingConte * @param blockType * @throws IOException */ - public void postEncoding(BlockType blockType) throws IOException; + void postEncoding(BlockType blockType) throws IOException; /** * Releases the resources used. */ - public void close(); + void close(); } Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java (original) +++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java Wed Jul 10 17:54:35 2013 @@ -1000,8 +1000,9 @@ public class Bytes { } interface Comparer { - abstract public int compareTo(T buffer1, int offset1, int length1, - T buffer2, int offset2, int length2); + int compareTo( + T buffer1, int offset1, int length1, T buffer2, int offset2, int length2 + ); } @VisibleForTesting Modified: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/ClassFinder.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/ClassFinder.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/ClassFinder.java (original) +++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/ClassFinder.java Wed Jul 10 17:54:35 2013 @@ -50,16 +50,16 @@ public class ClassFinder { private ClassFilter classFilter; private FileFilter fileFilter; - public static interface ResourcePathFilter { - public boolean isCandidatePath(String resourcePath, boolean isJar); + public interface ResourcePathFilter { + boolean isCandidatePath(String resourcePath, boolean isJar); }; - public static interface FileNameFilter { - public boolean isCandidateFile(String fileName, String absFilePath); + public interface FileNameFilter { + boolean isCandidateFile(String fileName, String absFilePath); }; - public static interface ClassFilter { - public boolean isCandidateClass(Class c); + public interface ClassFilter { + boolean isCandidateClass(Class c); }; public ClassFinder() { Modified: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java (original) +++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java Wed Jul 10 17:54:35 2013 @@ -86,14 +86,14 @@ public final class Waiter { * {@link Waiter#waitFor(Configuration, long, long, boolean, Predicate) methods. */ @InterfaceAudience.Private - public static interface Predicate { + public interface Predicate { /** * Perform a predicate evaluation. * @return the boolean result of the evaluation. * @throws Exception thrown if the predicate evaluation could not evaluate. */ - public boolean evaluate() throws E; + boolean evaluate() throws E; } Modified: hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java (original) +++ hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java Wed Jul 10 17:54:35 2013 @@ -22,37 +22,37 @@ package org.apache.hadoop.hbase.ipc; import org.apache.hadoop.hbase.metrics.BaseSource; public interface MetricsHBaseServerSource extends BaseSource { - public static final String AUTHORIZATION_SUCCESSES_NAME = "authorizationSuccesses"; - public static final String AUTHORIZATION_SUCCESSES_DESC = + String AUTHORIZATION_SUCCESSES_NAME = "authorizationSuccesses"; + String AUTHORIZATION_SUCCESSES_DESC = "Number of authorization successes."; - public static final String AUTHORIZATION_FAILURES_NAME = "authorizationFailures"; - public static final String AUTHORIZATION_FAILURES_DESC = + String AUTHORIZATION_FAILURES_NAME = "authorizationFailures"; + String AUTHORIZATION_FAILURES_DESC = "Number of authorization failures."; - public static final String AUTHENTICATION_SUCCESSES_NAME = "authenticationSuccesses"; - public static final String AUTHENTICATION_SUCCESSES_DESC = + String AUTHENTICATION_SUCCESSES_NAME = "authenticationSuccesses"; + String AUTHENTICATION_SUCCESSES_DESC = "Number of authentication successes."; - public static final String AUTHENTICATION_FAILURES_NAME = "authenticationFailures"; - public static final String AUTHENTICATION_FAILURES_DESC = + String AUTHENTICATION_FAILURES_NAME = "authenticationFailures"; + String AUTHENTICATION_FAILURES_DESC = "Number of authentication failures."; - public static final String SENT_BYTES_NAME = "sentBytes"; - public static final String SENT_BYTES_DESC = "Number of bytes sent."; - public static final String RECEIVED_BYTES_NAME = "receivedBytes"; - public static final String RECEIVED_BYTES_DESC = "Number of bytes received."; - public static final String QUEUE_CALL_TIME_NAME = "queueCallTime"; - public static final String QUEUE_CALL_TIME_DESC = "Queue Call Time."; - public static final String PROCESS_CALL_TIME_NAME = "processCallTime"; - public static final String PROCESS_CALL_TIME_DESC = "Processing call time."; - public static final String QUEUE_SIZE_NAME = "queueSize"; - public static final String QUEUE_SIZE_DESC = "Number of bytes in the call queues."; - public static final String GENERAL_QUEUE_NAME = "numCallsInGeneralQueue"; - public static final String GENERAL_QUEUE_DESC = "Number of calls in the general call queue."; - public static final String PRIORITY_QUEUE_NAME = "numCallsInPriorityQueue"; - public static final String REPLICATION_QUEUE_NAME = "numCallsInReplicationQueue"; - public static final String REPLICATION_QUEUE_DESC = + String SENT_BYTES_NAME = "sentBytes"; + String SENT_BYTES_DESC = "Number of bytes sent."; + String RECEIVED_BYTES_NAME = "receivedBytes"; + String RECEIVED_BYTES_DESC = "Number of bytes received."; + String QUEUE_CALL_TIME_NAME = "queueCallTime"; + String QUEUE_CALL_TIME_DESC = "Queue Call Time."; + String PROCESS_CALL_TIME_NAME = "processCallTime"; + String PROCESS_CALL_TIME_DESC = "Processing call time."; + String QUEUE_SIZE_NAME = "queueSize"; + String QUEUE_SIZE_DESC = "Number of bytes in the call queues."; + String GENERAL_QUEUE_NAME = "numCallsInGeneralQueue"; + String GENERAL_QUEUE_DESC = "Number of calls in the general call queue."; + String PRIORITY_QUEUE_NAME = "numCallsInPriorityQueue"; + String REPLICATION_QUEUE_NAME = "numCallsInReplicationQueue"; + String REPLICATION_QUEUE_DESC = "Number of calls in the replication call queue."; - public static final String PRIORITY_QUEUE_DESC = "Number of calls in the priority call queue."; - public static final String NUM_OPEN_CONNECTIONS_NAME = "numOpenConnections"; - public static final String NUM_OPEN_CONNECTIONS_DESC = "Number of open connections."; + String PRIORITY_QUEUE_DESC = "Number of calls in the priority call queue."; + String NUM_OPEN_CONNECTIONS_NAME = "numOpenConnections"; + String NUM_OPEN_CONNECTIONS_DESC = "Number of open connections."; void authorizationSuccess(); Modified: hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java (original) +++ hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java Wed Jul 10 17:54:35 2013 @@ -28,60 +28,60 @@ public interface MetricsMasterSource ext /** * The name of the metrics */ - static final String METRICS_NAME = "Server"; + String METRICS_NAME = "Server"; /** * The context metrics will be under. */ - static final String METRICS_CONTEXT = "master"; + String METRICS_CONTEXT = "master"; /** * The name of the metrics context that metrics will be under in jmx */ - static final String METRICS_JMX_CONTEXT = "Master,sub=" + METRICS_NAME; + String METRICS_JMX_CONTEXT = "Master,sub=" + METRICS_NAME; /** * Description */ - static final String METRICS_DESCRIPTION = "Metrics about HBase master server"; + String METRICS_DESCRIPTION = "Metrics about HBase master server"; // Strings used for exporting to metrics system. - static final String MASTER_ACTIVE_TIME_NAME = "masterActiveTime"; - static final String MASTER_START_TIME_NAME = "masterStartTime"; - static final String AVERAGE_LOAD_NAME = "averageLoad"; - static final String NUM_REGION_SERVERS_NAME = "numRegionServers"; - static final String NUM_DEAD_REGION_SERVERS_NAME = "numDeadRegionServers"; - static final String ZOOKEEPER_QUORUM_NAME = "zookeeperQuorum"; - static final String SERVER_NAME_NAME = "serverName"; - static final String CLUSTER_ID_NAME = "clusterId"; - static final String IS_ACTIVE_MASTER_NAME = "isActiveMaster"; - static final String SPLIT_TIME_NAME = "hlogSplitTime"; - static final String SPLIT_SIZE_NAME = "hlogSplitSize"; - static final String SNAPSHOT_TIME_NAME = "snapshotTime"; - static final String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime"; - static final String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime"; - static final String META_SPLIT_TIME_NAME = "metaHlogSplitTime"; - static final String META_SPLIT_SIZE_NAME = "metaHlogSplitSize"; - static final String CLUSTER_REQUESTS_NAME = "clusterRequests"; - static final String RIT_COUNT_NAME = "ritCount"; - static final String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold"; - static final String RIT_OLDEST_AGE_NAME = "ritOldestAge"; - static final String MASTER_ACTIVE_TIME_DESC = "Master Active Time"; - static final String MASTER_START_TIME_DESC = "Master Start Time"; - static final String AVERAGE_LOAD_DESC = "AverageLoad"; - static final String NUMBER_OF_REGION_SERVERS_DESC = "Number of RegionServers"; - static final String NUMBER_OF_DEAD_REGION_SERVERS_DESC = "Number of dead RegionServers"; - static final String ZOOKEEPER_QUORUM_DESC = "Zookeeper Quorum"; - static final String SERVER_NAME_DESC = "Server Name"; - static final String CLUSTER_ID_DESC = "Cluster Id"; - static final String IS_ACTIVE_MASTER_DESC = "Is Active Master"; - static final String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()"; - static final String SPLIT_SIZE_DESC = "Size of HLog files being split"; - static final String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()"; - static final String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()"; - static final String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()"; - static final String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()"; - static final String META_SPLIT_SIZE_DESC = "Size of META HLog files being split"; + String MASTER_ACTIVE_TIME_NAME = "masterActiveTime"; + String MASTER_START_TIME_NAME = "masterStartTime"; + String AVERAGE_LOAD_NAME = "averageLoad"; + String NUM_REGION_SERVERS_NAME = "numRegionServers"; + String NUM_DEAD_REGION_SERVERS_NAME = "numDeadRegionServers"; + String ZOOKEEPER_QUORUM_NAME = "zookeeperQuorum"; + String SERVER_NAME_NAME = "serverName"; + String CLUSTER_ID_NAME = "clusterId"; + String IS_ACTIVE_MASTER_NAME = "isActiveMaster"; + String SPLIT_TIME_NAME = "hlogSplitTime"; + String SPLIT_SIZE_NAME = "hlogSplitSize"; + String SNAPSHOT_TIME_NAME = "snapshotTime"; + String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime"; + String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime"; + String META_SPLIT_TIME_NAME = "metaHlogSplitTime"; + String META_SPLIT_SIZE_NAME = "metaHlogSplitSize"; + String CLUSTER_REQUESTS_NAME = "clusterRequests"; + String RIT_COUNT_NAME = "ritCount"; + String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold"; + String RIT_OLDEST_AGE_NAME = "ritOldestAge"; + String MASTER_ACTIVE_TIME_DESC = "Master Active Time"; + String MASTER_START_TIME_DESC = "Master Start Time"; + String AVERAGE_LOAD_DESC = "AverageLoad"; + String NUMBER_OF_REGION_SERVERS_DESC = "Number of RegionServers"; + String NUMBER_OF_DEAD_REGION_SERVERS_DESC = "Number of dead RegionServers"; + String ZOOKEEPER_QUORUM_DESC = "Zookeeper Quorum"; + String SERVER_NAME_DESC = "Server Name"; + String CLUSTER_ID_DESC = "Cluster Id"; + String IS_ACTIVE_MASTER_DESC = "Is Active Master"; + String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()"; + String SPLIT_SIZE_DESC = "Size of HLog files being split"; + String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()"; + String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()"; + String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()"; + String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()"; + String META_SPLIT_SIZE_DESC = "Size of META HLog files being split"; /** * Increment the number of requests the cluster has seen. Modified: hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java (original) +++ hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java Wed Jul 10 17:54:35 2013 @@ -24,7 +24,7 @@ package org.apache.hadoop.hbase.metrics; */ public interface BaseSource { - public static final String HBASE_METRICS_SYSTEM_NAME = "HBase"; + String HBASE_METRICS_SYSTEM_NAME = "HBase"; /** * Clear out the metrics and re-prepare the source. Modified: hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSource.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSource.java?rev=1501881&r1=1501880&r2=1501881&view=diff ============================================================================== --- hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSource.java (original) +++ hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSource.java Wed Jul 10 17:54:35 2013 @@ -29,22 +29,22 @@ public interface MetricsRegionAggregateS /** * The name of the metrics */ - static final String METRICS_NAME = "Regions"; + String METRICS_NAME = "Regions"; /** * The name of the metrics context that metrics will be under. */ - static final String METRICS_CONTEXT = "regionserver"; + String METRICS_CONTEXT = "regionserver"; /** * Description */ - static final String METRICS_DESCRIPTION = "Metrics about HBase RegionServer regions and tables"; + String METRICS_DESCRIPTION = "Metrics about HBase RegionServer regions and tables"; /** * The name of the metrics context that metrics will be under in jmx */ - static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; + String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; /** * Register a MetricsRegionSource as being open.