You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2014/09/24 05:18:26 UTC

[6/6] git commit: HBASE-12059 Create hbase-annotations module

HBASE-12059 Create hbase-annotations module


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c64e0891
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c64e0891
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c64e0891

Branch: refs/heads/0.98
Commit: c64e08919d5d29ec3b15ae8f18d5cf5640022370
Parents: 3897a02
Author: Elliott Clark <ec...@apache.org>
Authored: Mon Sep 22 18:46:35 2014 -0700
Committer: Elliott Clark <ec...@apache.org>
Committed: Tue Sep 23 20:16:57 2014 -0700

----------------------------------------------------------------------
 hbase-annotations/pom.xml                       |  43 ++++
 .../apache/hadoop/hbase/IntegrationTests.java   |  39 +++
 .../org/apache/hadoop/hbase/LargeTests.java     |  38 +++
 .../org/apache/hadoop/hbase/MediumTests.java    |  37 +++
 .../org/apache/hadoop/hbase/SmallTests.java     |  34 +++
 .../hbase/classification/InterfaceAudience.java |  73 ++++++
 .../classification/InterfaceStability.java      |  65 +++++
 ...ExcludePrivateAnnotationsStandardDoclet.java |  58 +++++
 .../IncludePublicAnnotationsStandardDoclet.java |  63 +++++
 .../classification/tools/RootDocProcessor.java  | 247 +++++++++++++++++++
 .../classification/tools/StabilityOptions.java  |  69 ++++++
 hbase-client/pom.xml                            |   8 +-
 .../hadoop/hbase/ClockOutOfSyncException.java   |   2 +-
 .../org/apache/hadoop/hbase/ClusterStatus.java  |   2 +-
 .../hadoop/hbase/DoNotRetryIOException.java     |   4 +-
 .../hadoop/hbase/DroppedSnapshotException.java  |   2 +-
 .../apache/hadoop/hbase/HColumnDescriptor.java  |   2 +-
 .../org/apache/hadoop/hbase/HRegionInfo.java    |   2 +-
 .../apache/hadoop/hbase/HRegionLocation.java    |   2 +-
 .../apache/hadoop/hbase/HTableDescriptor.java   |   2 +-
 .../hbase/InvalidFamilyOperationException.java  |   2 +-
 .../hadoop/hbase/MasterNotRunningException.java |   2 +-
 .../hadoop/hbase/NamespaceExistException.java   |   2 +-
 .../hbase/NamespaceNotFoundException.java       |   2 +-
 .../hbase/NotAllMetaRegionsOnlineException.java |   2 +-
 .../hadoop/hbase/NotServingRegionException.java |   2 +-
 .../hadoop/hbase/PleaseHoldException.java       |   2 +-
 .../apache/hadoop/hbase/RegionException.java    |   2 +-
 .../org/apache/hadoop/hbase/RegionLoad.java     |   2 +-
 .../hadoop/hbase/RegionTooBusyException.java    |   2 +-
 .../org/apache/hadoop/hbase/ServerLoad.java     |   2 +-
 .../org/apache/hadoop/hbase/ServerName.java     |   2 +-
 .../hadoop/hbase/TableExistsException.java      |   2 +-
 .../hadoop/hbase/TableInfoMissingException.java |   2 +-
 .../hadoop/hbase/TableNotDisabledException.java |   2 +-
 .../hadoop/hbase/TableNotEnabledException.java  |   2 +-
 .../hadoop/hbase/TableNotFoundException.java    |   2 +-
 .../hadoop/hbase/UnknownRegionException.java    |   2 +-
 .../hadoop/hbase/UnknownScannerException.java   |   2 +-
 .../hadoop/hbase/YouAreDeadException.java       |   2 +-
 .../hbase/ZooKeeperConnectionException.java     |   2 +-
 .../org/apache/hadoop/hbase/client/Action.java  |   2 +-
 .../org/apache/hadoop/hbase/client/Append.java  |   2 +-
 .../apache/hadoop/hbase/client/Attributes.java  |   2 +-
 .../client/ClientSmallReversedScanner.java      |   2 +-
 .../client/DelegatingRetryingCallable.java      |   2 +-
 .../org/apache/hadoop/hbase/client/Delete.java  |   2 +-
 .../apache/hadoop/hbase/client/Durability.java  |   2 +-
 .../org/apache/hadoop/hbase/client/Get.java     |   2 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   2 +-
 .../hadoop/hbase/client/HConnectable.java       |   2 +-
 .../apache/hadoop/hbase/client/HConnection.java |   4 +-
 .../hadoop/hbase/client/HConnectionKey.java     |   2 +-
 .../hadoop/hbase/client/HConnectionManager.java |   2 +-
 .../org/apache/hadoop/hbase/client/HTable.java  |   2 +-
 .../hadoop/hbase/client/HTableFactory.java      |   2 +-
 .../hbase/client/HTableInterfaceFactory.java    |   2 +-
 .../hadoop/hbase/client/HTableMultiplexer.java  |   2 +-
 .../apache/hadoop/hbase/client/Increment.java   |   2 +-
 .../hadoop/hbase/client/IsolationLevel.java     |   2 +-
 .../hbase/client/MasterKeepAliveConnection.java |   2 +-
 .../apache/hadoop/hbase/client/MultiAction.java |   2 +-
 .../hadoop/hbase/client/MultiResponse.java      |   2 +-
 .../apache/hadoop/hbase/client/Mutation.java    |   2 +-
 .../client/NoServerForRegionException.java      |   2 +-
 .../apache/hadoop/hbase/client/Operation.java   |   2 +-
 .../hbase/client/OperationWithAttributes.java   |   2 +-
 .../org/apache/hadoop/hbase/client/Put.java     |   2 +-
 .../org/apache/hadoop/hbase/client/Query.java   |   2 +-
 .../hbase/client/RegionOfflineException.java    |   2 +-
 .../org/apache/hadoop/hbase/client/Result.java  |   1 +
 .../hadoop/hbase/client/ResultScanner.java      |   2 +-
 .../hbase/client/RetriesExhaustedException.java |   2 +-
 .../RetriesExhaustedWithDetailsException.java   |   2 +-
 .../hadoop/hbase/client/RetryingCallable.java   |   2 +-
 .../hbase/client/ReversedScannerCallable.java   |   2 +-
 .../org/apache/hadoop/hbase/client/Row.java     |   2 +-
 .../hadoop/hbase/client/RowMutations.java       |   4 +-
 .../org/apache/hadoop/hbase/client/Scan.java    |   2 +-
 .../hbase/client/ScannerTimeoutException.java   |   2 +-
 .../hbase/client/UnmodifyableHRegionInfo.java   |   2 +-
 .../client/UnmodifyableHTableDescriptor.java    |   2 +-
 .../hbase/client/WrongRowIOException.java       |   4 +-
 .../hadoop/hbase/client/coprocessor/Batch.java  |   2 +-
 .../coprocessor/DoubleColumnInterpreter.java    |   2 +-
 .../client/replication/ReplicationAdmin.java    |   2 +-
 .../hbase/coprocessor/CoprocessorException.java |   2 +-
 .../hbase/exceptions/LockTimeoutException.java  |   2 +-
 .../hbase/exceptions/MergeRegionException.java  |   2 +-
 .../exceptions/OperationConflictException.java  |   2 +-
 .../exceptions/RegionInRecoveryException.java   |   2 +-
 .../hbase/exceptions/RegionMovedException.java  |   2 +-
 .../exceptions/RegionOpeningException.java      |   4 +-
 .../exceptions/UnknownProtocolException.java    |   2 +-
 .../hadoop/hbase/executor/ExecutorType.java     |   2 +-
 .../hadoop/hbase/filter/BinaryComparator.java   |   2 +-
 .../hbase/filter/BinaryPrefixComparator.java    |   2 +-
 .../hadoop/hbase/filter/BitComparator.java      |   2 +-
 .../hbase/filter/ByteArrayComparable.java       |   2 +-
 .../hbase/filter/ColumnCountGetFilter.java      |   2 +-
 .../hbase/filter/ColumnPaginationFilter.java    |   2 +-
 .../hadoop/hbase/filter/ColumnPrefixFilter.java |   2 +-
 .../hadoop/hbase/filter/ColumnRangeFilter.java  |   2 +-
 .../hadoop/hbase/filter/CompareFilter.java      |   2 +-
 .../hbase/filter/DependentColumnFilter.java     |   2 +-
 .../hadoop/hbase/filter/FamilyFilter.java       |   2 +-
 .../org/apache/hadoop/hbase/filter/Filter.java  |   2 +-
 .../apache/hadoop/hbase/filter/FilterList.java  |   2 +-
 .../hadoop/hbase/filter/FirstKeyOnlyFilter.java |   2 +-
 .../FirstKeyValueMatchingQualifiersFilter.java  |   2 +-
 .../hadoop/hbase/filter/FuzzyRowFilter.java     |   2 +-
 .../hbase/filter/InclusiveStopFilter.java       |   2 +-
 .../filter/IncompatibleFilterException.java     |   2 +-
 .../hbase/filter/InvalidRowFilterException.java |   2 +-
 .../hadoop/hbase/filter/KeyOnlyFilter.java      |   2 +-
 .../filter/MultipleColumnPrefixFilter.java      |   2 +-
 .../hadoop/hbase/filter/NullComparator.java     |   2 +-
 .../apache/hadoop/hbase/filter/PageFilter.java  |   2 +-
 .../hadoop/hbase/filter/ParseConstants.java     |   2 +-
 .../apache/hadoop/hbase/filter/ParseFilter.java |   2 +-
 .../hadoop/hbase/filter/PrefixFilter.java       |   2 +-
 .../hadoop/hbase/filter/QualifierFilter.java    |   2 +-
 .../hadoop/hbase/filter/RandomRowFilter.java    |   2 +-
 .../hbase/filter/RegexStringComparator.java     |   2 +-
 .../apache/hadoop/hbase/filter/RowFilter.java   |   2 +-
 .../filter/SingleColumnValueExcludeFilter.java  |   2 +-
 .../hbase/filter/SingleColumnValueFilter.java   |   2 +-
 .../apache/hadoop/hbase/filter/SkipFilter.java  |   4 +-
 .../hbase/filter/SubstringComparator.java       |   2 +-
 .../hadoop/hbase/filter/TimestampsFilter.java   |   2 +-
 .../apache/hadoop/hbase/filter/ValueFilter.java |   2 +-
 .../hadoop/hbase/filter/WhileMatchFilter.java   |   2 +-
 .../hadoop/hbase/ipc/BadAuthException.java      |   4 +-
 .../hadoop/hbase/ipc/BlockingRpcCallback.java   |   2 +-
 .../hbase/ipc/CallerDisconnectedException.java  |   2 +-
 .../hadoop/hbase/ipc/CoprocessorRpcChannel.java |   2 +-
 .../DelegatingPayloadCarryingRpcController.java |   2 +-
 .../hbase/ipc/FatalConnectionException.java     |   4 +-
 .../org/apache/hadoop/hbase/ipc/IPCUtil.java    |   2 +-
 .../hbase/ipc/PayloadCarryingRpcController.java |   2 +-
 .../hbase/ipc/RemoteWithExtrasException.java    |   4 +-
 .../org/apache/hadoop/hbase/ipc/RpcClient.java  |   2 +-
 .../hadoop/hbase/ipc/RpcControllerFactory.java  |   2 +-
 .../hbase/ipc/ServerNotRunningYetException.java |   2 +-
 .../hbase/ipc/StoppedRpcClientException.java    |   2 +-
 .../ipc/UnsupportedCellCodecException.java      |   4 +-
 .../UnsupportedCompressionCodecException.java   |   4 +-
 .../hadoop/hbase/ipc/WrongVersionException.java |   4 +-
 .../apache/hadoop/hbase/master/RegionState.java |   2 +-
 .../hadoop/hbase/regionserver/BloomType.java    |   4 +-
 .../hbase/regionserver/LeaseException.java      |   2 +-
 .../NoSuchColumnFamilyException.java            |   2 +-
 .../RegionAlreadyInTransitionException.java     |   2 +-
 .../RegionServerRunningException.java           |   2 +-
 .../RegionServerStoppedException.java           |   4 +-
 .../regionserver/WrongRegionException.java      |   2 +-
 .../wal/FailedLogCloseException.java            |   2 +-
 .../hbase/replication/ReplicationFactory.java   |   2 +-
 .../hbase/replication/ReplicationListener.java  |   2 +-
 .../ReplicationQueuesClientZKImpl.java          |   2 +-
 .../replication/ReplicationQueuesZKImpl.java    |   2 +-
 .../replication/ReplicationStateZKBase.java     |   2 +-
 .../replication/ReplicationTrackerZKImpl.java   |   2 +-
 .../hbase/security/AccessDeniedException.java   |   2 +-
 .../hadoop/hbase/security/EncryptionUtil.java   |   2 +-
 .../security/access/AccessControlClient.java    |   2 +-
 .../security/access/AccessControlConstants.java |   2 +-
 .../hbase/security/access/Permission.java       |   2 +-
 .../security/visibility/Authorizations.java     |   4 +-
 .../security/visibility/CellVisibility.java     |   2 +-
 .../visibility/InvalidLabelException.java       |   2 +-
 .../visibility/LabelAlreadyExistsException.java |   2 +-
 .../security/visibility/VisibilityClient.java   |   2 +-
 .../snapshot/CorruptedSnapshotException.java    |   2 +-
 .../hbase/snapshot/HBaseSnapshotException.java  |   2 +-
 .../snapshot/RestoreSnapshotException.java      |   2 +-
 .../snapshot/SnapshotCreationException.java     |   2 +-
 .../snapshot/SnapshotDoesNotExistException.java |   2 +-
 .../hbase/snapshot/SnapshotExistsException.java |   4 +-
 .../snapshot/TablePartiallyOpenException.java   |   2 +-
 .../snapshot/UnknownSnapshotException.java      |   4 +-
 .../hbase/util/FileSystemVersionException.java  |   2 +-
 .../apache/hadoop/hbase/util/JsonMapper.java    |   2 +-
 .../hadoop/hbase/zookeeper/HQuorumPeer.java     |   2 +-
 hbase-common/pom.xml                            |  12 +-
 .../main/java/org/apache/hadoop/hbase/Cell.java |   2 +-
 .../org/apache/hadoop/hbase/CellComparator.java |   2 +-
 .../org/apache/hadoop/hbase/CellScannable.java  |   2 +-
 .../org/apache/hadoop/hbase/CellScanner.java    |   4 +-
 .../java/org/apache/hadoop/hbase/CellUtil.java  |   2 +-
 .../apache/hadoop/hbase/HBaseConfiguration.java |   2 +-
 .../apache/hadoop/hbase/HBaseIOException.java   |   2 +-
 .../hadoop/hbase/HBaseInterfaceAudience.java    |   2 +-
 .../org/apache/hadoop/hbase/HConstants.java     |   2 +-
 .../hadoop/hbase/NamespaceDescriptor.java       |   2 +-
 .../java/org/apache/hadoop/hbase/TableName.java |   2 +-
 .../main/java/org/apache/hadoop/hbase/Tag.java  |   2 +-
 .../java/org/apache/hadoop/hbase/TagType.java   |   2 +-
 .../apache/hadoop/hbase/codec/BaseEncoder.java  |   2 +-
 .../hadoop/hbase/codec/CodecException.java      |   2 +-
 .../hadoop/hbase/io/ByteBufferOutputStream.java |   2 +-
 .../hadoop/hbase/io/CellOutputStream.java       |   4 +-
 .../hadoop/hbase/io/ImmutableBytesWritable.java |   2 +-
 .../hadoop/hbase/io/SizedCellScanner.java       |   4 +-
 .../hadoop/hbase/io/TagCompressionContext.java  |   2 +-
 .../org/apache/hadoop/hbase/io/TimeRange.java   |   4 +-
 .../hadoop/hbase/io/compress/Compression.java   |   2 +-
 .../apache/hadoop/hbase/io/crypto/Cipher.java   |   2 +-
 .../hadoop/hbase/io/crypto/CipherProvider.java  |   2 +-
 .../apache/hadoop/hbase/io/crypto/Context.java  |   2 +-
 .../hadoop/hbase/io/crypto/Decryptor.java       |   2 +-
 .../hbase/io/crypto/DefaultCipherProvider.java  |   2 +-
 .../hadoop/hbase/io/crypto/Encryption.java      |   2 +-
 .../hadoop/hbase/io/crypto/Encryptor.java       |   2 +-
 .../hadoop/hbase/io/crypto/KeyProvider.java     |   2 +-
 .../hbase/io/crypto/KeyStoreKeyProvider.java    |   2 +-
 .../apache/hadoop/hbase/io/crypto/aes/AES.java  |   2 +-
 .../hbase/io/crypto/aes/AESDecryptor.java       |   2 +-
 .../hbase/io/crypto/aes/AESEncryptor.java       |   2 +-
 .../hbase/io/encoding/DataBlockEncoding.java    |   2 +-
 .../hadoop/hbase/io/util/StreamUtils.java       |   2 +-
 .../hadoop/hbase/security/UserProvider.java     |   2 +-
 .../org/apache/hadoop/hbase/types/DataType.java |   2 +-
 .../hadoop/hbase/types/FixedLengthWrapper.java  |   2 +-
 .../apache/hadoop/hbase/types/OrderedBlob.java  |   2 +-
 .../hadoop/hbase/types/OrderedBlobVar.java      |   2 +-
 .../hadoop/hbase/types/OrderedBytesBase.java    |   2 +-
 .../hadoop/hbase/types/OrderedFloat32.java      |   2 +-
 .../hadoop/hbase/types/OrderedFloat64.java      |   2 +-
 .../apache/hadoop/hbase/types/OrderedInt16.java |   2 +-
 .../apache/hadoop/hbase/types/OrderedInt32.java |   2 +-
 .../apache/hadoop/hbase/types/OrderedInt64.java |   2 +-
 .../apache/hadoop/hbase/types/OrderedInt8.java  |   2 +-
 .../hadoop/hbase/types/OrderedNumeric.java      |   2 +-
 .../hadoop/hbase/types/OrderedString.java       |   2 +-
 .../org/apache/hadoop/hbase/types/RawByte.java  |   2 +-
 .../org/apache/hadoop/hbase/types/RawBytes.java |   2 +-
 .../hadoop/hbase/types/RawBytesFixedLength.java |   2 +-
 .../hadoop/hbase/types/RawBytesTerminated.java  |   2 +-
 .../apache/hadoop/hbase/types/RawDouble.java    |   2 +-
 .../org/apache/hadoop/hbase/types/RawFloat.java |   2 +-
 .../apache/hadoop/hbase/types/RawInteger.java   |   2 +-
 .../org/apache/hadoop/hbase/types/RawLong.java  |   2 +-
 .../org/apache/hadoop/hbase/types/RawShort.java |   2 +-
 .../apache/hadoop/hbase/types/RawString.java    |   2 +-
 .../hbase/types/RawStringFixedLength.java       |   2 +-
 .../hadoop/hbase/types/RawStringTerminated.java |   2 +-
 .../org/apache/hadoop/hbase/types/Struct.java   |   2 +-
 .../hadoop/hbase/types/StructBuilder.java       |   2 +-
 .../hadoop/hbase/types/StructIterator.java      |   2 +-
 .../hadoop/hbase/types/TerminatedWrapper.java   |   2 +-
 .../org/apache/hadoop/hbase/types/Union2.java   |   2 +-
 .../org/apache/hadoop/hbase/types/Union3.java   |   2 +-
 .../org/apache/hadoop/hbase/types/Union4.java   |   2 +-
 .../org/apache/hadoop/hbase/util/Base64.java    |   2 +-
 .../hadoop/hbase/util/ByteBufferUtils.java      |   2 +-
 .../org/apache/hadoop/hbase/util/ByteRange.java |   4 +-
 .../hadoop/hbase/util/ByteRangeUtils.java       |   2 +-
 .../org/apache/hadoop/hbase/util/Bytes.java     |   2 +-
 .../hadoop/hbase/util/ConcurrentIndex.java      |   4 +-
 .../java/org/apache/hadoop/hbase/util/Hash.java |   2 +-
 .../apache/hadoop/hbase/util/JenkinsHash.java   |   2 +-
 .../org/apache/hadoop/hbase/util/MD5Hash.java   |   2 +-
 .../apache/hadoop/hbase/util/MurmurHash.java    |   2 +-
 .../org/apache/hadoop/hbase/util/Order.java     |   2 +-
 .../apache/hadoop/hbase/util/OrderedBytes.java  |   2 +-
 .../java/org/apache/hadoop/hbase/util/Pair.java |   2 +-
 .../hadoop/hbase/util/PairOfSameType.java       |   2 +-
 .../hadoop/hbase/util/PositionedByteRange.java  |   2 +-
 .../apache/hadoop/hbase/util/PrettyPrinter.java |   2 +-
 .../hbase/util/SimplePositionedByteRange.java   |   2 +-
 .../apache/hadoop/hbase/util/VersionInfo.java   |   2 +-
 .../apache/hadoop/hbase/IntegrationTests.java   |  39 ---
 .../org/apache/hadoop/hbase/LargeTests.java     |  38 ---
 .../org/apache/hadoop/hbase/MediumTests.java    |  37 ---
 .../org/apache/hadoop/hbase/SmallTests.java     |  34 ---
 hbase-hadoop1-compat/pom.xml                    |   4 +
 .../apache/hadoop/hbase/mapreduce/JobUtil.java  |   6 +-
 hbase-hadoop2-compat/pom.xml                    |   9 +-
 .../apache/hadoop/hbase/mapreduce/JobUtil.java  |   4 +-
 .../hadoop/metrics2/util/MetricQuantile.java    |   2 +-
 hbase-it/pom.xml                                |   8 -
 hbase-prefix-tree/pom.xml                       |   8 -
 .../prefixtree/decode/ArraySearcherPool.java    |   2 +-
 .../codec/prefixtree/encode/EncoderPool.java    |   2 +-
 .../scanner/ReversibleCellScanner.java          |   2 +-
 .../util/byterange/impl/ByteRangeHashSet.java   |   2 +-
 .../util/byterange/impl/ByteRangeTreeSet.java   |   2 +-
 .../protobuf/HBaseZeroCopyByteString.java       |   2 +-
 hbase-server/pom.xml                            |   8 -
 .../org/apache/hadoop/hbase/HealthReport.java   |   2 +-
 .../apache/hadoop/hbase/LocalHBaseCluster.java  |   2 +-
 .../hbase/client/CoprocessorHConnection.java    |   4 +-
 .../hbase/client/TableSnapshotScanner.java      |   2 +-
 .../client/coprocessor/RowProcessorClient.java  |   2 +-
 .../BaseMasterAndRegionObserver.java            |   2 +-
 .../hbase/coprocessor/BaseMasterObserver.java   |   2 +-
 .../hbase/coprocessor/BaseRegionObserver.java   |   2 +-
 .../coprocessor/BaseRegionServerObserver.java   |   2 +-
 .../coprocessor/BaseRowProcessorEndpoint.java   |   2 +-
 .../hbase/coprocessor/BaseWALObserver.java      |   2 +-
 .../hbase/coprocessor/CoprocessorHost.java      |   2 +-
 .../hbase/coprocessor/CoprocessorService.java   |   2 +-
 .../hbase/coprocessor/EndpointObserver.java     |   2 +-
 .../MasterCoprocessorEnvironment.java           |   2 +-
 .../hbase/coprocessor/MasterObserver.java       |   2 +-
 .../coprocessor/MultiRowMutationEndpoint.java   |   2 +-
 .../hbase/coprocessor/ObserverContext.java      |   2 +-
 .../RegionCoprocessorEnvironment.java           |   2 +-
 .../hbase/coprocessor/RegionObserver.java       |   2 +-
 .../coprocessor/WALCoprocessorEnvironment.java  |   2 +-
 .../hadoop/hbase/coprocessor/WALObserver.java   |   2 +-
 .../hbase/errorhandling/ForeignException.java   |   4 +-
 .../ForeignExceptionDispatcher.java             |   2 +-
 .../errorhandling/ForeignExceptionListener.java |   2 +-
 .../errorhandling/ForeignExceptionSnare.java    |   2 +-
 .../hbase/errorhandling/TimeoutException.java   |   4 +-
 .../errorhandling/TimeoutExceptionInjector.java |   4 +-
 .../hadoop/hbase/io/DataInputInputStream.java   |   2 +-
 .../hbase/io/FSDataInputStreamWrapper.java      |   2 +-
 .../hadoop/hbase/io/hfile/BlockCacheKey.java    |   2 +-
 .../hbase/io/hfile/BlockCachesIterator.java     |   2 +-
 .../hadoop/hbase/io/hfile/BlockPriority.java    |   2 +-
 .../hadoop/hbase/io/hfile/CachedBlock.java      |   2 +-
 .../hbase/io/hfile/CombinedBlockCache.java      |   3 +-
 .../hbase/io/hfile/HFilePrettyPrinter.java      |   2 +-
 .../hfile/bucket/BucketAllocatorException.java  |   2 +-
 .../hbase/ipc/BalancedQueueRpcExecutor.java     |   2 +-
 .../apache/hadoop/hbase/ipc/BufferChain.java    |   2 +-
 .../hbase/ipc/EmptyServiceNameException.java    |   2 +-
 .../hadoop/hbase/ipc/MetricsHBaseServer.java    |   2 +-
 .../hadoop/hbase/ipc/PriorityFunction.java      |   2 +-
 .../hadoop/hbase/ipc/RWQueueRpcExecutor.java    |   2 +-
 .../apache/hadoop/hbase/ipc/RequestContext.java |   2 +-
 .../apache/hadoop/hbase/ipc/RpcCallContext.java |   2 +-
 .../apache/hadoop/hbase/ipc/RpcExecutor.java    |   2 +-
 .../apache/hadoop/hbase/ipc/RpcScheduler.java   |   2 +-
 .../hadoop/hbase/ipc/RpcSchedulerContext.java   |   2 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |   2 +-
 .../hadoop/hbase/ipc/RpcServerInterface.java    |   2 +-
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java    |   2 +-
 .../hbase/ipc/UnknownServiceException.java      |   2 +-
 .../org/apache/hadoop/hbase/mapred/Driver.java  |   2 +-
 .../hadoop/hbase/mapred/GroupingTableMap.java   |   2 +-
 .../hadoop/hbase/mapred/HRegionPartitioner.java |   2 +-
 .../hadoop/hbase/mapred/IdentityTableMap.java   |   2 +-
 .../hbase/mapred/IdentityTableReduce.java       |   2 +-
 .../apache/hadoop/hbase/mapred/RowCounter.java  |   2 +-
 .../hadoop/hbase/mapred/TableInputFormat.java   |   2 +-
 .../hbase/mapred/TableInputFormatBase.java      |   2 +-
 .../apache/hadoop/hbase/mapred/TableMap.java    |   2 +-
 .../hadoop/hbase/mapred/TableMapReduceUtil.java |   2 +-
 .../hadoop/hbase/mapred/TableOutputFormat.java  |   2 +-
 .../hadoop/hbase/mapred/TableRecordReader.java  |   2 +-
 .../hbase/mapred/TableRecordReaderImpl.java     |   2 +-
 .../apache/hadoop/hbase/mapred/TableReduce.java |   2 +-
 .../hbase/mapred/TableSnapshotInputFormat.java  |   2 +-
 .../apache/hadoop/hbase/mapred/TableSplit.java  |   2 +-
 .../hadoop/hbase/mapreduce/CellCounter.java     |   2 +-
 .../hadoop/hbase/mapreduce/CellCreator.java     |   2 +-
 .../hadoop/hbase/mapreduce/CopyTable.java       |   2 +-
 .../DefaultVisibilityExpressionResolver.java    |   2 +-
 .../apache/hadoop/hbase/mapreduce/Driver.java   |   2 +-
 .../apache/hadoop/hbase/mapreduce/Export.java   |   2 +-
 .../hbase/mapreduce/GroupingTableMapper.java    |   2 +-
 .../hbase/mapreduce/HFileOutputFormat.java      |   2 +-
 .../hbase/mapreduce/HFileOutputFormat2.java     |   2 +-
 .../hbase/mapreduce/HRegionPartitioner.java     |   2 +-
 .../hbase/mapreduce/IdentityTableMapper.java    |   2 +-
 .../hbase/mapreduce/IdentityTableReducer.java   |   2 +-
 .../apache/hadoop/hbase/mapreduce/Import.java   |   2 +-
 .../hadoop/hbase/mapreduce/ImportTsv.java       |   2 +-
 .../hbase/mapreduce/KeyValueSerialization.java  |   2 +-
 .../hbase/mapreduce/KeyValueSortReducer.java    |   4 +-
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |   2 +-
 .../hbase/mapreduce/MultiTableInputFormat.java  |   2 +-
 .../mapreduce/MultiTableInputFormatBase.java    |   2 +-
 .../hbase/mapreduce/MultiTableOutputFormat.java |   2 +-
 .../hadoop/hbase/mapreduce/PutCombiner.java     |   2 +-
 .../hadoop/hbase/mapreduce/PutSortReducer.java  |   2 +-
 .../hbase/mapreduce/ResultSerialization.java    |   2 +-
 .../hadoop/hbase/mapreduce/RowCounter.java      |   2 +-
 .../mapreduce/SimpleTotalOrderPartitioner.java  |   2 +-
 .../hbase/mapreduce/TableInputFormat.java       |   2 +-
 .../hbase/mapreduce/TableInputFormatBase.java   |   2 +-
 .../hbase/mapreduce/TableMapReduceUtil.java     |   2 +-
 .../hadoop/hbase/mapreduce/TableMapper.java     |   2 +-
 .../hbase/mapreduce/TableOutputCommitter.java   |   2 +-
 .../hbase/mapreduce/TableOutputFormat.java      |   2 +-
 .../hbase/mapreduce/TableRecordReader.java      |   2 +-
 .../hbase/mapreduce/TableRecordReaderImpl.java  |   2 +-
 .../hadoop/hbase/mapreduce/TableReducer.java    |   2 +-
 .../mapreduce/TableSnapshotInputFormat.java     |   2 +-
 .../mapreduce/TableSnapshotInputFormatImpl.java |   2 +-
 .../hadoop/hbase/mapreduce/TableSplit.java      |   2 +-
 .../hadoop/hbase/mapreduce/TextSortReducer.java |   2 +-
 .../hbase/mapreduce/TsvImporterMapper.java      |   2 +-
 .../hbase/mapreduce/TsvImporterTextMapper.java  |   2 +-
 .../mapreduce/VisibilityExpressionResolver.java |   4 +-
 .../hadoop/hbase/mapreduce/WALPlayer.java       |   2 +-
 .../hadoop/hbase/master/MetricsMaster.java      |   2 +-
 .../apache/hadoop/hbase/master/RegionPlan.java  |   2 +-
 .../hadoop/hbase/master/SnapshotSentinel.java   |   2 +-
 .../handler/DispatchMergingRegionHandler.java   |   2 +-
 .../handler/MetaServerShutdownHandler.java      |   2 +-
 .../snapshot/DisabledTableSnapshotHandler.java  |   2 +-
 .../snapshot/EnabledTableSnapshotHandler.java   |   2 +-
 .../master/snapshot/MasterSnapshotVerifier.java |   2 +-
 .../master/snapshot/SnapshotFileCache.java      |   2 +-
 .../master/snapshot/SnapshotHFileCleaner.java   |   2 +-
 .../master/snapshot/SnapshotLogCleaner.java     |   2 +-
 .../hbase/master/snapshot/SnapshotManager.java  |   2 +-
 .../monitoring/MonitoredRPCHandlerImpl.java     |   2 +-
 .../hbase/monitoring/StateDumpServlet.java      |   2 +-
 .../hbase/procedure/MasterProcedureManager.java |   2 +-
 .../hbase/procedure/ProcedureManager.java       |   2 +-
 .../hbase/procedure/ProcedureManagerHost.java   |   2 +-
 .../hbase/procedure/ProcedureMemberRpcs.java    |   2 +-
 .../procedure/RegionServerProcedureManager.java |   2 +-
 .../procedure/ZKProcedureCoordinatorRpcs.java   |   4 +-
 .../hadoop/hbase/procedure/ZKProcedureUtil.java |   2 +-
 .../hbase/regionserver/BaseRowProcessor.java    |   2 +-
 .../hbase/regionserver/MemStoreChunkPool.java   |   2 +-
 .../hbase/regionserver/MetaLogRoller.java       |   2 +-
 .../hbase/regionserver/MetricsRegionServer.java |   2 +-
 .../regionserver/RegionCoprocessorHost.java     |   2 +-
 .../RegionServerCoprocessorHost.java            |   2 +-
 .../regionserver/ReversedRegionScannerImpl.java |   2 +-
 .../hadoop/hbase/regionserver/RowProcessor.java |   2 +-
 .../hbase/regionserver/RpcSchedulerFactory.java |   2 +-
 .../regionserver/SimpleRpcSchedulerFactory.java |   2 +-
 .../apache/hadoop/hbase/regionserver/Store.java |   2 +-
 .../regionserver/StoreConfigInformation.java    |   2 +-
 .../compactions/CompactionRequest.java          |   2 +-
 .../snapshot/FlushSnapshotSubprocedure.java     |   2 +-
 .../snapshot/RegionServerSnapshotManager.java   |   2 +-
 .../regionserver/wal/HLogPrettyPrinter.java     |   2 +-
 .../regionserver/wal/WALActionsListener.java    |   1 -
 .../org/apache/hadoop/hbase/rest/Constants.java |   2 +-
 .../apache/hadoop/hbase/rest/client/Client.java |   2 +-
 .../hadoop/hbase/rest/client/Cluster.java       |   2 +-
 .../hadoop/hbase/rest/client/RemoteAdmin.java   |   2 +-
 .../hadoop/hbase/rest/client/RemoteHTable.java  |   2 +-
 .../hadoop/hbase/rest/client/Response.java      |   2 +-
 .../hadoop/hbase/security/SecurityUtil.java     |   2 +-
 .../hbase/security/access/AuthResult.java       |   2 +-
 .../HbaseObjectWritableFor96Migration.java      |   2 +-
 .../hbase/security/token/FsDelegationToken.java |   2 +-
 .../security/visibility/ExpressionExpander.java |   2 +-
 .../security/visibility/ScanLabelGenerator.java |   2 +-
 .../visibility/VisibilityExpEvaluator.java      |   2 +-
 .../visibility/VisibilityLabelFilter.java       |   2 +-
 .../visibility/VisibilityLabelService.java      |   2 +-
 .../hadoop/hbase/snapshot/ExportSnapshot.java   |   2 +-
 .../hadoop/hbase/snapshot/SnapshotInfo.java     |   2 +-
 .../hbase/snapshot/SnapshotManifestV1.java      |   2 +-
 .../hbase/util/CompoundBloomFilterWriter.java   |   2 +-
 .../hadoop/hbase/util/CompressionTest.java      |   2 +-
 .../apache/hadoop/hbase/util/ConfigUtil.java    |   5 +-
 .../hadoop/hbase/util/DirectMemoryUtils.java    |   2 +-
 .../apache/hadoop/hbase/util/FSHDFSUtils.java   |   2 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   2 +-
 .../apache/hadoop/hbase/util/HashedBytes.java   |   2 +-
 .../hadoop/hbase/util/HttpServerUtil.java       |   2 +-
 .../hadoop/hbase/util/JvmPauseMonitor.java      |   2 +-
 .../apache/hadoop/hbase/util/JvmVersion.java    |   2 +-
 .../org/apache/hadoop/hbase/util/Merge.java     |   2 +-
 .../hadoop/hbase/util/RegionSizeCalculator.java |   2 +-
 .../hadoop/hbase/util/SortedCopyOnWriteSet.java |   2 +-
 .../hbase/util/hbck/OfflineMetaRepair.java      |   2 +-
 .../hbase/zookeeper/MiniZooKeeperCluster.java   |   2 +-
 .../hadoop/hbase/zookeeper/ZKServerTool.java    |   2 +-
 .../hbase/zookeeper/ZooKeeperMainServer.java    |   2 +-
 hbase-shell/pom.xml                             |   8 -
 hbase-testing-util/pom.xml                      |  10 -
 hbase-thrift/pom.xml                            |   8 -
 pom.xml                                         |  87 ++++---
 477 files changed, 1317 insertions(+), 737 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-annotations/pom.xml b/hbase-annotations/pom.xml
new file mode 100644
index 0000000..1ba5c3c
--- /dev/null
+++ b/hbase-annotations/pom.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <!--
+  /**
+   * 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.
+   */
+  -->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>0.98.7-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+
+  <artifactId>hbase-annotations</artifactId>
+  <name>HBase - Annotations</name>
+  <description>Copy of Hadoop's annotations for HBase</description>
+
+  <dependencies>
+    <dependency>
+      <groupId>jdk.tools</groupId>
+      <artifactId>jdk.tools</artifactId>
+      <version>1.7</version>
+      <scope>system</scope>
+      <systemPath>${java.home}/../lib/tools.jar</systemPath>
+    </dependency>
+  </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/IntegrationTests.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/IntegrationTests.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/IntegrationTests.java
new file mode 100644
index 0000000..d429e24
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/IntegrationTests.java
@@ -0,0 +1,39 @@
+/**
+ * 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;
+
+/**
+ * Tag a test as 'integration/system' test, meaning that the test class has the following
+ * characteristics: <ul>
+ *  <li> Possibly takes hours to complete</li>
+ *  <li> Can be run on a mini cluster or an actual cluster</li>
+ *  <li> Can make changes to the given cluster (starting stopping daemons, etc)</li>
+ *  <li> Should not be run in parallel of other integration tests</li>
+ * </ul>
+ *
+ * Integration / System tests should have a class name starting with "IntegrationTest", and
+ * should be annotated with @Category(IntegrationTests.class). Integration tests can be run
+ * using the IntegrationTestsDriver class or from mvn verify.
+ *
+ * @see SmallTests
+ * @see MediumTests
+ * @see LargeTests
+ */
+public interface IntegrationTests {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/LargeTests.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/LargeTests.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/LargeTests.java
new file mode 100644
index 0000000..958ffd7
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/LargeTests.java
@@ -0,0 +1,38 @@
+/*
+ *
+ * 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;
+
+/**
+ * Tag a test as 'large', meaning that the test class has the following
+ * characteristics:
+ *  - executed in an isolated JVM. Tests can however be executed in different
+ *    JVM on the same machine simultaneously.
+ *  - will not have to be executed by the developer before submitting a bug
+ *  - ideally, last less than 2 minutes to help parallelization
+ *
+ *  It the worst case compared to small or medium, use it only for tests that
+ *    you cannot put in the other categories
+ *
+ * @see SmallTests
+ * @see MediumTests
+ * @see IntegrationTests
+ */
+public interface LargeTests {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/MediumTests.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/MediumTests.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/MediumTests.java
new file mode 100644
index 0000000..a51a2c9
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/MediumTests.java
@@ -0,0 +1,37 @@
+/*
+ *
+ * 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;
+
+/**
+ * Tag a test as 'Medium', meaning that the test class has the following
+ * characteristics:
+ *  - executed in an isolated JVM. Tests can however be executed in different
+ *    JVM on the same machine simultaneously.
+ *  - will have to be executed by the developer before submitting a bug
+ *  - ideally, last less than 1 minutes to help parallelization
+ *
+ *  Use it for tests that cannot be tagged as 'Small'.
+ *
+ * @see SmallTests
+ * @see LargeTests
+ * @see IntegrationTests
+ */
+public interface MediumTests {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/SmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/SmallTests.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/SmallTests.java
new file mode 100644
index 0000000..6953667
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/SmallTests.java
@@ -0,0 +1,34 @@
+/*
+ *
+ * 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;
+
+/**
+ * Tag a test as 'small', meaning that the test class has the following
+ * characteristics:
+ *  - can be run simultaneously with other small tests in the same JVM
+ *  - ideally, last less than 15 seconds
+ *  - does not use a cluster
+ *
+ * @see MediumTests
+ * @see LargeTests
+ * @see IntegrationTests
+ */
+public interface SmallTests {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceAudience.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceAudience.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceAudience.java
new file mode 100644
index 0000000..a76b2d9
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceAudience.java
@@ -0,0 +1,73 @@
+/*
+ * 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.classification;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Annotation to inform users of a package, class or method's intended audience.
+ * Currently the audience can be {@link Public}, {@link LimitedPrivate} or
+ * {@link Private}. <br>
+ * All public classes must have InterfaceAudience annotation. <br>
+ * <ul>
+ * <li>Public classes that are not marked with this annotation must be
+ * considered by default as {@link Private}.</li>
+ *
+ * <li>External applications must only use classes that are marked
+ * {@link Public}. Avoid using non public classes as these classes
+ * could be removed or change in incompatible ways.</li>
+ *
+ * <li>Hadoop projects must only use classes that are marked
+ * {@link LimitedPrivate} or {@link Public}</li>
+ *
+ * <li> Methods may have a different annotation that it is more restrictive
+ * compared to the audience classification of the class. Example: A class
+ * might be {@link Public}, but a method may be {@link LimitedPrivate}
+ * </li></ul>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class InterfaceAudience {
+  /**
+   * Intended for use by any project or application.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Public {};
+
+  /**
+   * Intended only for the project(s) specified in the annotation.
+   * For example, "Common", "HDFS", "MapReduce", "ZooKeeper", "HBase".
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface LimitedPrivate {
+    String[] value();
+  };
+
+  /**
+   * Intended for use only within Hadoop itself.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Private {};
+
+  private InterfaceAudience() {} // Audience can't exist on its own
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceStability.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceStability.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceStability.java
new file mode 100644
index 0000000..7ec47f6
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceStability.java
@@ -0,0 +1,65 @@
+/*
+ * 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.classification;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.hbase.classification.InterfaceAudience.Private;
+import org.apache.hadoop.hbase.classification.InterfaceAudience.Public;
+
+/**
+ * Annotation to inform users of how much to rely on a particular package,
+ * class or method not changing over time. Currently the stability can be
+ * {@link Stable}, {@link Evolving} or {@link Unstable}. <br>
+ *
+ * <ul><li>All classes that are annotated with {@link Public} or
+ * {@link LimitedPrivate} must have InterfaceStability annotation. </li>
+ * <li>Classes that are {@link Private} are to be considered unstable unless
+ * a different InterfaceStability annotation states otherwise.</li>
+ * <li>Incompatible changes must not be made to classes marked as stable.</li>
+ * </ul>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class InterfaceStability {
+  /**
+   * Can evolve while retaining compatibility for minor release boundaries.;
+   * can break compatibility only at major release (ie. at m.0).
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Stable {};
+
+  /**
+   * Evolving, but can break compatibility at minor release (i.e. m.x)
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Evolving {};
+
+  /**
+   * No guarantee is provided as to reliability or stability across any
+   * level of release granularity.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Unstable {};
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/ExcludePrivateAnnotationsStandardDoclet.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/ExcludePrivateAnnotationsStandardDoclet.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/ExcludePrivateAnnotationsStandardDoclet.java
new file mode 100644
index 0000000..efb5cdc
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/ExcludePrivateAnnotationsStandardDoclet.java
@@ -0,0 +1,58 @@
+/*
+ * 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.classification.tools;
+
+import com.sun.javadoc.DocErrorReporter;
+import com.sun.javadoc.LanguageVersion;
+import com.sun.javadoc.RootDoc;
+import com.sun.tools.doclets.standard.Standard;
+
+/**
+ * A <a href="http://java.sun.com/javase/6/docs/jdk/api/javadoc/doclet/">Doclet</a>
+ * for excluding elements that are annotated with
+ * {@link org.apache.hadoop.classification.InterfaceAudience.Private} or
+ * {@link org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate}.
+ * It delegates to the Standard Doclet, and takes the same options.
+ */
+public class ExcludePrivateAnnotationsStandardDoclet {
+
+  public static LanguageVersion languageVersion() {
+    return LanguageVersion.JAVA_1_5;
+  }
+
+  public static boolean start(RootDoc root) {
+    System.out.println(
+	ExcludePrivateAnnotationsStandardDoclet.class.getSimpleName());
+    return Standard.start(RootDocProcessor.process(root));
+  }
+
+  public static int optionLength(String option) {
+    Integer length = StabilityOptions.optionLength(option);
+    if (length != null) {
+      return length;
+    }
+    return Standard.optionLength(option);
+  }
+
+  public static boolean validOptions(String[][] options,
+      DocErrorReporter reporter) {
+    StabilityOptions.validOptions(options, reporter);
+    String[][] filteredOptions = StabilityOptions.filterOptions(options);
+    return Standard.validOptions(filteredOptions, reporter);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/IncludePublicAnnotationsStandardDoclet.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/IncludePublicAnnotationsStandardDoclet.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/IncludePublicAnnotationsStandardDoclet.java
new file mode 100644
index 0000000..c283c91
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/IncludePublicAnnotationsStandardDoclet.java
@@ -0,0 +1,63 @@
+/*
+ * 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.classification.tools;
+
+import com.sun.javadoc.DocErrorReporter;
+import com.sun.javadoc.LanguageVersion;
+import com.sun.javadoc.RootDoc;
+import com.sun.tools.doclets.standard.Standard;
+
+/**
+ * A <a href="http://java.sun.com/javase/6/docs/jdk/api/javadoc/doclet/">Doclet</a>
+ * that only includes class-level elements that are annotated with
+ * {@link org.apache.hadoop.hbase.classification.InterfaceAudience.Public}.
+ * Class-level elements with no annotation are excluded.
+ * In addition, all elements that are annotated with
+ * {@link org.apache.hadoop.hbase.classification.InterfaceAudience.Private} or
+ * {@link org.apache.hadoop.hbase.classification.InterfaceAudience.LimitedPrivate}
+ * are also excluded.
+ * It delegates to the Standard Doclet, and takes the same options.
+ */
+public class IncludePublicAnnotationsStandardDoclet {
+
+  public static LanguageVersion languageVersion() {
+    return LanguageVersion.JAVA_1_5;
+  }
+
+  public static boolean start(RootDoc root) {
+    System.out.println(
+        IncludePublicAnnotationsStandardDoclet.class.getSimpleName());
+    RootDocProcessor.treatUnannotatedClassesAsPrivate = true;
+    return Standard.start(RootDocProcessor.process(root));
+  }
+
+  public static int optionLength(String option) {
+    Integer length = StabilityOptions.optionLength(option);
+    if (length != null) {
+      return length;
+    }
+    return Standard.optionLength(option);
+  }
+
+  public static boolean validOptions(String[][] options,
+      DocErrorReporter reporter) {
+    StabilityOptions.validOptions(options, reporter);
+    String[][] filteredOptions = StabilityOptions.filterOptions(options);
+    return Standard.validOptions(filteredOptions, reporter);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/RootDocProcessor.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/RootDocProcessor.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/RootDocProcessor.java
new file mode 100644
index 0000000..78e8163
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/RootDocProcessor.java
@@ -0,0 +1,247 @@
+/*
+ * 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.classification.tools;
+
+import com.sun.javadoc.AnnotationDesc;
+import com.sun.javadoc.AnnotationTypeDoc;
+import com.sun.javadoc.ClassDoc;
+import com.sun.javadoc.ConstructorDoc;
+import com.sun.javadoc.Doc;
+import com.sun.javadoc.FieldDoc;
+import com.sun.javadoc.MethodDoc;
+import com.sun.javadoc.PackageDoc;
+import com.sun.javadoc.ProgramElementDoc;
+import com.sun.javadoc.RootDoc;
+
+import java.lang.reflect.Array;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.WeakHashMap;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Process the {@link RootDoc} by substituting with (nested) proxy objects that
+ * exclude elements with Private or LimitedPrivate annotations.
+ * <p>
+ * Based on code from http://www.sixlegs.com/blog/java/exclude-javadoc-tag.html.
+ */
+class RootDocProcessor {
+
+  static String stability = StabilityOptions.UNSTABLE_OPTION;
+  static boolean treatUnannotatedClassesAsPrivate = false;
+
+  public static RootDoc process(RootDoc root) {
+    return (RootDoc) process(root, RootDoc.class);
+  }
+
+  private static Object process(Object obj, Class<?> type) {
+    if (obj == null) {
+      return null;
+    }
+    Class<?> cls = obj.getClass();
+    if (cls.getName().startsWith("com.sun.")) {
+      return getProxy(obj);
+    } else if (obj instanceof Object[]) {
+      Class<?> componentType = type.isArray() ? type.getComponentType()
+	  : cls.getComponentType();
+      Object[] array = (Object[]) obj;
+      Object[] newArray = (Object[]) Array.newInstance(componentType,
+	  array.length);
+      for (int i = 0; i < array.length; ++i) {
+        newArray[i] = process(array[i], componentType);
+      }
+      return newArray;
+    }
+    return obj;
+  }
+
+  private static Map<Object, Object> proxies =
+    new WeakHashMap<Object, Object>();
+
+  private static Object getProxy(Object obj) {
+    Object proxy = proxies.get(obj);
+    if (proxy == null) {
+      proxy = Proxy.newProxyInstance(obj.getClass().getClassLoader(),
+        obj.getClass().getInterfaces(), new ExcludeHandler(obj));
+      proxies.put(obj, proxy);
+    }
+    return proxy;
+  }
+
+  private static class ExcludeHandler implements InvocationHandler {
+    private Object target;
+
+    public ExcludeHandler(Object target) {
+      this.target = target;
+    }
+
+    @Override
+    public Object invoke(Object proxy, Method method, Object[] args)
+	throws Throwable {
+      String methodName = method.getName();
+      if (target instanceof Doc) {
+	if (methodName.equals("isIncluded")) {
+	  Doc doc = (Doc) target;
+	  return !exclude(doc) && doc.isIncluded();
+	}
+	if (target instanceof RootDoc) {
+	  if (methodName.equals("classes")) {
+	    return filter(((RootDoc) target).classes(), ClassDoc.class);
+	  } else if (methodName.equals("specifiedClasses")) {
+	    return filter(((RootDoc) target).specifiedClasses(), ClassDoc.class);
+	  } else if (methodName.equals("specifiedPackages")) {
+	    return filter(((RootDoc) target).specifiedPackages(), PackageDoc.class);
+	  }
+	} else if (target instanceof ClassDoc) {
+	  if (isFiltered(args)) {
+	    if (methodName.equals("methods")) {
+	      return filter(((ClassDoc) target).methods(true), MethodDoc.class);
+	    } else if (methodName.equals("fields")) {
+	      return filter(((ClassDoc) target).fields(true), FieldDoc.class);
+	    } else if (methodName.equals("innerClasses")) {
+	      return filter(((ClassDoc) target).innerClasses(true),
+		  ClassDoc.class);
+	    } else if (methodName.equals("constructors")) {
+	      return filter(((ClassDoc) target).constructors(true),
+		  ConstructorDoc.class);
+	    }
+	  }
+	} else if (target instanceof PackageDoc) {
+	  if (methodName.equals("allClasses")) {
+	    if (isFiltered(args)) {
+	      return filter(((PackageDoc) target).allClasses(true),
+		ClassDoc.class);
+	    } else {
+	      return filter(((PackageDoc) target).allClasses(), ClassDoc.class);
+	    }
+	  } else if (methodName.equals("annotationTypes")) {
+	    return filter(((PackageDoc) target).annotationTypes(),
+		AnnotationTypeDoc.class);
+	  } else if (methodName.equals("enums")) {
+	    return filter(((PackageDoc) target).enums(),
+		ClassDoc.class);
+	  } else if (methodName.equals("errors")) {
+	    return filter(((PackageDoc) target).errors(),
+		ClassDoc.class);
+	  } else if (methodName.equals("exceptions")) {
+	    return filter(((PackageDoc) target).exceptions(),
+		ClassDoc.class);
+	  } else if (methodName.equals("interfaces")) {
+	    return filter(((PackageDoc) target).interfaces(),
+		ClassDoc.class);
+	  } else if (methodName.equals("ordinaryClasses")) {
+	    return filter(((PackageDoc) target).ordinaryClasses(),
+		ClassDoc.class);
+	  }
+	}
+      }
+
+      if (args != null) {
+	if (methodName.equals("compareTo") || methodName.equals("equals")
+	    || methodName.equals("overrides")
+	    || methodName.equals("subclassOf")) {
+	  args[0] = unwrap(args[0]);
+	}
+      }
+      try {
+	return process(method.invoke(target, args), method.getReturnType());
+      } catch (InvocationTargetException e) {
+	throw e.getTargetException();
+      }
+    }
+
+    private static boolean exclude(Doc doc) {
+      AnnotationDesc[] annotations = null;
+      if (doc instanceof ProgramElementDoc) {
+	annotations = ((ProgramElementDoc) doc).annotations();
+      } else if (doc instanceof PackageDoc) {
+	annotations = ((PackageDoc) doc).annotations();
+      }
+      if (annotations != null) {
+	for (AnnotationDesc annotation : annotations) {
+	  String qualifiedTypeName = annotation.annotationType().qualifiedTypeName();
+	  if (qualifiedTypeName.equals(
+	        InterfaceAudience.Private.class.getCanonicalName())
+	    || qualifiedTypeName.equals(
+                InterfaceAudience.LimitedPrivate.class.getCanonicalName())) {
+	    return true;
+	  }
+	  if (stability.equals(StabilityOptions.EVOLVING_OPTION)) {
+	    if (qualifiedTypeName.equals(
+		InterfaceStability.Unstable.class.getCanonicalName())) {
+	      return true;
+	    }
+	  }
+	  if (stability.equals(StabilityOptions.STABLE_OPTION)) {
+	    if (qualifiedTypeName.equals(
+		InterfaceStability.Unstable.class.getCanonicalName())
+              || qualifiedTypeName.equals(
+  		InterfaceStability.Evolving.class.getCanonicalName())) {
+	      return true;
+	    }
+	  }
+	}
+        for (AnnotationDesc annotation : annotations) {
+          String qualifiedTypeName =
+            annotation.annotationType().qualifiedTypeName();
+          if (qualifiedTypeName.equals(
+              InterfaceAudience.Public.class.getCanonicalName())) {
+            return false;
+          }
+        }
+      }
+      if (treatUnannotatedClassesAsPrivate) {
+        return doc.isClass() || doc.isInterface() || doc.isAnnotationType();
+      }
+      return false;
+    }
+
+    private static Object[] filter(Doc[] array, Class<?> componentType) {
+      if (array == null || array.length == 0) {
+	return array;
+      }
+      List<Object> list = new ArrayList<Object>(array.length);
+      for (Doc entry : array) {
+	if (!exclude(entry)) {
+	  list.add(process(entry, componentType));
+	}
+      }
+      return list.toArray((Object[]) Array.newInstance(componentType, list
+	  .size()));
+    }
+
+    private Object unwrap(Object proxy) {
+      if (proxy instanceof Proxy)
+	return ((ExcludeHandler) Proxy.getInvocationHandler(proxy)).target;
+      return proxy;
+    }
+
+    private boolean isFiltered(Object[] args) {
+      return args != null && Boolean.TRUE.equals(args[0]);
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/StabilityOptions.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/StabilityOptions.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/StabilityOptions.java
new file mode 100644
index 0000000..f3169ba
--- /dev/null
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/StabilityOptions.java
@@ -0,0 +1,69 @@
+/*
+ * 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.classification.tools;
+
+import com.sun.javadoc.DocErrorReporter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+class StabilityOptions {
+  public static final String STABLE_OPTION = "-stable";
+  public static final String EVOLVING_OPTION = "-evolving";
+  public static final String UNSTABLE_OPTION = "-unstable";
+
+  public static Integer optionLength(String option) {
+    String opt = option.toLowerCase();
+    if (opt.equals(UNSTABLE_OPTION)) return 1;
+    if (opt.equals(EVOLVING_OPTION)) return 1;
+    if (opt.equals(STABLE_OPTION)) return 1;
+    return null;
+  }
+
+  public static void validOptions(String[][] options,
+      DocErrorReporter reporter) {
+    for (int i = 0; i < options.length; i++) {
+      String opt = options[i][0].toLowerCase();
+      if (opt.equals(UNSTABLE_OPTION)) {
+	RootDocProcessor.stability = UNSTABLE_OPTION;
+      } else if (opt.equals(EVOLVING_OPTION)) {
+	RootDocProcessor.stability = EVOLVING_OPTION;
+      } else if (opt.equals(STABLE_OPTION)) {
+	RootDocProcessor.stability = STABLE_OPTION;
+      }
+    }
+  }
+
+  public static String[][] filterOptions(String[][] options) {
+    List<String[]> optionsList = new ArrayList<String[]>();
+    for (int i = 0; i < options.length; i++) {
+      if (!options[i][0].equalsIgnoreCase(UNSTABLE_OPTION)
+	  && !options[i][0].equalsIgnoreCase(EVOLVING_OPTION)
+	  && !options[i][0].equalsIgnoreCase(STABLE_OPTION)) {
+	optionsList.add(options[i]);
+      }
+    }
+    String[][] filteredOptions = new String[optionsList.size()][];
+    int i = 0;
+    for (String[] option : optionsList) {
+      filteredOptions[i++] = option;
+    }
+    return filteredOptions;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 15e8bd0..dec701f 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -78,6 +78,10 @@
     <!-- Intra-project dependencies -->
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-common</artifactId>
     </dependency>
     <dependency>
@@ -222,10 +226,6 @@
           </exclusion>
         </exclusions>
         </dependency>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-        </dependency>
       </dependencies>
     </profile>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
index 1998593..33b6d57 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * This exception is thrown by the master when a region server clock skew is

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
index 236304c..1ff6058 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
@@ -28,7 +28,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
index 6bacfde..521a591 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.HBaseIOException;
 
 /**
@@ -57,4 +57,4 @@ public class DoNotRetryIOException extends HBaseIOException {
   public DoNotRetryIOException(Throwable cause) {
     super(cause);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
index f2e939d..1e27ad7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
@@ -15,7 +15,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
index e7cdddb..1f4721a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
@@ -28,7 +28,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.compress.Compression;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index da2ac0d..27b1667 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
index 6af221c..2a8a790 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Addressing;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index 702618c..c48f8f4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
index cc7e449..b6d6d80 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
index b81e9af..5e42a2a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java
index a0a0823..96abe3a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when a namespace exists but should not

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java
index f4f5dcc..b1deedc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when a namespace can not be located

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
index da26274..888951d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
@@ -20,7 +20,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when an operation requires the root and all meta regions to be online

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java
index 007f4ab..b0143f4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
index a60bd90..46f1eaa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.HBaseIOException;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java
index 274d68e..7b1b156 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.HBaseIOException;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
index 28e6d6b..d2b1e21 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
@@ -21,7 +21,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
index f5217bc..5cb1a91 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown by a region server if it will block and wait to serve a request.

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
index afc5ba0..b4fb5b7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
@@ -21,7 +21,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
index 48b4a79..f200649 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
@@ -22,7 +22,7 @@ import com.google.common.net.InetAddresses;
 import com.google.protobuf.InvalidProtocolBufferException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.MetaRegionServer;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
index acd0e1e..5609fb1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
@@ -15,7 +15,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.TableName;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java
index 6a28b22..fb233d7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java
index f3ec12e..7fa138b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java
index c28ec77..c3aa4ea 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java
index 1c4d829..d23f290 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java
index 046670f..da1c28a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when we are asked to operate on a region we know nothing about.

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64e0891/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java
index 856c029..b84c3fa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 
 /**