From bbf3bae72ac0ebbc4e325128b6b8c8015efc220a Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Mon, 29 Jan 2018 12:36:40 -0800 Subject: [PATCH] HBASE-19873 Add a CategoryBasedTimeout ClassRule for all UTs --- .../exemplars/client/TestHelloHBase.java | 8 +- .../shaded_client/TestHelloHBase.java | 8 +- .../hadoop/hbase/TestHColumnDescriptor.java | 6 ++ .../hadoop/hbase/TestHTableDescriptor.java | 7 +- .../TestInterfaceAudienceAnnotations.java | 17 ++-- .../hadoop/hbase/TestRegionLocations.java | 6 +- .../hadoop/hbase/client/TestAsyncProcess.java | 15 ++-- .../hadoop/hbase/client/TestAttributes.java | 10 ++- .../hbase/client/TestBufferedMutator.java | 8 +- .../client/TestBufferedMutatorParams.java | 11 ++- .../client/TestClientExponentialBackoff.java | 15 +++- .../hbase/client/TestClientNoCluster.java | 10 ++- .../hbase/client/TestClientScanner.java | 7 +- .../TestColumnFamilyDescriptorBuilder.java | 6 ++ .../hbase/client/TestDelayingRunner.java | 9 ++- .../hbase/client/TestDeleteTimeStamp.java | 32 +++++--- .../apache/hadoop/hbase/client/TestGet.java | 17 ++-- .../client/TestHTableMultiplexerViaMocks.java | 38 +++++---- .../TestImmutableHColumnDescriptor.java | 11 ++- .../client/TestImmutableHRegionInfo.java | 9 ++- .../client/TestImmutableHTableDescriptor.java | 11 ++- .../hadoop/hbase/client/TestIncrement.java | 9 ++- .../hbase/client/TestInterfaceAlign.java | 9 ++- .../hbase/client/TestMetricsConnection.java | 38 +++++---- .../hadoop/hbase/client/TestMutation.java | 8 +- .../hadoop/hbase/client/TestOperation.java | 10 ++- .../hbase/client/TestProcedureFuture.java | 15 ++-- .../hadoop/hbase/client/TestPutDotHas.java | 8 +- .../hbase/client/TestRegionInfoDisplay.java | 15 ++-- ...tRetriesExhaustedWithDetailsException.java | 30 +++---- .../client/TestReversedScannerCallable.java | 6 ++ .../hbase/client/TestRowComparator.java | 7 ++ .../apache/hadoop/hbase/client/TestScan.java | 18 +++-- .../client/TestSimpleRequestController.java | 9 ++- .../hbase/client/TestSnapshotFromAdmin.java | 13 +++- .../client/TestTableDescriptorBuilder.java | 10 ++- .../exceptions/TestClientExceptionsUtil.java | 15 ++-- .../hadoop/hbase/filter/TestComparators.java | 9 ++- .../hbase/filter/TestKeyOnlyFilter.java | 7 +- .../hbase/filter/TestLongComparator.java | 16 ++-- .../hbase/ipc/TestCellBlockBuilder.java | 7 +- .../hbase/ipc/TestFailedServersLog.java | 32 +++++--- .../hbase/ipc/TestHBaseRpcControllerImpl.java | 8 +- .../apache/hadoop/hbase/ipc/TestIPCUtil.java | 7 +- .../TestRpcClientDeprecatedNameMapping.java | 6 ++ .../hadoop/hbase/quotas/TestQuotaFilter.java | 7 +- .../quotas/TestQuotaSettingsFactory.java | 31 +++++--- .../hbase/quotas/TestSpaceLimitSettings.java | 31 +++++--- .../hbase/quotas/TestThrottleSettings.java | 29 ++++--- .../TestReplicationPeerConfig.java | 7 +- .../hbase/security/TestEncryptionUtil.java | 11 ++- .../security/TestHBaseSaslRpcClient.java | 8 +- .../hadoop/hbase/security/TestSaslUtil.java | 11 ++- .../shaded/protobuf/TestProtobufUtil.java | 18 +++-- .../hbase/zookeeper/TestZNodePaths.java | 6 ++ hbase-common/pom.xml | 11 --- .../hadoop/hbase/CategoryBasedTimeout.java | 78 ------------------- .../hadoop/hbase/TestByteBufferKeyValue.java | 32 ++++---- .../apache/hadoop/hbase/TestCellBuilder.java | 6 +- .../hadoop/hbase/TestCellComparator.java | 9 ++- .../org/apache/hadoop/hbase/TestCellUtil.java | 7 +- .../apache/hadoop/hbase/TestChoreService.java | 8 +- .../apache/hadoop/hbase/TestClassFinder.java | 7 +- .../hbase/TestCompoundConfiguration.java | 11 ++- .../hadoop/hbase/TestHBaseConfiguration.java | 8 +- .../hbase/TestIndividualBytesFieldCell.java | 8 +- .../apache/hadoop/hbase/TestTableName.java | 7 +- .../org/apache/hadoop/hbase/TestTagUtil.java | 6 +- .../org/apache/hadoop/hbase/TestTimeout.java | 13 ++-- .../hadoop/hbase/codec/TestCellCodec.java | 7 +- .../hbase/codec/TestCellCodecWithTags.java | 6 ++ .../hadoop/hbase/codec/TestKeyValueCodec.java | 8 +- .../codec/TestKeyValueCodecWithTags.java | 6 ++ .../io/TestByteBufferListOutputStream.java | 7 +- .../hadoop/hbase/io/TestByteBufferPool.java | 7 +- .../io/TestMultiByteBuffInputStream.java | 7 +- .../hbase/io/TestTagCompressionContext.java | 8 +- .../hbase/io/crypto/TestCipherProvider.java | 32 ++++---- .../hbase/io/crypto/TestEncryption.java | 32 ++++---- .../hbase/io/crypto/TestKeyProvider.java | 32 ++++---- .../io/crypto/TestKeyStoreKeyProvider.java | 32 ++++---- .../hadoop/hbase/io/crypto/aes/TestAES.java | 32 ++++---- .../hbase/io/crypto/aes/TestCommonsAES.java | 32 ++++---- .../TestThrottledInputStream.java | 31 +++++--- .../hbase/io/util/TestLRUDictionary.java | 9 ++- .../hadoop/hbase/nio/TestMultiByteBuff.java | 9 ++- .../hadoop/hbase/nio/TestSingleByteBuff.java | 9 ++- .../hbase/types/TestCopyOnWriteMaps.java | 8 +- .../hbase/types/TestFixedLengthWrapper.java | 8 +- .../hadoop/hbase/types/TestOrderedBlob.java | 6 ++ .../hbase/types/TestOrderedBlobVar.java | 6 ++ .../hadoop/hbase/types/TestOrderedString.java | 6 ++ .../hadoop/hbase/types/TestRawString.java | 6 ++ .../apache/hadoop/hbase/types/TestStruct.java | 7 +- .../hbase/types/TestStructNullExtension.java | 7 +- .../hbase/types/TestTerminatedWrapper.java | 6 ++ .../apache/hadoop/hbase/types/TestUnion2.java | 6 ++ .../apache/hadoop/hbase/util/TestAvlUtil.java | 34 ++++---- .../apache/hadoop/hbase/util/TestBase64.java | 10 ++- .../hbase/util/TestByteBufferArray.java | 7 +- .../hbase/util/TestByteBufferUtils.java | 33 ++++---- .../TestByteRangeWithKVSerialization.java | 7 +- .../apache/hadoop/hbase/util/TestBytes.java | 9 ++- .../hadoop/hbase/util/TestCommonFSUtils.java | 9 ++- .../hbase/util/TestConcatenatedLists.java | 11 ++- .../util/TestCoprocessorClassLoader.java | 10 ++- .../apache/hadoop/hbase/util/TestCounter.java | 11 ++- .../hadoop/hbase/util/TestDrainBarrier.java | 9 ++- .../hbase/util/TestDynamicClassLoader.java | 11 ++- .../util/TestEnvironmentEdgeManager.java | 9 ++- .../hadoop/hbase/util/TestJRubyFormat.java | 11 ++- .../hadoop/hbase/util/TestKeyLocker.java | 9 ++- .../hbase/util/TestLoadTestKVGenerator.java | 32 ++++---- .../apache/hadoop/hbase/util/TestOrder.java | 7 +- .../hadoop/hbase/util/TestOrderedBytes.java | 7 +- .../hadoop/hbase/util/TestShowProperties.java | 10 ++- .../util/TestSimpleMutableByteRange.java | 8 +- .../TestSimplePositionedMutableByteRange.java | 8 +- .../apache/hadoop/hbase/util/TestThreads.java | 35 +++++---- .../hadoop/hbase/util/TestVersionInfo.java | 6 ++ .../hadoop/hbase/util/TestWeakObjectPool.java | 16 ++-- .../hadoop/hbase/zookeeper/TestZKConfig.java | 9 ++- .../client/TestAsyncAggregationClient.java | 7 +- .../client/TestRpcControllerFactory.java | 15 ++-- .../TestAsyncCoprocessorEndpoint.java | 14 ++-- .../TestBatchCoprocessorEndpoint.java | 16 ++-- .../hbase/coprocessor/TestClassLoading.java | 34 ++++---- .../coprocessor/TestCoprocessorEndpoint.java | 16 ++-- ...processorServiceBackwardCompatibility.java | 27 ++++--- .../TestCoprocessorTableEndpoint.java | 19 +++-- .../hbase/coprocessor/TestImportExport.java | 11 ++- .../TestRegionServerCoprocessorEndpoint.java | 15 ++-- .../coprocessor/TestRowProcessorEndpoint.java | 23 +++--- .../hbase/coprocessor/TestSecureExport.java | 18 +++-- .../hbase/ipc/TestCoprocessorRpcUtils.java | 11 ++- ...onServerBulkLoadWithOldSecureEndpoint.java | 16 +++- .../TestServerCustomProtocol.java | 20 +++-- ...plicationSyncUpToolWithBulkLoadedData.java | 32 +++++--- .../example/TestAsyncClientExample.java | 6 ++ .../client/example/TestHttpProxyExample.java | 11 ++- .../example/TestRefreshHFilesEndpoint.java | 15 ++-- .../example/TestScanModifyingObserver.java | 7 +- .../example/TestValueReplacingCompaction.java | 7 +- .../TestWriteHeavyIncrementObserver.java | 6 ++ ...crementObserverWithMemStoreCompaction.java | 6 ++ .../TestZooKeeperScanPolicyObserver.java | 7 +- .../mapreduce/TestMapReduceExamples.java | 44 ++++++----- .../apache/hadoop/hbase/types/TestPBCell.java | 10 ++- hbase-hadoop-compat/pom.xml | 6 ++ .../TestCompatibilitySingletonFactory.java | 16 ++-- .../TestMetricsMasterSourceFactory.java | 9 ++- .../TestMetricsRegionServerSourceFactory.java | 10 ++- .../wal/TestMetricsWALSource.java | 9 ++- .../TestMetricsReplicationSourceFactory.java | 10 ++- .../hbase/rest/TestMetricsRESTSource.java | 9 ++- .../TestMetricsThriftServerSourceFactory.java | 10 ++- .../zookeeper/TestMetricsZooKeeperSource.java | 6 ++ hbase-hadoop2-compat/pom.xml | 6 ++ .../TestMetricsMasterProcSourceImpl.java | 19 +++-- .../master/TestMetricsMasterSourceImpl.java | 19 +++-- .../hbase/metrics/TestBaseSourceImpl.java | 23 +++--- .../TestMetricsRegionServerSourceImpl.java | 19 +++-- .../TestMetricsRegionSourceImpl.java | 10 ++- .../TestMetricsTableSourceImpl.java | 9 ++- .../wal/TestMetricsWALSourceImpl.java | 18 +++-- ...stMetricsReplicationSourceFactoryImpl.java | 14 +++- .../TestMetricsReplicationSourceImpl.java | 13 +++- .../hbase/rest/TestMetricsRESTSourceImpl.java | 23 +++--- ...tMetricsThriftServerSourceFactoryImpl.java | 23 +++--- .../TestMetricsZooKeeperSourceImpl.java | 6 ++ .../hadoop/hbase/http/TestGlobalFilter.java | 9 ++- .../hadoop/hbase/http/TestHtmlQuoting.java | 7 +- .../hadoop/hbase/http/TestHttpRequestLog.java | 23 +++--- .../http/TestHttpRequestLogAppender.java | 10 ++- .../hadoop/hbase/http/TestHttpServer.java | 9 ++- .../hbase/http/TestHttpServerLifecycle.java | 6 ++ .../hbase/http/TestHttpServerWebapps.java | 9 ++- .../hadoop/hbase/http/TestPathFilter.java | 9 ++- .../hadoop/hbase/http/TestSSLHttpServer.java | 11 ++- .../hadoop/hbase/http/TestServletFilter.java | 9 ++- .../hbase/http/TestSpnegoHttpServer.java | 26 ++++--- .../hbase/http/conf/TestConfServlet.java | 10 ++- .../hbase/http/jmx/TestJMXJsonServlet.java | 31 ++++---- .../http/lib/TestStaticUserWebFilter.java | 12 ++- .../hadoop/hbase/http/log/TestLogLevel.java | 40 ++++++---- .../hadoop/hbase/TestIntegrationTestBase.java | 6 +- .../actions/TestChangeSplitPolicyAction.java | 7 ++ .../hbase/TestPerformanceEvaluation.java | 22 +++--- .../hadoop/hbase/mapred/TestDriver.java | 13 +++- .../hbase/mapred/TestGroupingTableMap.java | 14 ++-- .../hbase/mapred/TestIdentityTableMap.java | 12 ++- .../TestMultiTableSnapshotInputFormat.java | 12 ++- .../hadoop/hbase/mapred/TestRowCounter.java | 12 ++- .../hadoop/hbase/mapred/TestSplitTable.java | 8 +- .../hbase/mapred/TestTableInputFormat.java | 8 +- .../hbase/mapred/TestTableMapReduce.java | 15 ++-- .../hbase/mapred/TestTableMapReduceUtil.java | 13 +++- ...estTableOutputFormatConnectionExhaust.java | 13 +++- .../mapred/TestTableSnapshotInputFormat.java | 14 ++-- .../MultiTableInputFormatTestBase.java | 8 +- .../TableSnapshotInputFormatTestBase.java | 14 +--- .../TestCellBasedHFileOutputFormat2.java | 14 ++-- .../mapreduce/TestCellBasedImportExport2.java | 11 ++- .../mapreduce/TestCellBasedWALPlayer2.java | 14 +++- .../hbase/mapreduce/TestCellCounter.java | 19 +++-- .../hadoop/hbase/mapreduce/TestCopyTable.java | 8 +- .../mapreduce/TestGroupingTableMapper.java | 57 ++++++++------ .../mapreduce/TestHFileOutputFormat2.java | 14 ++-- .../mapreduce/TestHRegionPartitioner.java | 35 +++++---- .../hadoop/hbase/mapreduce/TestHashTable.java | 13 +++- .../hbase/mapreduce/TestImportExport.java | 11 ++- .../TestImportTSVWithOperationAttributes.java | 22 +++--- .../mapreduce/TestImportTSVWithTTLs.java | 17 ++-- .../TestImportTSVWithVisibilityLabels.java | 7 +- .../hadoop/hbase/mapreduce/TestImportTsv.java | 8 +- .../hbase/mapreduce/TestImportTsvParser.java | 12 ++- .../hadoop/hbase/mapreduce/TestJarFinder.java | 18 +++-- .../mapreduce/TestMultiTableInputFormat.java | 11 ++- .../TestMultiTableInputFormatBase.java | 38 ++++----- .../TestMultiTableSnapshotInputFormat.java | 15 ++-- ...TestMultiTableSnapshotInputFormatImpl.java | 53 +++++++------ .../TestMultithreadedTableMapper.java | 10 ++- .../mapreduce/TestRegionSizeCalculator.java | 6 ++ .../hbase/mapreduce/TestRowCounter.java | 14 ++-- .../TestSimpleTotalOrderPartitioner.java | 11 ++- .../hadoop/hbase/mapreduce/TestSyncTable.java | 13 ++-- .../hbase/mapreduce/TestTableInputFormat.java | 8 +- .../mapreduce/TestTableInputFormatBase.java | 9 ++- .../mapreduce/TestTableInputFormatScan1.java | 8 +- .../mapreduce/TestTableInputFormatScan2.java | 8 +- .../hbase/mapreduce/TestTableMapReduce.java | 9 ++- .../mapreduce/TestTableMapReduceBase.java | 6 -- .../mapreduce/TestTableMapReduceUtil.java | 30 ++++--- .../TestTableSnapshotInputFormat.java | 30 +++---- .../hbase/mapreduce/TestTableSplit.java | 16 ++-- .../hbase/mapreduce/TestTimeRangeMapRed.java | 23 +++--- .../hadoop/hbase/mapreduce/TestWALPlayer.java | 14 +++- .../hbase/mapreduce/TestWALRecordReader.java | 7 ++ .../replication/TestVerifyReplication.java | 6 ++ .../hbase/snapshot/TestExportSnapshot.java | 14 ++-- .../snapshot/TestExportSnapshotHelpers.java | 12 ++- .../snapshot/TestExportSnapshotNoCluster.java | 13 ++-- .../hbase/snapshot/TestMobExportSnapshot.java | 8 +- .../snapshot/TestMobSecureExportSnapshot.java | 14 ++-- .../snapshot/TestSecureExportSnapshot.java | 16 ++-- hbase-metrics-api/pom.xml | 6 ++ .../metrics/TestMetricRegistriesLoader.java | 6 ++ hbase-metrics/pom.xml | 6 ++ .../hbase/metrics/impl/TestCounterImpl.java | 6 ++ .../metrics/impl/TestDropwizardMeter.java | 7 +- .../metrics/impl/TestFastLongHistogram.java | 7 +- .../hadoop/hbase/metrics/impl/TestGauge.java | 7 +- .../hbase/metrics/impl/TestHistogramImpl.java | 12 +-- .../metrics/impl/TestMetricRegistryImpl.java | 13 ++-- .../metrics/impl/TestRefCountingMap.java | 10 ++- .../hbase/metrics/impl/TestTimerImpl.java | 7 +- .../hbase/procedure2/TestChildProcedures.java | 17 ++-- .../hbase/procedure2/TestProcedureEvents.java | 20 +++-- .../procedure2/TestProcedureExecution.java | 18 +++-- .../procedure2/TestProcedureExecutor.java | 16 ++-- .../TestProcedureInMemoryChore.java | 9 ++- .../procedure2/TestProcedureMetrics.java | 19 +++-- .../hbase/procedure2/TestProcedureNonce.java | 50 ++++++------ .../procedure2/TestProcedureRecovery.java | 52 +++++++------ .../procedure2/TestProcedureReplayOrder.java | 47 ++++++----- .../TestProcedureSchedulerConcurrency.java | 38 +++++---- .../procedure2/TestProcedureSuspended.java | 9 ++- .../procedure2/TestProcedureToString.java | 12 ++- .../hbase/procedure2/TestProcedureUtil.java | 24 +++--- .../procedure2/TestStateMachineProcedure.java | 18 +++-- .../hbase/procedure2/TestYieldProcedures.java | 20 +++-- .../store/TestProcedureStoreTracker.java | 24 +++--- .../wal/TestStressWALProcedureStore.java | 22 +++--- .../store/wal/TestWALProcedureStore.java | 25 +++--- .../procedure2/util/TestDelayedUtil.java | 14 +++- .../hadoop/hbase/rest/TestDeleteRow.java | 10 ++- .../hbase/rest/TestGZIPResponseWrapper.java | 17 ++-- .../hbase/rest/TestGetAndPutResource.java | 16 ++-- .../hadoop/hbase/rest/TestGzipFilter.java | 15 ++-- .../hbase/rest/TestMultiRowResource.java | 10 ++- .../rest/TestNamespacesInstanceResource.java | 28 +++---- .../hbase/rest/TestNamespacesResource.java | 17 ++-- .../hadoop/hbase/rest/TestResourceFilter.java | 8 +- .../hbase/rest/TestScannerResource.java | 13 ++-- .../hbase/rest/TestScannersWithFilters.java | 12 +-- .../hbase/rest/TestScannersWithLabels.java | 11 ++- .../hadoop/hbase/rest/TestSchemaResource.java | 14 ++-- .../hadoop/hbase/rest/TestStatusResource.java | 12 ++- .../hadoop/hbase/rest/TestTableResource.java | 17 ++-- .../hadoop/hbase/rest/TestTableScan.java | 10 ++- .../hbase/rest/TestVersionResource.java | 21 ++--- .../rest/client/TestRemoteAdminRetries.java | 9 ++- .../rest/client/TestRemoteHTableRetries.java | 13 +++- .../hbase/rest/client/TestRemoteTable.java | 44 ++++++----- .../hbase/rest/client/TestXmlParsing.java | 25 +++--- .../hbase/rest/model/TestCellModel.java | 11 ++- .../hbase/rest/model/TestCellSetModel.java | 12 +-- .../rest/model/TestColumnSchemaModel.java | 10 ++- .../hbase/rest/model/TestModelBase.java | 10 +-- .../model/TestNamespacesInstanceModel.java | 11 ++- .../hbase/rest/model/TestNamespacesModel.java | 12 +-- .../hadoop/hbase/rest/model/TestRowModel.java | 12 +-- .../hbase/rest/model/TestScannerModel.java | 11 ++- .../model/TestStorageClusterStatusModel.java | 14 ++-- .../model/TestStorageClusterVersionModel.java | 11 ++- .../hbase/rest/model/TestTableInfoModel.java | 13 ++-- .../hbase/rest/model/TestTableListModel.java | 13 ++-- .../rest/model/TestTableRegionModel.java | 14 ++-- .../rest/model/TestTableSchemaModel.java | 13 ++-- .../hbase/rest/model/TestVersionModel.java | 14 +++- .../TestRSGroupBasedLoadBalancer.java | 8 +- .../hadoop/hbase/rsgroup/TestRSGroups.java | 9 ++- .../rsgroup/TestRSGroupsOfflineMode.java | 7 ++ .../hbase/rsgroup/TestRSGroupsWithACL.java | 22 ++++-- .../TestAcidGuaranteesWithAdaptivePolicy.java | 6 +- .../TestAcidGuaranteesWithBasicPolicy.java | 6 +- .../TestAcidGuaranteesWithEagerPolicy.java | 6 +- ...stAcidGuaranteesWithNoInMemCompaction.java | 6 +- .../hadoop/hbase/TestCheckTestClasses.java | 9 ++- .../hbase/TestClientClusterMetrics.java | 6 ++ .../hadoop/hbase/TestClientClusterStatus.java | 6 ++ .../hadoop/hbase/TestClusterBootOrder.java | 6 +- .../TestFSTableDescriptorForceCreation.java | 8 +- .../hbase/TestFullLogReconstruction.java | 7 +- .../hadoop/hbase/TestGlobalMemStoreSize.java | 11 ++- .../hbase/TestHBaseOnOtherDfsCluster.java | 15 ++-- .../hadoop/hbase/TestHBaseTestingUtility.java | 9 ++- .../TestHColumnDescriptorDefaultVersions.java | 8 +- .../hbase/TestHDFSBlocksDistribution.java | 18 +++-- .../hadoop/hbase/TestHRegionLocation.java | 7 +- .../apache/hadoop/hbase/TestIOFencing.java | 9 ++- .../hbase/TestIPv6NIOServerSocketChannel.java | 13 ++-- .../apache/hadoop/hbase/TestInfoServers.java | 8 +- .../hadoop/hbase/TestJMXConnectorServer.java | 11 ++- .../apache/hadoop/hbase/TestJMXListener.java | 14 ++-- .../hadoop/hbase/TestLocalHBaseCluster.java | 8 +- .../hadoop/hbase/TestMetaTableAccessor.java | 9 ++- .../hbase/TestMetaTableAccessorNoCluster.java | 15 +++- .../hadoop/hbase/TestMetaTableLocator.java | 22 ++++-- .../hadoop/hbase/TestMovedRegionsCleaner.java | 17 ++-- .../hadoop/hbase/TestMultiVersions.java | 14 ++-- .../apache/hadoop/hbase/TestNamespace.java | 14 ++-- .../hbase/TestNodeHealthCheckChore.java | 6 +- .../TestPartialResultsFromClientSide.java | 7 +- .../apache/hadoop/hbase/TestRegionLoad.java | 14 ++-- .../hadoop/hbase/TestRegionMetrics.java | 7 +- .../hadoop/hbase/TestRegionRebalancing.java | 9 ++- .../hadoop/hbase/TestSerialization.java | 14 ++-- .../apache/hadoop/hbase/TestServerLoad.java | 9 ++- .../hadoop/hbase/TestServerMetrics.java | 9 ++- .../apache/hadoop/hbase/TestServerName.java | 7 +- ...stServerSideScanMetricsFromClientSide.java | 30 ++++--- .../org/apache/hadoop/hbase/TestSize.java | 8 +- .../TestStochasticBalancerJmxMetrics.java | 7 +- .../hadoop/hbase/TestTagRewriteCell.java | 20 +++-- .../apache/hadoop/hbase/TestZooKeeper.java | 7 +- .../hbase/backup/TestHFileArchiving.java | 7 +- .../TestZooKeeperTableArchiveClient.java | 7 +- .../hadoop/hbase/client/TestAdmin1.java | 10 ++- .../hadoop/hbase/client/TestAdmin2.java | 9 ++- .../TestAllowPartialScanResultCache.java | 7 +- .../hbase/client/TestAlwaysSetScannerId.java | 19 +++-- .../client/TestAppendFromClientSide.java | 17 ++-- .../hbase/client/TestAsyncAdminBuilder.java | 9 ++- .../hbase/client/TestAsyncBufferMutator.java | 7 +- .../client/TestAsyncClusterAdminApi.java | 6 ++ .../client/TestAsyncClusterAdminApi2.java | 7 +- .../client/TestAsyncDecommissionAdminApi.java | 6 ++ .../client/TestAsyncMetaRegionLocator.java | 7 +- .../client/TestAsyncNamespaceAdminApi.java | 7 +- .../client/TestAsyncNonMetaRegionLocator.java | 9 ++- ...ncNonMetaRegionLocatorConcurrenyLimit.java | 7 +- .../client/TestAsyncProcedureAdminApi.java | 8 +- .../hbase/client/TestAsyncQuotaAdminApi.java | 8 +- .../hbase/client/TestAsyncRegionAdminApi.java | 7 +- .../client/TestAsyncRegionLocatorTimeout.java | 7 +- .../client/TestAsyncReplicationAdminApi.java | 7 +- ...tAsyncReplicationAdminApiWithClusters.java | 7 +- .../client/TestAsyncResultScannerCursor.java | 7 +- ...stAsyncSingleRequestRpcRetryingCaller.java | 7 +- .../client/TestAsyncSnapshotAdminApi.java | 7 +- .../hadoop/hbase/client/TestAsyncTable.java | 7 +- .../hbase/client/TestAsyncTableAdminApi.java | 7 +- .../hbase/client/TestAsyncTableBatch.java | 7 +- .../TestAsyncTableGetMultiThreaded.java | 8 +- ...leGetMultiThreadedWithBasicCompaction.java | 6 ++ ...leGetMultiThreadedWithEagerCompaction.java | 6 ++ .../client/TestAsyncTableNoncedRetry.java | 8 +- .../hbase/client/TestAsyncTableScan.java | 7 +- .../hbase/client/TestAsyncTableScanAll.java | 7 +- .../client/TestAsyncTableScanMetrics.java | 7 +- .../client/TestAsyncTableScanRenewLease.java | 7 +- .../hbase/client/TestAsyncTableScanner.java | 7 +- ...AsyncTableScannerCloseWhileSuspending.java | 7 +- .../hbase/client/TestAsyncToolAdminApi.java | 6 ++ ...tAvoidCellReferencesIntoShippedBlocks.java | 8 +- .../client/TestBatchScanResultCache.java | 7 +- .../client/TestBlockEvictionFromClient.java | 8 +- .../hbase/client/TestCIBadHostname.java | 10 ++- .../client/TestCIDeleteOperationTimeout.java | 6 ++ .../hbase/client/TestCIDeleteRpcTimeout.java | 6 ++ .../client/TestCIGetOperationTimeout.java | 6 ++ .../hbase/client/TestCIGetRpcTimeout.java | 6 ++ .../client/TestCIIncrementRpcTimeout.java | 6 ++ .../client/TestCIPutOperationTimeout.java | 6 ++ .../hbase/client/TestCIPutRpcTimeout.java | 6 ++ .../hadoop/hbase/client/TestCISleep.java | 6 ++ .../hbase/client/TestCheckAndMutate.java | 17 ++-- .../client/TestClientOperationInterrupt.java | 24 +++--- .../hbase/client/TestClientPushback.java | 19 +++-- .../client/TestClientScannerRPCTimeout.java | 19 +++-- .../hbase/client/TestClientTimeouts.java | 12 ++- .../client/TestCloneSnapshotFromClient.java | 8 +- ...eSnapshotFromClientWithRegionReplicas.java | 7 ++ .../TestCompleteResultScanResultCache.java | 7 +- .../client/TestConnectionImplementation.java | 14 ++-- .../hbase/client/TestConnectionUtils.java | 21 ++--- .../hbase/client/TestDropTimeoutRequest.java | 26 +++---- .../hadoop/hbase/client/TestEnableTable.java | 22 ++++-- .../hadoop/hbase/client/TestFastFail.java | 12 ++- .../hbase/client/TestFlushFromClient.java | 9 ++- .../hbase/client/TestFromClientSide.java | 8 +- .../hbase/client/TestFromClientSide3.java | 10 ++- .../client/TestFromClientSideNoCodec.java | 8 +- .../TestFromClientSideScanExcpetion.java | 7 +- ...lientSideScanExcpetionWithCoprocessor.java | 7 ++ .../TestFromClientSideWithCoprocessor.java | 13 +++- .../hbase/client/TestGetProcedureResult.java | 6 ++ .../hbase/client/TestHBaseAdminNoCluster.java | 39 +++++----- .../hbase/client/TestHTableMultiplexer.java | 14 ++-- .../TestHTableMultiplexerFlushCache.java | 19 +++-- .../client/TestIllegalTableDescriptor.java | 9 ++- ...ncrementFromClientSideWithCoprocessor.java | 9 ++- .../client/TestIncrementsFromClientSide.java | 9 ++- .../hbase/client/TestIntraRowPagination.java | 32 ++++---- .../hadoop/hbase/client/TestLeaseRenewal.java | 9 ++- .../client/TestLimitedScanWithFilter.java | 7 +- .../hadoop/hbase/client/TestMetaCache.java | 35 +++++---- .../hbase/client/TestMetaWithReplicas.java | 15 ++-- .../TestMobCloneSnapshotFromClient.java | 7 +- .../TestMobRestoreSnapshotFromClient.java | 7 +- .../TestMobSnapshotCloneIndependence.java | 12 +-- .../client/TestMobSnapshotFromClient.java | 8 +- .../hbase/client/TestMultiParallel.java | 10 ++- .../hbase/client/TestMultiRespectsLimits.java | 11 ++- .../hbase/client/TestMultipleTimestamps.java | 9 ++- .../client/TestMvccConsistentScanner.java | 7 +- .../client/TestPutDeleteEtcCellIteration.java | 11 ++- .../hbase/client/TestPutWithDelete.java | 12 ++- .../hbase/client/TestRawAsyncScanCursor.java | 7 +- ...estRawAsyncTableLimitedScanWithFilter.java | 7 +- .../client/TestRawAsyncTablePartialScan.java | 7 +- .../hbase/client/TestRawAsyncTableScan.java | 7 +- .../hbase/client/TestReplicaWithCluster.java | 14 ++-- .../hbase/client/TestReplicasClient.java | 20 +++-- .../client/TestRestoreSnapshotFromClient.java | 13 ++-- ...eSnapshotFromClientWithRegionReplicas.java | 7 ++ .../hadoop/hbase/client/TestResult.java | 12 +-- .../client/TestResultFromCoprocessor.java | 13 +++- .../hbase/client/TestResultScannerCursor.java | 7 +- .../client/TestResultSizeEstimation.java | 14 +++- .../client/TestScanWithoutFetchingData.java | 21 +++-- .../hbase/client/TestScannerTimeout.java | 7 +- .../client/TestScannersFromClientSide.java | 60 +++++++------- .../client/TestScannersFromClientSide2.java | 7 +- .../hbase/client/TestServerBusyException.java | 19 +++-- .../client/TestServerLoadDurability.java | 7 ++ .../client/TestShortCircuitConnection.java | 13 +++- .../hadoop/hbase/client/TestSizeFailures.java | 10 ++- .../client/TestSmallReversedScanner.java | 10 ++- .../client/TestSnapshotCloneIndependence.java | 12 ++- .../hbase/client/TestSnapshotFromClient.java | 16 +++- ...tSnapshotFromClientWithRegionReplicas.java | 6 ++ .../hbase/client/TestSnapshotMetadata.java | 9 ++- .../hbase/client/TestSnapshotWithAcl.java | 12 ++- .../hbase/client/TestSplitOrMergeStatus.java | 9 ++- .../hbase/client/TestTableFavoredNodes.java | 9 ++- .../client/TestTableSnapshotScanner.java | 8 +- .../hbase/client/TestTimestampsFilter.java | 11 ++- .../hbase/client/TestUpdateConfiguration.java | 11 ++- .../hbase/client/TestZKAsyncRegistry.java | 7 ++ .../hbase/client/locking/TestEntityLocks.java | 28 ++++--- .../replication/TestReplicationAdmin.java | 11 ++- .../TestReplicationAdminWithClusters.java | 30 ++++--- ...cationAdminWithTwoDifferentZKClusters.java | 30 ++++--- .../hbase/codec/TestCellMessageCodec.java | 9 ++- .../hbase/conf/TestConfigurationManager.java | 8 +- .../hbase/constraint/TestConstraint.java | 12 ++- .../hbase/constraint/TestConstraints.java | 18 +++-- .../coprocessor/TestAppendTimeRange.java | 8 +- .../TestCoprocessorConfiguration.java | 23 ++++-- .../coprocessor/TestCoprocessorHost.java | 16 +++- .../coprocessor/TestCoprocessorInterface.java | 12 ++- .../coprocessor/TestCoprocessorMetrics.java | 28 ++++--- .../TestCoprocessorShortCircuitRPC.java | 22 +++--- .../coprocessor/TestCoprocessorStop.java | 24 +++--- .../TestCoreMasterCoprocessor.java | 26 ++++--- .../TestCoreRegionCoprocessor.java | 27 +++---- .../TestCoreRegionServerCoprocessor.java | 27 +++---- .../coprocessor/TestIncrementTimeRange.java | 9 ++- ...stMasterCoprocessorExceptionWithAbort.java | 23 +++--- ...tMasterCoprocessorExceptionWithRemove.java | 21 ++--- .../hbase/coprocessor/TestMasterObserver.java | 13 +++- ...gativeMemStoreSizeWithSlowCoprocessor.java | 33 +++++--- .../TestOpenTableInCoprocessor.java | 32 ++++---- .../TestPassCustomCellViaRegionObserver.java | 15 ++-- .../TestReadOnlyConfiguration.java | 41 +++++----- .../coprocessor/TestRegionObserverBypass.java | 15 ++-- ...verForAddingMutationsFromCoprocessors.java | 15 ++-- .../TestRegionObserverInterface.java | 17 ++-- ...stRegionObserverPreFlushAndPreCompact.java | 24 +++--- .../TestRegionObserverScannerOpenHook.java | 10 ++- .../TestRegionObserverStacking.java | 17 ++-- ...onServerCoprocessorExceptionWithAbort.java | 16 ++-- ...nServerCoprocessorExceptionWithRemove.java | 16 ++-- .../hbase/coprocessor/TestWALObserver.java | 9 ++- .../TestForeignExceptionDispatcher.java | 7 ++ .../TestForeignExceptionSerialization.java | 10 ++- .../TestTimeoutExceptionInjector.java | 6 ++ .../hbase/executor/TestExecutorService.java | 25 ++++-- .../TestFavoredNodeAssignmentHelper.java | 10 ++- .../TestStartcodeAgnosticServerName.java | 6 ++ .../filter/TestBigDecimalComparator.java | 23 +++--- .../hbase/filter/TestBitComparator.java | 34 ++++---- .../filter/TestColumnPaginationFilter.java | 13 +++- .../hbase/filter/TestColumnPrefixFilter.java | 7 +- .../hbase/filter/TestColumnRangeFilter.java | 14 ++-- .../filter/TestComparatorSerialization.java | 12 ++- .../filter/TestDependentColumnFilter.java | 12 ++- .../hadoop/hbase/filter/TestFilter.java | 16 ++-- .../filter/TestFilterFromRegionSide.java | 15 ++-- .../hadoop/hbase/filter/TestFilterList.java | 10 ++- .../hbase/filter/TestFilterListOnMini.java | 7 +- .../TestFilterListOrOperatorWithBlkCnt.java | 9 ++- .../hbase/filter/TestFilterSerialization.java | 16 ++-- .../filter/TestFilterWithScanLimits.java | 12 ++- .../hbase/filter/TestFilterWrapper.java | 19 +++-- ...FirstKeyValueMatchingQualifiersFilter.java | 11 ++- .../TestFuzzyRowAndColumnRangeFilter.java | 27 ++++--- .../hbase/filter/TestFuzzyRowFilter.java | 9 ++- .../filter/TestFuzzyRowFilterEndToEnd.java | 55 +++++++------ .../hbase/filter/TestInclusiveStopFilter.java | 14 +++- .../filter/TestInvocationRecordFilter.java | 9 ++- .../hbase/filter/TestMultiRowRangeFilter.java | 9 ++- .../TestMultipleColumnPrefixFilter.java | 36 +++++---- .../hbase/filter/TestNullComparator.java | 26 ++++--- .../hadoop/hbase/filter/TestPageFilter.java | 9 ++- .../hadoop/hbase/filter/TestParseFilter.java | 12 ++- .../hadoop/hbase/filter/TestPrefixFilter.java | 15 ++-- .../hbase/filter/TestRandomRowFilter.java | 17 ++-- .../hbase/filter/TestRegexComparator.java | 8 +- .../hbase/filter/TestScanRowPrefix.java | 22 +++--- .../TestSingleColumnValueExcludeFilter.java | 20 +++-- .../filter/TestSingleColumnValueFilter.java | 9 ++- .../hadoop/hbase/fs/TestBlockReorder.java | 9 ++- .../hbase/io/TestByteBufferOutputStream.java | 8 +- .../apache/hadoop/hbase/io/TestFileLink.java | 8 +- .../apache/hadoop/hbase/io/TestHFileLink.java | 21 +++-- .../hbase/io/TestHalfStoreFileReader.java | 12 ++- .../apache/hadoop/hbase/io/TestHeapSize.java | 12 ++- .../hbase/io/TestImmutableBytesWritable.java | 21 ++--- .../apache/hadoop/hbase/io/TestMetricsIO.java | 7 +- .../TestFanOutOneBlockAsyncDFSOutput.java | 6 ++ .../io/asyncfs/TestLocalAsyncOutput.java | 7 +- .../TestSaslFanOutOneBlockAsyncDFSOutput.java | 7 +- .../io/asyncfs/TestSendBufSizePredictor.java | 6 ++ .../TestBufferedDataBlockEncoder.java | 36 +++++---- .../io/encoding/TestChangingEncoding.java | 49 +++++++----- .../io/encoding/TestDataBlockEncoders.java | 49 ++++++------ .../io/encoding/TestDataBlockEncoding.java | 31 +++++--- .../hbase/io/encoding/TestEncodedSeekers.java | 36 +++++---- .../TestLoadAndSwitchEncodeOnDisk.java | 38 +++++---- .../TestSeekBeforeWithReverseScan.java | 12 ++- .../encoding/TestSeekToBlockWithEncoders.java | 35 +++++---- .../io/hfile/TestBlockCacheReporting.java | 17 ++-- .../hbase/io/hfile/TestCacheConfig.java | 13 +++- .../hbase/io/hfile/TestCacheOnWrite.java | 14 ++-- .../hbase/io/hfile/TestCachedBlockQueue.java | 12 ++- .../hadoop/hbase/io/hfile/TestChecksum.java | 28 ++++--- .../io/hfile/TestCombinedBlockCache.java | 25 +++--- .../hbase/io/hfile/TestFixedFileTrailer.java | 10 ++- .../hfile/TestForceCacheImportantBlocks.java | 37 +++++---- .../hadoop/hbase/io/hfile/TestHFile.java | 10 ++- .../hadoop/hbase/io/hfile/TestHFileBlock.java | 12 ++- .../hbase/io/hfile/TestHFileBlockIndex.java | 13 ++-- .../hfile/TestHFileBlockPositionalRead.java | 9 ++- .../io/hfile/TestHFileDataBlockEncoder.java | 37 +++++---- .../hbase/io/hfile/TestHFileEncryption.java | 10 ++- .../TestHFileInlineToRootChunkConversion.java | 39 ++++++---- .../hadoop/hbase/io/hfile/TestHFileSeek.java | 49 ++++++------ .../hbase/io/hfile/TestHFileWriterV3.java | 11 ++- .../hfile/TestLazyDataBlockDecompression.java | 8 +- .../hbase/io/hfile/TestLruBlockCache.java | 16 ++-- .../hbase/io/hfile/TestLruCachedBlock.java | 12 ++- .../hadoop/hbase/io/hfile/TestPrefetch.java | 14 ++-- .../hadoop/hbase/io/hfile/TestReseekTo.java | 10 ++- .../io/hfile/TestScannerFromBucketCache.java | 11 ++- .../TestScannerSelectionUsingKeyRange.java | 35 +++++---- .../hfile/TestScannerSelectionUsingTTL.java | 32 ++++---- .../hfile/TestSeekBeforeWithInlineBlocks.java | 71 +++++++++-------- .../hadoop/hbase/io/hfile/TestSeekTo.java | 8 +- .../io/hfile/bucket/TestBucketCache.java | 31 ++++---- .../hfile/bucket/TestBucketWriterThread.java | 65 +++++++++------- .../hfile/bucket/TestByteBufferIOEngine.java | 32 ++++---- .../io/hfile/bucket/TestFileIOEngine.java | 32 ++++---- .../io/hfile/bucket/TestFileMmapEngine.java | 33 ++++---- .../hadoop/hbase/ipc/TestBlockingIPC.java | 8 +- .../hadoop/hbase/ipc/TestBufferChain.java | 12 ++- .../hadoop/hbase/ipc/TestCallRunner.java | 7 ++ .../hbase/ipc/TestFifoRpcScheduler.java | 50 ++++++------ .../hadoop/hbase/ipc/TestHBaseClient.java | 10 ++- .../apache/hadoop/hbase/ipc/TestNettyIPC.java | 8 +- .../hadoop/hbase/ipc/TestNettyRpcServer.java | 8 +- .../hadoop/hbase/ipc/TestProtoBufRpc.java | 17 ++-- .../hadoop/hbase/ipc/TestRpcClientLeaks.java | 17 ++-- .../hbase/ipc/TestRpcHandlerException.java | 14 +++- .../hadoop/hbase/ipc/TestRpcMetrics.java | 20 +++-- .../hadoop/hbase/ipc/TestRpcServer.java | 7 +- .../ipc/TestRpcServerSlowConnectionSetup.java | 24 +++--- .../hbase/ipc/TestSimpleRpcScheduler.java | 37 +++++---- .../hbase/master/TestActiveMasterManager.java | 11 ++- .../hbase/master/TestAssignmentListener.java | 9 ++- .../master/TestAssignmentManagerMetrics.java | 13 +++- .../hbase/master/TestCatalogJanitor.java | 13 ++-- .../TestCatalogJanitorInMemoryStates.java | 13 ++-- .../hbase/master/TestClockSkewDetection.java | 12 ++- .../master/TestClusterStatusPublisher.java | 15 ++-- .../hbase/master/TestDLSAsyncFSWAL.java | 6 ++ .../hadoop/hbase/master/TestDLSFSHLog.java | 6 ++ .../hadoop/hbase/master/TestDeadServer.java | 20 +++-- .../hadoop/hbase/master/TestGetInfoPort.java | 6 ++ .../master/TestGetLastFlushedSequenceId.java | 10 ++- .../hbase/master/TestHMasterCommandLine.java | 10 ++- .../hbase/master/TestHMasterRPCException.java | 19 +++-- .../hadoop/hbase/master/TestMaster.java | 10 ++- .../master/TestMasterBalanceThrottling.java | 9 ++- .../master/TestMasterCoprocessorServices.java | 29 ++++--- .../hbase/master/TestMasterFailover.java | 16 ++-- ...TestMasterFailoverBalancerPersistence.java | 7 +- .../hbase/master/TestMasterFileSystem.java | 7 ++ .../TestMasterFileSystemWithWALDir.java | 6 ++ .../hbase/master/TestMasterMetrics.java | 14 +++- .../master/TestMasterMetricsWrapper.java | 8 +- .../hbase/master/TestMasterNoCluster.java | 21 ++--- ...TestMasterOperationsForRegionReplicas.java | 8 +- .../hbase/master/TestMasterQosFunction.java | 15 +++- .../TestMasterRestartAfterDisablingTable.java | 8 +- .../hbase/master/TestMasterShutdown.java | 9 ++- .../hbase/master/TestMasterStatusServlet.java | 8 +- .../hbase/master/TestMasterTransitions.java | 9 ++- .../hbase/master/TestMetaShutdownHandler.java | 10 ++- .../hbase/master/TestRegionPlacement.java | 8 +- .../hbase/master/TestRegionPlacement2.java | 8 +- .../hadoop/hbase/master/TestRegionPlan.java | 9 ++- .../hadoop/hbase/master/TestRegionState.java | 12 ++- .../hbase/master/TestRestartCluster.java | 9 ++- .../hbase/master/TestRollingRestart.java | 12 ++- .../master/TestShutdownBackupMaster.java | 13 ++-- .../hbase/master/TestSplitLogManager.java | 9 ++- .../hbase/master/TestTableStateManager.java | 16 ++-- .../hadoop/hbase/master/TestWarmupRegion.java | 9 ++- .../assignment/TestAssignmentManager.java | 17 ++-- .../assignment/TestAssignmentOnRSCrash.java | 9 ++- .../TestMergeTableRegionsProcedure.java | 13 ++-- .../master/assignment/TestRegionStates.java | 9 ++- .../assignment/TestRogueRSAssignment.java | 33 ++++---- .../TestSplitTableRegionProcedure.java | 13 ++-- .../master/balancer/BalancerTestBase.java | 6 -- .../master/balancer/TestBaseLoadBalancer.java | 8 +- .../balancer/TestDefaultLoadBalancer.java | 8 +- .../balancer/TestFavoredNodeTableImport.java | 13 +++- .../TestFavoredStochasticBalancerPickers.java | 8 +- .../TestFavoredStochasticLoadBalancer.java | 6 ++ .../balancer/TestRegionLocationFinder.java | 7 ++ .../balancer/TestRegionsOnMasterOptions.java | 22 +++--- .../master/balancer/TestServerAndLoad.java | 12 ++- .../balancer/TestStochasticLoadBalancer.java | 7 ++ .../balancer/TestStochasticLoadBalancer2.java | 19 ++--- ...tStochasticLoadBalancerBalanceCluster.java | 6 ++ ...estStochasticLoadBalancerLargeCluster.java | 6 ++ .../TestStochasticLoadBalancerMidCluster.java | 6 ++ ...stStochasticLoadBalancerRegionReplica.java | 6 ++ ...ticLoadBalancerRegionReplicaSameHosts.java | 6 ++ ...ticLoadBalancerRegionReplicaWithRacks.java | 6 ++ ...estStochasticLoadBalancerSmallCluster.java | 6 ++ .../master/cleaner/TestCleanerChore.java | 7 +- .../master/cleaner/TestHFileCleaner.java | 10 ++- .../master/cleaner/TestHFileLinkCleaner.java | 13 ++-- .../hbase/master/cleaner/TestLogsCleaner.java | 6 ++ .../cleaner/TestReplicationHFileCleaner.java | 40 ++++++---- .../cleaner/TestReplicationZKNodeCleaner.java | 8 +- .../cleaner/TestSnapshotFromMaster.java | 27 ++++--- .../hbase/master/locking/TestLockManager.java | 15 ++-- .../master/locking/TestLockProcedure.java | 21 ++--- .../TestSimpleRegionNormalizer.java | 14 +++- .../TestSimpleRegionNormalizerOnCluster.java | 9 ++- .../procedure/TestCloneSnapshotProcedure.java | 14 +++- .../TestCreateNamespaceProcedure.java | 9 ++- .../procedure/TestCreateTableProcedure.java | 8 +- ...DeleteColumnFamilyProcedureFromClient.java | 10 ++- .../TestDeleteNamespaceProcedure.java | 9 ++- .../procedure/TestDeleteTableProcedure.java | 12 +-- .../procedure/TestDisableTableProcedure.java | 12 +-- .../procedure/TestEnableTableProcedure.java | 12 +-- .../TestFastFailOnProcedureNotRegistered.java | 8 +- .../TestMasterFailoverWithProcedures.java | 13 ++-- .../procedure/TestMasterProcedureEvents.java | 8 +- .../TestMasterProcedureScheduler.java | 14 +++- ...stMasterProcedureSchedulerConcurrency.java | 18 +++-- .../TestMasterProcedureWalLease.java | 14 ++-- .../TestModifyNamespaceProcedure.java | 9 ++- .../procedure/TestModifyTableProcedure.java | 12 +-- .../master/procedure/TestProcedureAdmin.java | 13 ++-- .../TestRestoreSnapshotProcedure.java | 22 +++--- .../TestSafemodeBringsDownMaster.java | 9 ++- .../procedure/TestServerCrashProcedure.java | 8 +- ...TableDescriptorModificationFromClient.java | 9 ++- .../procedure/TestTruncateTableProcedure.java | 12 +-- .../TestWALProcedureStoreOnHDFS.java | 20 +++-- .../master/snapshot/TestAssignProcedure.java | 42 +++++----- .../snapshot/TestSnapshotFileCache.java | 9 ++- .../snapshot/TestSnapshotHFileCleaner.java | 13 +++- .../master/snapshot/TestSnapshotManager.java | 14 +++- .../hadoop/hbase/mob/TestCachedMobFile.java | 8 +- .../hbase/mob/TestDefaultMobStoreFlusher.java | 40 +++++----- .../hbase/mob/TestExpiredMobFileCleaner.java | 7 +- .../hbase/mob/TestMobDataBlockEncoding.java | 40 +++++----- .../apache/hadoop/hbase/mob/TestMobFile.java | 8 +- .../hadoop/hbase/mob/TestMobFileCache.java | 9 ++- .../hadoop/hbase/mob/TestMobFileLink.java | 9 ++- .../hadoop/hbase/mob/TestMobFileName.java | 8 +- .../mob/compactions/TestMobCompactor.java | 10 ++- .../TestPartitionedMobCompactionRequest.java | 9 ++- .../TestPartitionedMobCompactor.java | 9 ++- .../TestMemoryBoundedLogMessageBuffer.java | 12 ++- .../hbase/monitoring/TestTaskMonitor.java | 32 ++++---- .../hbase/namespace/TestNamespaceAuditor.java | 28 +++---- .../procedure/TestFailedProcCleanup.java | 36 +++++---- .../hadoop/hbase/procedure/TestProcedure.java | 11 ++- .../procedure/TestProcedureCoordinator.java | 11 ++- .../procedure/TestProcedureDescriber.java | 18 +++-- .../hbase/procedure/TestProcedureManager.java | 6 ++ .../hbase/procedure/TestProcedureMember.java | 12 ++- .../hbase/procedure/TestZKProcedure.java | 12 ++- .../procedure/TestZKProcedureControllers.java | 11 ++- .../hbase/protobuf/TestProtobufUtil.java | 12 ++- .../protobuf/TestReplicationProtobuf.java | 9 ++- .../quotas/TestActivePolicyEnforcement.java | 28 ++++--- .../TestFileSystemUtilizationChore.java | 24 +++--- .../quotas/TestGlobalQuotaSettingsImpl.java | 29 ++++--- .../hbase/quotas/TestMasterQuotaManager.java | 23 ++++-- .../quotas/TestMasterSpaceQuotaObserver.java | 25 +++--- ...TestMasterSpaceQuotaObserverWithMocks.java | 24 +++--- .../TestNamespaceQuotaViolationStore.java | 25 +++--- .../hadoop/hbase/quotas/TestQuotaAdmin.java | 27 ++++--- .../hbase/quotas/TestQuotaObserverChore.java | 25 +++--- .../TestQuotaObserverChoreRegionReports.java | 25 +++--- ...TestQuotaObserverChoreWithMiniCluster.java | 29 ++++--- .../hadoop/hbase/quotas/TestQuotaState.java | 35 +++++---- .../hbase/quotas/TestQuotaStatusRPCs.java | 25 +++--- .../hbase/quotas/TestQuotaTableUtil.java | 20 +++-- .../hbase/quotas/TestQuotaThrottle.java | 16 ++-- .../hadoop/hbase/quotas/TestRateLimiter.java | 8 +- .../TestRegionServerSpaceQuotaManager.java | 26 ++++--- .../hbase/quotas/TestRegionSizeUse.java | 25 +++--- .../TestSnapshotQuotaObserverChore.java | 24 ++++-- ...aceQuotaViolationPolicyRefresherChore.java | 24 +++--- .../hadoop/hbase/quotas/TestSpaceQuotas.java | 25 +++--- .../quotas/TestSpaceQuotasWithSnapshots.java | 31 +++++--- .../quotas/TestSuperUserQuotaPermissions.java | 25 +++--- .../quotas/TestTableQuotaViolationStore.java | 25 +++--- .../TestTableSpaceQuotaViolationNotifier.java | 27 ++++--- .../hbase/quotas/TestTablesWithQuotas.java | 25 +++--- ...oadCheckingViolationPolicyEnforcement.java | 24 +++--- ...isableTableViolationPolicyEnforcement.java | 23 ++++-- ...stNoInsertsViolationPolicyEnforcement.java | 23 ++++-- ...CompactionsViolationPolicyEnforcement.java | 23 ++++-- ...estNoWritesViolationPolicyEnforcement.java | 23 ++++-- .../regionserver/TestAtomicOperation.java | 8 +- .../hbase/regionserver/TestBlocksRead.java | 26 ++++--- .../hbase/regionserver/TestBlocksScanned.java | 11 ++- .../hbase/regionserver/TestBulkLoad.java | 21 +++-- .../TestCacheOnWriteInSchema.java | 10 ++- .../hbase/regionserver/TestCellFlatSet.java | 9 ++- .../regionserver/TestCellSkipListSet.java | 13 +++- .../TestClearRegionBlockCache.java | 17 ++-- .../hbase/regionserver/TestClusterId.java | 35 +++++---- .../hbase/regionserver/TestColumnSeeking.java | 14 ++-- .../regionserver/TestCompactSplitThread.java | 13 +++- .../regionserver/TestCompactingMemStore.java | 18 +++-- .../TestCompactingToCellFlatMapMemStore.java | 21 +++-- .../hbase/regionserver/TestCompaction.java | 10 ++- .../TestCompactionArchiveConcurrentClose.java | 7 ++ .../TestCompactionArchiveIOException.java | 7 ++ .../TestCompactionFileNotFound.java | 9 ++- .../TestCompactionInDeadRegionServer.java | 8 +- .../TestCompactionLifeCycleTracker.java | 7 +- .../regionserver/TestCompactionState.java | 13 +++- .../TestCompactionWithCoprocessor.java | 11 ++- .../regionserver/TestCompoundBloomFilter.java | 15 ++-- .../TestDateTieredCompactionPolicy.java | 7 +- ...estDateTieredCompactionPolicyOverflow.java | 7 +- .../TestDefaultCompactSelection.java | 7 +- .../regionserver/TestDefaultMemStore.java | 14 ++-- .../regionserver/TestDefaultStoreEngine.java | 11 ++- .../regionserver/TestDeleteMobTable.java | 7 +- .../TestEncryptionKeyRotation.java | 11 ++- .../TestEncryptionRandomKeying.java | 10 ++- .../TestEndToEndSplitTransaction.java | 9 ++- .../regionserver/TestFSErrorsExposed.java | 11 ++- .../regionserver/TestFailedAppendAndSync.java | 10 ++- .../TestFlushLifeCycleTracker.java | 7 +- .../regionserver/TestFlushRegionEntry.java | 7 ++ .../TestGetClosestAtOrBefore.java | 15 ++-- .../hbase/regionserver/TestHMobStore.java | 10 ++- .../hbase/regionserver/TestHRegion.java | 14 ++-- .../regionserver/TestHRegionFileSystem.java | 16 ++-- .../hbase/regionserver/TestHRegionInfo.java | 17 ++-- .../regionserver/TestHRegionOnCluster.java | 8 +- .../regionserver/TestHRegionReplayEvents.java | 6 ++ .../TestHRegionServerBulkLoad.java | 12 ++- ...estHRegionServerBulkLoadWithOldClient.java | 18 +++-- .../TestHRegionWithInMemoryFlush.java | 13 ++-- .../hadoop/hbase/regionserver/TestHStore.java | 7 ++ .../hbase/regionserver/TestHStoreFile.java | 18 +++-- .../regionserver/TestHdfsSnapshotHRegion.java | 7 +- .../regionserver/TestHeapMemoryManager.java | 10 ++- .../regionserver/TestJoinedScanners.java | 17 ++-- .../hbase/regionserver/TestKeepDeletes.java | 12 ++- .../hbase/regionserver/TestKeyValueHeap.java | 12 ++- .../regionserver/TestKeyValueScanFixture.java | 12 +-- .../regionserver/TestMajorCompaction.java | 9 ++- .../TestMasterAddressTracker.java | 13 +++- .../regionserver/TestMemStoreChunkPool.java | 49 ++++++------ .../hbase/regionserver/TestMemStoreLAB.java | 23 +++--- .../TestMemstoreLABWithoutPool.java | 10 ++- .../TestMetricsHeapMemoryManager.java | 34 ++++---- .../hbase/regionserver/TestMetricsRegion.java | 15 ++-- .../regionserver/TestMetricsRegionServer.java | 13 +++- .../TestMetricsTableAggregate.java | 6 ++ .../TestMetricsTableLatencies.java | 6 ++ .../hbase/regionserver/TestMinVersions.java | 13 +++- .../TestMiniBatchOperationInProgress.java | 18 +++-- .../regionserver/TestMinorCompaction.java | 10 ++- .../regionserver/TestMobStoreCompaction.java | 9 ++- .../regionserver/TestMobStoreScanner.java | 8 +- .../regionserver/TestMultiColumnScanner.java | 12 ++- .../regionserver/TestMultiLogThreshold.java | 28 ++++--- .../TestMultiVersionConcurrencyControl.java | 8 +- ...stMultiVersionConcurrencyControlBasic.java | 7 ++ .../regionserver/TestMutateRowsRecovery.java | 8 +- .../TestNewVersionBehaviorFromClientSide.java | 13 +++- .../TestObservedExceptionsInBatch.java | 6 ++ .../hbase/regionserver/TestParallelPut.java | 25 +++--- .../TestPerColumnFamilyFlush.java | 8 +- .../hbase/regionserver/TestPriorityRpc.java | 17 ++-- .../hbase/regionserver/TestQosFunction.java | 43 +++++----- .../TestRSKilledWhenInitializing.java | 21 +++-- .../regionserver/TestRSStatusServlet.java | 11 ++- .../regionserver/TestRecoveredEdits.java | 14 +++- .../regionserver/TestRegionFavoredNodes.java | 11 ++- .../regionserver/TestRegionIncrement.java | 14 ++-- .../regionserver/TestRegionInfoBuilder.java | 17 ++-- .../TestRegionMergeTransactionOnCluster.java | 43 +++++----- .../hbase/regionserver/TestRegionOpen.java | 19 +++-- .../TestRegionReplicaFailover.java | 12 ++- .../regionserver/TestRegionReplicas.java | 10 ++- .../TestRegionReplicasAreDistributed.java | 7 +- .../TestRegionReplicasWithModifyTable.java | 8 +- ...estRegionReplicasWithRestartScenarios.java | 16 ++-- .../regionserver/TestRegionServerAbort.java | 19 +++-- .../TestRegionServerAccounting.java | 7 +- .../TestRegionServerHostname.java | 8 +- .../regionserver/TestRegionServerMetrics.java | 12 +-- .../TestRegionServerNoMaster.java | 10 ++- .../TestRegionServerOnlineConfigChange.java | 13 +++- .../TestRegionServerReadRequestMetrics.java | 10 ++- .../TestRegionServerRegionSpaceUseReport.java | 24 +++--- .../TestRegionServerReportForDuty.java | 16 ++-- .../regionserver/TestRegionSplitPolicy.java | 7 +- .../regionserver/TestRemoveRegionMetrics.java | 16 ++-- .../regionserver/TestResettingCounters.java | 13 +++- .../regionserver/TestReversibleScanners.java | 34 ++++---- .../hbase/regionserver/TestRowTooBig.java | 12 ++- .../regionserver/TestRpcSchedulerFactory.java | 10 ++- .../regionserver/TestSCVFWithMiniCluster.java | 18 +++-- .../regionserver/TestScanWithBloomError.java | 10 ++- .../hbase/regionserver/TestScanner.java | 13 ++-- .../TestScannerHeartbeatMessages.java | 8 +- .../TestScannerRetriableFailure.java | 26 ++++--- .../regionserver/TestScannerWithBulkload.java | 8 +- .../TestScannerWithCorruptHFile.java | 9 ++- .../regionserver/TestSeekOptimizations.java | 14 ++-- .../regionserver/TestServerNonceManager.java | 10 ++- .../TestSettingTimeoutOnBlockingPoint.java | 10 ++- .../TestSimpleTimeRangeTracker.java | 7 +- .../regionserver/TestSplitLogWorker.java | 9 ++- .../TestSplitTransactionOnCluster.java | 14 ++-- .../regionserver/TestSplitWalDataLoss.java | 7 +- .../hbase/regionserver/TestStoreFileInfo.java | 11 ++- .../TestStoreFileRefresherChore.java | 7 +- ...estStoreFileScannerWithTagCompression.java | 8 +- .../hbase/regionserver/TestStoreScanner.java | 18 ++--- .../regionserver/TestStripeStoreEngine.java | 7 +- .../TestStripeStoreFileManager.java | 9 ++- .../regionserver/TestSwitchToStreamRead.java | 7 +- .../TestSyncTimeRangeTracker.java | 7 +- .../hadoop/hbase/regionserver/TestTags.java | 10 ++- .../TestTimestampFilterSeekHint.java | 15 ++-- .../hbase/regionserver/TestWALLockup.java | 13 +++- .../TestWALMonotonicallyIncreasingSeqId.java | 9 ++- .../TestWalAndCompactingMemStoreFlush.java | 15 ++-- .../hbase/regionserver/TestWideScanner.java | 12 ++- .../TestCompactedHFilesDischarger.java | 12 ++- .../compactions/TestDateTieredCompactor.java | 7 +- .../compactions/TestFIFOCompactionPolicy.java | 7 +- .../compactions/TestOffPeakHours.java | 9 ++- .../TestStripeCompactionPolicy.java | 8 +- .../compactions/TestStripeCompactor.java | 8 +- .../TestCompactionScanQueryMatcher.java | 7 +- .../TestExplicitColumnTracker.java | 11 ++- .../TestNewVersionBehaviorTracker.java | 14 ++-- .../querymatcher/TestScanDeleteTracker.java | 10 ++- .../TestScanWildcardColumnTracker.java | 11 ++- .../TestUserScanQueryMatcher.java | 9 ++- ...estCompactionWithThroughputController.java | 8 +- .../TestFlushWithThroughputController.java | 31 +++++--- .../regionserver/wal/TestAsyncFSWAL.java | 17 ++-- .../wal/TestAsyncLogRollPeriod.java | 6 ++ .../regionserver/wal/TestAsyncLogRolling.java | 12 +-- .../wal/TestAsyncProtobufLog.java | 20 +++-- .../regionserver/wal/TestAsyncWALReplay.java | 17 ++-- .../wal/TestAsyncWALReplayCompressed.java | 6 ++ .../regionserver/wal/TestCompressor.java | 15 ++-- .../wal/TestCustomWALCellCodec.java | 6 ++ .../regionserver/wal/TestDurability.java | 7 ++ .../hbase/regionserver/wal/TestFSHLog.java | 13 +++- .../regionserver/wal/TestFSWALEntry.java | 14 ++-- .../regionserver/wal/TestLogRollAbort.java | 7 ++ .../regionserver/wal/TestLogRollPeriod.java | 6 ++ .../regionserver/wal/TestLogRolling.java | 12 +-- .../wal/TestLogRollingNoCluster.java | 12 +-- .../regionserver/wal/TestMetricsWAL.java | 22 +++--- .../regionserver/wal/TestProtobufLog.java | 9 ++- .../wal/TestSecureAsyncWALReplay.java | 11 +-- .../regionserver/wal/TestSecureWALReplay.java | 8 +- .../wal/TestSequenceIdAccounting.java | 8 +- .../regionserver/wal/TestSyncFuture.java | 6 ++ .../wal/TestWALActionsListener.java | 7 +- .../wal/TestWALCellCodecWithCompression.java | 8 +- .../hbase/regionserver/wal/TestWALReplay.java | 13 ++-- ...TestWALReplayBoundedLogWriterCreation.java | 9 ++- .../wal/TestWALReplayCompressed.java | 6 ++ .../replication/TestMasterReplication.java | 6 ++ .../TestMultiSlaveReplication.java | 13 ++-- .../replication/TestNamespaceReplication.java | 9 ++- .../TestPerTableCFReplication.java | 16 ++-- ...tReplicationChangingPeerRegionservers.java | 11 ++- .../TestReplicationDisableInactivePeer.java | 13 +++- .../TestReplicationDroppedTables.java | 23 +++--- .../TestReplicationEmptyWALRecovery.java | 6 ++ .../replication/TestReplicationEndpoint.java | 9 ++- .../TestReplicationKillMasterRS.java | 8 +- ...TestReplicationKillMasterRSCompressed.java | 7 +- .../replication/TestReplicationKillRS.java | 13 +++- .../TestReplicationKillSlaveRS.java | 8 +- .../TestReplicationSmallTests.java | 6 ++ .../replication/TestReplicationSource.java | 33 ++++---- .../TestReplicationStateZKImpl.java | 7 +- .../replication/TestReplicationStatus.java | 7 ++ .../TestReplicationSyncUpTool.java | 15 ++-- .../TestReplicationTrackerZKImpl.java | 10 ++- .../TestReplicationWALEntryFilters.java | 10 ++- .../replication/TestReplicationWithTags.java | 7 +- .../replication/TestSerialReplication.java | 12 ++- .../master/TestTableCFsUpdater.java | 22 +++--- ...plicationEndpointWithMultipleAsyncWAL.java | 7 ++ ...estReplicationEndpointWithMultipleWAL.java | 7 ++ ...asterRSCompressedWithMultipleAsyncWAL.java | 7 ++ ...KillMasterRSCompressedWithMultipleWAL.java | 7 ++ ...icationSyncUpToolWithMultipleAsyncWAL.java | 7 ++ ...tReplicationSyncUpToolWithMultipleWAL.java | 7 ++ .../regionserver/TestGlobalThrottler.java | 12 ++- .../TestRegionReplicaReplicationEndpoint.java | 18 +++-- ...ionReplicaReplicationEndpointNoMaster.java | 25 +++--- .../regionserver/TestReplicationSink.java | 25 +++--- .../TestReplicationSinkManager.java | 16 ++-- .../TestReplicationSourceManager.java | 6 ++ .../TestReplicationSourceManagerZkImpl.java | 53 +++++++------ .../TestReplicationThrottler.java | 8 +- .../regionserver/TestReplicator.java | 11 ++- .../regionserver/TestWALEntrySinkFilter.java | 15 ++-- .../regionserver/TestWALEntryStream.java | 6 ++ .../hadoop/hbase/security/TestSecureIPC.java | 20 +++-- .../hadoop/hbase/security/TestUser.java | 18 +++-- .../TestUsersOperationsWithSecureHadoop.java | 10 ++- .../access/TestAccessControlFilter.java | 15 ++-- .../security/access/TestAccessController.java | 9 ++- .../access/TestAccessController2.java | 14 +++- .../access/TestAccessController3.java | 11 ++- .../TestCellACLWithMultipleVersions.java | 12 ++- .../hbase/security/access/TestCellACLs.java | 12 ++- ...estCoprocessorWhitelistMasterObserver.java | 17 ++-- .../access/TestNamespaceCommands.java | 15 +++- .../access/TestScanEarlyTermination.java | 12 ++- .../security/access/TestTablePermissions.java | 15 +++- .../access/TestWithDisabledAuthorization.java | 12 ++- .../access/TestZKPermissionWatcher.java | 12 ++- .../security/token/TestAuthenticationKey.java | 14 ++-- .../TestDelegationTokenWithEncryption.java | 7 +- .../token/TestGenerateDelegationToken.java | 8 +- .../token/TestTokenAuthentication.java | 24 +++--- .../security/token/TestZKSecretWatcher.java | 11 ++- .../token/TestZKSecretWatcherRefreshKeys.java | 10 ++- .../TestDefaultScanLabelGeneratorStack.java | 9 ++- .../TestEnforcingScanLabelGenerator.java | 7 +- .../visibility/TestExpressionExpander.java | 10 ++- .../visibility/TestExpressionParser.java | 10 ++- ...bilityLabelReplicationWithExpAsString.java | 8 +- ...bilityLabelsOpWithDifferentUsersNoACL.java | 17 ++-- .../TestVisibilityLabelsReplication.java | 8 +- .../TestVisibilityLabelsWithACL.java | 14 ++-- ...sibilityLabelsWithCustomVisLabService.java | 7 +- ...ilityLabelsWithDefaultVisLabelService.java | 13 +++- .../TestVisibilityLabelsWithDeletes.java | 8 +- .../TestVisibilityLabelsWithSLGStack.java | 9 ++- .../TestVisibilityLablesWithGroups.java | 10 ++- .../TestVisibilityWithCheckAuths.java | 8 +- .../TestWithDisabledAuthorization.java | 10 ++- .../snapshot/TestFlushSnapshotFromClient.java | 20 ++--- .../TestMobFlushSnapshotFromClient.java | 11 ++- ...TestMobRestoreFlushSnapshotFromClient.java | 8 +- .../TestMobRestoreSnapshotHelper.java | 10 ++- .../snapshot/TestRegionSnapshotTask.java | 32 ++++---- .../TestRestoreFlushSnapshotFromClient.java | 8 +- .../snapshot/TestRestoreSnapshotHelper.java | 15 +++- .../snapshot/TestSnapshotClientRetries.java | 15 ++-- .../TestSnapshotDescriptionUtils.java | 11 ++- .../hbase/snapshot/TestSnapshotManifest.java | 20 +++-- .../hadoop/hbase/tool/TestCanaryTool.java | 39 ++++++---- .../hbase/tool/TestLoadIncrementalHFiles.java | 12 ++- ...estLoadIncrementalHFilesSplitRecovery.java | 12 ++- .../tool/TestSecureLoadIncrementalHFiles.java | 8 +- ...ureLoadIncrementalHFilesSplitRecovery.java | 6 ++ .../hadoop/hbase/trace/TestHTraceHooks.java | 14 +++- .../hbase/util/TestBloomFilterChunk.java | 12 +-- .../TestBoundedPriorityBlockingQueue.java | 37 +++++---- .../hadoop/hbase/util/TestByteBuffUtils.java | 33 ++++---- .../hbase/util/TestCompressionTest.java | 24 +++--- .../hbase/util/TestConfigurationUtil.java | 33 ++++---- .../hbase/util/TestConnectionCache.java | 12 ++- .../hbase/util/TestCoprocessorScanPolicy.java | 11 ++- .../util/TestDefaultEnvironmentEdge.java | 19 +++-- .../hadoop/hbase/util/TestEncryptionTest.java | 13 ++-- .../hadoop/hbase/util/TestFSHDFSUtils.java | 14 ++-- .../hbase/util/TestFSTableDescriptors.java | 12 ++- .../apache/hadoop/hbase/util/TestFSUtils.java | 9 ++- .../hadoop/hbase/util/TestFSVisitor.java | 13 +++- .../util/TestFromClientSide3WoUnsafe.java | 8 +- .../hbase/util/TestHBaseFsckComparator.java | 9 ++- .../hbase/util/TestHBaseFsckEncryption.java | 10 ++- .../hadoop/hbase/util/TestHBaseFsckMOB.java | 22 +++--- .../hbase/util/TestHFileArchiveUtil.java | 16 ++-- .../apache/hadoop/hbase/util/TestIdLock.java | 11 ++- .../hbase/util/TestIdReadWriteLock.java | 13 ++-- .../util/TestIncrementingEnvironmentEdge.java | 9 ++- .../hadoop/hbase/util/TestJSONMetricUtil.java | 27 ++++--- .../util/TestMiniClusterLoadEncoded.java | 32 ++++---- .../util/TestMiniClusterLoadParallel.java | 31 +++++--- .../util/TestMiniClusterLoadSequential.java | 31 +++++--- .../apache/hadoop/hbase/util/TestPoolMap.java | 10 ++- .../hadoop/hbase/util/TestRegionMover.java | 11 ++- .../hbase/util/TestRegionSplitCalculator.java | 16 ++-- .../hadoop/hbase/util/TestRegionSplitter.java | 13 +++- .../hadoop/hbase/util/TestRootPath.java | 13 ++-- .../hadoop/hbase/util/TestSortedList.java | 15 ++-- .../hadoop/hbase/util/TestStealJobQueue.java | 24 +++--- .../TestBoundedRegionGroupingStrategy.java | 8 +- .../hadoop/hbase/wal/TestFSHLogProvider.java | 9 ++- .../hadoop/hbase/wal/TestSecureWAL.java | 6 ++ .../hadoop/hbase/wal/TestWALFactory.java | 7 ++ .../hadoop/hbase/wal/TestWALFiltering.java | 22 ++++-- .../hadoop/hbase/wal/TestWALMethods.java | 19 +++-- .../wal/TestWALOpenAfterDNRollingStart.java | 8 +- .../hbase/wal/TestWALReaderOnSecureWAL.java | 9 ++- .../hadoop/hbase/wal/TestWALRootDir.java | 7 ++ .../apache/hadoop/hbase/wal/TestWALSplit.java | 8 +- .../TestWALSplitBoundedLogWriterCreation.java | 8 +- .../hbase/wal/TestWALSplitCompressed.java | 8 +- .../hbase/zookeeper/TestZooKeeperACL.java | 9 ++- .../hbase/client/TestReplicationShell.java | 8 +- .../apache/hadoop/hbase/client/TestShell.java | 8 +- .../hbase/client/TestShellNoCluster.java | 8 +- .../client/rsgroup/TestShellRSGroups.java | 15 ++-- .../hadoop/hbase/thrift/TestCallQueue.java | 39 +++++----- .../hbase/thrift/TestThriftHttpServer.java | 35 +++++---- .../hadoop/hbase/thrift/TestThriftServer.java | 11 ++- .../hbase/thrift/TestThriftServerCmdLine.java | 35 +++++---- .../TestThriftHBaseServiceHandler.java | 11 ++- ...stThriftHBaseServiceHandlerWithLabels.java | 8 +- ...ThriftHBaseServiceHandlerWithReadOnly.java | 10 ++- .../hbase/zookeeper/TestHQuorumPeer.java | 8 +- .../hbase/zookeeper/TestInstancePending.java | 8 +- .../hbase/zookeeper/TestReadOnlyZKClient.java | 6 ++ .../zookeeper/TestRecoverableZooKeeper.java | 7 +- .../hbase/zookeeper/TestZKLeaderManager.java | 8 +- .../hbase/zookeeper/TestZKMainServer.java | 9 ++- .../hadoop/hbase/zookeeper/TestZKMulti.java | 10 ++- .../hbase/zookeeper/TestZKNodeTracker.java | 9 ++- .../hadoop/hbase/zookeeper/TestZKUtil.java | 6 ++ .../hbase/zookeeper/TestZKUtilNoServer.java | 19 +++-- pom.xml | 6 ++ 1112 files changed, 10501 insertions(+), 5237 deletions(-) delete mode 100644 hbase-common/src/test/java/org/apache/hadoop/hbase/CategoryBasedTimeout.java diff --git a/hbase-archetypes/hbase-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/client/TestHelloHBase.java b/hbase-archetypes/hbase-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/client/TestHelloHBase.java index be028bec0d8..9a92e606ffb 100644 --- a/hbase-archetypes/hbase-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/client/TestHelloHBase.java +++ b/hbase-archetypes/hbase-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/client/TestHelloHBase.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,7 +20,7 @@ package org.apache.hbase.archetypes.exemplars.client; import static org.junit.Assert.assertEquals; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.client.Admin; @@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -42,6 +42,10 @@ import org.junit.experimental.categories.Category; @Category(MediumTests.class) public class TestHelloHBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHelloHBase.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-archetypes/hbase-shaded-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/shaded_client/TestHelloHBase.java b/hbase-archetypes/hbase-shaded-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/shaded_client/TestHelloHBase.java index 1656e35c087..0f0f7d91ade 100644 --- a/hbase-archetypes/hbase-shaded-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/shaded_client/TestHelloHBase.java +++ b/hbase-archetypes/hbase-shaded-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/shaded_client/TestHelloHBase.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,7 +20,7 @@ package org.apache.hbase.archetypes.exemplars.shaded_client; import static org.junit.Assert.assertEquals; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.client.Admin; @@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -42,6 +42,10 @@ import org.junit.experimental.categories.Category; @Category(MediumTests.class) public class TestHelloHBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHelloHBase.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java index 5733c86c5d6..1429e1cc83e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.util.BuilderStyleTest; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PrettyPrinter; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,6 +42,11 @@ import org.junit.rules.ExpectedException; @Category({MiscTests.class, SmallTests.class}) @Deprecated public class TestHColumnDescriptor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHColumnDescriptor.class); + @Rule public ExpectedException expectedEx = ExpectedException.none(); @Test diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java index bad78264aca..7b230eb6044 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java @@ -25,13 +25,13 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.Arrays; import java.util.regex.Pattern; - import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.BuilderStyleTest; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -45,6 +45,11 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) @Deprecated public class TestHTableDescriptor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHTableDescriptor.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHTableDescriptor.class); @Rule diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java index ae040bf4923..493e23a8398 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import java.io.IOException; @@ -27,18 +26,18 @@ import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.Set; - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.Triple; import org.apache.hadoop.hbase.ClassFinder.And; import org.apache.hadoop.hbase.ClassFinder.FileNameFilter; import org.apache.hadoop.hbase.ClassFinder.Not; import org.apache.hadoop.hbase.ClassTestFinder.TestClassFilter; import org.apache.hadoop.hbase.ClassTestFinder.TestFileNameFilter; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.Triple; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -72,6 +71,10 @@ import org.slf4j.LoggerFactory; @Category(SmallTests.class) public class TestInterfaceAudienceAnnotations { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestInterfaceAudienceAnnotations.class); + private static final String HBASE_PROTOBUF = "org.apache.hadoop.hbase.protobuf.generated"; private static final Logger LOG = LoggerFactory.getLogger(TestInterfaceAudienceAnnotations.class); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestRegionLocations.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestRegionLocations.java index dddfb82f244..f43ce4a52bb 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestRegionLocations.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestRegionLocations.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; @@ -26,12 +25,17 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ClientTests.class, SmallTests.class}) public class TestRegionLocations { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionLocations.class); + ServerName sn0 = ServerName.valueOf("host0", 10, 10); ServerName sn1 = ServerName.valueOf("host1", 10, 10); ServerName sn2 = ServerName.valueOf("host2", 10, 10); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java index fc5979390de..3a746828b90 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; @@ -50,8 +49,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CallQueueTooBigException; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; @@ -70,19 +69,21 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; -import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({ClientTests.class, MediumTests.class}) public class TestAsyncProcess { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncProcess.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAsyncProcess.class); private static final TableName DUMMY_TABLE = TableName.valueOf("DUMMY_TABLE"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java index 2eca92f0ad1..cc6f6f5ae4e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,20 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import java.util.Arrays; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ClientTests.class, SmallTests.class}) public class TestAttributes { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAttributes.class); + private static final byte [] ROW = new byte [] {'r'}; @Test public void testPutAttributes() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java index 50befce82ae..f8e12954b1e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java @@ -20,13 +20,14 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,6 +35,11 @@ import org.junit.rules.TestName; @Category({SmallTests.class, ClientTests.class}) public class TestBufferedMutator { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBufferedMutator.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutatorParams.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutatorParams.java index 2c2935e63f1..73953d0db75 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutatorParams.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutatorParams.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -21,6 +20,7 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; + import java.util.Collection; import java.util.List; import java.util.concurrent.Callable; @@ -29,9 +29,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -39,6 +41,11 @@ import org.junit.rules.TestName; @Category({ ClientTests.class, SmallTests.class }) public class TestBufferedMutatorParams { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBufferedMutatorParams.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java index dc0459d4935..80d2e2584e9 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java @@ -17,25 +17,32 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.backoff.ExponentialClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ServerStatistics; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; @Category({ClientTests.class, SmallTests.class}) public class TestClientExponentialBackoff { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientExponentialBackoff.class); + ServerName server = Mockito.mock(ServerName.class); byte[] regionname = Bytes.toBytes("region"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index ec8f8010dcc..3cab09dec58 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -34,12 +34,12 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; - import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -60,17 +60,20 @@ import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Stopwatch; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest; @@ -102,6 +105,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpeci */ @Category({ClientTests.class, SmallTests.class}) public class TestClientNoCluster extends Configured implements Tool { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientNoCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestClientNoCluster.class); private Configuration conf; public static final ServerName META_SERVERNAME = diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java index 41d92222b4f..cfe9e07616a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java @@ -33,10 +33,10 @@ import java.io.IOException; import java.util.Iterator; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.RegionLocations; @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -62,6 +63,10 @@ import org.mockito.stubbing.Answer; @Category(SmallTests.class) public class TestClientScanner { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientScanner.class); + Scan scan; ExecutorService pool; Configuration conf; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestColumnFamilyDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestColumnFamilyDescriptorBuilder.java index 1022ee2d4a0..9a24e76b314 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestColumnFamilyDescriptorBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestColumnFamilyDescriptorBuilder.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; @@ -35,6 +36,7 @@ import org.apache.hadoop.hbase.util.BuilderStyleTest; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PrettyPrinter; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +45,10 @@ import org.junit.rules.ExpectedException; @Category({MiscTests.class, SmallTests.class}) public class TestColumnFamilyDescriptorBuilder { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestColumnFamilyDescriptorBuilder.class); + @Rule public ExpectedException expectedEx = ExpectedException.none(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDelayingRunner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDelayingRunner.java index b8400538f01..22e718bf95f 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDelayingRunner.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDelayingRunner.java @@ -21,18 +21,23 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.concurrent.atomic.AtomicLong; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestDelayingRunner { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDelayingRunner.class); + private static final TableName DUMMY_TABLE = TableName.valueOf("DUMMY_TABLE"); private static final byte[] DUMMY_BYTES_1 = Bytes.toBytes("DUMMY_BYTES_1"); @@ -52,7 +57,7 @@ public class TestDelayingRunner { @Override public void run() { endTime.set(EnvironmentEdgeManager.currentTime()); - } + } }); long startTime = EnvironmentEdgeManager.currentTime(); runner.run(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDeleteTimeStamp.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDeleteTimeStamp.java index debf2bd9b27..e855055fd88 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDeleteTimeStamp.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDeleteTimeStamp.java @@ -1,30 +1,42 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.hadoop.hbase.client; import java.util.List; import java.util.Map.Entry; import java.util.NavigableMap; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ClientTests.class, SmallTests.class}) public class TestDeleteTimeStamp { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDeleteTimeStamp.class); + private static final byte[] ROW = Bytes.toBytes("testRow"); private static final byte[] FAMILY = Bytes.toBytes("testFamily"); private static final byte[] QUALIFIER = Bytes.toBytes("testQualifier"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestGet.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestGet.java index 6a2bb396695..f5662238243 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestGet.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestGet.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; @@ -27,20 +25,18 @@ import static org.junit.Assert.fail; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; -import java.nio.ByteBuffer; import java.lang.reflect.InvocationTargetException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.Set; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.KeyOnlyFilter; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.visibility.Authorizations; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -48,12 +44,21 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Base64; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; + // TODO: cover more test cases @Category({ClientTests.class, SmallTests.class}) public class TestGet { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGet.class); + private static final byte [] ROW = new byte [] {'r'}; private static final String PB_GET = "CgNyb3ciEwoPdGVzdC5Nb2NrRmlsdGVyEgAwATgB"; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java index ef59eed42af..cce4939279d 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -16,14 +17,6 @@ */ package org.apache.hadoop.hbase.client; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import java.io.IOException; - import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; import static org.mockito.Mockito.doCallRealMethod; @@ -32,9 +25,22 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + @Category(SmallTests.class) public class TestHTableMultiplexerViaMocks { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHTableMultiplexerViaMocks.class); + private HTableMultiplexer mockMultiplexer; private ClusterConnection mockConnection; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHColumnDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHColumnDescriptor.java index 79ec9593183..1ac483da91f 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHColumnDescriptor.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHColumnDescriptor.java @@ -15,12 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; +import static org.junit.Assert.fail; + import java.util.Arrays; import java.util.List; import java.util.function.Consumer; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.MemoryCompactionPolicy; @@ -31,7 +33,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.BuilderStyleTest; import org.apache.hadoop.hbase.util.Bytes; -import static org.junit.Assert.fail; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -39,6 +41,11 @@ import org.junit.rules.TestName; @Category({ClientTests.class, SmallTests.class}) public class TestImmutableHColumnDescriptor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImmutableHColumnDescriptor.class); + @Rule public TestName name = new TestName(); private static final List> TEST_FUNCTION = Arrays.asList( diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java index 4644641115c..68afeec2663 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java @@ -17,14 +17,17 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.fail; + import java.util.Arrays; import java.util.List; import java.util.function.Consumer; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import static org.junit.Assert.fail; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -36,6 +39,10 @@ import org.junit.rules.TestName; @Category({ClientTests.class, SmallTests.class}) public class TestImmutableHRegionInfo { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImmutableHRegionInfo.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java index fe1e4a3c368..b83c01ab29a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java @@ -15,13 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; +import static org.junit.Assert.fail; + import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.function.Consumer; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -29,7 +31,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.BuilderStyleTest; import org.apache.hadoop.hbase.util.Bytes; -import static org.junit.Assert.fail; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -37,6 +39,11 @@ import org.junit.rules.TestName; @Category({ClientTests.class, SmallTests.class}) public class TestImmutableHTableDescriptor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImmutableHTableDescriptor.class); + @Rule public TestName name = new TestName(); private static final List> TEST_FUNCTION = Arrays.asList( diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestIncrement.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestIncrement.java index e7bd5bc04ae..75bad5ea416 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestIncrement.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestIncrement.java @@ -21,14 +21,21 @@ import static org.junit.Assert.assertEquals; import java.util.Map; import java.util.NavigableMap; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; + @Category({ClientTests.class, SmallTests.class}) public class TestIncrement { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrement.class); + @Test public void testIncrementInstance() { final long expected = 13; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java index 8ddb392fd09..ed72ac1df7a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestInterfaceAlign.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertTrue; @@ -27,10 +25,11 @@ import java.lang.reflect.Method; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; - import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -39,6 +38,10 @@ import org.slf4j.LoggerFactory; @Category({ ClientTests.class, SmallTests.class }) public class TestInterfaceAlign { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestInterfaceAlign.class); + private static final Logger LOG = LoggerFactory.getLogger(TestInterfaceAlign.class); /** diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java index c0cbaae36a5..97a672db6fa 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java @@ -17,37 +17,45 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertEquals; + import com.codahale.metrics.RatioGauge; import com.codahale.metrics.RatioGauge.Ratio; -import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; +import java.io.IOException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MetricsTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; -import java.io.IOException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicBoolean; -import static org.junit.Assert.assertEquals; +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; @Category({ClientTests.class, MetricsTests.class, SmallTests.class}) public class TestMetricsConnection { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsConnection.class); + private static MetricsConnection METRICS; private static final ExecutorService BATCH_POOL = Executors.newFixedThreadPool(2); @BeforeClass diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMutation.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMutation.java index 9b8144db863..3c26db7112c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMutation.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMutation.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertNotEquals; @@ -29,17 +27,23 @@ import org.apache.hadoop.hbase.Cell.Type; import org.apache.hadoop.hbase.CellBuilderFactory; import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ SmallTests.class, ClientTests.class }) public class TestMutation { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMutation.class); + @Test public void testAppendCopyConstructor() throws IOException { Append origin = new Append(Bytes.toBytes("ROW-01")); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java index cf40a690938..14b75c2cffb 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,17 +21,16 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import com.fasterxml.jackson.databind.ObjectMapper; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.filter.BinaryComparator; @@ -65,6 +63,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.BuilderStyleTest; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -74,6 +73,11 @@ import org.junit.experimental.categories.Category; */ @Category({ClientTests.class, SmallTests.class}) public class TestOperation { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOperation.class); + private static byte [] ROW = Bytes.toBytes("testRow"); private static byte [] FAMILY = Bytes.toBytes("testFamily"); private static byte [] QUALIFIER = Bytes.toBytes("testQualifier"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java index f94a6a6a698..44bd6e801db 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; @@ -27,18 +25,25 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse; + @Category({ClientTests.class, SmallTests.class}) public class TestProcedureFuture { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureFuture.class); + private static class TestFuture extends HBaseAdmin.ProcedureFuture { private boolean postOperationResultCalled = false; private boolean waitOperationResultCalled = false; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestPutDotHas.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestPutDotHas.java index b90374b0da1..ef9d4c96d28 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestPutDotHas.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestPutDotHas.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,11 +17,13 @@ */ package org.apache.hadoop.hbase.client; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -32,6 +34,10 @@ import org.junit.experimental.categories.Category; */ public class TestPutDotHas { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPutDotHas.class); + public static final byte[] ROW_01 = Bytes.toBytes("row-01"); public static final byte[] QUALIFIER_01 = Bytes.toBytes("qualifier-01"); public static final byte[] VALUE_01 = Bytes.toBytes("value-01"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java index a20128564bf..1a6f2f7d5b1 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -19,9 +19,8 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.nio.charset.StandardCharsets; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.RegionState; @@ -29,17 +28,19 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; @Category({MasterTests.class, SmallTests.class}) public class TestRegionInfoDisplay { - @Rule - public final TestRule timeout = CategoryBasedTimeout.builder() - .withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionInfoDisplay.class); + @Rule public TestName name = new TestName(); @Test diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRetriesExhaustedWithDetailsException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRetriesExhaustedWithDetailsException.java index 46f44faa070..7b584e94861 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRetriesExhaustedWithDetailsException.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRetriesExhaustedWithDetailsException.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,34 +17,28 @@ */ package org.apache.hadoop.hbase.client; -import org.apache.avro.generic.GenericData; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.mockito.Mockito; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.lang.reflect.Array; -import java.util.ArrayList; -import java.util.List; - -import static org.junit.Assert.assertTrue; @Category({SmallTests.class}) public class TestRetriesExhaustedWithDetailsException { - private static final Logger LOG = - LoggerFactory.getLogger(TestRetriesExhaustedWithDetailsException.class); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRetriesExhaustedWithDetailsException.class); + @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()). - withLookingForStuckThread(true). - build(); /** * Assert that a RetriesExhaustedException that has RegionTooBusyException outputs region name. diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java index 7ad7fcbfddc..1b554f76176 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; @@ -28,6 +29,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -39,6 +41,10 @@ import org.mockito.runners.MockitoJUnitRunner; @Category({ ClientTests.class, SmallTests.class }) public class TestReversedScannerCallable { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReversedScannerCallable.class); + @Mock private ClusterConnection connection; @Mock diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRowComparator.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRowComparator.java index 72c1b3e2379..64983089ae0 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRowComparator.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRowComparator.java @@ -27,14 +27,21 @@ import java.util.List; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ClientTests.class, SmallTests.class}) public class TestRowComparator { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowComparator.class); + private static final List DEFAULT_ROWS = IntStream.range(1, 9) .mapToObj(String::valueOf).map(Bytes::toBytes).collect(Collectors.toList()); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java index a687384deb2..18dcb46ce9c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,33 +15,38 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; import java.util.Arrays; import java.util.Set; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.security.visibility.Authorizations; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; // TODO: cover more test cases @Category({ClientTests.class, SmallTests.class}) public class TestScan { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScan.class); + @Test public void testAttributesSerialization() throws IOException { Scan scan = new Scan(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java index 37d5cb28277..3dc86de2f35 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -31,8 +31,8 @@ import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -44,12 +44,17 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ClientTests.class, SmallTests.class}) public class TestSimpleRequestController { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSimpleRequestController.class); + private static final TableName DUMMY_TABLE = TableName.valueOf("DUMMY_TABLE"); private static final byte[] DUMMY_BYTES_1 = Bytes.toBytes("DUMMY_BYTES_1"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java index 0ed2acbd582..a353a4085df 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java @@ -21,17 +21,16 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -39,14 +38,22 @@ import org.junit.rules.TestName; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; + /** * Test snapshot logic from the client */ @Category({SmallTests.class, ClientTests.class}) public class TestSnapshotFromAdmin { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotFromAdmin.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotFromAdmin.class); @Rule diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java index 639d97401a4..7794a048d42 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.client; -import org.apache.hadoop.hbase.*; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -25,12 +24,14 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.regex.Pattern; - +import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.BuilderStyleTest; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +44,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, SmallTests.class}) public class TestTableDescriptorBuilder { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableDescriptorBuilder.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableDescriptorBuilder.class); @Rule diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/exceptions/TestClientExceptionsUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/exceptions/TestClientExceptionsUtil.java index 27ede5a3c76..71c8405d910 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/exceptions/TestClientExceptionsUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/exceptions/TestClientExceptionsUtil.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,20 +17,26 @@ */ package org.apache.hadoop.hbase.exceptions; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import static org.junit.Assert.*; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; - -import static org.junit.Assert.*; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; @SuppressWarnings("ThrowableInstanceNeverThrown") @Category({ SmallTests.class, ClientTests.class }) public class TestClientExceptionsUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientExceptionsUtil.class); + @Test public void testFindException() throws Exception { IOException ioe = new IOException("Tesst"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java index e3a6927d5a1..ac17cba1711 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java @@ -22,20 +22,25 @@ import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; import org.apache.hadoop.hbase.ByteBufferKeyValue; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestComparators { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestComparators.class); + @Test public void testCellFieldsCompare() throws Exception { byte[] r0 = Bytes.toBytes("row0"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java index 28fe2bf6f41..4f9273963b7 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java @@ -22,8 +22,8 @@ import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; import java.util.Collection; import org.apache.hadoop.hbase.ByteBufferKeyValue; - import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyCell; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -43,6 +44,10 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) public class TestKeyOnlyFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeyOnlyFilter.class); + @Parameterized.Parameter public boolean lenAsVal; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestLongComparator.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestLongComparator.java index 9ecf95a35aa..340fc4d4eef 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestLongComparator.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestLongComparator.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -17,17 +17,23 @@ */ package org.apache.hadoop.hbase.filter; -import java.nio.ByteBuffer; +import static org.junit.Assert.assertEquals; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; - @Category(SmallTests.class) public class TestLongComparator { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLongComparator.class); + private long values[] = { Long.MIN_VALUE, -10000000000L, -1000000L, 0L, 1000000L, 10000000000L, Long.MAX_VALUE }; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java index 1ae2c4a9a11..18c1853f175 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java @@ -22,11 +22,11 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; - import org.apache.commons.lang3.time.StopWatch; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; @@ -42,6 +42,7 @@ import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.compress.GzipCodec; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -50,6 +51,10 @@ import org.slf4j.LoggerFactory; @Category({ ClientTests.class, SmallTests.class }) public class TestCellBlockBuilder { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellBlockBuilder.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCellBlockBuilder.class); private CellBlockBuilder builder; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestFailedServersLog.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestFailedServersLog.java index 03e3ca5039f..eb1877f189d 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestFailedServersLog.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestFailedServersLog.java @@ -1,14 +1,20 @@ /** - * 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. + * 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.ipc; import static org.hamcrest.CoreMatchers.is; @@ -16,8 +22,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import java.net.InetSocketAddress; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.log4j.Appender; @@ -26,6 +32,7 @@ import org.apache.log4j.LogManager; import org.apache.log4j.spi.LoggingEvent; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -38,6 +45,11 @@ import org.mockito.runners.MockitoJUnitRunner; @RunWith(MockitoJUnitRunner.class) @Category({ ClientTests.class, SmallTests.class }) public class TestFailedServersLog { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFailedServersLog.class); + static final int TEST_PORT = 9999; private InetSocketAddress addr; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java index 6990b30b035..2c1dd4f3d38 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.ipc; import static org.junit.Assert.assertEquals; @@ -24,19 +23,24 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, SmallTests.class }) public class TestHBaseRpcControllerImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseRpcControllerImpl.class); + @Test public void testListOfCellScannerables() throws IOException { final int count = 10; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java index 7c4ac029acd..4f6d6aa77f8 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java @@ -23,16 +23,21 @@ import static org.junit.Assert.assertTrue; import java.net.ConnectException; import java.net.InetSocketAddress; import java.net.SocketTimeoutException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, SmallTests.class }) public class TestIPCUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIPCUtil.class); + @Test public void testWrapException() throws Exception { final InetSocketAddress address = InetSocketAddress.createUnresolved("localhost", 0); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientDeprecatedNameMapping.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientDeprecatedNameMapping.java index 7ac5c2eab9f..ca2829a8065 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientDeprecatedNameMapping.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientDeprecatedNameMapping.java @@ -21,16 +21,22 @@ import static org.hamcrest.CoreMatchers.instanceOf; import static org.junit.Assert.assertThat; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, SmallTests.class }) public class TestRpcClientDeprecatedNameMapping { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRpcClientDeprecatedNameMapping.class); + @Test public void test() { Configuration conf = HBaseConfiguration.create(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaFilter.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaFilter.java index 728f37396c5..31c0bd67d9c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaFilter.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaFilter.java @@ -15,18 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.quotas; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.BuilderStyleTest; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ClientTests.class, SmallTests.class}) public class TestQuotaFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaFilter.class); + @Test public void testClassMethodsAreBuilderStyle() { /* ReplicationPeerConfig should have a builder style setup where setXXX/addXXX methods diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java index edf6926e250..2f1942c5be2 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaSettingsFactory.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -25,8 +26,13 @@ import static org.junit.Assert.fail; import java.util.List; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; @@ -35,9 +41,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitR import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; /** * Test class for {@link QuotaSettingsFactory}. @@ -45,6 +48,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestQuotaSettingsFactory { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaSettingsFactory.class); + @Test public void testAllQuotasAddedToList() { final SpaceQuota spaceQuota = SpaceQuota.newBuilder() diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java index 636b5e1dfcc..2406d10ed0a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceLimitSettings.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -22,15 +23,17 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; /** * Test class for {@link SpaceLimitSettings}. @@ -38,6 +41,10 @@ import org.junit.experimental.categories.Category; @Category({SmallTests.class}) public class TestSpaceLimitSettings { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSpaceLimitSettings.class); + @Test(expected = IllegalArgumentException.class) public void testInvalidTableQuotaSizeLimit() { new SpaceLimitSettings(TableName.valueOf("foo"), -1, SpaceViolationPolicy.NO_INSERTS); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestThrottleSettings.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestThrottleSettings.java index b6756fa9846..91241ae7ebf 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestThrottleSettings.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestThrottleSettings.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -22,18 +23,24 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; @Category({SmallTests.class}) public class TestThrottleSettings { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestThrottleSettings.class); + @Test public void testMerge() throws IOException { TimedQuota tq1 = TimedQuota.newBuilder().setSoftLimit(10) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationPeerConfig.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationPeerConfig.java index edda760fd6c..881ef459fa6 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationPeerConfig.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationPeerConfig.java @@ -15,18 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.BuilderStyleTest; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ClientTests.class, SmallTests.class}) public class TestReplicationPeerConfig { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationPeerConfig.class); + @Test public void testClassMethodsAreBuilderStyle() { /* ReplicationPeerConfig should have a builder style setup where setXXX/addXXX methods diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java index 907c809d502..59bebcf4a9a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -24,21 +24,26 @@ import static org.junit.Assert.fail; import java.security.Key; import java.security.KeyException; import java.security.SecureRandom; - import javax.crypto.spec.SecretKeySpec; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.io.crypto.aes.AES; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ClientTests.class, SmallTests.class}) public class TestEncryptionUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEncryptionUtil.class); + // There does not seem to be a ready way to test either getKeyFromBytesOrMasterKey // or createEncryptionContext, and the existing code under MobUtils appeared to be // untested. Not ideal! diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java index 700862e3d39..5e97ada2be3 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java @@ -28,7 +28,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.IOException; - import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; import javax.security.auth.callback.NameCallback; @@ -39,7 +38,7 @@ import javax.security.sasl.RealmCallback; import javax.security.sasl.RealmChoiceCallback; import javax.security.sasl.Sasl; import javax.security.sasl.SaslClient; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.security.AbstractHBaseSaslRpcClient.SaslClientCallbackHandler; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -51,6 +50,7 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -62,6 +62,10 @@ import org.apache.hbase.thirdparty.com.google.common.base.Strings; @Category({SecurityTests.class, SmallTests.class}) public class TestHBaseSaslRpcClient { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseSaslRpcClient.class); + static { System.setProperty("java.security.krb5.realm", "DOMAIN.COM"); System.setProperty("java.security.krb5.kdc", "DOMAIN.COM"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestSaslUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestSaslUtil.java index aea14a7d400..36f29dec240 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestSaslUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestSaslUtil.java @@ -19,19 +19,24 @@ package org.apache.hadoop.hbase.security; import static org.junit.Assert.assertEquals; +import java.util.Map; +import javax.security.sasl.Sasl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; -import javax.security.sasl.Sasl; -import java.util.Map; - @Category({SecurityTests.class, SmallTests.class}) public class TestSaslUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSaslUtil.class); + @Rule public ExpectedException exception = ExpectedException.none(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java index 7e44021dacd..77c0650ad3e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java @@ -22,22 +22,29 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.nio.ByteBuffer; - import org.apache.hadoop.hbase.ByteBufferKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.protobuf.Any; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; import org.apache.hbase.thirdparty.com.google.protobuf.BytesValue; + import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column; @@ -49,13 +56,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationPr import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Test; -import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestProtobufUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProtobufUtil.class); + public TestProtobufUtil() { } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZNodePaths.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZNodePaths.java index 102dde19271..1a45379edf6 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZNodePaths.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZNodePaths.java @@ -20,15 +20,21 @@ package org.apache.hadoop.hbase.zookeeper; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.ZKTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ZKTests.class, SmallTests.class }) public class TestZNodePaths { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZNodePaths.class); + @Test public void testIsClientReadable() { ZNodePaths znodePaths = new ZNodePaths(HBaseConfiguration.create()); diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml index 3284440482b..f5fa8ed4a46 100644 --- a/hbase-common/pom.xml +++ b/hbase-common/pom.xml @@ -129,17 +129,6 @@ - - maven-surefire-plugin - - - - listener - org.apache.hadoop.hbase.ResourceCheckerJUnitListener - - - - org.apache.maven.plugins diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/CategoryBasedTimeout.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/CategoryBasedTimeout.java deleted file mode 100644 index 8a31ff3789d..00000000000 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/CategoryBasedTimeout.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase; - -import java.lang.annotation.Annotation; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.experimental.categories.Category; -import org.junit.rules.Timeout; - -/** - * Set a test method timeout based off the test categories small, medium, large. - * Based on junit Timeout TestRule; see https://github.com/junit-team/junit/wiki/Rules - */ -public class CategoryBasedTimeout extends Timeout { - - public static Timeout forClass(Class clazz) { - return CategoryBasedTimeout.builder().withTimeout(clazz).withLookingForStuckThread(true) - .build(); - } - - public static Builder builder() { - return new CategoryBasedTimeout.Builder(); - } - - @Deprecated - public CategoryBasedTimeout(int millis) { - super(millis); - } - - public CategoryBasedTimeout(long timeout, TimeUnit timeUnit) { - super(timeout, timeUnit); - } - - protected CategoryBasedTimeout(Builder builder) { - super(builder); - } - - public static class Builder extends Timeout.Builder { - public Timeout.Builder withTimeout(Class clazz) { - Annotation annotation = clazz.getAnnotation(Category.class); - if (annotation != null) { - Category category = (Category)annotation; - for (Class c: category.value()) { - if (c == SmallTests.class) { - // See SmallTests. Supposed to run 15 seconds. - return withTimeout(30, TimeUnit.SECONDS); - } else if (c == MediumTests.class) { - // See MediumTests. Supposed to run 50 seconds. - return withTimeout(180, TimeUnit.SECONDS); - } else if (c == LargeTests.class) { - // Let large tests have a ten minute timeout. - return withTimeout(10, TimeUnit.MINUTES); - } - } - } - return this; - } - } -} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java index 97a4276a02b..172e36d145c 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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; @@ -22,17 +23,22 @@ import static org.junit.Assert.assertFalse; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MiscTests.class, SmallTests.class }) public class TestByteBufferKeyValue { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteBufferKeyValue.class); + private static final String QUAL2 = "qual2"; private static final String FAM2 = "fam2"; private static final String QUAL1 = "qual1"; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellBuilder.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellBuilder.java index 1443878fb1f..051c16e65e1 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellBuilder.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellBuilder.java @@ -15,19 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestCellBuilder { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellBuilder.class); + private static final byte OLD_DATA = 87; private static final byte NEW_DATA = 100; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java index 4746bec262f..8652d820374 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -21,16 +21,21 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; - import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; + @Category({MiscTests.class, SmallTests.class}) public class TestCellComparator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellComparator.class); + private CellComparator comparator = CellComparator.getInstance(); byte[] row1 = Bytes.toBytes("row1"); byte[] row2 = Bytes.toBytes("row2"); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java index 1f95db98355..069bcfbc413 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; @@ -35,12 +34,18 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @Category({MiscTests.class, SmallTests.class}) public class TestCellUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellUtil.class); + /** * CellScannable used in test. Returns a {@link TestCellScanner} */ diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java index a28bad865da..dad1ce7c1bc 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java @@ -17,13 +17,11 @@ */ package org.apache.hadoop.hbase; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.hbase.TestChoreService.ScheduledChoreSamples.CountingChore; import org.apache.hadoop.hbase.TestChoreService.ScheduledChoreSamples.DoNothingChore; import org.apache.hadoop.hbase.TestChoreService.ScheduledChoreSamples.FailInitialChore; @@ -31,6 +29,7 @@ import org.apache.hadoop.hbase.TestChoreService.ScheduledChoreSamples.SampleStop import org.apache.hadoop.hbase.TestChoreService.ScheduledChoreSamples.SleepingChore; import org.apache.hadoop.hbase.TestChoreService.ScheduledChoreSamples.SlowChore; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -38,6 +37,11 @@ import org.slf4j.LoggerFactory; @Category(SmallTests.class) public class TestChoreService { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestChoreService.class); + public static final Logger log = LoggerFactory.getLogger(TestChoreService.class); /** diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClassFinder.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClassFinder.java index e9c3e604a50..0b17359eb1f 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClassFinder.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClassFinder.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -40,11 +39,11 @@ import java.util.jar.JarOutputStream; import java.util.jar.Manifest; import javax.tools.JavaCompiler; import javax.tools.ToolProvider; - import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,6 +54,10 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) public class TestClassFinder { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClassFinder.class); + private static final Logger LOG = LoggerFactory.getLogger(TestClassFinder.class); @Rule public TestName name = new TestName(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCompoundConfiguration.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCompoundConfiguration.java index 2b565f4ca30..6ae3efd15f4 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCompoundConfiguration.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCompoundConfiguration.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -25,17 +23,22 @@ import static org.junit.Assert.fail; import java.util.HashMap; import java.util.Map; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestCompoundConfiguration { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompoundConfiguration.class); + private Configuration baseConf; private int baseConfSize; @@ -51,7 +54,7 @@ public class TestCompoundConfiguration { @Test public void testBasicFunctionality() throws ClassNotFoundException { CompoundConfiguration compoundConf = new CompoundConfiguration() - .add(baseConf); + .add(baseConf); assertEquals("1", compoundConf.get("A")); assertEquals(2, compoundConf.getInt("B", 0)); assertEquals(3, compoundConf.getInt("C", 0)); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseConfiguration.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseConfiguration.java index 54252c78796..cb422c0e769 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseConfiguration.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseConfiguration.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; @@ -27,20 +26,25 @@ import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.AfterClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; @Category({MiscTests.class, SmallTests.class}) public class TestHBaseConfiguration { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseConfiguration.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHBaseConfiguration.class); private static HBaseCommonTestingUtility UTIL = new HBaseCommonTestingUtility(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestIndividualBytesFieldCell.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestIndividualBytesFieldCell.java index e2556773d70..ba4066b5bfd 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestIndividualBytesFieldCell.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestIndividualBytesFieldCell.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertArrayEquals; @@ -24,17 +23,22 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.nio.ByteBuffer; - import org.apache.hadoop.hbase.io.ByteArrayOutputStream; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestIndividualBytesFieldCell { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIndividualBytesFieldCell.class); + private static IndividualBytesFieldCell ic0 = null; private static KeyValue kv0 = null; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTableName.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTableName.java index 05d6bec2c25..43a384ade93 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTableName.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTableName.java @@ -25,10 +25,10 @@ import static org.junit.Assert.fail; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; - import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestWatcher; @@ -39,6 +39,11 @@ import org.junit.runner.Description; */ @Category({MiscTests.class, MediumTests.class}) public class TestTableName extends TestWatcher { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableName.class); + private TableName tableName; /** diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTagUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTagUtil.java index f43a4e0e147..2ae7288e555 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTagUtil.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTagUtil.java @@ -20,15 +20,19 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; import java.util.List; - import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MiscTests.class, SmallTests.class }) public class TestTagUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTagUtil.class); + @Test public void testCarryForwardTTLTag() throws Exception { // No tags so far and the TTL tag must get added to the Tags list diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimeout.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimeout.java index fc4a2be4a6a..343108e08b5 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimeout.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimeout.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -18,18 +18,17 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Ignore; -import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; @Category({SmallTests.class}) public class TestTimeout { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder() - .withTimeout(this.getClass()) - .withLookingForStuckThread(true) - .build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTimeout.class); @Test public void run1() throws InterruptedException { diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java index 72131718650..d4a3a471777 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java @@ -26,13 +26,14 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -42,6 +43,10 @@ import org.apache.hbase.thirdparty.com.google.common.io.CountingOutputStream; @Category({MiscTests.class, SmallTests.class}) public class TestCellCodec { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellCodec.class); + @Test public void testEmptyWorks() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java index 11d8832c313..5687d23d828 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java @@ -30,6 +30,7 @@ import java.util.List; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; @@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +48,10 @@ import org.apache.hbase.thirdparty.com.google.common.io.CountingOutputStream; @Category({MiscTests.class, SmallTests.class}) public class TestCellCodecWithTags { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellCodecWithTags.class); + @Test public void testCellWithTag() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java index 71feab635e8..18cdc1186a3 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java @@ -26,11 +26,12 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -39,6 +40,11 @@ import org.apache.hbase.thirdparty.com.google.common.io.CountingOutputStream; @Category({MiscTests.class, SmallTests.class}) public class TestKeyValueCodec { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeyValueCodec.class); + @Test public void testEmptyWorks() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java index e77bf7bb082..37336d1602b 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java @@ -30,6 +30,7 @@ import java.util.List; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; @@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +48,10 @@ import org.apache.hbase.thirdparty.com.google.common.io.CountingOutputStream; @Category({MiscTests.class, SmallTests.class}) public class TestKeyValueCodecWithTags { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeyValueCodecWithTags.class); + @Test public void testKeyValueWithTag() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferListOutputStream.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferListOutputStream.java index d28064c303d..2f7a869b700 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferListOutputStream.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferListOutputStream.java @@ -22,17 +22,22 @@ import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ IOTests.class, SmallTests.class }) public class TestByteBufferListOutputStream { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteBufferListOutputStream.class); + @Test public void testWrites() throws Exception { ByteBufferPool pool = new ByteBufferPool(10, 3); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferPool.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferPool.java index cf1f8ca2ebc..44d2f453584 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferPool.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferPool.java @@ -20,15 +20,20 @@ package org.apache.hadoop.hbase.io; import static org.junit.Assert.assertEquals; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ IOTests.class, SmallTests.class }) public class TestByteBufferPool { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteBufferPool.class); + @Test public void testOffheapBBPool() throws Exception { boolean directByteBuffer = true; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestMultiByteBuffInputStream.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestMultiByteBuffInputStream.java index ed96e872776..c2f17cbcf29 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestMultiByteBuffInputStream.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestMultiByteBuffInputStream.java @@ -23,17 +23,22 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.nio.MultiByteBuff; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ IOTests.class, SmallTests.class }) public class TestMultiByteBuffInputStream { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiByteBuffInputStream.class); + @Test public void testReads() throws Exception { ByteArrayOutputStream bos = new ByteArrayOutputStream(100); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java index 96181e17bec..b456f950c5a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io; import static org.junit.Assert.assertTrue; @@ -25,11 +24,11 @@ import java.io.DataOutputStream; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.ByteBufferExtendedCell; import org.apache.hadoop.hbase.ByteBufferKeyValue; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.io.util.LRUDictionary; @@ -38,12 +37,17 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestTagCompressionContext { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTagCompressionContext.class); + private static final byte[] ROW = Bytes.toBytes("r1"); private static final byte[] CF = Bytes.toBytes("f"); private static final byte[] Q = Bytes.toBytes("q"); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestCipherProvider.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestCipherProvider.java index 0f45e5d0716..0b749c5fde1 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestCipherProvider.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestCipherProvider.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.crypto; @@ -25,19 +26,24 @@ import java.io.InputStream; import java.io.OutputStream; import java.security.Key; import java.util.Arrays; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.crypto.aes.AES; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestCipherProvider { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCipherProvider.class); + public static class MyCipherProvider implements CipherProvider { private Configuration conf; @Override diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestEncryption.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestEncryption.java index 8ae20d5cd92..2a468897f2e 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestEncryption.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestEncryption.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.crypto; @@ -23,13 +24,14 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.security.Key; import javax.crypto.spec.SecretKeySpec; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -38,6 +40,10 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) public class TestEncryption { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEncryption.class); + private static final Logger LOG = LoggerFactory.getLogger(TestEncryption.class); @Test diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestKeyProvider.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestKeyProvider.java index 036ad603e3f..15e045f8efd 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestKeyProvider.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestKeyProvider.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.crypto; @@ -21,19 +22,24 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.security.Key; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.crypto.aes.AES; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestKeyProvider { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeyProvider.class); + @Test public void testTestProvider() { Configuration conf = HBaseConfiguration.create(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestKeyStoreKeyProvider.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestKeyStoreKeyProvider.java index 7037a346a23..e094a1daa82 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestKeyStoreKeyProvider.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestKeyStoreKeyProvider.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.crypto; @@ -27,12 +28,13 @@ import java.security.KeyStore; import java.security.MessageDigest; import java.util.Properties; import javax.crypto.spec.SecretKeySpec; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -41,6 +43,10 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) public class TestKeyStoreKeyProvider { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeyStoreKeyProvider.class); + private static final Logger LOG = LoggerFactory.getLogger(TestKeyStoreKeyProvider.class); static final HBaseCommonTestingUtility TEST_UTIL = new HBaseCommonTestingUtility(); static final String ALIAS = "test"; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/aes/TestAES.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/aes/TestAES.java index d0f2600e568..8e8dff30f8a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/aes/TestAES.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/aes/TestAES.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.crypto.aes; @@ -31,9 +32,9 @@ import java.security.SecureRandom; import java.security.SecureRandomSpi; import java.security.Security; import javax.crypto.spec.SecretKeySpec; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.io.crypto.Cipher; import org.apache.hadoop.hbase.io.crypto.DefaultCipherProvider; @@ -42,12 +43,17 @@ import org.apache.hadoop.hbase.io.crypto.Encryptor; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestAES { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAES.class); + // Validation for AES in CTR mode with a 128 bit key // From NIST Special Publication 800-38A @Test diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/aes/TestCommonsAES.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/aes/TestCommonsAES.java index de9b787596b..d285c7be879 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/aes/TestCommonsAES.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/aes/TestCommonsAES.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.crypto.aes; @@ -31,9 +32,9 @@ import java.security.SecureRandom; import java.security.SecureRandomSpi; import java.security.Security; import javax.crypto.spec.SecretKeySpec; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.io.crypto.Cipher; import org.apache.hadoop.hbase.io.crypto.DefaultCipherProvider; @@ -42,12 +43,17 @@ import org.apache.hadoop.hbase.io.crypto.Encryptor; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestCommonsAES { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCommonsAES.class); + // Validation for AES in CTR mode with a 128 bit key // From NIST Special Publication 800-38A @Test diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/hadoopbackport/TestThrottledInputStream.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/hadoopbackport/TestThrottledInputStream.java index 00bd1fbdeb4..63a4f40cc90 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/hadoopbackport/TestThrottledInputStream.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/hadoopbackport/TestThrottledInputStream.java @@ -1,31 +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 +/** + * 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 + * 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. + * 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.io.hadoopbackport; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestThrottledInputStream { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestThrottledInputStream.class); + @Test public void testCalSleepTimeMs() { // case 0: initial - no read, no sleep diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestLRUDictionary.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestLRUDictionary.java index c53c9f52595..bc8af936164 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestLRUDictionary.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestLRUDictionary.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io.util; import static org.junit.Assert.assertEquals; @@ -25,12 +24,13 @@ import static org.junit.Assert.assertTrue; import java.math.BigInteger; import java.util.Arrays; import java.util.Random; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -39,6 +39,11 @@ import org.junit.experimental.categories.Category; */ @Category({MiscTests.class, SmallTests.class}) public class TestLRUDictionary { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLRUDictionary.class); + LRUDictionary testee; @Before diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java index 3b724b1cf2c..16ff404fbdc 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -28,18 +26,23 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.nio.BufferOverflowException; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ObjectIntPair; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MiscTests.class, SmallTests.class }) public class TestMultiByteBuff { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiByteBuff.class); + @Test public void testWritesAndReads() { // Absolute reads diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestSingleByteBuff.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestSingleByteBuff.java index 98a1cc0fd8d..d471985142f 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestSingleByteBuff.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestSingleByteBuff.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -22,15 +20,20 @@ package org.apache.hadoop.hbase.nio; import static org.junit.Assert.assertEquals; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MiscTests.class, SmallTests.class }) public class TestSingleByteBuff { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSingleByteBuff.class); + @Test public void testPositionalReads() { // Off heap buffer diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestCopyOnWriteMaps.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestCopyOnWriteMaps.java index eca6c472fba..e3c8980f461 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestCopyOnWriteMaps.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestCopyOnWriteMaps.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.types; import static org.junit.Assert.assertEquals; @@ -28,16 +27,21 @@ import java.util.Map; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ThreadLocalRandom; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestCopyOnWriteMaps { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCopyOnWriteMaps.class); + private static final int MAX_RAND = 10 * 1000 * 1000; private ConcurrentNavigableMap m; private ConcurrentSkipListMap csm; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java index c2c5a6db706..d3cb4fba529 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java @@ -20,18 +20,24 @@ package org.apache.hadoop.hbase.types; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestFixedLengthWrapper { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFixedLengthWrapper.class); + static final byte[][] VALUES = new byte[][] { Bytes.toBytes(""), Bytes.toBytes("1"), Bytes.toBytes("22"), Bytes.toBytes("333"), Bytes.toBytes("4444"), Bytes.toBytes("55555"), Bytes.toBytes("666666"), @@ -55,7 +61,7 @@ public class TestFixedLengthWrapper { assertEquals(limit, type.encode(buff, val)); buff.setPosition(0); byte[] actual = type.decode(buff); - assertTrue("Decoding output differs from expected", + assertTrue("Decoding output differs from expected", Bytes.equals(val, 0, val.length, actual, 0, val.length)); buff.setPosition(0); assertEquals(limit, type.skip(buff)); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java index c796fea1d19..448a660afcb 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java @@ -19,17 +19,23 @@ package org.apache.hadoop.hbase.types; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestOrderedBlob { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOrderedBlob.class); + static final byte[][] VALUES = new byte[][] { null, Bytes.toBytes(""), Bytes.toBytes("1"), Bytes.toBytes("22"), Bytes.toBytes("333"), Bytes.toBytes("4444"), Bytes.toBytes("55555"), Bytes.toBytes("666666"), diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java index d9c40e57dc0..99e8fd2c42c 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java @@ -19,17 +19,23 @@ package org.apache.hadoop.hbase.types; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestOrderedBlobVar { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOrderedBlobVar.class); + static final byte[][] VALUES = new byte[][] { null, Bytes.toBytes(""), Bytes.toBytes("1"), Bytes.toBytes("22"), Bytes.toBytes("333"), Bytes.toBytes("4444"), Bytes.toBytes("55555"), Bytes.toBytes("666666"), diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java index 6e9e9d07e2c..d616627a983 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java @@ -19,16 +19,22 @@ package org.apache.hadoop.hbase.types; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestOrderedString { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOrderedString.class); + static final String[] VALUES = new String[] { null, "", "1", "22", "333", "4444", "55555", "666666", "7777777", "88888888", "999999999" }; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java index 90f7e212aa9..b1bfe75a861 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java @@ -20,18 +20,24 @@ package org.apache.hadoop.hbase.types; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestRawString { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRawString.class); + static final String[] VALUES = new String[] { "", "1", "22", "333", "4444", "55555", "666666", "7777777", "88888888", "999999999", }; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java index 1cdc9873cf6..d1a0fa633dd 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java @@ -24,13 +24,14 @@ import java.lang.reflect.Constructor; import java.util.Arrays; import java.util.Collection; import java.util.Comparator; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -47,6 +48,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({MiscTests.class, SmallTests.class}) public class TestStruct { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStruct.class); + @Parameterized.Parameter(value = 0) public Struct generic; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStructNullExtension.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStructNullExtension.java index 2b2efe68f73..a1a620567b7 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStructNullExtension.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStructNullExtension.java @@ -23,17 +23,22 @@ import static org.junit.Assert.assertNull; import java.math.BigDecimal; import java.util.Arrays; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestStructNullExtension { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStructNullExtension.class); + /** * Verify null extension respects the type's isNullable field. */ diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java index 310067b2245..02b36abda83 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java @@ -20,18 +20,24 @@ package org.apache.hadoop.hbase.types; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestTerminatedWrapper { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTerminatedWrapper.class); + static final String[] VALUES_STRINGS = new String[] { "", "1", "22", "333", "4444", "55555", "666666", "7777777", "88888888", "999999999", }; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java index 932be9525d3..1463abd4b91 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java @@ -20,17 +20,23 @@ package org.apache.hadoop.hbase.types; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestUnion2 { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestUnion2.class); + /** * An example Union */ diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestAvlUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestAvlUtil.java index 6c4e08eff0a..53544ef9445 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestAvlUtil.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestAvlUtil.java @@ -1,20 +1,20 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.util; import static org.junit.Assert.assertEquals; @@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue; import java.util.Random; import java.util.TreeMap; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.AvlUtil.AvlIterableList; @@ -34,11 +34,17 @@ import org.apache.hadoop.hbase.util.AvlUtil.AvlNode; import org.apache.hadoop.hbase.util.AvlUtil.AvlNodeVisitor; import org.apache.hadoop.hbase.util.AvlUtil.AvlTree; import org.apache.hadoop.hbase.util.AvlUtil.AvlTreeIterator; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestAvlUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAvlUtil.class); + private static final TestAvlKeyComparator KEY_COMPARATOR = new TestAvlKeyComparator(); @Test diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBase64.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBase64.java index e6098049e80..2d40aba3b17 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBase64.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBase64.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import java.io.UnsupportedEncodingException; import java.util.Map; import java.util.TreeMap; - import junit.framework.TestCase; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -33,6 +32,11 @@ import org.junit.experimental.categories.Category; */ @Category({MiscTests.class, SmallTests.class}) public class TestBase64 extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBase64.class); + // Note: uris is sorted. We need to prove that the ordered Base64 // preserves that ordering private String[] uris = { diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java index 9ee356f56ae..3fc1c230f5a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java @@ -23,16 +23,21 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestByteBufferArray { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteBufferArray.class); + @Test public void testAsSubBufferWhenEndOffsetLandInLastBuffer() throws Exception { int capacity = 4 * 1024 * 1024; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java index f4687fa5deb..997a0bb4b58 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.util; @@ -43,7 +44,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -51,6 +52,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.io.WritableUtils; import org.junit.AfterClass; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -59,6 +61,11 @@ import org.junit.runners.Parameterized; @Category({MiscTests.class, SmallTests.class}) @RunWith(Parameterized.class) public class TestByteBufferUtils { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteBufferUtils.class); + private static final String UNSAFE_AVAIL_NAME = "UNSAFE_AVAIL"; private static final String UNSAFE_UNALIGNED_NAME = "UNSAFE_UNALIGNED"; private byte[] array; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java index 297bd62147f..b7de95bfed0 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java @@ -19,19 +19,24 @@ package org.apache.hadoop.hbase.util; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.ArrayBackedTag; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestByteRangeWithKVSerialization { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteRangeWithKVSerialization.class); + static void writeCell(PositionedByteRange pbr, KeyValue kv) throws Exception { pbr.putInt(kv.getKeyLength()); pbr.putInt(kv.getValueLength()); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java index 546840c114c..cea615e081a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java @@ -28,17 +28,22 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Random; - import junit.framework.TestCase; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.io.WritableUtils; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; - @Category({MiscTests.class, SmallTests.class}) public class TestBytes extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBytes.class); + public void testNullHashCode() { byte [] b = null; Exception ee = null; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCommonFSUtils.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCommonFSUtils.java index a3330af1341..49c5256afad 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCommonFSUtils.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCommonFSUtils.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,15 +24,16 @@ import static org.junit.Assert.assertTrue; import java.io.ByteArrayOutputStream; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -44,6 +44,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, MediumTests.class}) public class TestCommonFSUtils { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCommonFSUtils.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCommonFSUtils.class); private HBaseCommonTestingUtility htu; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestConcatenatedLists.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestConcatenatedLists.java index 18670a1015e..f3c780ec40e 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestConcatenatedLists.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestConcatenatedLists.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -28,14 +27,20 @@ import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestConcatenatedLists { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConcatenatedLists.class); + @Test public void testUnsupportedOps() { // If adding support, add tests. diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java index 32af01bfe1f..94cf37db67a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -27,13 +26,14 @@ import static org.junit.Assert.fail; import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.io.IOUtils; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +43,10 @@ import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestCoprocessorClassLoader { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorClassLoader.class); + private static final HBaseCommonTestingUtility TEST_UTIL = new HBaseCommonTestingUtility(); private static final Configuration conf = TEST_UTIL.getConfiguration(); static { diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCounter.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCounter.java index 1c25ee3423d..1b8bcc0a823 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCounter.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCounter.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -16,16 +16,23 @@ * limitations under the License. */ package org.apache.hadoop.hbase.util; -import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CountDownLatch; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, MediumTests.class}) public class TestCounter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCounter.class); + private static final int[] THREAD_COUNTS = {1, 10, 100}; private static final int DATA_COUNT = 1000000; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java index b5fa18597bc..5c3d053bc45 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -22,14 +21,20 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestDrainBarrier { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDrainBarrier.class); + @Test public void testBeginEndStopWork() throws Exception { DrainBarrier barrier = new DrainBarrier(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDynamicClassLoader.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDynamicClassLoader.java index 5b80cd8fea1..b85921588f9 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDynamicClassLoader.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDynamicClassLoader.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -22,12 +21,13 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; import java.io.File; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -38,6 +38,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, SmallTests.class}) public class TestDynamicClassLoader { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDynamicClassLoader.class); + private static final Logger LOG = LoggerFactory.getLogger(TestDynamicClassLoader.class); private static final HBaseCommonTestingUtility TEST_UTIL = new HBaseCommonTestingUtility(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestEnvironmentEdgeManager.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestEnvironmentEdgeManager.java index 6089924d5ec..bd6f35bc208 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestEnvironmentEdgeManager.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestEnvironmentEdgeManager.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -26,14 +25,20 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, MediumTests.class}) public class TestEnvironmentEdgeManager { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEnvironmentEdgeManager.class); + @Test public void testManageSingleton() { EnvironmentEdgeManager.reset(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestJRubyFormat.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestJRubyFormat.java index 779f483ab90..ffe4c56571d 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestJRubyFormat.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestJRubyFormat.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import static org.junit.Assert.assertEquals; import java.util.LinkedHashMap; import java.util.Map; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -31,6 +31,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category(SmallTests.class) public class TestJRubyFormat { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestJRubyFormat.class); + @Test public void testPrint() { Map map = new LinkedHashMap<>(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java index edf2f78eba9..1f6928fcdef 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java @@ -15,19 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import java.util.concurrent.locks.ReentrantLock; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestKeyLocker { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeyLocker.class); + @Test public void testLocker(){ KeyLocker locker = new KeyLocker<>(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestLoadTestKVGenerator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestLoadTestKVGenerator.java index 242d6bef53c..ff0a3b20415 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestLoadTestKVGenerator.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestLoadTestKVGenerator.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.util; @@ -22,15 +23,20 @@ import static org.junit.Assert.assertTrue; import java.util.HashSet; import java.util.Random; import java.util.Set; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestLoadTestKVGenerator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLoadTestKVGenerator.class); + private static final int MIN_LEN = 10; private static final int MAX_LEN = 20; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestOrder.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestOrder.java index 8029e44d555..c286bc6d86e 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestOrder.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestOrder.java @@ -23,15 +23,20 @@ import static org.junit.Assert.assertArrayEquals; import java.util.Arrays; import java.util.Collections; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestOrder { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOrder.class); + byte[][] VALS = { Bytes.toBytes("foo"), Bytes.toBytes("bar"), Bytes.toBytes("baz") }; @Test diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestOrderedBytes.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestOrderedBytes.java index 2107ee891b7..c0146fd178b 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestOrderedBytes.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestOrderedBytes.java @@ -25,15 +25,20 @@ import static org.junit.Assert.fail; import java.math.BigDecimal; import java.util.Arrays; import java.util.Collections; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestOrderedBytes { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOrderedBytes.class); + // integer constants for testing Numeric code paths static final Long[] I_VALS = { 0L, 1L, 10L, 99L, 100L, 1234L, 9999L, 10000L, 10001L, 12345L, 123450L, Long.MAX_VALUE, diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestShowProperties.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestShowProperties.java index 6b6d9b072bf..1e5392e49d7 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestShowProperties.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestShowProperties.java @@ -15,25 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import java.util.Properties; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * This test is there to dump the properties. It allows to detect possible env issues when * executing the tests on various environment. */ @Category({MiscTests.class, SmallTests.class}) public class TestShowProperties { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestShowProperties.class); + private static final Logger LOG = LoggerFactory.getLogger(TestShowProperties.class); @Test diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleMutableByteRange.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleMutableByteRange.java index 88d482978bc..763f3056df0 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleMutableByteRange.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleMutableByteRange.java @@ -17,15 +17,21 @@ */ package org.apache.hadoop.hbase.util; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestSimpleMutableByteRange { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSimpleMutableByteRange.class); + @Test public void testEmpty(){ Assert.assertTrue(SimpleMutableByteRange.isEmpty(null)); @@ -69,7 +75,7 @@ public class TestSimpleMutableByteRange { r.setLength(2);//verify we retained the 2nd byte, but dangerous in real code Assert.assertTrue(Bytes.equals(new byte[]{1, 3}, r.deepCopyToNewArray())); } - + @Test public void testPutandGetPrimitiveTypes() throws Exception { ByteRange r = new SimpleMutableByteRange(100); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimplePositionedMutableByteRange.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimplePositionedMutableByteRange.java index ecc8c602da4..82613097f2a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimplePositionedMutableByteRange.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimplePositionedMutableByteRange.java @@ -18,15 +18,21 @@ package org.apache.hadoop.hbase.util; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestSimplePositionedMutableByteRange { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSimplePositionedMutableByteRange.class); + @Test public void testPosition() { PositionedByteRange r = new SimplePositionedMutableByteRange(new byte[5], 1, 3); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestThreads.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestThreads.java index e6a06c1c7b7..48b66fa32f7 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestThreads.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestThreads.java @@ -1,29 +1,29 @@ -/* - * Copyright The Apache Software Foundation +/** + * 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 * - * 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 + * 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. + * 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.util; import static org.junit.Assert.assertTrue; import java.util.concurrent.atomic.AtomicBoolean; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -31,6 +31,11 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) public class TestThreads { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestThreads.class); + private static final Logger LOG = LoggerFactory.getLogger(TestThreads.class); private static final int SLEEP_TIME_MS = 3000; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestVersionInfo.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestVersionInfo.java index 6e71b275f42..325ad089e2a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestVersionInfo.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestVersionInfo.java @@ -19,13 +19,19 @@ package org.apache.hadoop.hbase.util; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestVersionInfo { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVersionInfo.class); + @Test public void testCompareVersion() { assertTrue(VersionInfo.compareVersion("1.0.0", "0.98.11") > 0); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestWeakObjectPool.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestWeakObjectPool.java index 12d98773879..63470252586 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestWeakObjectPool.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestWeakObjectPool.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,27 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; - -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; import org.junit.Before; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; @Category({MiscTests.class, SmallTests.class}) public class TestWeakObjectPool { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWeakObjectPool.class); + ObjectPool pool; @Before diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java index 8309f81f455..3b59b4eb97f 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -22,18 +22,23 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Properties; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestZKConfig { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKConfig.class); + @Test public void testZKConfigLoading() throws Exception { Configuration conf = HBaseConfiguration.create(); diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAggregationClient.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAggregationClient.java index 12e5b8d83ae..d50ceb9c1c3 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAggregationClient.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAggregationClient.java @@ -22,8 +22,8 @@ import static org.junit.Assert.assertEquals; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import java.util.stream.LongStream; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.coprocessor.AsyncAggregationClient; @@ -35,12 +35,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, CoprocessorTests.class }) public class TestAsyncAggregationClient { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncAggregationClient.class); + private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("TestAsyncAggregationClient"); diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java index 986ea2f55b7..2d60733b83e 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java @@ -25,10 +25,10 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -37,23 +37,28 @@ import org.apache.hadoop.hbase.coprocessor.ProtobufCoprocessorService; import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hbase.thirdparty.com.google.common.collect.ConcurrentHashMultiset; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hbase.thirdparty.com.google.common.collect.Multiset; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; - import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.apache.hbase.thirdparty.com.google.common.collect.ConcurrentHashMultiset; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.collect.Multiset; + @Category({MediumTests.class, ClientTests.class}) public class TestRpcControllerFactory { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRpcControllerFactory.class); + public static class StaticRpcControllerFactory extends RpcControllerFactory { public StaticRpcControllerFactory(Configuration conf) { diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAsyncCoprocessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAsyncCoprocessorEndpoint.java index 2e474bc9fc5..9d4b07df5b8 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAsyncCoprocessorEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAsyncCoprocessorEndpoint.java @@ -21,11 +21,14 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collections; - import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -41,19 +44,20 @@ import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; - @RunWith(Parameterized.class) @Category({ ClientTests.class, MediumTests.class }) public class TestAsyncCoprocessorEndpoint extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncCoprocessorEndpoint.class); + private static final FileNotFoundException WHAT_TO_THROW = new FileNotFoundException("/file.txt"); private static final String DUMMY_VALUE = "val"; diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java index 5433792986d..c108db28a2a 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -21,11 +20,13 @@ package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.protobuf.ByteString; +import com.google.protobuf.ServiceException; import java.util.Collections; import java.util.Map; import java.util.TreeMap; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -47,19 +48,22 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.protobuf.ByteString; -import com.google.protobuf.ServiceException; - /** * TestEndpoint: test cases to verify the batch execution of coprocessor Endpoint */ @Category({CoprocessorTests.class, MediumTests.class}) public class TestBatchCoprocessorEndpoint { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBatchCoprocessorEndpoint.class); + private static final Logger LOG = LoggerFactory.getLogger(TestBatchCoprocessorEndpoint.class); private static final TableName TEST_TABLE = diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java index 37e5a78cb12..922977c03a4 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,13 +17,25 @@ */ package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.*; +import java.util.*; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.RegionLoad; +import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -36,29 +47,22 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.ClassLoaderTestHelper; import org.apache.hadoop.hbase.util.CoprocessorClassLoader; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.ServerLoad; -import org.apache.hadoop.hbase.RegionLoad; - -import java.io.*; -import java.util.*; - import org.junit.*; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; - /** * Test coprocessors class loading. */ @Category({CoprocessorTests.class, MediumTests.class}) public class TestClassLoading { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClassLoading.class); + private static final Logger LOG = LoggerFactory.getLogger(TestClassLoading.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java index 376c0715866..87409a7813c 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -24,13 +23,15 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.TreeMap; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -55,19 +56,22 @@ import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; - /** * TestEndpoint: test cases to verify coprocessor Endpoint */ @Category({CoprocessorTests.class, MediumTests.class}) public class TestCoprocessorEndpoint { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorEndpoint.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCoprocessorEndpoint.class); private static final TableName TEST_TABLE = diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorServiceBackwardCompatibility.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorServiceBackwardCompatibility.java index c51da8196f0..e7181bb0541 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorServiceBackwardCompatibility.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorServiceBackwardCompatibility.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,22 +7,23 @@ * "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 + * 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. + * 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.coprocessor; +import static org.junit.Assert.assertEquals; + import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcController; import com.google.protobuf.Service; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.*; @@ -30,16 +31,20 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; - /** * Tests to ensure that 2.0 is backward compatible in loading CoprocessorService. */ @Category({SmallTests.class}) public class TestCoprocessorServiceBackwardCompatibility { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorServiceBackwardCompatibility.class); + private static HBaseTestingUtility TEST_UTIL = null; private static Configuration CONF = null; diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java index cd022de8005..fbcbb54f680 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -20,9 +19,11 @@ package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertEquals; +import com.google.protobuf.ByteString; +import com.google.protobuf.ServiceException; import java.io.IOException; import java.util.Map; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -38,17 +39,19 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import com.google.protobuf.ByteString; -import com.google.protobuf.ServiceException; import org.junit.rules.TestName; @Category({CoprocessorTests.class, MediumTests.class}) public class TestCoprocessorTableEndpoint { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorTableEndpoint.class); + private static final byte[] TEST_FAMILY = Bytes.toBytes("TestFamily"); private static final byte[] TEST_QUALIFIER = Bytes.toBytes("TestQualifier"); private static final byte[] ROW = Bytes.toBytes("testRow"); @@ -85,7 +88,7 @@ public class TestCoprocessorTableEndpoint { } @Test - public void testDynamicCoprocessorTableEndpoint() throws Throwable { + public void testDynamicCoprocessorTableEndpoint() throws Throwable { final TableName tableName = TableName.valueOf(name.getMethodName()); HTableDescriptor desc = new HTableDescriptor(tableName); @@ -142,7 +145,7 @@ public class TestCoprocessorTableEndpoint { table.put(put); } } finally { - table.close(); + table.close(); } } diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestImportExport.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestImportExport.java index e0d4fd2a201..8a4c7b21b55 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestImportExport.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestImportExport.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -28,6 +29,10 @@ import org.junit.experimental.categories.Category; @Category({MediumTests.class}) public class TestImportExport extends org.apache.hadoop.hbase.mapreduce.TestImportExport { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImportExport.class); + @BeforeClass public static void beforeClass() throws Throwable { UTIL.getConfiguration().setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java index d6421080c1b..f1808845d0a 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java @@ -20,10 +20,13 @@ package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; import java.io.FileNotFoundException; import java.util.Collections; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos; @@ -38,15 +41,17 @@ import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; - @Category({CoprocessorTests.class, MediumTests.class}) public class TestRegionServerCoprocessorEndpoint { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionServerCoprocessorEndpoint.class); + public static final FileNotFoundException WHAT_TO_THROW = new FileNotFoundException("/file.txt"); private static HBaseTestingUtility TEST_UTIL = null; private static Configuration CONF = null; diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java index 8a79400bcb4..01e5b59a11b 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.protobuf.Message; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -32,24 +32,21 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; - -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.testclassification.CoprocessorTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.IsolationLevel; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.coprocessor.RowProcessorClient; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorRequest; @@ -68,17 +65,19 @@ import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.RowProcesso import org.apache.hadoop.hbase.regionserver.BaseRowProcessor; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.testclassification.CoprocessorTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.protobuf.Message; - /** * Verifies ProcessEndpoint works. * The tested RowProcessor performs two scans and a read-modify-write. @@ -86,6 +85,10 @@ import com.google.protobuf.Message; @Category({CoprocessorTests.class, MediumTests.class}) public class TestRowProcessorEndpoint { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowProcessorEndpoint.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRowProcessorEndpoint.class); private static final TableName TABLE = TableName.valueOf("testtable"); diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestSecureExport.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestSecureExport.java index 76ef82504ea..852f1abc740 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestSecureExport.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestSecureExport.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertEquals; + import com.google.protobuf.ServiceException; import java.io.File; import java.io.IOException; @@ -27,12 +28,12 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Properties; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -55,9 +56,9 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.access.AccessControlConstants; import org.apache.hadoop.hbase.security.access.AccessControlLists; +import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.SecureTestUtil; import org.apache.hadoop.hbase.security.access.SecureTestUtil.AccessTestAction; -import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.visibility.Authorizations; import org.apache.hadoop.hbase.security.visibility.CellVisibility; import org.apache.hadoop.hbase.security.visibility.VisibilityClient; @@ -74,9 +75,9 @@ import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.junit.After; import org.junit.AfterClass; -import static org.junit.Assert.assertEquals; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -86,6 +87,11 @@ import org.slf4j.LoggerFactory; @Category({MediumTests.class}) public class TestSecureExport { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecureExport.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSecureExport.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static MiniKdc KDC; diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/ipc/TestCoprocessorRpcUtils.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/ipc/TestCoprocessorRpcUtils.java index 15a27478aa2..170a303845b 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/ipc/TestCoprocessorRpcUtils.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/ipc/TestCoprocessorRpcUtils.java @@ -1,5 +1,4 @@ -/* - +/** * 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 @@ -16,20 +15,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.ipc; import static org.junit.Assert.assertEquals; import com.google.protobuf.Descriptors; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos; import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestCoprocessorRpcUtils { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorRpcUtils.class); + @Test public void testServiceName() throws Exception { // verify that we de-namespace build in HBase rpc services diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java index b134a7c19f7..7196851b920 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java @@ -21,9 +21,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread; import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext; @@ -35,22 +35,25 @@ import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest; + /** * Tests bulk loading of HFiles with old secure Endpoint client for backward compatibility. Will be * removed when old non-secure client for backward compatibility is not supported. @@ -59,6 +62,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({RegionServerTests.class, LargeTests.class}) @Ignore // BROKEN. FIX OR REMOVE. public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionServerBulkLoad { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionServerBulkLoadWithOldSecureEndpoint.class); + public TestHRegionServerBulkLoadWithOldSecureEndpoint(int duration) { super(duration); } diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java index 0b17abf0dc1..b306b763441 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -22,11 +21,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; import java.io.IOException; import java.util.Collections; import java.util.Map; - import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; @@ -57,18 +60,19 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; -import com.google.protobuf.ServiceException; - @Category({RegionServerTests.class, MediumTests.class}) public class TestServerCustomProtocol { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerCustomProtocol.class); + private static final Logger LOG = LoggerFactory.getLogger(TestServerCustomProtocol.class); static final String WHOAREYOU = "Who are you?"; static final String NOBODY = "nobody"; diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java index 1b74b7d7245..e248e9af975 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java @@ -1,12 +1,19 @@ /** - * 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. + * 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.replication; @@ -20,20 +27,21 @@ import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.UUID; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,6 +49,10 @@ import org.slf4j.LoggerFactory; @Category({ ReplicationTests.class, LargeTests.class }) public class TestReplicationSyncUpToolWithBulkLoadedData extends TestReplicationSyncUpTool { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSyncUpToolWithBulkLoadedData.class); + private static final Logger LOG = LoggerFactory .getLogger(TestReplicationSyncUpToolWithBulkLoadedData.class); diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/client/example/TestAsyncClientExample.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/client/example/TestAsyncClientExample.java index b6d4f5c0b68..bee18ddd7e7 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/client/example/TestAsyncClientExample.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/client/example/TestAsyncClientExample.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client.example; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -27,12 +28,17 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, MediumTests.class }) public class TestAsyncClientExample { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncClientExample.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName TABLE_NAME = TableName.valueOf("test"); diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/client/example/TestHttpProxyExample.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/client/example/TestHttpProxyExample.java index afa257a02ff..531f818853a 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/client/example/TestHttpProxyExample.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/client/example/TestHttpProxyExample.java @@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.client.example; import static org.junit.Assert.assertEquals; -import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams; - import java.nio.charset.StandardCharsets; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -38,12 +36,19 @@ import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams; + @Category({ ClientTests.class, MediumTests.class }) public class TestHttpProxyExample { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHttpProxyExample.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName TABLE_NAME = TableName.valueOf("test"); diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java index 77f98999ead..b28db1196ea 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java @@ -6,16 +6,15 @@ * 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 - *

+ * + * 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.coprocessor.example; import static org.junit.Assert.assertEquals; @@ -24,10 +23,10 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -58,6 +58,11 @@ import org.slf4j.LoggerFactory; @Category(MediumTests.class) public class TestRefreshHFilesEndpoint { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRefreshHFilesEndpoint.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRefreshHFilesEndpoint.class); private static final HBaseTestingUtility HTU = new HBaseTestingUtility(); private static final int NUM_MASTER = 1; diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestScanModifyingObserver.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestScanModifyingObserver.java index d5d0ba1226a..f90a0f43494 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestScanModifyingObserver.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestScanModifyingObserver.java @@ -25,7 +25,7 @@ import static org.junit.Assert.assertNull; import java.io.IOException; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -41,12 +41,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ CoprocessorTests.class, MediumTests.class }) public class TestScanModifyingObserver { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScanModifyingObserver.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName NAME = TableName.valueOf("TestScanModifications"); private static final byte[] FAMILY = Bytes.toBytes("f"); diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestValueReplacingCompaction.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestValueReplacingCompaction.java index 206cdf64866..0dbdfe18b2a 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestValueReplacingCompaction.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestValueReplacingCompaction.java @@ -25,7 +25,7 @@ import static org.junit.Assert.assertNull; import java.io.IOException; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -41,12 +41,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ CoprocessorTests.class, MediumTests.class }) public class TestValueReplacingCompaction { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestValueReplacingCompaction.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName NAME = TableName.valueOf("TestValueReplacement"); private static final byte[] FAMILY = Bytes.toBytes("f"); diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestWriteHeavyIncrementObserver.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestWriteHeavyIncrementObserver.java index 18e819f2d1f..e84cd722c69 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestWriteHeavyIncrementObserver.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestWriteHeavyIncrementObserver.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.coprocessor.example; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Result; @@ -30,12 +31,17 @@ import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ CoprocessorTests.class, MediumTests.class }) public class TestWriteHeavyIncrementObserver extends WriteHeavyIncrementObserverTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWriteHeavyIncrementObserver.class); + @BeforeClass public static void setUp() throws Exception { WriteHeavyIncrementObserverTestBase.setUp(); diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestWriteHeavyIncrementObserverWithMemStoreCompaction.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestWriteHeavyIncrementObserverWithMemStoreCompaction.java index eeb1fa81563..ae93d884481 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestWriteHeavyIncrementObserverWithMemStoreCompaction.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestWriteHeavyIncrementObserverWithMemStoreCompaction.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.coprocessor.example; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.MemoryCompactionPolicy; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; @@ -27,6 +28,7 @@ import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,6 +36,10 @@ import org.junit.experimental.categories.Category; public class TestWriteHeavyIncrementObserverWithMemStoreCompaction extends WriteHeavyIncrementObserverTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWriteHeavyIncrementObserverWithMemStoreCompaction.class); + @BeforeClass public static void setUp() throws Exception { WriteHeavyIncrementObserverTestBase.setUp(); diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java index 2c40cbecec3..3c89fa5f268 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java @@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -38,12 +38,17 @@ import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ CoprocessorTests.class, MediumTests.class }) public class TestZooKeeperScanPolicyObserver { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZooKeeperScanPolicyObserver.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static TableName NAME = TableName.valueOf("TestCP"); diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMapReduceExamples.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMapReduceExamples.java index 089dafdbe5f..3b7f7839661 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMapReduceExamples.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMapReduceExamples.java @@ -1,30 +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 + * 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 + * 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 + * 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.mapreduce; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.IndexBuilder.Map; import org.apache.hadoop.hbase.mapreduce.SampleUploader.Uploader; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.LauncherSecurityManager; import org.apache.hadoop.io.LongWritable; @@ -33,19 +41,19 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Mapper.Context; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; - -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - @Category({MapReduceTests.class, LargeTests.class}) public class TestMapReduceExamples { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMapReduceExamples.class); + private static HBaseTestingUtility util = new HBaseTestingUtility(); /** diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java index 77c9e22d530..b5a880b4e3f 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java @@ -24,20 +24,26 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.CellProtos; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PositionedByteRange; import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({SmallTests.class, MiscTests.class}) public class TestPBCell { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPBCell.class); + private static final PBCell CODEC = new PBCell(); /** diff --git a/hbase-hadoop-compat/pom.xml b/hbase-hadoop-compat/pom.xml index cb5b5862993..c662c4bffcd 100644 --- a/hbase-hadoop-compat/pom.xml +++ b/hbase-hadoop-compat/pom.xml @@ -82,6 +82,12 @@ test-jar test + + org.apache.hbase + hbase-common + test-jar + test + org.apache.hbase.thirdparty hbase-shaded-miscellaneous diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java index 168f6c74fa5..a57c935651a 100644 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java +++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java @@ -15,9 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -25,19 +27,19 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; - -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MetricsTests; - +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; - @Category({MetricsTests.class, SmallTests.class}) public class TestCompatibilitySingletonFactory { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompatibilitySingletonFactory.class); + private static final int ITERATIONS = 100000; private static final Random RANDOM = new Random(); diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceFactory.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceFactory.java index 6e2961f9979..a49c8a7b892 100644 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceFactory.java +++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceFactory.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -30,6 +31,10 @@ import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsMasterSourceFactory { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsMasterSourceFactory.class); + @Test(expected=RuntimeException.class) public void testGetInstanceNoHadoopCompat() throws Exception { //This should throw an exception because there is no compat lib on the class path. diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceFactory.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceFactory.java index fbf1994f7d4..3c2a21d1533 100644 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceFactory.java +++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceFactory.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -31,6 +31,10 @@ import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsRegionServerSourceFactory { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsRegionServerSourceFactory.class); + @Test(expected=RuntimeException.class) public void testGetInstanceNoHadoopCompat() throws Exception { //This should throw an exception because there is no compat lib on the class path. diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWALSource.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWALSource.java index aabfdfe1634..2a804158e9e 100644 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWALSource.java +++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWALSource.java @@ -15,18 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.wal; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsWALSource { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsWALSource.class); + @Test(expected=RuntimeException.class) public void testGetInstanceNoHadoopCompat() throws Exception { //This should throw an exception because there is no compat lib on the class path. diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceFactory.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceFactory.java index 3a8bb8910d6..19e041193ee 100644 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceFactory.java +++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceFactory.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication.regionserver; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSource; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -31,6 +31,10 @@ import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsReplicationSourceFactory { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsReplicationSourceFactory.class); + @Test(expected=RuntimeException.class) public void testGetInstanceNoHadoopCompat() throws Exception { //This should throw an exception because there is no compat lib on the class path. diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/rest/TestMetricsRESTSource.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/rest/TestMetricsRESTSource.java index ee2f164e20a..fc4caae0b20 100644 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/rest/TestMetricsRESTSource.java +++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/rest/TestMetricsRESTSource.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.rest.MetricsRESTSource; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -31,6 +31,9 @@ import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsRESTSource { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsRESTSource.class); @Test(expected=RuntimeException.class) public void testGetInstanceNoHadoopCompat() throws Exception { diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactory.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactory.java index f33135eb5aa..c49a2a51ee5 100644 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactory.java +++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactory.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.thrift; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; -import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -31,6 +31,10 @@ import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsThriftServerSourceFactory { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsThriftServerSourceFactory.class); + @Test(expected=RuntimeException.class) public void testGetInstanceNoHadoopCompat() throws RuntimeException { diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSource.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSource.java index 1844d347e48..ca7ba2402e6 100644 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSource.java +++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSource.java @@ -18,14 +18,20 @@ package org.apache.hadoop.hbase.zookeeper; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsZooKeeperSource { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsZooKeeperSource.class); + @Test(expected=RuntimeException.class) public void testGetInstanceNoHadoopCompat() throws Exception { //This should throw an exception because there is no compat lib on the class path. diff --git a/hbase-hadoop2-compat/pom.xml b/hbase-hadoop2-compat/pom.xml index 87d553df6ba..821bbff130a 100644 --- a/hbase-hadoop2-compat/pom.xml +++ b/hbase-hadoop2-compat/pom.xml @@ -127,6 +127,12 @@ limitations under the License. org.apache.hbase hbase-common + + org.apache.hbase + hbase-common + test-jar + test + org.apache.hbase hbase-metrics diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterProcSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterProcSourceImpl.java index 3fe60b81455..d95c282ecf9 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterProcSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterProcSourceImpl.java @@ -15,24 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master; -import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.MetricsTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + /** * Test for MetricsMasterProcSourceImpl */ @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsMasterProcSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsMasterProcSourceImpl.class); + @Test public void testGetInstance() throws Exception { MetricsMasterProcSourceFactory metricsMasterProcSourceFactory = CompatibilitySingletonFactory diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceImpl.java index f9508f5b47e..3434263e6b7 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceImpl.java @@ -15,24 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master; -import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MetricsTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + /** * Test for MetricsMasterSourceImpl */ @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsMasterSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsMasterSourceImpl.class); + @Test public void testGetInstance() throws Exception { MetricsMasterSourceFactory metricsMasterSourceFactory = CompatibilitySingletonFactory diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java index 655824fd013..063071b4317 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java @@ -15,26 +15,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.metrics; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MetricsTests; -import org.apache.hadoop.metrics2.lib.MutableFastCounter; -import org.apache.hadoop.metrics2.lib.MutableGaugeLong; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.metrics2.lib.MutableFastCounter; +import org.apache.hadoop.metrics2.lib.MutableGaugeLong; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + /** * Test of default BaseSource for hadoop 2 */ @Category({MetricsTests.class, SmallTests.class}) public class TestBaseSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBaseSourceImpl.class); + private static BaseSourceImpl bmsi; @BeforeClass diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceImpl.java index 05a5522af81..f970bad3959 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceImpl.java @@ -15,24 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MetricsTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + /** * Test for MetricsRegionServerSourceImpl */ @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsRegionServerSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsRegionServerSourceImpl.class); + @Test public void testGetInstance() throws Exception { MetricsRegionServerSourceFactory metricsRegionServerSourceFactory = diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java index 53d42bb25bd..30ea9060ee3 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java @@ -15,23 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; - import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsRegionSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsRegionSourceImpl.class); + @SuppressWarnings("SelfComparison") @Test public void testCompareToHashCodeEquals() throws Exception { diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableSourceImpl.java index 9a1f016020d..970a7cc723e 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableSourceImpl.java @@ -15,16 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,6 +35,10 @@ import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsTableSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsTableSourceImpl.class); + @SuppressWarnings("SelfComparison") @Test public void testCompareToHashCode() throws Exception { diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWALSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWALSourceImpl.java index ddfed45ebbc..d8ec0af92bb 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWALSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWALSourceImpl.java @@ -17,18 +17,24 @@ */ package org.apache.hadoop.hbase.regionserver.wal; -import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MetricsTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsWALSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsWALSourceImpl.class); + @Test public void testGetInstance() throws Exception { MetricsWALSource walSource = diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceFactoryImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceFactoryImpl.java index abbd49fb509..bd23b0e9acb 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceFactoryImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceFactoryImpl.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,17 +17,23 @@ */ package org.apache.hadoop.hbase.replication.regionserver; +import static org.junit.Assert.*; + import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.*; - @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsReplicationSourceFactoryImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsReplicationSourceFactoryImpl.class); + @Test public void testGetInstance() throws Exception { diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceImpl.java index acbadbe602d..67513078c77 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetricsReplicationSourceImpl.java @@ -15,23 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication.regionserver; +import static org.junit.Assert.assertTrue; + import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSource; import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceImpl; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertTrue; - @Category({MetricsTests.class, SmallTests.class}) /** Test for MetricsReplicationSourceImpl */ public class TestMetricsReplicationSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsReplicationSourceImpl.class); + @Test public void testGetInstance() throws Exception { MetricsReplicationSource rms = CompatibilitySingletonFactory diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/rest/TestMetricsRESTSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/rest/TestMetricsRESTSourceImpl.java index 8069388f1de..e41156bbc56 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/rest/TestMetricsRESTSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/rest/TestMetricsRESTSourceImpl.java @@ -15,26 +15,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest; -import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.rest.MetricsRESTSource; -import org.apache.hadoop.hbase.rest.MetricsRESTSourceImpl; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MetricsTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.rest.MetricsRESTSource; +import org.apache.hadoop.hbase.rest.MetricsRESTSourceImpl; +import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + /** * Test for hadoop 2's version of MetricsRESTSource */ @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsRESTSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsRESTSourceImpl.class); + @Test public void ensureCompatRegistered() throws Exception { assertNotNull(CompatibilitySingletonFactory.getInstance(MetricsRESTSource.class)); diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactoryImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactoryImpl.java index c3a32b9f4f0..6808e158e09 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactoryImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactoryImpl.java @@ -15,27 +15,32 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.thrift; -import org.apache.hadoop.hbase.CompatibilitySingletonFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MetricsTests; -import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory; -import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactoryImpl; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MetricsTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory; +import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactoryImpl; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + /** * Test for hadoop 2's version of MetricsThriftServerSourceFactory */ @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsThriftServerSourceFactoryImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsThriftServerSourceFactoryImpl.class); + @Test public void testCompatabilityRegistered() throws Exception { assertNotNull(CompatibilitySingletonFactory.getInstance(MetricsThriftServerSourceFactory.class)); diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSourceImpl.java index 36998d5b9a0..6e2571d3266 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSourceImpl.java @@ -21,14 +21,20 @@ import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MetricsTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MetricsTests.class, SmallTests.class}) public class TestMetricsZooKeeperSourceImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsZooKeeperSourceImpl.class); + @Test public void testGetInstance() throws Exception { MetricsZooKeeperSource zkSource = CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class); diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java index f2461a16aa8..81902901f81 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java @@ -24,7 +24,6 @@ import java.net.URL; import java.net.URLConnection; import java.util.Set; import java.util.TreeSet; - import javax.servlet.Filter; import javax.servlet.FilterChain; import javax.servlet.FilterConfig; @@ -32,11 +31,12 @@ import javax.servlet.ServletException; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletRequest; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.net.NetUtils; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -44,6 +44,11 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) public class TestGlobalFilter extends HttpServerFunctionalTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGlobalFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(HttpServer.class); static final Set RECORDS = new TreeSet<>(); diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java index 5bc026c7fce..f47991d9289 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java @@ -20,9 +20,10 @@ package org.apache.hadoop.hbase.http; import static org.junit.Assert.*; import javax.servlet.http.HttpServletRequest; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -30,6 +31,10 @@ import org.mockito.Mockito; @Category({MiscTests.class, SmallTests.class}) public class TestHtmlQuoting { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHtmlQuoting.class); + @Test public void testNeedsQuoting() throws Exception { assertTrue(HtmlQuoting.needsQuoting("abcde>")); assertTrue(HtmlQuoting.needsQuoting(" RECORDS = new TreeSet<>(); diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java index 282530f0ec9..c62ca6561cf 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java @@ -22,20 +22,20 @@ import java.io.File; import java.io.InputStream; import java.net.URI; import java.net.URL; - import javax.net.ssl.HttpsURLConnection; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.http.ssl.KeyStoreTestUtil; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.http.ssl.KeyStoreTestUtil; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.ssl.SSLFactory; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -48,6 +48,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, SmallTests.class}) public class TestSSLHttpServer extends HttpServerFunctionalTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSSLHttpServer.class); + private static final String BASEDIR = System.getProperty("test.build.dir", "target/test-dir") + "/" + TestSSLHttpServer.class.getSimpleName(); diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java index 756487199c5..6c70dbc9d83 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java @@ -23,7 +23,6 @@ import java.io.InputStreamReader; import java.net.URL; import java.net.URLConnection; import java.util.Random; - import javax.servlet.Filter; import javax.servlet.FilterChain; import javax.servlet.FilterConfig; @@ -31,13 +30,14 @@ import javax.servlet.ServletException; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletRequest; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.StringUtils; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +46,11 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) public class TestServletFilter extends HttpServerFunctionalTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServletFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(HttpServer.class); static volatile String uri = null; diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java index 13e2519da5c..1262b0c1d43 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -23,11 +24,10 @@ import java.net.URL; import java.security.Principal; import java.security.PrivilegedExceptionAction; import java.util.Set; - import javax.security.auth.Subject; import javax.security.auth.kerberos.KerberosTicket; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.http.TestHttpServer.EchoServlet; import org.apache.hadoop.hbase.http.resource.JerseyResource; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -57,6 +57,7 @@ import org.ietf.jgss.GSSName; import org.ietf.jgss.Oid; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -68,6 +69,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, SmallTests.class}) public class TestSpnegoHttpServer extends HttpServerFunctionalTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSpnegoHttpServer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSpnegoHttpServer.class); private static final String KDC_SERVER_HOST = "localhost"; private static final String CLIENT_PRINCIPAL = "client"; diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java index 174d63564db..53e129fa872 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java @@ -22,16 +22,15 @@ import java.io.StringWriter; import java.util.HashSet; import java.util.Map; import java.util.Set; - import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; - import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.eclipse.jetty.util.ajax.JSON; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.w3c.dom.Document; @@ -46,6 +45,11 @@ import org.xml.sax.InputSource; */ @Category({MiscTests.class, SmallTests.class}) public class TestConfServlet extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConfServlet.class); + private static final String TEST_KEY = "testconfservlet.key"; private static final String TEST_VAL = "testval"; diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java index 9a6399d5935..9de8b2eb910 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -14,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.http.jmx; import java.net.HttpURLConnection; @@ -22,15 +22,15 @@ import java.net.URL; import java.net.URLEncoder; import java.util.regex.Matcher; import java.util.regex.Pattern; - import javax.servlet.http.HttpServletResponse; - -import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.http.HttpServer; import org.apache.hadoop.hbase.http.HttpServerFunctionalTest; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -38,6 +38,11 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) public class TestJMXJsonServlet extends HttpServerFunctionalTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestJMXJsonServlet.class); + private static final Logger LOG = LoggerFactory.getLogger(TestJMXJsonServlet.class); private static HttpServer server; private static URL baseUrl; diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java index 3adca50bbaa..39855ee86ef 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java @@ -25,13 +25,14 @@ import javax.servlet.FilterConfig; import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletRequestWrapper; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.http.ServerConfigurationKeys; import org.apache.hadoop.hbase.http.lib.StaticUserWebFilter.StaticUserFilter; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.ArgumentCaptor; @@ -39,6 +40,11 @@ import org.mockito.Mockito; @Category({MiscTests.class, SmallTests.class}) public class TestStaticUserWebFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStaticUserWebFilter.class); + private FilterConfig mockConfig(String username) { FilterConfig mock = Mockito.mock(FilterConfig.class); Mockito.doReturn(username).when(mock).getInitParameter( diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java index 84d2493ba36..e552ce247bd 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java @@ -1,20 +1,20 @@ /** -* 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. -*/ + * 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.http.log; import static org.junit.Assert.assertTrue; @@ -24,7 +24,7 @@ import java.io.InputStreamReader; import java.io.PrintStream; import java.net.URI; import java.net.URL; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.http.HttpServer; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -32,6 +32,7 @@ import org.apache.hadoop.net.NetUtils; import org.apache.log4j.Level; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.LoggerFactory; @@ -39,6 +40,11 @@ import org.slf4j.impl.Log4jLoggerAdapter; @Category({MiscTests.class, SmallTests.class}) public class TestLogLevel { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLogLevel.class); + static final PrintStream out = System.out; @Test (timeout=60000) diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/TestIntegrationTestBase.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/TestIntegrationTestBase.java index 7330909c89c..dfdb5f050f7 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/TestIntegrationTestBase.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/TestIntegrationTestBase.java @@ -20,16 +20,20 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; import java.util.Properties; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestIntegrationTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIntegrationTestBase.class); + @Test public void testMonkeyPropertiesParsing() { final Configuration conf = new Configuration(false); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TestChangeSplitPolicyAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TestChangeSplitPolicyAction.java index 091da9ac6c3..0bdc07ca72f 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TestChangeSplitPolicyAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TestChangeSplitPolicyAction.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.chaos.actions; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -26,12 +27,18 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @Category({MediumTests.class}) public class TestChangeSplitPolicyAction extends Action { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestChangeSplitPolicyAction.class); + private final static IntegrationTestingUtility TEST_UTIL = new IntegrationTestingUtility(); private static ChangeSplitPolicyAction action; private Admin admin; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java index 106b7e9ac05..d5b25e2df1b 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java @@ -19,17 +19,22 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.*; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.codahale.metrics.UniformReservoir; +import com.fasterxml.jackson.core.JsonGenerationException; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStreamReader; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.util.LinkedList; import java.util.NoSuchElementException; import java.util.Queue; import java.util.Random; -import java.util.LinkedList; - import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -37,19 +42,18 @@ import org.apache.hadoop.hbase.PerformanceEvaluation.RandomReadTest; import org.apache.hadoop.hbase.PerformanceEvaluation.TestOptions; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Snapshot; -import com.codahale.metrics.UniformReservoir; -import com.fasterxml.jackson.core.JsonGenerationException; -import com.fasterxml.jackson.databind.JsonMappingException; -import com.fasterxml.jackson.databind.ObjectMapper; - @Category({MiscTests.class, SmallTests.class}) public class TestPerformanceEvaluation { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPerformanceEvaluation.class); + private static final HBaseTestingUtility HTU = new HBaseTestingUtility(); @Test diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestDriver.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestDriver.java index fa03a171f4e..327b7afec2f 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestDriver.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestDriver.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,19 +17,25 @@ */ package org.apache.hadoop.hbase.mapred; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.util.ProgramDriver; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; - @Category({MapReduceTests.class, SmallTests.class}) public class TestDriver { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDriver.class); + @Test public void testDriverMainMethod() throws Throwable { ProgramDriver programDriverMock = mock(ProgramDriver.class); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestGroupingTableMap.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestGroupingTableMap.java index 584b4fe0796..12db348ba8b 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestGroupingTableMap.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestGroupingTableMap.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,26 +23,27 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; import java.io.IOException; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.testclassification.MapReduceTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -52,6 +52,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; @Category({MapReduceTests.class, SmallTests.class}) public class TestGroupingTableMap { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGroupingTableMap.class); + @Test @SuppressWarnings({ "deprecation", "unchecked" }) public void shouldNotCallCollectonSinceFindUniqueKeyValueMoreThanOnes() diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestIdentityTableMap.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestIdentityTableMap.java index be65d84d381..25576c1ef42 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestIdentityTableMap.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestIdentityTableMap.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,13 +22,14 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import java.io.IOException; - -import org.apache.hadoop.hbase.testclassification.MapReduceTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -37,6 +37,10 @@ import org.mockito.Mockito; @Category({MapReduceTests.class, SmallTests.class}) public class TestIdentityTableMap { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIdentityTableMap.class); + @Test @SuppressWarnings({ "deprecation", "unchecked" }) public void shouldCollectPredefinedTimes() throws IOException { diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestMultiTableSnapshotInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestMultiTableSnapshotInputFormat.java index 9497592baff..1dd3e69f977 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestMultiTableSnapshotInputFormat.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestMultiTableSnapshotInputFormat.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.mapred; import static org.junit.Assert.assertTrue; @@ -23,9 +22,9 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Iterator; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -38,6 +37,7 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.RunningJob; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,6 +48,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; public class TestMultiTableSnapshotInputFormat extends org.apache.hadoop.hbase.mapreduce.TestMultiTableSnapshotInputFormat { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiTableSnapshotInputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMultiTableSnapshotInputFormat.class); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestRowCounter.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestRowCounter.java index f1e0f1ab72a..b977c3d3fb4 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestRowCounter.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestRowCounter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -29,16 +28,17 @@ import static org.mockito.Mockito.times; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.testclassification.MapReduceTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapred.RowCounter.RowCounterMapper; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -48,6 +48,10 @@ import org.apache.hbase.thirdparty.com.google.common.base.Joiner; @Category({MapReduceTests.class, SmallTests.class}) public class TestRowCounter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCounter.class); + @Test @SuppressWarnings("deprecation") public void shouldPrintUsage() throws Exception { diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestSplitTable.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestSplitTable.java index 2897e7bee52..7b097d264ce 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestSplitTable.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestSplitTable.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,11 +21,13 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,6 +35,11 @@ import org.junit.rules.TestName; @Category({MapReduceTests.class, SmallTests.class}) public class TestSplitTable { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSplitTable.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java index 369f1c11745..91bf3efd43a 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -31,10 +30,10 @@ import static org.mockito.Mockito.spy; import java.io.IOException; import java.util.Arrays; import java.util.Map; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -62,6 +61,7 @@ import org.apache.hadoop.mapred.lib.NullOutputFormat; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.invocation.InvocationOnMock; @@ -75,6 +75,10 @@ import org.slf4j.LoggerFactory; @Category({MapReduceTests.class, LargeTests.class}) public class TestTableInputFormat { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableInputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableInputFormat.class); private final static HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java index d300e7d7451..e3684761306 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,15 +21,15 @@ import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; - import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TestTableMapReduceBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; @@ -38,6 +37,7 @@ import org.apache.hadoop.mapred.MapReduceBase; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.RunningJob; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,6 +50,11 @@ import org.slf4j.LoggerFactory; @Category({MapReduceTests.class, LargeTests.class}) @SuppressWarnings("deprecation") public class TestTableMapReduce extends TestTableMapReduceBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableMapReduce.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableMapReduce.class.getName()); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java index c74607793b7..fe160062669 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -27,17 +26,17 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; @@ -49,16 +48,22 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; @Category({MapReduceTests.class, LargeTests.class}) public class TestTableMapReduceUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableMapReduceUtil.class); + private static final Logger LOG = LoggerFactory .getLogger(TestTableMapReduceUtil.class); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableOutputFormatConnectionExhaust.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableOutputFormatConnectionExhaust.java index 785380f5a0a..746ac532ac9 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableOutputFormatConnectionExhaust.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableOutputFormatConnectionExhaust.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.hbase.mapred; +import static org.junit.Assert.fail; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -26,15 +30,12 @@ import org.apache.hadoop.mapred.RecordWriter; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - -import static org.junit.Assert.fail; - /** * Spark creates many instances of TableOutputFormat within a single process. We need to make * sure we can have many instances and not leak connections. @@ -44,6 +45,10 @@ import static org.junit.Assert.fail; @Category(MediumTests.class) public class TestTableOutputFormatConnectionExhaust { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableOutputFormatConnectionExhaust.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableOutputFormatConnectionExhaust.class); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java index 8b4e9185856..b61ed07d8af 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java @@ -15,20 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.mapred; import static org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormatImpl.SNAPSHOT_INPUTFORMAT_LOCALITY_ENABLED_DEFAULT; import static org.mockito.Mockito.mock; +import java.io.IOException; +import java.util.Iterator; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormatTestBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.RegionSplitter; @@ -44,17 +46,19 @@ import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.mapred.lib.NullOutputFormat; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.IOException; -import java.util.Iterator; - @Category({VerySlowMapReduceTests.class, LargeTests.class}) public class TestTableSnapshotInputFormat extends TableSnapshotInputFormatTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableSnapshotInputFormat.class); + private static final byte[] aaa = Bytes.toBytes("aaa"); private static final byte[] after_zzz = Bytes.toBytes("zz{"); // 'z' + 1 => '{' private static final String COLUMNS = diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java index b2be09fb094..8df7a6c7470 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -28,11 +28,9 @@ import java.util.List; import java.util.Locale; import java.util.Map; import java.util.NavigableMap; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Result; @@ -47,9 +45,7 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,8 +55,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; * Base set of tests and setup for input formats touching multiple tables. */ public abstract class MultiTableInputFormatTestBase { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); static final Logger LOG = LoggerFactory.getLogger(TestMultiTableInputFormat.class); public static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); static final String TABLE_NAME = "scantest"; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java index d753d40add9..385ad3e6b23 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java @@ -18,11 +18,14 @@ package org.apache.hadoop.hbase.mapreduce; +import static org.junit.Assert.assertFalse; + +import java.io.IOException; +import java.util.Arrays; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -39,21 +42,12 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.junit.Assert; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertFalse; - -import java.io.IOException; -import java.util.Arrays; - public abstract class TableSnapshotInputFormatTestBase { private static final Logger LOG = LoggerFactory.getLogger(TableSnapshotInputFormatTestBase.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); protected final HBaseTestingUtility UTIL = new HBaseTestingUtility(); protected static final int NUM_REGION_SERVERS = 2; protected static final byte[][] FAMILIES = {Bytes.toBytes("f1"), Bytes.toBytes("f2")}; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java index 62a73064fe8..42ef33786a1 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java @@ -39,7 +39,6 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.stream.Collectors; import java.util.stream.Stream; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -49,10 +48,10 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.hbase.ArrayBackedTag; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -94,7 +93,6 @@ import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.ReflectionUtils; -import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -105,11 +103,10 @@ import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.junit.ClassRule; import org.junit.Ignore; -import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.mockito.Mockito; /** @@ -120,8 +117,11 @@ import org.mockito.Mockito; */ @Category({VerySlowMapReduceTests.class, LargeTests.class}) public class TestCellBasedHFileOutputFormat2 { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellBasedHFileOutputFormat2.class); + private final static int ROWSPERSPLIT = 1024; public static final byte[] FAMILY_NAME = TestHRegionFileSystem.FAMILY_NAME; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java index 87c79ecbd7c..cb179cce70e 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java @@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -34,7 +34,6 @@ import java.net.URL; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -42,11 +41,12 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; @@ -82,6 +82,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -95,6 +96,10 @@ import org.mockito.stubbing.Answer; @Category({VerySlowMapReduceTests.class, MediumTests.class}) public class TestCellBasedImportExport2 { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellBasedImportExport2.class); + private static final Log LOG = LogFactory.getLog(TestCellBasedImportExport2.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final byte[] ROW1 = Bytes.toBytesBinary("\\x32row1"); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java index 03bd0b13791..d25603c9996 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.mapreduce; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -28,12 +28,12 @@ import static org.mockito.Mockito.when; import java.io.ByteArrayOutputStream; import java.io.PrintStream; import java.util.ArrayList; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -46,20 +46,21 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALCellMapper; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.LauncherSecurityManager; import org.apache.hadoop.hbase.util.MapReduceExtendedCell; import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Mapper.Context; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -72,6 +73,11 @@ import org.mockito.stubbing.Answer; */ @Category({MapReduceTests.class, LargeTests.class}) public class TestCellBasedWALPlayer2 { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellBasedWALPlayer2.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static MiniHBaseCluster cluster; private static Path rootDir; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java index ff623cb8e03..0fb4f76cbd2 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java @@ -15,14 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.mapreduce; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.*; import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; @@ -35,19 +40,19 @@ import org.apache.hadoop.hbase.util.LauncherSecurityManager; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.*; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - @Category({MapReduceTests.class, LargeTests.class}) public class TestCellCounter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellCounter.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final byte[] ROW1 = Bytes.toBytesBinary("\\x01row1"); private static final byte[] ROW2 = Bytes.toBytesBinary("\\x01row2"); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java index 0bec03b91de..ef5ce5dccab 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java @@ -25,9 +25,9 @@ import static org.junit.Assert.fail; import java.io.ByteArrayOutputStream; import java.io.PrintStream; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.util.LauncherSecurityManager; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +52,11 @@ import org.junit.rules.TestName; */ @Category({MapReduceTests.class, LargeTests.class}) public class TestCopyTable { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCopyTable.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] ROW1 = Bytes.toBytes("row1"); private static final byte[] ROW2 = Bytes.toBytes("row2"); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestGroupingTableMapper.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestGroupingTableMapper.java index ec7ddee1808..46a449a4359 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestGroupingTableMapper.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestGroupingTableMapper.java @@ -1,40 +1,47 @@ /** - * 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 + * 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 + * 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 + * 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.mapreduce; -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.testclassification.MapReduceTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.mapreduce.Mapper; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static org.mockito.Mockito.*; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.mapreduce.Mapper; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + @Category({MapReduceTests.class, SmallTests.class}) public class TestGroupingTableMapper { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGroupingTableMapper.class); + /** * Test GroupingTableMapper class */ diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index 8e2c4c8951c..d9c0ae9b532 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -39,7 +38,6 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.stream.Collectors; import java.util.stream.Stream; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -47,10 +45,10 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.hbase.ArrayBackedTag; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -102,11 +100,10 @@ import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.junit.ClassRule; import org.junit.Ignore; -import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -120,8 +117,11 @@ import org.slf4j.LoggerFactory; @Category({VerySlowMapReduceTests.class, LargeTests.class}) //TODO : Remove this in 3.0 public class TestHFileOutputFormat2 { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileOutputFormat2.class); + private final static int ROWSPERSPLIT = 1024; public static final byte[] FAMILY_NAME = TestHRegionFileSystem.FAMILY_NAME; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java index c0debb43431..8251a3e4136 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java @@ -1,38 +1,47 @@ /** - * 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 + * 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 + * 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 + * 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.mapreduce; +import static org.junit.Assert.assertEquals; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import static org.junit.Assert.assertEquals; - @Category({MapReduceTests.class, MediumTests.class}) public class TestHRegionPartitioner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionPartitioner.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); @Rule diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHashTable.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHashTable.java index 3ccc01a5eca..167bccd1953 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHashTable.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHashTable.java @@ -21,10 +21,10 @@ import static org.junit.Assert.assertEquals; import java.util.HashMap; import java.util.Map; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; @@ -36,22 +36,27 @@ import org.apache.hadoop.io.MapFile; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; -import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; +import org.apache.hbase.thirdparty.com.google.common.collect.Maps; + /** * Basic test for the HashTable M/R tool */ @Category(LargeTests.class) public class TestHashTable { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHashTable.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHashTable.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index 65d53f3cd3b..6d323c9b419 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -34,17 +34,17 @@ import java.net.URL; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; @@ -79,6 +79,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -95,6 +96,10 @@ import org.slf4j.LoggerFactory; //TODO : Remove this in 3.0 public class TestImportExport { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImportExport.class); + private static final Logger LOG = LoggerFactory.getLogger(TestImportExport.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final byte[] ROW1 = Bytes.toBytesBinary("\\x32row1"); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java index 5d4c8a3ad70..9951e64ddfd 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java @@ -26,24 +26,19 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; import java.util.UUID; - import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; -import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -51,26 +46,33 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MapReduceTests.class, LargeTests.class}) public class TestImportTSVWithOperationAttributes implements Configurable { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImportTSVWithOperationAttributes.class); + private static final Logger LOG = LoggerFactory.getLogger(TestImportTSVWithOperationAttributes.class); protected static final String NAME = TestImportTsv.class.getSimpleName(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java index 9ddbc65dda0..aaa495efcc4 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java @@ -25,29 +25,30 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; import java.util.UUID; - import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; -import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,6 +59,10 @@ import org.slf4j.LoggerFactory; @Category({MapReduceTests.class, LargeTests.class}) public class TestImportTSVWithTTLs implements Configurable { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImportTSVWithTTLs.class); + protected static final Logger LOG = LoggerFactory.getLogger(TestImportTSVWithTTLs.class); protected static final String NAME = TestImportTsv.class.getSimpleName(); protected static HBaseTestingUtility util = new HBaseTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java index 6d6b72916b9..2e5f3be21b0 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java @@ -28,7 +28,6 @@ import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.UUID; - import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -37,6 +36,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -68,6 +68,7 @@ import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -78,6 +79,10 @@ import org.slf4j.LoggerFactory; @Category({MapReduceTests.class, LargeTests.class}) public class TestImportTSVWithVisibilityLabels implements Configurable { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImportTSVWithVisibilityLabels.class); + private static final Logger LOG = LoggerFactory.getLogger(TestImportTSVWithVisibilityLabels.class); protected static final String NAME = TestImportTsv.class.getSimpleName(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java index 9484a94a395..68c6b6b4bca 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -31,7 +30,6 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; - import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -40,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -65,6 +64,7 @@ import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -75,6 +75,10 @@ import org.slf4j.LoggerFactory; @Category({VerySlowMapReduceTests.class, LargeTests.class}) public class TestImportTsv implements Configurable { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImportTsv.class); + private static final Logger LOG = LoggerFactory.getLogger(TestImportTsv.class); protected static final String NAME = TestImportTsv.class.getSimpleName(); protected static HBaseTestingUtility util = new HBaseTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsvParser.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsvParser.java index 5b82451bd9a..86c23a05343 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsvParser.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsvParser.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.mapreduce; import static org.junit.Assert.assertEquals; @@ -25,15 +24,16 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.util.ArrayList; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.MapReduceTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser; import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException; import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.ParsedLine; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -47,6 +47,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; @Category({MapReduceTests.class, SmallTests.class}) public class TestImportTsvParser { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImportTsvParser.class); + private void assertBytesEquals(byte[] a, byte[] b) { assertEquals(Bytes.toStringBinary(a), Bytes.toStringBinary(b)); } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestJarFinder.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestJarFinder.java index 7eeee707ea2..d1f48bb299e 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestJarFinder.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestJarFinder.java @@ -15,15 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.mapreduce; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Assert; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.slf4j.LoggerFactory; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.File; @@ -37,6 +30,13 @@ import java.util.Properties; import java.util.jar.JarInputStream; import java.util.jar.JarOutputStream; import java.util.jar.Manifest; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.LoggerFactory; /** * This file was forked from hadoop/common/branches/branch-2@1350012. @@ -44,6 +44,10 @@ import java.util.jar.Manifest; @Category(SmallTests.class) public class TestJarFinder { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestJarFinder.class); + @Test public void testJar() throws Exception { diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java index 0c5207b700c..b4a53ea65e8 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java @@ -17,17 +17,18 @@ */ package org.apache.hadoop.hbase.mapreduce; +import java.io.IOException; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; import org.apache.hadoop.mapreduce.Job; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import java.io.IOException; -import java.util.List; - /** * Tests various scan start and stop row scenarios. This is set in a scan and * tested in a MapReduce job to see if that is handed over and done properly @@ -36,6 +37,10 @@ import java.util.List; @Category({VerySlowMapReduceTests.class, LargeTests.class}) public class TestMultiTableInputFormat extends MultiTableInputFormatTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiTableInputFormat.class); + @BeforeClass public static void setupLogging() { TEST_UTIL.enableDebug(MultiTableInputFormat.class); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java index 47d7c0ca0d2..efc59c7844a 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,8 +17,17 @@ */ package org.apache.hadoop.hbase.mapreduce; +import static org.apache.hadoop.hbase.client.Scan.SCAN_ATTRIBUTES_TABLE_NAME; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -44,40 +53,27 @@ import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.junit.After; -import org.junit.AfterClass; import org.junit.Assert; -import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.apache.hadoop.hbase.client.Scan.SCAN_ATTRIBUTES_TABLE_NAME; - /** * Tests of MultiTableInputFormatBase. */ @Category({SmallTests.class}) public class TestMultiTableInputFormatBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiTableInputFormatBase.class); + @Rule public final TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder() - .withTimeout(this.getClass()) - .withLookingForStuckThread(true) - .build(); /** * Test getSplits only puts up one Connection. diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java index e26289267e6..1d5a5e07807 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.mapreduce; +import edu.umd.cs.findbugs.annotations.Nullable; import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Map; - import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -35,17 +35,20 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.mapreduce.Job; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.apache.hbase.thirdparty.com.google.common.base.Function; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; import org.apache.hbase.thirdparty.com.google.common.collect.Multimaps; -import edu.umd.cs.findbugs.annotations.Nullable; - @Category({ VerySlowMapReduceTests.class, LargeTests.class }) public class TestMultiTableSnapshotInputFormat extends MultiTableInputFormatTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiTableSnapshotInputFormat.class); + protected Path restoreDir; @BeforeClass diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormatImpl.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormatImpl.java index 05e93c73310..685e5194e36 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormatImpl.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormatImpl.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -15,39 +15,44 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.mapreduce; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hbase.thirdparty.com.google.common.collect.Maps; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; - -import java.io.IOException; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Objects; - import static org.junit.Assert.assertEquals; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.verify; +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.Mockito; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.collect.Maps; + @Category({ SmallTests.class }) public class TestMultiTableSnapshotInputFormatImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiTableSnapshotInputFormatImpl.class); + private MultiTableSnapshotInputFormatImpl subject; private Map> snapshotScans; private Path restoreDir; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java index 357f3750ef5..dacc15c74b2 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java @@ -25,13 +25,13 @@ import java.io.IOException; import java.util.Iterator; import java.util.Map; import java.util.NavigableMap; - import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -46,6 +46,7 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -58,6 +59,11 @@ import org.slf4j.LoggerFactory; */ @Category({MapReduceTests.class, LargeTests.class}) public class TestMultithreadedTableMapper { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultithreadedTableMapper.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMultithreadedTableMapper.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java index 1227595bb35..85d0f1c8ddd 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionMetrics; @@ -35,6 +36,7 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -42,6 +44,10 @@ import org.mockito.Mockito; @Category({MiscTests.class, SmallTests.class}) public class TestRegionSizeCalculator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionSizeCalculator.class); + private Configuration configuration = new Configuration(); private final long megabyte = 1024L * 1024L; private final ServerName sn = ServerName.valueOf("local-rs", DEFAULT_REGIONSERVER_PORT, diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java index aba17147537..b07de7fb93a 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.mapreduce; import static org.junit.Assert.assertEquals; @@ -26,8 +25,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; import java.util.ArrayList; - -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; @@ -40,10 +38,9 @@ import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Job; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,8 +49,11 @@ import org.slf4j.LoggerFactory; */ @Category({MapReduceTests.class, LargeTests.class}) public class TestRowCounter { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCounter.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRowCounter.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static String TABLE_NAME = "testRowCounter"; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSimpleTotalOrderPartitioner.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSimpleTotalOrderPartitioner.java index 5629cb4c9e8..9a91013f22e 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSimpleTotalOrderPartitioner.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSimpleTotalOrderPartitioner.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,19 +21,25 @@ import static org.junit.Assert.assertEquals; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.experimental.categories.Category; - +import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; /** * Test of simple partitioner. */ @Category({MapReduceTests.class, SmallTests.class}) public class TestSimpleTotalOrderPartitioner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSimpleTotalOrderPartitioner.class); + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); Configuration conf = TEST_UTIL.getConfiguration(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSyncTable.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSyncTable.java index 2bfeb92eac5..543a169764a 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSyncTable.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSyncTable.java @@ -21,12 +21,11 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.Arrays; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; @@ -41,13 +40,14 @@ import org.apache.hadoop.mapreduce.Counters; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Throwables; /** @@ -55,8 +55,11 @@ import org.apache.hbase.thirdparty.com.google.common.base.Throwables; */ @Category(LargeTests.class) public class TestSyncTable { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncTable.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSyncTable.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java index 5453054373f..b7ea5d0ec16 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -31,8 +30,8 @@ import static org.mockito.Mockito.spy; import java.io.IOException; import java.util.Arrays; import java.util.Map; - import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -58,6 +57,7 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.invocation.InvocationOnMock; @@ -72,6 +72,10 @@ import org.slf4j.LoggerFactory; @Category(LargeTests.class) public class TestTableInputFormat { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableInputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableInputFormat.class); private final static HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java index 699e773ce9e..5fa4b546915 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,13 +22,19 @@ import static org.junit.Assert.*; import java.net.Inet6Address; import java.net.InetAddress; import java.net.UnknownHostException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({SmallTests.class}) public class TestTableInputFormatBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableInputFormatBase.class); + @Test public void testTableInputFormatBaseReverseDNSForIPv6() throws UnknownHostException { diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScan1.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScan1.java index 553869ef8bb..e76f5ad857e 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScan1.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScan1.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,9 +18,10 @@ package org.apache.hadoop.hbase.mapreduce; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -32,6 +32,10 @@ import org.junit.experimental.categories.Category; @Category({VerySlowMapReduceTests.class, LargeTests.class}) public class TestTableInputFormatScan1 extends TestTableInputFormatScanBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableInputFormatScan1.class); + /** * Tests a MR scan using specific start and stop rows. * diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScan2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScan2.java index 02f893fca32..83158d8f970 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScan2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScan2.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,9 +18,10 @@ package org.apache.hadoop.hbase.mapreduce; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -32,6 +32,10 @@ import org.junit.experimental.categories.Category; @Category({VerySlowMapReduceTests.class, LargeTests.class}) public class TestTableInputFormatScan2 extends TestTableInputFormatScanBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableInputFormatScan2.class); + /** * Tests a MR scan using specific start and stop rows. * diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java index 9c38a0ddf3a..786da1a0204 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,9 +25,9 @@ import java.io.File; import java.io.IOException; import java.util.Map; import java.util.NavigableMap; - import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -45,6 +44,7 @@ import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -58,6 +58,11 @@ import org.slf4j.LoggerFactory; @Category({VerySlowMapReduceTests.class, LargeTests.class}) public class TestTableMapReduce extends TestTableMapReduceBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableMapReduce.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableMapReduce.class); @Override diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java index 60e2622856c..f86f20d6412 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java @@ -25,9 +25,7 @@ import java.io.IOException; import java.util.Iterator; import java.util.Map; import java.util.NavigableMap; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -42,9 +40,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TestRule; import org.slf4j.Logger; /** @@ -54,8 +50,6 @@ import org.slf4j.Logger; * implementations. */ public abstract class TestTableMapReduceBase { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); protected static final TableName MULTI_REGION_TABLE_NAME = TableName.valueOf("mrtest"); protected static final TableName TABLE_FOR_NEGATIVE_TESTS = TableName.valueOf("testfailuretable"); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceUtil.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceUtil.java index 506bf4f4d5c..73851e08852 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceUtil.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceUtil.java @@ -1,31 +1,35 @@ /** - * 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 + * 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 + * 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 + * 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.mapreduce; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -35,6 +39,10 @@ import org.junit.experimental.categories.Category; @Category({MapReduceTests.class, SmallTests.class}) public class TestTableMapReduceUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableMapReduceUtil.class); + /* * initTableSnapshotMapperJob is tested in {@link TestTableSnapshotInputFormat} because * the method depends on an online cluster. diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java index ca5c3ae4dfe..ac9862deb9c 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java @@ -15,33 +15,37 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.mapreduce; import static org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormatImpl.SNAPSHOT_INPUTFORMAT_LOCALITY_ENABLED_DEFAULT; import static org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormatImpl.SNAPSHOT_INPUTFORMAT_LOCALITY_ENABLED_KEY; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.io.IOException; +import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TestTableSnapshotScanner; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat.TableSnapshotRegionSplit; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.RegionSplitter; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; @@ -51,28 +55,24 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; import org.junit.After; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import java.util.Arrays; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.RegionSplitter; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({VerySlowMapReduceTests.class, LargeTests.class}) public class TestTableSnapshotInputFormat extends TableSnapshotInputFormatTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableSnapshotInputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableSnapshotInputFormat.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); private static final byte[] bbb = Bytes.toBytes("bbb"); private static final byte[] yyy = Bytes.toBytes("yyy"); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSplit.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSplit.java index 4382c9c6bc1..9be163448c8 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSplit.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSplit.java @@ -17,24 +17,30 @@ */ package org.apache.hadoop.hbase.mapreduce; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.HashSet; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.util.ReflectionUtils; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.util.HashSet; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - @Category({MapReduceTests.class, SmallTests.class}) public class TestTableSplit { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableSplit.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java index 3f0c591f2f0..6c4d0b6f06b 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,11 +17,19 @@ */ package org.apache.hadoop.hbase.mapreduce; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -47,21 +54,19 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; - @Category({MapReduceTests.class, LargeTests.class}) public class TestTimeRangeMapRed { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTimeRangeMapRed.class); + private final static Logger log = LoggerFactory.getLogger(TestTimeRangeMapRed.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java index a35c88e30f4..eedada88f89 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.mapreduce; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -28,12 +28,12 @@ import static org.mockito.Mockito.when; import java.io.ByteArrayOutputStream; import java.io.PrintStream; import java.util.ArrayList; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -46,19 +46,20 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALKeyValueMapper; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.LauncherSecurityManager; import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Mapper.Context; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -72,6 +73,11 @@ import org.mockito.stubbing.Answer; @Category({MapReduceTests.class, LargeTests.class}) //TODO : Remove this in 3.0 public class TestWALPlayer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALPlayer.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static MiniHBaseCluster cluster; private static Path rootDir; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java index c8db903650a..87d100b9994 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java @@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -51,6 +52,7 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -61,6 +63,11 @@ import org.slf4j.LoggerFactory; */ @Category({ MapReduceTests.class, MediumTests.class }) public class TestWALRecordReader { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALRecordReader.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWALRecordReader.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Configuration conf; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java index c0b28397323..1f080c82042 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.mapreduce.Job; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -69,6 +71,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({ ReplicationTests.class, LargeTests.class }) public class TestVerifyReplication extends TestReplicationBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVerifyReplication.class); + private static final Logger LOG = LoggerFactory.getLogger(TestVerifyReplication.class); private static final String PEER_ID = "2"; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java index 8703ca02065..6dca6e74457 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.snapshot; import static org.apache.hadoop.util.ToolRunner.run; @@ -30,12 +29,11 @@ import java.util.HashSet; import java.util.List; import java.util.Objects; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -50,13 +48,14 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; @@ -65,8 +64,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.Snapshot */ @Category({VerySlowMapReduceTests.class, LargeTests.class}) public class TestExportSnapshot { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestExportSnapshot.class); + private static final Logger LOG = LoggerFactory.getLogger(TestExportSnapshot.class); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotHelpers.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotHelpers.java index e31e81ebf63..d104d830985 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotHelpers.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotHelpers.java @@ -15,26 +15,32 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.snapshot; import static org.junit.Assert.assertEquals; import java.util.ArrayList; import java.util.List; - -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Pair; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo; + /** * Test Export Snapshot Tool helpers */ @Category({RegionServerTests.class, SmallTests.class}) public class TestExportSnapshotHelpers { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestExportSnapshotHelpers.class); + /** * Verfy the result of getBalanceSplits() method. * The result are groups of files, used as input list for the "export" mappers. diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java index f3d08ba52f7..0720b363559 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java @@ -15,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.snapshot; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -31,10 +30,9 @@ import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,8 +41,11 @@ import org.slf4j.LoggerFactory; */ @Category({MapReduceTests.class, MediumTests.class}) public class TestExportSnapshotNoCluster { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestExportSnapshotNoCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestExportSnapshotNoCluster.class); protected final static HBaseCommonTestingUtility TEST_UTIL = new HBaseCommonTestingUtility(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java index 5ba575dfa0b..ed605b90be5 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java @@ -15,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.snapshot; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.experimental.categories.Category; @@ -35,6 +35,10 @@ import org.junit.experimental.categories.Category; @Category({VerySlowRegionServerTests.class, LargeTests.class}) public class TestMobExportSnapshot extends TestExportSnapshot { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobExportSnapshot.class); + public static void setUpBaseConf(Configuration conf) { TestExportSnapshot.setUpBaseConf(conf); conf.setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java index d28e98687ae..b4e0afd2848 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -19,14 +17,15 @@ */ package org.apache.hadoop.hbase.snapshot; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.security.HadoopSecurityEnabledUserProviderForTesting; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.access.AccessControlLists; import org.apache.hadoop.hbase.security.access.SecureTestUtil; - +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.experimental.categories.Category; @@ -35,6 +34,11 @@ import org.junit.experimental.categories.Category; */ @Category({VerySlowRegionServerTests.class, LargeTests.class}) public class TestMobSecureExportSnapshot extends TestMobExportSnapshot { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobSecureExportSnapshot.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { setUpBaseConf(TEST_UTIL.getConfiguration()); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java index f5278396f1f..8804b83fb2a 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -19,27 +17,27 @@ */ package org.apache.hadoop.hbase.snapshot; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.security.HadoopSecurityEnabledUserProviderForTesting; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.access.AccessControlLists; import org.apache.hadoop.hbase.security.access.SecureTestUtil; - import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; /** * Reruns TestExportSnapshot using ExportSnapshot in secure mode. */ @Category({VerySlowRegionServerTests.class, LargeTests.class}) public class TestSecureExportSnapshot extends TestExportSnapshot { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecureExportSnapshot.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { setUpBaseConf(TEST_UTIL.getConfiguration()); diff --git a/hbase-metrics-api/pom.xml b/hbase-metrics-api/pom.xml index 3389f7e9f27..2cb04a0c9ab 100644 --- a/hbase-metrics-api/pom.xml +++ b/hbase-metrics-api/pom.xml @@ -90,6 +90,12 @@ org.apache.hbase hbase-common + + org.apache.hbase + hbase-common + test-jar + test + org.slf4j diff --git a/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java b/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java index 3c5a7bfc86c..59f26999bd2 100644 --- a/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java +++ b/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java @@ -21,7 +21,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.mockito.Mockito.mock; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -33,6 +35,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category(SmallTests.class) public class TestMetricRegistriesLoader { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricRegistriesLoader.class); + @Test public void testLoadSinleInstance() { MetricRegistries loader = mock(MetricRegistries.class); diff --git a/hbase-metrics/pom.xml b/hbase-metrics/pom.xml index 3abca8300ab..c215e5faed0 100644 --- a/hbase-metrics/pom.xml +++ b/hbase-metrics/pom.xml @@ -94,6 +94,12 @@ org.apache.hbase hbase-common + + org.apache.hbase + hbase-common + test-jar + test + diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java index e23f2f8cdbc..5b5e26f13a8 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java @@ -19,9 +19,11 @@ package org.apache.hadoop.hbase.metrics.impl; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.metrics.Counter; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -31,6 +33,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestCounterImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCounterImpl.class); + private Counter counter; @Before public void setup() { diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java index e48d0519d71..072f18a3b15 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java @@ -18,9 +18,10 @@ package org.apache.hadoop.hbase.metrics.impl; import com.codahale.metrics.Meter; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -31,6 +32,10 @@ import org.mockito.Mockito; @Category(SmallTests.class) public class TestDropwizardMeter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDropwizardMeter.class); + private Meter meter; @Before public void setup() { diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java index 6024d25ee69..371f767f74c 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java @@ -21,10 +21,11 @@ import static org.junit.Assert.assertEquals; import java.util.Arrays; import java.util.Random; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,6 +35,10 @@ import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestFastLongHistogram { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFastLongHistogram.class); + private static void doTestUniform(FastLongHistogram hist) { long[] VALUES = { 0, 10, 20, 30, 40, 50 }; double[] qs = new double[VALUES.length]; diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGauge.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGauge.java index 786c052f372..52d29fc700c 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGauge.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGauge.java @@ -20,9 +20,10 @@ package org.apache.hadoop.hbase.metrics.impl; import static org.junit.Assert.assertEquals; import java.util.concurrent.atomic.AtomicLong; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.metrics.Gauge; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -32,6 +33,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestGauge { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGauge.class); + @Test public void testGetValue() { SimpleGauge gauge = new SimpleGauge(); diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java index 1ae3c2c38c2..2e53869650f 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -16,16 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - package org.apache.hadoop.hbase.metrics.impl; import static org.junit.Assert.assertEquals; import java.util.stream.IntStream; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.metrics.Snapshot; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -35,6 +33,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestHistogramImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHistogramImpl.class); + @Test public void testUpdate() { HistogramImpl histogram = new HistogramImpl(); diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java index bf086173fb7..1115529a051 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -16,8 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - package org.apache.hadoop.hbase.metrics.impl; import static org.junit.Assert.assertEquals; @@ -26,7 +23,7 @@ import static org.junit.Assert.assertTrue; import java.util.Map; import java.util.Optional; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.metrics.Counter; import org.apache.hadoop.hbase.metrics.Gauge; import org.apache.hadoop.hbase.metrics.Meter; @@ -35,11 +32,17 @@ import org.apache.hadoop.hbase.metrics.MetricRegistryInfo; import org.apache.hadoop.hbase.metrics.Timer; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestMetricRegistryImpl { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricRegistryImpl.class); + private MetricRegistryInfo info; private MetricRegistryImpl registry; diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java index 191735b7e3b..c5ed1edb9eb 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -25,9 +24,10 @@ import static org.junit.Assert.assertTrue; import java.util.Collection; import java.util.Set; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -36,6 +36,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category(SmallTests.class) public class TestRefCountingMap { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRefCountingMap.class); + private RefCountingMap map; @Before diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestTimerImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestTimerImpl.java index eef99404abd..d9d3632b731 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestTimerImpl.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestTimerImpl.java @@ -20,10 +20,11 @@ package org.apache.hadoop.hbase.metrics.impl; import static org.junit.Assert.assertEquals; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.metrics.Timer; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -33,6 +34,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestTimerImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTimerImpl.class); + private Timer timer; @Before diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestChildProcedures.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestChildProcedures.java index 79ce73617ce..3d99b31abdc 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestChildProcedures.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestChildProcedures.java @@ -15,29 +15,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; -import java.io.IOException; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import java.io.IOException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - @Category({MasterTests.class, SmallTests.class}) public class TestChildProcedures { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestChildProcedures.class); + private static final Logger LOG = LoggerFactory.getLogger(TestChildProcedures.class); private static final int PROCEDURE_EXECUTOR_SLOTS = 1; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureEvents.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureEvents.java index f36da823599..05dab7f1e75 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureEvents.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureEvents.java @@ -15,33 +15,39 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; +import static org.junit.Assert.assertEquals; + import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.NoopProcedure; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; -import org.apache.hbase.thirdparty.com.google.protobuf.Int32Value; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; - +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; +import org.apache.hbase.thirdparty.com.google.protobuf.Int32Value; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; @Category({MasterTests.class, SmallTests.class}) public class TestProcedureEvents { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureEvents.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureEvents.class); private TestProcEnv procEnv; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java index ae781cd9139..e79088471ce 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java @@ -15,34 +15,40 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Objects; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; @Category({MasterTests.class, SmallTests.class}) public class TestProcedureExecution { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureExecution.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureExecution.class); private static final int PROCEDURE_EXECUTOR_SLOTS = 1; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecutor.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecutor.java index e5b50707722..db19974f096 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecutor.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecutor.java @@ -15,32 +15,36 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; +import static org.junit.Assert.assertEquals; + import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.NoopProcedure; import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Threads; - import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; - @Category({MasterTests.class, SmallTests.class}) public class TestProcedureExecutor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureExecutor.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureExecutor.class); private TestProcEnv procEnv; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureInMemoryChore.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureInMemoryChore.java index 33d93d548fa..86293e13477 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureInMemoryChore.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureInMemoryChore.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; import static org.junit.Assert.assertFalse; @@ -24,13 +23,14 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -38,6 +38,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, SmallTests.class}) public class TestProcedureInMemoryChore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureInMemoryChore.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureInMemoryChore.class); private static final int PROCEDURE_EXECUTOR_SLOTS = 1; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java index 0550a91abe1..94a293d75fe 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java @@ -15,30 +15,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; - @Category({MasterTests.class, SmallTests.class}) public class TestProcedureMetrics { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureMetrics.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureMetrics.class); private static final int PROCEDURE_EXECUTOR_SLOTS = 1; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureNonce.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureNonce.java index bebfae001a3..76129879af4 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureNonce.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureNonce.java @@ -15,37 +15,41 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; -import java.io.IOException; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.CountDownLatch; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseCommonTestingUtility; -import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.NonceKey; -import org.apache.hadoop.hbase.util.Threads; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.NonceKey; +import org.apache.hadoop.hbase.util.Threads; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + @Category({MasterTests.class, SmallTests.class}) public class TestProcedureNonce { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureNonce.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureNonce.class); private static final int PROCEDURE_EXECUTOR_SLOTS = 2; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java index f4ba07d49ef..16af8433c3b 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java @@ -15,37 +15,43 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; -import java.io.IOException; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.CountDownLatch; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseCommonTestingUtility; -import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; -import org.apache.hbase.thirdparty.com.google.protobuf.Int32Value; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.Threads; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Threads; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.protobuf.Int32Value; + @Category({MasterTests.class, SmallTests.class}) public class TestProcedureRecovery { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureRecovery.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureRecovery.class); private static final int PROCEDURE_EXECUTOR_SLOTS = 1; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java index 821901ca980..217f98228a2 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java @@ -15,35 +15,40 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; -import java.io.IOException; -import java.util.ArrayList; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseCommonTestingUtility; -import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; -import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; -import org.apache.hbase.thirdparty.com.google.protobuf.Int64Value; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.MasterTests; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.IOException; +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; +import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.protobuf.Int64Value; + @Category({MasterTests.class, LargeTests.class}) public class TestProcedureReplayOrder { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureReplayOrder.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureReplayOrder.class); private static final int NUM_THREADS = 16; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSchedulerConcurrency.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSchedulerConcurrency.java index 6116736d3cf..35ca30bc2e4 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSchedulerConcurrency.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSchedulerConcurrency.java @@ -15,31 +15,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; -import java.io.IOException; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.ConcurrentSkipListSet; - -import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.NoopProcedure; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.util.Threads; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.NoopProcedure; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Threads; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + @Category({MasterTests.class, MediumTests.class}) public class TestProcedureSchedulerConcurrency { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureSchedulerConcurrency.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureEvents.class); private SimpleProcedureScheduler procSched; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java index 1a426505eab..393f57a9637 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; import static org.junit.Assert.assertEquals; @@ -24,7 +23,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; @@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Threads; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -40,6 +40,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, SmallTests.class}) public class TestProcedureSuspended { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureSuspended.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureSuspended.class); private static final int PROCEDURE_EXECUTOR_SLOTS = 1; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java index 9bbef912796..d0aee10e950 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java @@ -20,15 +20,23 @@ package org.apache.hadoop.hbase.procedure2; import static org.junit.Assert.assertTrue; import java.io.IOException; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; + @Category({MasterTests.class, SmallTests.class}) public class TestProcedureToString { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureToString.class); + /** * A do-nothing environment for BasicProcedure. */ diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureUtil.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureUtil.java index dec58541510..0fcb4f4283e 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureUtil.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureUtil.java @@ -15,21 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; -import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MasterTests; - -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; + @Category({MasterTests.class, SmallTests.class}) public class TestProcedureUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureUtil.class); + @Test public void testValidation() throws Exception { ProcedureUtil.validateClass(new TestProcedure(10)); diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java index fdaa65eb2b9..95f0e0679da 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java @@ -15,32 +15,36 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.NoopProcedure; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; - +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - @Category({MasterTests.class, SmallTests.class}) public class TestStateMachineProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStateMachineProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestStateMachineProcedure.class); private static final Exception TEST_FAILURE_EXCEPTION = new Exception("test failure") { diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java index e6fc4bbd8a5..7fa768243b0 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java @@ -15,34 +15,38 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.util.ArrayList; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.TimeUnit; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; - +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - @Category({MasterTests.class, SmallTests.class}) public class TestYieldProcedures { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestYieldProcedures.class); + private static final Logger LOG = LoggerFactory.getLogger(TestYieldProcedures.class); private static final int PROCEDURE_EXECUTOR_SLOTS = 1; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java index e4766f6f15a..ffc6ab8de0d 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java @@ -15,26 +15,30 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2.store; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + import java.util.Random; - -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker.BitSetNode; import org.apache.hadoop.hbase.testclassification.MasterTests; - +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker.BitSetNode; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -@Category({MasterTests.class, SmallTests.class}) +@Category({MasterTests.class, MediumTests.class}) public class TestProcedureStoreTracker { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureStoreTracker.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureStoreTracker.class); @Test diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java index 31c9cf3ee20..443386d0845 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java @@ -15,16 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2.store.wal; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.IOException; import java.util.Random; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.LoadCounter; @@ -32,22 +36,22 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure; import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; - import org.junit.After; import org.junit.Before; -import org.junit.Test; +import org.junit.ClassRule; import org.junit.Ignore; +import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - @Category({MasterTests.class, LargeTests.class}) public class TestStressWALProcedureStore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStressWALProcedureStore.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWALProcedureStore.class); private static final int PROCEDURE_STORE_SLOTS = 8; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java index bc7c9a7f852..6c7594f6774 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java @@ -15,9 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2.store.wal; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; @@ -27,11 +31,11 @@ import java.util.Arrays; import java.util.Comparator; import java.util.HashSet; import java.util.Set; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; @@ -42,25 +46,26 @@ import org.apache.hadoop.hbase.procedure2.SequentialProcedure; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator; import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker; -import org.apache.hbase.thirdparty.com.google.protobuf.Int64Value; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.io.IOUtils; - import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import org.apache.hbase.thirdparty.com.google.protobuf.Int64Value; @Category({MasterTests.class, SmallTests.class}) public class TestWALProcedureStore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALProcedureStore.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWALProcedureStore.class); private static final int PROCEDURE_STORE_SLOTS = 1; diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestDelayedUtil.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestDelayedUtil.java index dcb133e9ef0..0d494fcdd6b 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestDelayedUtil.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestDelayedUtil.java @@ -15,20 +15,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2.util; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import static org.junit.Assert.assertEquals; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; - @Category({MasterTests.class, SmallTests.class}) public class TestDelayedUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDelayedUtil.class); + private static final Logger LOG = LoggerFactory.getLogger(TestDelayedUtil.class); @Test diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestDeleteRow.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestDeleteRow.java index 15141c3fa7a..f4f9c757208 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestDeleteRow.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestDeleteRow.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -21,15 +21,21 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import javax.xml.bind.JAXBException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.rest.client.Response; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RestTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, MediumTests.class}) public class TestDeleteRow extends RowResourceBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDeleteRow.class); + @Test public void testDeleteNonExistentColumn() throws Exception { Response response = putValueJson(TABLE, ROW_1, COLUMN_1, VALUE_1); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGZIPResponseWrapper.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGZIPResponseWrapper.java index 18beb24df1c..42e38fc99a6 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGZIPResponseWrapper.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGZIPResponseWrapper.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,23 +25,27 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.IOException; - import javax.servlet.ServletOutputStream; import javax.servlet.http.HttpServletResponse; - -import org.apache.hadoop.hbase.testclassification.RestTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.rest.filter.GZIPResponseStream; import org.apache.hadoop.hbase.rest.filter.GZIPResponseWrapper; +import org.apache.hadoop.hbase.testclassification.RestTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestGZIPResponseWrapper { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGZIPResponseWrapper.class); + private final HttpServletResponse response = mock(HttpServletResponse.class); private final GZIPResponseWrapper wrapper = new GZIPResponseWrapper(response); - + /** * wrapper should set all headers except "content-length" */ @@ -67,7 +70,7 @@ public class TestGZIPResponseWrapper { wrapper.sendRedirect("location"); verify(response).sendRedirect("location"); - + wrapper.flushBuffer(); verify(response).flushBuffer(); } diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java index 094c937ffeb..529c4339b26 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java @@ -26,11 +26,9 @@ import java.io.StringWriter; import java.net.URLEncoder; import java.util.HashMap; import java.util.List; - import javax.xml.bind.JAXBException; - -import org.apache.http.Header; import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.rest.client.Response; import org.apache.hadoop.hbase.rest.model.CellModel; @@ -41,12 +39,18 @@ import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.http.Header; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, MediumTests.class}) public class TestGetAndPutResource extends RowResourceBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGetAndPutResource.class); + private static final MetricsAssertHelper METRICS_ASSERT = CompatibilityFactory.getInstance(MetricsAssertHelper.class); @@ -110,7 +114,7 @@ public class TestGetAndPutResource extends RowResourceBase { public void testSingleCellGetPutPB() throws IOException, JAXBException { Response response = getValuePB(TABLE, ROW_1, COLUMN_1); assertEquals(404, response.getCode()); - + response = putValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1); assertEquals(200, response.getCode()); checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1); @@ -523,7 +527,7 @@ public class TestGetAndPutResource extends RowResourceBase { response = deleteRow(TABLE, ROW_2); assertEquals(200, response.getCode()); } - + @Test public void testMetrics() throws IOException, JAXBException { final String path = "/" + TABLE + "/" + ROW_4 + "/" + COLUMN_1; @@ -550,7 +554,7 @@ public class TestGetAndPutResource extends RowResourceBase { METRICS_ASSERT.assertCounterGt("successfulDelete", 0l, RESTServlet.getInstance(conf, userProvider).getMetrics().getSource()); } - + @Test public void testMultiColumnGetXML() throws Exception { String path = "/" + TABLE + "/fakerow"; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java index c383adf5e31..32d330564d3 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -26,9 +25,7 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; - -import org.apache.http.Header; -import org.apache.http.message.BasicHeader; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -43,13 +40,21 @@ import org.apache.hadoop.hbase.rest.client.Response; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.http.Header; +import org.apache.http.message.BasicHeader; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, MediumTests.class}) public class TestGzipFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGzipFilter.class); + private static final TableName TABLE = TableName.valueOf("TestGzipFilter"); private static final String CFA = "a"; private static final String COLUMN_1 = CFA + ":1"; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java index c0dbd6b9f5f..928abfd75d0 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,17 +21,15 @@ import static org.junit.Assert.assertEquals; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider; - import java.io.IOException; import java.util.Collection; - import javax.ws.rs.core.MediaType; import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; import javax.xml.bind.Marshaller; import javax.xml.bind.Unmarshaller; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -52,6 +49,7 @@ import org.apache.http.Header; import org.apache.http.message.BasicHeader; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -61,6 +59,10 @@ import org.junit.runners.Parameterized; @RunWith(Parameterized.class) public class TestMultiRowResource { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiRowResource.class); + private static final TableName TABLE = TableName.valueOf("TestRowResource"); private static final String CFA = "a"; private static final String CFB = "b"; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java index 6d3e89d620a..e0beeab4909 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,6 +17,10 @@ */ package org.apache.hadoop.hbase.rest; +import static org.junit.Assert.*; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.StringWriter; @@ -26,12 +29,11 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; - import javax.ws.rs.core.MediaType; import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -48,20 +50,20 @@ import org.apache.hadoop.hbase.rest.model.TestNamespacesInstanceModel; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; - -import static org.junit.Assert.*; - import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider; - @Category({RestTests.class, MediumTests.class}) public class TestNamespacesInstanceResource { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNamespacesInstanceResource.class); + private static String NAMESPACE1 = "TestNamespacesInstanceResource1"; private static Map NAMESPACE1_PROPS = new HashMap<>(); private static String NAMESPACE2 = "TestNamespacesInstanceResource2"; @@ -131,7 +133,7 @@ public class TestNamespacesInstanceResource { checkNamespaceProperties(nd.getConfiguration(), testProps); } - private void checkNamespaceProperties(Map namespaceProps, + private void checkNamespaceProperties(Map namespaceProps, Map testProps){ assertTrue(namespaceProps.size() == testProps.size()); for(String key: testProps.keySet()){ @@ -176,7 +178,7 @@ public class TestNamespacesInstanceResource { // Create namespace via admin. NamespaceDescriptor.Builder nsBuilder = NamespaceDescriptor.create(nsName); - NamespaceDescriptor nsd = nsBuilder.build(); + NamespaceDescriptor nsd = nsBuilder.build(); nsd.setConfiguration("key1", "value1"); admin.createNamespace(nsd); @@ -415,7 +417,7 @@ public class TestNamespacesInstanceResource { // Check cannot post tables that already exist. response = client.post(namespacePath3, Constants.MIMETYPE_BINARY, new byte[]{}); assertEquals(403, response.getCode()); - response = client.post(namespacePath4, Constants.MIMETYPE_PROTOBUF, + response = client.post(namespacePath4, Constants.MIMETYPE_PROTOBUF, model4.createProtobufOutput()); assertEquals(403, response.getCode()); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java index 54f470cb983..5cda16caec1 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,13 +17,14 @@ */ package org.apache.hadoop.hbase.rest; +import static org.junit.Assert.*; + import java.io.ByteArrayInputStream; import java.io.IOException; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.client.Admin; @@ -36,16 +36,19 @@ import org.apache.hadoop.hbase.rest.model.TestNamespacesModel; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; - -import static org.junit.Assert.*; - import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, MediumTests.class}) public class TestNamespacesResource { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNamespacesResource.class); + private static String NAMESPACE1 = "TestNamespacesInstanceResource1"; private static String NAMESPACE2 = "TestNamespacesInstanceResource2"; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestResourceFilter.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestResourceFilter.java index 11d465f3373..ba0390d465d 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestResourceFilter.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestResourceFilter.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.rest; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.rest.client.Client; import org.apache.hadoop.hbase.rest.client.Cluster; @@ -27,12 +28,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RestTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, MediumTests.class}) public class TestResourceFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestResourceFilter.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final HBaseRESTTestingUtility REST_TEST_UTIL = new HBaseRESTTestingUtility(); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java index b3a5168efb0..da09473ced8 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest; import static org.junit.Assert.assertEquals; @@ -31,14 +29,13 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Random; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; import javax.xml.bind.Marshaller; import javax.xml.bind.Unmarshaller; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -62,6 +59,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.http.Header; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -69,6 +67,11 @@ import org.slf4j.LoggerFactory; @Category({RestTests.class, MediumTests.class}) public class TestScannerResource { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerResource.class); + private static final Logger LOG = LoggerFactory.getLogger(TestScannerResource.class); private static final TableName TABLE = TableName.valueOf("TestScannerResource"); private static final TableName TABLE_TO_BE_DISABLED = TableName.valueOf("ScannerResourceDisable"); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java index 18d83c80dc0..bf7b60c5d9a 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest; import static org.junit.Assert.assertEquals; @@ -29,13 +27,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; import java.util.List; - import javax.xml.bind.JAXBContext; import javax.xml.bind.Marshaller; import javax.xml.bind.Unmarshaller; - import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -74,6 +71,7 @@ import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -82,6 +80,10 @@ import org.slf4j.LoggerFactory; @Category({RestTests.class, MediumTests.class}) public class TestScannersWithFilters { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannersWithFilters.class); + private static final Logger LOG = LoggerFactory.getLogger(TestScannersWithFilters.class); private static final TableName TABLE = TableName.valueOf("TestScannersWithFilters"); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java index a6065315a42..4cbcdd8151f 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -27,14 +27,13 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Iterator; import java.util.List; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; import javax.xml.bind.Marshaller; import javax.xml.bind.Unmarshaller; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -66,11 +65,17 @@ import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, MediumTests.class}) public class TestScannersWithLabels { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannersWithLabels.class); + private static final TableName TABLE = TableName.valueOf("TestScannersWithLabels"); private static final String CFA = "a"; private static final String CFB = "b"; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java index d69958965dd..ab9ac12d317 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -26,11 +25,10 @@ import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.StringWriter; import java.util.Collection; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; @@ -49,6 +47,7 @@ import org.apache.http.message.BasicHeader; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -59,6 +58,11 @@ import org.slf4j.LoggerFactory; @Category({RestTests.class, MediumTests.class}) @RunWith(Parameterized.class) public class TestSchemaResource { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSchemaResource.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSchemaResource.class); private static String TABLE1 = "TestSchemaResource1"; @@ -72,7 +76,7 @@ public class TestSchemaResource { private static Configuration conf; private static TestTableSchemaModel testTableSchemaModel; private static Header extraHdr = null; - + private static boolean csrfEnabled = true; @Parameterized.Parameters diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java index e7d2f2b3751..43d586f812b 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -25,11 +24,10 @@ import static org.junit.Assert.assertTrue; import java.io.ByteArrayInputStream; import java.io.IOException; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -42,6 +40,7 @@ import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -49,6 +48,11 @@ import org.slf4j.LoggerFactory; @Category({RestTests.class, MediumTests.class}) public class TestStatusResource { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStatusResource.class); + private static final Logger LOG = LoggerFactory.getLogger(TestStatusResource.class); private static final byte[] META_REGION_NAME = Bytes.toBytes(TableName.META_TABLE_NAME + ",,1"); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java index d7251c08e17..5fa3072281d 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest; import static org.junit.Assert.assertEquals; @@ -28,11 +26,10 @@ import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Iterator; import java.util.List; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; - import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -60,6 +57,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.StringUtils; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -67,6 +65,11 @@ import org.slf4j.LoggerFactory; @Category({RestTests.class, MediumTests.class}) public class TestTableResource { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableResource.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableResource.class); private static TableName TABLE = TableName.valueOf("TestTableResource"); @@ -115,12 +118,12 @@ public class TestTableResource { } } Connection connection = TEST_UTIL.getConnection(); - + Table table = connection.getTable(TABLE); table.put(puts); table.close(); // get the initial layout (should just be one region) - + RegionLocator regionLocator = connection.getRegionLocator(TABLE); List m = regionLocator.getAllRegionLocations(); assertEquals(1, m.size()); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java index 1d1b5cd1ee5..766d0b9b178 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java @@ -27,7 +27,6 @@ import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonToken; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider; - import java.io.DataInputStream; import java.io.EOFException; import java.io.IOException; @@ -38,7 +37,6 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; - import javax.ws.rs.core.MediaType; import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; @@ -49,8 +47,8 @@ import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlRootElement; import javax.xml.parsers.SAXParserFactory; import javax.xml.stream.XMLStreamException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -70,6 +68,7 @@ import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -79,6 +78,11 @@ import org.xml.sax.XMLReader; @Category({RestTests.class, MediumTests.class}) public class TestTableScan { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableScan.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableScan.class); private static final TableName TABLE = TableName.valueOf("TestScanResource"); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestVersionResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestVersionResource.java index 0f8d7f22e82..bbd5e082f4c 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestVersionResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestVersionResource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -24,14 +23,12 @@ import static org.junit.Assert.assertTrue; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider; - import java.io.ByteArrayInputStream; import java.io.IOException; - import javax.ws.rs.core.MediaType; import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.rest.client.Client; import org.apache.hadoop.hbase.rest.client.Cluster; @@ -43,6 +40,7 @@ import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -50,10 +48,15 @@ import org.slf4j.LoggerFactory; @Category({RestTests.class, MediumTests.class}) public class TestVersionResource { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVersionResource.class); + private static final Logger LOG = LoggerFactory.getLogger(TestVersionResource.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static final HBaseRESTTestingUtility REST_TEST_UTIL = + private static final HBaseRESTTestingUtility REST_TEST_UTIL = new HBaseRESTTestingUtility(); private static Client client; private static JAXBContext context; @@ -62,7 +65,7 @@ public class TestVersionResource { public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(); REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration()); - client = new Client(new Cluster().add("localhost", + client = new Client(new Cluster().add("localhost", REST_TEST_UTIL.getServletPort())); context = JAXBContext.newInstance( VersionModel.class, @@ -79,7 +82,7 @@ public class TestVersionResource { assertNotNull(model); assertNotNull(model.getRESTVersion()); assertEquals(RESTServlet.VERSION_STRING, model.getRESTVersion()); - String osVersion = model.getOSVersion(); + String osVersion = model.getOSVersion(); assertNotNull(osVersion); assertTrue(osVersion.contains(System.getProperty("os.name"))); assertTrue(osVersion.contains(System.getProperty("os.version"))); @@ -168,7 +171,7 @@ public class TestVersionResource { Response response = client.get("/version/cluster",Constants.MIMETYPE_XML); assertEquals(200, response.getCode()); assertEquals(Constants.MIMETYPE_XML, response.getHeader("content-type")); - StorageClusterVersionModel clusterVersionModel = + StorageClusterVersionModel clusterVersionModel = (StorageClusterVersionModel) context.createUnmarshaller().unmarshal( new ByteArrayInputStream(response.getBody())); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java index 706402c0d1c..4d0359dd7f1 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -28,8 +28,8 @@ import static org.mockito.Mockito.when; import java.io.IOException; import java.util.regex.Pattern; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +47,10 @@ import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestRemoteAdminRetries { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRemoteAdminRetries.class); + private static final int SLEEP_TIME = 50; private static final int RETRIES = 3; private static final long MAX_TIME = SLEEP_TIME * (RETRIES - 1); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteHTableRetries.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteHTableRetries.java index b68df70d27b..fda5958f5c0 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteHTableRetries.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteHTableRetries.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -29,18 +29,19 @@ import static org.mockito.Mockito.when; import java.io.IOException; import java.util.Arrays; import java.util.regex.Pattern; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.testclassification.RestTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.testclassification.RestTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -50,6 +51,10 @@ import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestRemoteHTableRetries { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRemoteHTableRetries.class); + private static final int SLEEP_TIME = 50; private static final int RETRIES = 3; private static final long MAX_TIME = SLEEP_TIME * (RETRIES - 1); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java index 1fa17ca5ba8..c50f8022f67 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -29,11 +28,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; - -import org.apache.http.Header; -import org.apache.http.message.BasicHeader; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -50,16 +47,23 @@ import org.apache.hadoop.hbase.rest.HBaseRESTTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.http.Header; +import org.apache.http.message.BasicHeader; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, MediumTests.class}) public class TestRemoteTable { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRemoteTable.class); + // Verify that invalid URL characters and arbitrary bytes are escaped when // constructing REST URLs per HBASE-7621. RemoteHTable should support row keys // and qualifiers containing any byte for all table operations. @@ -96,7 +100,7 @@ public class TestRemoteTable { private static final long TS_1 = TS_2 - ONE_HOUR; private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static final HBaseRESTTestingUtility REST_TEST_UTIL = + private static final HBaseRESTTestingUtility REST_TEST_UTIL = new HBaseRESTTestingUtility(); private RemoteHTable remoteTable; @@ -129,16 +133,16 @@ public class TestRemoteTable { table.put(put); } remoteTable = new RemoteHTable( - new Client(new Cluster().add("localhost", + new Client(new Cluster().add("localhost", REST_TEST_UTIL.getServletPort())), TEST_UTIL.getConfiguration(), TABLE.toBytes()); } - + @After public void after() throws Exception { remoteTable.close(); } - + @AfterClass public static void tearDownAfterClass() throws Exception { REST_TEST_UTIL.shutdownServletContainer(); @@ -340,7 +344,7 @@ public class TestRemoteTable { value = result.getValue(COLUMN_2, QUALIFIER_2); assertNotNull(value); assertTrue(Bytes.equals(VALUE_2, value)); - + assertTrue(Bytes.equals(Bytes.toBytes("TestRemoteTable" + VALID_TABLE_NAME_CHARS), remoteTable.getTableName())); } @@ -402,9 +406,9 @@ public class TestRemoteTable { assertNull(value1); assertNull(value2); } - + /** - * Test RemoteHTable.Scanner + * Test RemoteHTable.Scanner */ @Test public void testScanner() throws IOException { @@ -443,7 +447,7 @@ public class TestRemoteTable { results = scanner.next(1); assertNull(results); scanner.close(); - + scanner = remoteTable.getScanner(COLUMN_1); results = scanner.next(4); assertNotNull(results); @@ -454,7 +458,7 @@ public class TestRemoteTable { assertTrue(Bytes.equals(ROW_4, results[3].getRow())); scanner.close(); - + scanner = remoteTable.getScanner(COLUMN_1,QUALIFIER_1); results = scanner.next(4); assertNotNull(results); @@ -467,7 +471,7 @@ public class TestRemoteTable { assertTrue(remoteTable.isAutoFlush()); } - + @Test public void testCheckAndDelete() throws IOException { Get get = new Get(ROW_1); @@ -494,9 +498,9 @@ public class TestRemoteTable { assertFalse(remoteTable.checkAndMutate(ROW_1, COLUMN_1).qualifier(QUALIFIER_1) .ifEquals(VALUE_2).thenPut(put)); } - + /** - * Test RemoteHable.Scanner.iterator method + * Test RemoteHable.Scanner.iterator method */ @Test public void testIteratorScaner() throws IOException { @@ -525,7 +529,7 @@ public class TestRemoteTable { } assertEquals(4, counter); } - + /** * Test a some methods of class Response. */ @@ -547,8 +551,8 @@ public class TestRemoteTable { response.setHeaders(headers); assertEquals("value1.1", response.getHeader("header1")); response.setBody(Bytes.toBytes("body")); - assertTrue(response.hasBody()); + assertTrue(response.hasBody()); } - + } diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java index e86039a388e..26190f66f47 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -24,13 +25,14 @@ import static org.mockito.Mockito.when; import java.io.IOException; import javax.xml.bind.UnmarshalException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.rest.Constants; import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.StringUtils; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -41,6 +43,11 @@ import org.slf4j.LoggerFactory; */ @Category(SmallTests.class) public class TestXmlParsing { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestXmlParsing.class); + private static final Logger LOG = LoggerFactory.getLogger(TestXmlParsing.class); @Test diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestCellModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestCellModel.java index d87450520c7..b8305d56a18 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestCellModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestCellModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; @@ -26,16 +24,21 @@ import static org.junit.Assert.assertTrue; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; - +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestCellModel extends TestModelBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellModel.class); + private static final long TIMESTAMP = 1245219839331L; private static final byte[] COLUMN = Bytes.toBytes("testcolumn"); private static final byte[] VALUE = Bytes.toBytes("testvalue"); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestCellSetModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestCellSetModel.java index b4d92900354..1d40effb47d 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestCellSetModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestCellSetModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; @@ -24,16 +22,20 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.Iterator; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; - +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestCellSetModel extends TestModelBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellSetModel.class); + private static final byte[] ROW1 = Bytes.toBytes("testrow1"); private static final byte[] COLUMN1 = Bytes.toBytes("testcolumn1"); private static final byte[] VALUE1 = Bytes.toBytes("testvalue1"); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestColumnSchemaModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestColumnSchemaModel.java index 8ddf5698a9c..a52358cbe52 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestColumnSchemaModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestColumnSchemaModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,20 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestColumnSchemaModel extends TestModelBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestColumnSchemaModel.class); + protected static final String COLUMN_NAME = "testcolumn"; protected static final boolean BLOCKCACHE = true; protected static final int BLOCKSIZE = 16384; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestModelBase.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestModelBase.java index 977f6c27230..d56efbc6467 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestModelBase.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestModelBase.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; @@ -23,24 +22,17 @@ import static org.junit.Assert.assertEquals; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider; - import java.io.IOException; import java.io.StringReader; import java.io.StringWriter; - import javax.ws.rs.core.MediaType; import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; - -import org.apache.hadoop.hbase.testclassification.RestTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.provider.JAXBContextResolver; import org.apache.hadoop.hbase.util.Base64; import org.junit.Test; -import org.junit.experimental.categories.Category; -@Category({RestTests.class, SmallTests.class}) public abstract class TestModelBase { protected String AS_XML; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java index 06948987786..e7a9188b5e3 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; @@ -24,15 +22,20 @@ import static org.junit.Assert.assertNotSame; import java.util.HashMap; import java.util.Map; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestNamespacesInstanceModel extends TestModelBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNamespacesInstanceModel.class); + public static final Map NAMESPACE_PROPERTIES = new HashMap<>(); public static final String NAMESPACE_NAME = "namespaceName"; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java index 9ff75517a48..5da776ab735 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; @@ -24,16 +22,20 @@ import static org.junit.Assert.assertTrue; import java.util.Arrays; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - @Category({RestTests.class, SmallTests.class}) public class TestNamespacesModel extends TestModelBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNamespacesModel.class); + public static final String NAMESPACE_NAME_1 = "testNamespace1"; public static final String NAMESPACE_NAME_2 = "testNamespace2"; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestRowModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestRowModel.java index 88593c5042d..99f8e3df447 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestRowModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestRowModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; @@ -24,20 +22,24 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.Iterator; - import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; - +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestRowModel extends TestModelBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowModel.class); + private static final byte[] ROW1 = Bytes.toBytes("testrow1"); private static final byte[] COLUMN1 = Bytes.toBytes("testcolumn1"); private static final byte[] VALUE1 = Bytes.toBytes("testvalue1"); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestScannerModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestScannerModel.java index 06c1c1ca094..85c55886f4f 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestScannerModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestScannerModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,19 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestScannerModel extends TestModelBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerModel.class); + private static final String PRIVATE = "private"; private static final String PUBLIC = "public"; private static final byte[] START_ROW = Bytes.toBytes("abracadabra"); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java index 58b538f7e7c..77ca7614b1f 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; @@ -24,17 +22,21 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.Iterator; - +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; - +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestStorageClusterStatusModel extends TestModelBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStorageClusterStatusModel.class); + public TestStorageClusterStatusModel() throws Exception { super(StorageClusterStatusModel.class); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterVersionModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterVersionModel.java index eb36fd5fe2b..b6101462aa0 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterVersionModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterVersionModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,17 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestStorageClusterVersionModel extends TestModelBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStorageClusterVersionModel.class); + private static final String VERSION = "0.0.1-testing"; public TestStorageClusterVersionModel() throws Exception { diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableInfoModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableInfoModel.java index c52967157ab..2ada01c5887 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableInfoModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableInfoModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; @@ -24,15 +22,20 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.Iterator; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; - +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestTableInfoModel extends TestModelBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableInfoModel.class); + private static final String TABLE = "testtable"; private static final byte[] START_KEY = Bytes.toBytes("abracadbra"); private static final byte[] END_KEY = Bytes.toBytes("zzyzx"); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableListModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableListModel.java index 0cf8915f133..eca14978c90 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableListModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableListModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,21 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import java.util.Iterator; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; - +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestTableListModel extends TestModelBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableListModel.class); + private static final String TABLE1 = "table1"; private static final String TABLE2 = "table2"; private static final String TABLE3 = "table3"; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java index ea49515e964..4285c9bfbde 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,23 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; - +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestTableRegionModel extends TestModelBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableRegionModel.class); + private static final String TABLE = "testtable"; private static final byte[] START_KEY = Bytes.toBytes("abracadbra"); private static final byte[] END_KEY = Bytes.toBytes("zzyzx"); @@ -67,7 +71,7 @@ public class TestTableRegionModel extends TestModelBase { assertTrue(Bytes.equals(model.getEndKey(), END_KEY)); assertEquals(ID, model.getId()); assertEquals(LOCATION, model.getLocation()); - assertEquals(model.getName(), + assertEquals(model.getName(), TABLE + "," + Bytes.toString(START_KEY) + "," + Long.toString(ID) + ".ad9860f031282c46ed431d7af8f94aca."); } diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableSchemaModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableSchemaModel.java index 55d149afbb5..6b50ab70048 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableSchemaModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableSchemaModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; @@ -24,10 +22,10 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.Iterator; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; - +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -35,6 +33,11 @@ import org.slf4j.LoggerFactory; @Category({RestTests.class, SmallTests.class}) public class TestTableSchemaModel extends TestModelBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableSchemaModel.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableSchemaModel.class); public static final String TABLE_NAME = "testTable"; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestVersionModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestVersionModel.java index c3e7a4ce0bd..b35295059cf 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestVersionModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestVersionModel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -20,20 +19,27 @@ package org.apache.hadoop.hbase.rest.model; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({RestTests.class, SmallTests.class}) public class TestVersionModel extends TestModelBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVersionModel.class); + private static final String REST_VERSION = "0.0.1"; - private static final String OS_VERSION = + private static final String OS_VERSION = "Linux 2.6.18-128.1.6.el5.centos.plusxen amd64"; private static final String JVM_VERSION = "Sun Microsystems Inc. 1.6.0_13-11.3-b02"; private static final String JETTY_VERSION = "6.1.14"; private static final String JERSEY_VERSION = "1.1.0-ea"; - + public TestVersionModel() throws Exception { super(VersionModel.class); AS_XML = diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java index f36199e522c..0ef0a0118c4 100644 --- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java +++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java @@ -34,9 +34,9 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.ServerName; @@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -63,6 +64,7 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -70,6 +72,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category(SmallTests.class) public class TestRSGroupBasedLoadBalancer { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRSGroupBasedLoadBalancer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRSGroupBasedLoadBalancer.class); private static RSGroupBasedLoadBalancer loadBalancer; private static SecureRandom rand; diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java index 6a333e41a11..9116f3bdfac 100644 --- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java +++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java @@ -23,7 +23,7 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.Iterator; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -47,17 +47,24 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; @Category({MediumTests.class}) public class TestRSGroups extends TestRSGroupsBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRSGroups.class); + protected static final Logger LOG = LoggerFactory.getLogger(TestRSGroups.class); private static HMaster master; private static boolean INIT = false; diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java index b8a4a752f92..4685c011093 100644 --- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java +++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.rsgroup; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCluster; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,6 +57,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets; // assignment with a timeout. @Category(MediumTests.class) public class TestRSGroupsOfflineMode { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRSGroupsOfflineMode.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRSGroupsOfflineMode.class); private static HMaster master; private static Admin hbaseAdmin; diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java index 196fc70a906..25bb4339c92 100644 --- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java +++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,14 +7,13 @@ * "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 + * 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. + * 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.rsgroup; @@ -24,6 +23,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -53,6 +54,11 @@ import org.slf4j.LoggerFactory; */ @Category({SecurityTests.class}) public class TestRSGroupsWithACL extends SecureTestUtil{ + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRSGroupsWithACL.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRSGroupsWithACL.class); private static TableName TEST_TABLE = TableName.valueOf("testtable1"); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithAdaptivePolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithAdaptivePolicy.java index 436624a6f3e..ff770815dd2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithAdaptivePolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithAdaptivePolicy.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,11 +18,16 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category(LargeTests.class) public class TestAcidGuaranteesWithAdaptivePolicy extends AcidGuaranteesTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAcidGuaranteesWithAdaptivePolicy.class); + @Override protected MemoryCompactionPolicy getMemoryCompactionPolicy() { return MemoryCompactionPolicy.ADAPTIVE; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithBasicPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithBasicPolicy.java index 53f5cba07f9..02c6a98a7a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithBasicPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithBasicPolicy.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,11 +18,16 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category(LargeTests.class) public class TestAcidGuaranteesWithBasicPolicy extends AcidGuaranteesTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAcidGuaranteesWithBasicPolicy.class); + @Override protected MemoryCompactionPolicy getMemoryCompactionPolicy() { return MemoryCompactionPolicy.BASIC; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithEagerPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithEagerPolicy.java index 9216457e778..5a9f2eebe8e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithEagerPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithEagerPolicy.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,11 +18,16 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category(LargeTests.class) public class TestAcidGuaranteesWithEagerPolicy extends AcidGuaranteesTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAcidGuaranteesWithEagerPolicy.class); + @Override protected MemoryCompactionPolicy getMemoryCompactionPolicy() { return MemoryCompactionPolicy.EAGER; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithNoInMemCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithNoInMemCompaction.java index 3ffc2c256da..74374b0b018 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithNoInMemCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuaranteesWithNoInMemCompaction.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,11 +18,16 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category(LargeTests.class) public class TestAcidGuaranteesWithNoInMemCompaction extends AcidGuaranteesTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAcidGuaranteesWithNoInMemCompaction.class); + protected MemoryCompactionPolicy getMemoryCompactionPolicy() { return MemoryCompactionPolicy.NONE; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java index 23ca57ff34e..18669812618 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java @@ -15,24 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertTrue; import java.util.List; - import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - /** * Checks tests are categorized. */ @Category({MiscTests.class, SmallTests.class}) public class TestCheckTestClasses { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCheckTestClasses.class); + /** * Throws an assertion if we find a test class without category (small/medium/large/integration). * List all the test classes without category in the assertion message. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java index b2688420d79..a2605f272a5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java @@ -43,11 +43,17 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestClientClusterMetrics { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientClusterMetrics.class); + private static HBaseTestingUtility UTIL; private static Admin ADMIN; private final static int SLAVES = 5; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java index d7e6f144f25..bfe28e4e404 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -48,6 +49,11 @@ import org.junit.experimental.categories.Category; */ @Category(SmallTests.class) public class TestClientClusterStatus { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientClusterStatus.class); + private static HBaseTestingUtility UTIL; private static Admin ADMIN; private final static int SLAVES = 5; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterBootOrder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterBootOrder.java index 4097efbc038..2d16a90ef1f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterBootOrder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterBootOrder.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertTrue; @@ -26,6 +25,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -35,6 +35,10 @@ import org.junit.experimental.categories.Category; @Category({MiscTests.class, MediumTests.class}) public class TestClusterBootOrder { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClusterBootOrder.class); + private static final long SLEEP_INTERVAL = 1000; private static final long SLEEP_TIME = 4000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java index 95997f2e2b6..c01fc45d5fd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,7 +21,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -30,6 +28,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FSTableDescriptors; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -37,6 +36,11 @@ import org.junit.rules.TestName; @Category({MiscTests.class, SmallTests.class}) public class TestFSTableDescriptorForceCreation { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSTableDescriptorForceCreation.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java index 6640a9846f6..da3bf2c3982 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; @@ -28,12 +26,17 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MiscTests.class, LargeTests.class }) public class TestFullLogReconstruction { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFullLogReconstruction.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java index a690987c4f5..f66a8287df1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java @@ -1,8 +1,7 @@ /** - * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional infomation + * 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 @@ -24,7 +23,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionLocator; @@ -36,12 +34,14 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; /** @@ -50,6 +50,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; */ @Category({MiscTests.class, MediumTests.class}) public class TestGlobalMemStoreSize { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGlobalMemStoreSize.class); + private static final Logger LOG = LoggerFactory.getLogger(TestGlobalMemStoreSize.class); private static int regionServerNum = 4; private static int regionNum = 16; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java index 55a1b5b9043..9234ea68010 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.hbase; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.UUID; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.Put; @@ -24,21 +28,22 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.util.UUID; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * Test that an HBase cluster can run on top of an existing MiniDfsCluster */ @Category(MediumTests.class) public class TestHBaseOnOtherDfsCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseOnOtherDfsCluster.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java index cf54b8f875c..74d04fc015c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,7 +17,6 @@ */ package org.apache.hadoop.hbase; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -34,7 +32,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; @@ -49,6 +46,7 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -64,6 +62,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, LargeTests.class}) public class TestHBaseTestingUtility { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseTestingUtility.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHBaseTestingUtility.class); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java index 47f0add5a75..e7a9b450bd0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,7 +20,6 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; import java.io.IOException; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -35,12 +33,12 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; - /** * Verify that the HColumnDescriptor version is set correctly by default, hbase-site.xml, and user * input @@ -48,6 +46,10 @@ import org.junit.rules.TestName; @Category({MiscTests.class, MediumTests.class}) public class TestHColumnDescriptorDefaultVersions { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHColumnDescriptorDefaultVersions.class); + @Rule public TestName name = new TestName(); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHDFSBlocksDistribution.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHDFSBlocksDistribution.java index f21d79d165e..e9ec333e31e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHDFSBlocksDistribution.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHDFSBlocksDistribution.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,18 +17,23 @@ */ package org.apache.hadoop.hbase; -import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import static junit.framework.Assert.assertEquals; import java.util.HashMap; import java.util.Map; - -import static junit.framework.Assert.assertEquals; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestHDFSBlocksDistribution { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHDFSBlocksDistribution.class); + @Test public void testAddHostsAndBlockWeight() throws Exception { HDFSBlocksDistribution distribution = new HDFSBlocksDistribution(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java index 6bbde239d92..cffe4b63b15 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,11 +24,17 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestHRegionLocation { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionLocation.class); + /** * HRegionLocations are equal if they have the same 'location' -- i.e. host and * port -- even if they are carrying different regions. Verify that is indeed diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java index 12103618e8e..2df484a5cb8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java @@ -25,7 +25,6 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -52,11 +51,14 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.wal.WAL; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; @@ -81,6 +83,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDes */ @Category({MiscTests.class, LargeTests.class}) public class TestIOFencing { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIOFencing.class); + private static final Logger LOG = LoggerFactory.getLogger(TestIOFencing.class); static { // Uncomment the following lines if more verbosity is needed for diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIPv6NIOServerSocketChannel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIPv6NIOServerSocketChannel.java index ceb9f562572..44c10efcab5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIPv6NIOServerSocketChannel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIPv6NIOServerSocketChannel.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import java.io.IOException; @@ -25,14 +24,12 @@ import java.net.InetSocketAddress; import java.net.ServerSocket; import java.nio.channels.ServerSocketChannel; import java.util.Locale; - import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,11 +46,13 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, SmallTests.class}) public class TestIPv6NIOServerSocketChannel { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIPv6NIOServerSocketChannel.class); + private static final Logger LOG = LoggerFactory.getLogger(TestIPv6NIOServerSocketChannel.class); - @Rule - public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); /** * Creates and binds a regular ServerSocket. */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java index f3d31154965..13ada17d096 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,7 +24,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.io.InputStream; import java.net.URL; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -33,6 +31,7 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +45,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, MediumTests.class}) public class TestInfoServers { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestInfoServers.class); + private static final Logger LOG = LoggerFactory.getLogger(TestInfoServers.class); private final static HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java index d09b0a41fb2..fee1439a70b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -19,11 +18,9 @@ package org.apache.hadoop.hbase; import java.io.IOException; - import javax.management.remote.JMXConnector; import javax.management.remote.JMXConnectorFactory; import javax.naming.ServiceUnavailableException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -39,6 +36,7 @@ import org.apache.hadoop.hbase.util.Threads; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -49,6 +47,11 @@ import org.slf4j.LoggerFactory; */ @Category({ MiscTests.class, MediumTests.class }) public class TestJMXConnectorServer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestJMXConnectorServer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestJMXConnectorServer.class); private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXListener.java index 4c7f129d795..8332180b829 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXListener.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,12 +17,12 @@ */ package org.apache.hadoop.hbase; -import java.io.IOException; +import static org.junit.Assert.fail; +import java.io.IOException; import javax.management.MBeanServerConnection; import javax.management.remote.JMXConnector; import javax.management.remote.JMXConnectorFactory; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -31,8 +30,8 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.AfterClass; import org.junit.Assert; -import static org.junit.Assert.fail; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,10 +39,13 @@ import org.junit.rules.ExpectedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - - @Category({MiscTests.class, MediumTests.class}) public class TestJMXListener { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestJMXListener.class); + private static final Logger LOG = LoggerFactory.getLogger(TestJMXListener.class); private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static int connectorPort = UTIL.randomFreePort(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java index 4c6b848a296..4403739623b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestLocalHBaseCluster.java @@ -20,18 +20,22 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.*; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.zookeeper.KeeperException; - +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, MediumTests.class}) public class TestLocalHBaseCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLocalHBaseCluster.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java index 729583ca78f..3831e9c5249 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -33,7 +32,6 @@ import static org.mockito.Mockito.verify; import java.io.IOException; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -60,12 +58,14 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -74,6 +74,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({MiscTests.class, MediumTests.class}) @SuppressWarnings("deprecation") public class TestMetaTableAccessor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetaTableAccessor.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableAccessor.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static Connection connection; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java index 620abefe975..686f281c450 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java @@ -26,22 +26,19 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; - import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -49,15 +46,25 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; + /** * Test MetaTableAccessor but without spinning up a cluster. * We mock regionserver back and forth (we do spin up a zk cluster). */ @Category({MiscTests.class, MediumTests.class}) public class TestMetaTableAccessorNoCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetaTableAccessorNoCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableAccessorNoCluster.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final Abortable ABORTABLE = new Abortable() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java index 63d2cc2b485..db93c0c526b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,13 +18,12 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; import java.io.IOException; import java.net.ConnectException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; @@ -33,11 +31,6 @@ import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.master.RegionState; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Threads; @@ -48,19 +41,32 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse; + /** * Test {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator} */ @Category({MiscTests.class, MediumTests.class}) public class TestMetaTableLocator { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetaTableLocator.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableLocator.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final ServerName SN = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMovedRegionsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMovedRegionsCleaner.java index 24a88304746..8932646ef9b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMovedRegionsCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMovedRegionsCleaner.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,26 +17,26 @@ */ package org.apache.hadoop.hbase; - +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; - /** * Test whether background cleanup of MovedRegion entries is happening */ -@Category({ MiscTests.class, MediumTests.class }) public class TestMovedRegionsCleaner { +@Category({ MiscTests.class, MediumTests.class }) +public class TestMovedRegionsCleaner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMovedRegionsCleaner.class); - private static final Logger LOG = LoggerFactory.getLogger(TestRegionRebalancing.class); private final HBaseTestingUtility UTIL = new HBaseTestingUtility(); public static int numCalls = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java index acf78610562..f3befd36996 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -27,7 +26,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; - import org.apache.hadoop.hbase.TimestampTestBase.FlushCache; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; @@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -56,10 +55,15 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, MediumTests.class}) public class TestMultiVersions { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiVersions.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMultiVersions.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private Admin admin; - + private static final int NUM_SLAVES = 3; @Rule @@ -85,7 +89,7 @@ public class TestMultiVersions { * Tests user specifiable time stamps putting, getting and scanning. Also * tests same in presence of deletes. Test cores are written so can be * run against an HRegion and against an HTable: i.e. both local and remote. - * + * *

Port of old TestTimestamp test to here so can better utilize the spun * up cluster running more than a single test per spin up. Keep old tests' * crazyness. @@ -186,7 +190,7 @@ public class TestMultiVersions { * Port of old TestScanMultipleVersions test here so can better utilize the * spun up cluster running more than just a single test. Keep old tests * crazyness. - * + * *

Tests five cases of scans and timestamps. * @throws Exception */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java index b9c4e5c6e0b..f1c42cee52c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -27,7 +26,6 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.Set; import java.util.concurrent.Callable; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.Admin; @@ -42,17 +40,23 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; + @Category({MiscTests.class, MediumTests.class}) public class TestNamespace { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNamespace.class); + private static final Logger LOG = LoggerFactory.getLogger(TestNamespace.class); private static HMaster master; protected final static int NUM_SLAVES_BASE = 4; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNodeHealthCheckChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNodeHealthCheckChore.java index 22daf490934..15f04114076 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNodeHealthCheckChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNodeHealthCheckChore.java @@ -26,7 +26,6 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.PrintWriter; import java.util.UUID; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -35,6 +34,7 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.util.Shell; import org.junit.After; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -43,6 +43,10 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) public class TestNodeHealthCheckChore { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNodeHealthCheckChore.class); + private static final Logger LOG = LoggerFactory.getLogger(TestNodeHealthCheckChore.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final int SCRIPT_TIMEOUT = 5000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java index 12f43d48678..3038744de0e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.LinkedHashSet; import java.util.List; import java.util.Set; - import org.apache.hadoop.hbase.client.ClientScanner; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Put; @@ -50,6 +49,7 @@ import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -69,6 +69,11 @@ import org.slf4j.LoggerFactory; */ @Category(MediumTests.class) public class TestPartialResultsFromClientSide { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPartialResultsFromClientSide.class); + private static final Logger LOG = LoggerFactory.getLogger(TestPartialResultsFromClientSide.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java index 5eace2b6103..801d2d8b5e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -39,16 +37,22 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.collect.Maps; + @Category({MiscTests.class, MediumTests.class}) public class TestRegionLoad { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionLoad.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionLoad.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static Admin admin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java index 34196e27e9a..1f9c5199d87 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -36,6 +34,7 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -44,6 +43,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Maps; @Category({ MiscTests.class, MediumTests.class }) public class TestRegionMetrics { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionMetrics.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static Admin admin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java index 5f2898fadd9..222421961f5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -26,7 +26,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; - import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -47,6 +47,7 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; /** @@ -58,6 +59,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @RunWith(value = Parameterized.class) public class TestRegionRebalancing { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionRebalancing.class); + @Parameters public static Collection data() { Object[][] balancers = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java index f60c1e5c85f..9f0cf6ba18a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,7 +17,6 @@ */ package org.apache.hadoop.hbase; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -33,7 +31,6 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; import java.util.Set; - import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.BinaryComparator; @@ -41,20 +38,27 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.PrefixFilter; import org.apache.hadoop.hbase.filter.RowFilter; import org.apache.hadoop.hbase.io.TimeRange; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.DataInputBuffer; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; + /** * Test HBase Writables serializations */ @Category({MiscTests.class, SmallTests.class}) public class TestSerialization { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSerialization.class); + @Test public void testKeyValue() throws Exception { final String name = "testKeyValue2"; byte[] row = name.getBytes(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java index 49b678f49e8..61db6b89ed5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -17,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; @@ -26,16 +23,22 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; @Category({ MiscTests.class, SmallTests.class }) public class TestServerLoad { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerLoad.class); + @Test public void testRegionLoadAggregation() { ServerLoad sl = new ServerLoad(ServerName.valueOf("localhost,1,1"), createServerLoadProto()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java index da39289618b..b3c903c7730 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -17,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; @@ -25,16 +22,22 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; @Category({ MiscTests.class, SmallTests.class }) public class TestServerMetrics { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerMetrics.class); + @Test public void testRegionLoadAggregation() { ServerMetrics metrics = ServerMetricsBuilder.toServerMetrics( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java index cfc5c2f1ae4..a6e1401038f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java @@ -23,16 +23,21 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import java.util.regex.Pattern; - import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Addressing; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestServerName { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerName.class); + @Test public void testGetHostNameMinusDomain() { assertEquals("2607:f0d0:1002:51::4", diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java index 4aeedb9d688..162be3575de 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java @@ -1,12 +1,19 @@ /** - * 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. + * 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; @@ -16,7 +23,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -38,11 +44,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(MediumTests.class) public class TestServerSideScanMetricsFromClientSide { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerSideScanMetricsFromClientSide.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Table TABLE = null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSize.java index 30d88bd6f5b..ecb317f5381 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSize.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -17,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; @@ -26,12 +23,17 @@ import java.util.Set; import java.util.TreeSet; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestSize { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSize.class); + @Test public void testConversion() { Size kbSize = new Size(1024D, Size.Unit.MEGABYTE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java index c5e64641440..c418b5f2826 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase; import static org.junit.Assert.assertTrue; @@ -46,6 +45,7 @@ import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.net.DNSToSwitchMapping; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.FixMethodOrder; import org.junit.Ignore; import org.junit.Test; @@ -58,6 +58,11 @@ import org.slf4j.LoggerFactory; @FixMethodOrder(MethodSorters.NAME_ASCENDING) @Ignore public class TestStochasticBalancerJmxMetrics extends BalancerTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticBalancerJmxMetrics.class); + private static final Logger LOG = LoggerFactory.getLogger(TestStochasticBalancerJmxMetrics.class); private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static int connectorPort = 61120; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestTagRewriteCell.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestTagRewriteCell.java index 686c93408ce..dc47661166d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestTagRewriteCell.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestTagRewriteCell.java @@ -1,10 +1,11 @@ -/* - * 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 +/** + * 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 * @@ -21,12 +22,17 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestTagRewriteCell { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTagRewriteCell.class); + @Test public void testHeapSize() { Cell originalCell = CellUtil.createCell(Bytes.toBytes("row"), Bytes.toBytes("value")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java index 0bdcb3e7624..c1735a7b024 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -49,6 +48,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,6 +58,11 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, LargeTests.class}) public class TestZooKeeper { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZooKeeper.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZooKeeper.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java index 95de9bcc677..578724fef0c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java @@ -25,13 +25,13 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.regionserver.CompactingMemStore; @@ -50,6 +50,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -64,6 +65,10 @@ import org.slf4j.LoggerFactory; @Category({MediumTests.class, MiscTests.class}) public class TestHFileArchiving { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileArchiving.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHFileArchiving.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final byte[] TEST_FAM = Bytes.toBytes("fam"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java index 4eb1ae4973b..e29e16884ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java @@ -26,12 +26,12 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ChoreService; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Stoppable; @@ -58,6 +58,7 @@ import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -73,6 +74,10 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, MediumTests.class}) public class TestZooKeeperTableArchiveClient { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZooKeeperTableArchiveClient.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZooKeeperTableArchiveClient.class); private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU(); private static final String STRING_TABLE_NAME = "test"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java index f81a36d5997..0e8a3e69460 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -32,7 +31,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -59,12 +58,14 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest; @@ -75,6 +76,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTable */ @Category({LargeTests.class, ClientTests.class}) public class TestAdmin1 { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAdmin1.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAdmin1.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private Admin admin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 57bd1584483..fee7f58a6c4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -33,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics.Option; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -61,6 +61,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -70,7 +71,6 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; - /** * Class to test HBaseAdmin. * Spins up the minicluster once at test start and then takes it down afterward. @@ -78,6 +78,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; */ @Category({LargeTests.class, ClientTests.class}) public class TestAdmin2 { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAdmin2.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAdmin2.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private Admin admin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAllowPartialScanResultCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAllowPartialScanResultCache.java index 3fe43a5a343..14123b31f5b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAllowPartialScanResultCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAllowPartialScanResultCache.java @@ -23,19 +23,24 @@ import static org.junit.Assert.assertSame; import java.io.IOException; import java.util.Arrays; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ SmallTests.class, ClientTests.class }) public class TestAllowPartialScanResultCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAllowPartialScanResultCache.class); + private static byte[] CF = Bytes.toBytes("cf"); private AllowPartialScanResultCache resultCache; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java index 5e38f161752..2ede1ddc3fd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java @@ -21,29 +21,36 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; + /** * Testcase to make sure that we always set scanner id in ScanResponse. See HBASE-18000. */ @Category({ RegionServerTests.class, MediumTests.class }) public class TestAlwaysSetScannerId { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAlwaysSetScannerId.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName TABLE_NAME = TableName.valueOf("test"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAppendFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAppendFromClientSide.java index ac1ac035db5..186b51f1e1f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAppendFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAppendFromClientSide.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,8 +17,13 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; @@ -27,21 +31,22 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.IOException; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; - /** * Run Append tests that use the HBase clients; */ @Category(LargeTests.class) public class TestAppendFromClientSide { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAppendFromClientSide.class); + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte [] ROW = Bytes.toBytes("testRow"); private static byte [] FAMILY = Bytes.toBytes("testFamily"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java index 8b3b18166e0..b1e527aba06 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.apache.hadoop.hbase.NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR; +import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.junit.Assert.fail; import java.io.IOException; @@ -29,8 +29,8 @@ import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; - import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Threads; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -56,6 +57,10 @@ import org.slf4j.LoggerFactory; @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncAdminBuilder { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncAdminBuilder.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAsyncAdminBuilder.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static AsyncConnection ASYNC_CONN; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java index 6a5a00e7947..9fe4ca74f16 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java @@ -33,7 +33,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import java.util.stream.IntStream; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -41,12 +41,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncBufferMutator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncBufferMutator.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java index 5b33ad0906f..e73b73fe97d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java @@ -34,6 +34,7 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics.Option; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.RegionMetrics; @@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -59,6 +61,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Maps; @Category({ ClientTests.class, MediumTests.class }) public class TestAsyncClusterAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncClusterAdminApi.class); + private final Path cnfPath = FileSystems.getDefault().getPath("target/test-classes/hbase-site.xml"); private final Path cnf2Path = FileSystems.getDefault().getPath("target/test-classes/hbase-site2.xml"); private final Path cnf3Path = FileSystems.getDefault().getPath("target/test-classes/hbase-site3.xml"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi2.java index 3344c4b470d..94528ea6628 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi2.java @@ -22,8 +22,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.concurrent.TimeUnit; - import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -33,6 +33,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -42,6 +43,10 @@ import org.junit.experimental.categories.Category; @Category({ ClientTests.class, MediumTests.class }) public class TestAsyncClusterAdminApi2 extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncClusterAdminApi2.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 60000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java index 8968b394c43..9321ced158b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java @@ -26,9 +26,11 @@ import java.util.HashMap; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.hbase.ClusterMetrics.Option; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -38,6 +40,10 @@ import org.junit.runners.Parameterized; @Category({ ClientTests.class, MediumTests.class }) public class TestAsyncDecommissionAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncDecommissionAdminApi.class); + @Test(timeout = 30000) public void testAsyncDecommissionRegionServers() throws Exception { List decommissionedRegionServers = admin.listDecommissionedRegionServers().get(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java index 53148bb4eb6..7c08d6db736 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java @@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; import java.util.Optional; - import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; @@ -33,12 +33,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncMetaRegionLocator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncMetaRegionLocator.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static AsyncRegistry REGISTRY; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java index ada8824ab8d..ed692cb4e44 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java @@ -25,7 +25,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.util.concurrent.Callable; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -52,6 +53,10 @@ import org.junit.runners.Parameterized; @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncNamespaceAdminApi.class); + private String prefix = "TestNamespace"; private static HMaster master; private static ZKNamespaceManager zkNamespaceManager; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index c70ad2669bd..5f229c2ceaa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -33,8 +33,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.IntStream; - import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -49,13 +50,17 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncNonMetaRegionLocator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocator.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java index 88a06298b7a..f31fbfa8cb5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java @@ -32,9 +32,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.IntStream; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; @@ -50,12 +50,17 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncNonMetaRegionLocatorConcurrenyLimit { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocatorConcurrenyLimit.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java index d50e0391bc3..83845186021 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; @@ -26,7 +25,7 @@ import static org.junit.Assert.assertTrue; import java.util.HashMap; import java.util.Map; import java.util.Random; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.procedure.ProcedureManagerHost; @@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -49,6 +49,10 @@ import org.junit.runners.Parameterized; @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncProcedureAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncProcedureAdminApi.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 60000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java index a3b160fb202..fc8a0ca5ad7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; @@ -24,7 +23,7 @@ import static org.junit.Assert.fail; import java.util.Objects; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.quotas.QuotaCache; @@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -47,6 +47,10 @@ import org.junit.runners.Parameterized; @Category({ ClientTests.class, MediumTests.class }) public class TestAsyncQuotaAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncQuotaAdminApi.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java index 91f797145a1..b9afb6826f3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java @@ -31,8 +31,8 @@ import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; - import org.apache.hadoop.hbase.AsyncMetaTableAccessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; @@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -64,6 +65,10 @@ import org.junit.runners.Parameterized; @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncRegionAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncRegionAdminApi.class); + @Test public void testGetRegionLocation() throws Exception { RawAsyncHBaseAdmin rawAdmin = (RawAsyncHBaseAdmin) ASYNC_CONN.getAdmin(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java index f241aceaf2d..758aa303fca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java @@ -30,9 +30,9 @@ import java.io.IOException; import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; @@ -48,12 +48,17 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncRegionLocatorTimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncRegionLocatorTimeout.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java index 28a7562d054..9f3740ff15b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java @@ -32,7 +32,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletionException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.replication.ReplicationException; @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.After; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -54,6 +55,10 @@ import org.junit.runners.Parameterized; @Category({LargeTests.class, ClientTests.class}) public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncReplicationAdminApi.class); + private final String ID_ONE = "1"; private final String KEY_ONE = "127.0.0.1:2181:/hbase"; private final String ID_SECOND = "2"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java index efea20e7718..0dc2b38209d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java @@ -29,8 +29,8 @@ import java.util.Map; import java.util.concurrent.CompletionException; import java.util.concurrent.ForkJoinPool; import java.util.regex.Pattern; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -54,6 +55,10 @@ import org.junit.runners.Parameterized; @Category({LargeTests.class, ClientTests.class}) public class TestAsyncReplicationAdminApiWithClusters extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncReplicationAdminApiWithClusters.class); + private final static String ID_SECOND = "2"; private static HBaseTestingUtility TEST_UTIL2; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java index 5aebb4aabbe..638ae8cef02 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java @@ -18,15 +18,20 @@ package org.apache.hadoop.hbase.client; import java.util.concurrent.ForkJoinPool; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncResultScannerCursor extends AbstractTestResultScannerCursor { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncResultScannerCursor.class); + private static AsyncConnection CONN; @BeforeClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java index 7ea69e51ddf..a6c2efb8afb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java @@ -29,8 +29,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; @@ -40,12 +40,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncSingleRequestRpcRetryingCaller { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncSingleRequestRpcRetryingCaller.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSnapshotAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSnapshotAdminApi.java index 7501192d8ba..db72dd2a233 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSnapshotAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSnapshotAdminApi.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertArrayEquals; @@ -27,11 +26,13 @@ import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.regex.Pattern; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -41,6 +42,10 @@ import org.junit.runners.Parameterized; @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncSnapshotAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncSnapshotAdminApi.class); + private static final Pattern MATCH_ALL = Pattern.compile(".*"); String snapshotName1 = "snapshotName1"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTable.java index cecae4782e3..37182ecd290 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTable.java @@ -36,8 +36,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; import java.util.stream.IntStream; - import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -60,6 +61,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncTable { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTable.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java index 8b2dce3904a..597a3d7fbe5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java @@ -34,9 +34,9 @@ import java.util.Set; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.regex.Pattern; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.AsyncMetaTableAccessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -62,6 +63,10 @@ import org.junit.runners.Parameterized; @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncTableAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableAdminApi.class); + @Test public void testTableExist() throws Exception { boolean exist; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java index 7848251cf29..5c76381b679 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java @@ -39,8 +39,8 @@ import java.util.concurrent.TimeoutException; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -54,6 +54,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -65,6 +66,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncTableBatch { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableBatch.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java index 12a976e124a..7632716bac8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java @@ -35,8 +35,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.IntStream; - import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MemoryCompactionPolicy; import org.apache.hadoop.hbase.ServerName; @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,6 +59,11 @@ import org.junit.experimental.categories.Category; */ @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncTableGetMultiThreaded { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableGetMultiThreaded.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithBasicCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithBasicCompaction.java index e32fd1c62c8..9325f4df62d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithBasicCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithBasicCompaction.java @@ -17,10 +17,12 @@ */ package org.apache.hadoop.hbase.client; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.MemoryCompactionPolicy; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.experimental.categories.Category; @@ -29,6 +31,10 @@ import org.junit.experimental.categories.Category; public class TestAsyncTableGetMultiThreadedWithBasicCompaction extends TestAsyncTableGetMultiThreaded { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableGetMultiThreadedWithBasicCompaction.class); + @BeforeClass public static void setUp() throws Exception { setUp(MemoryCompactionPolicy.BASIC); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithEagerCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithEagerCompaction.java index 4de4d078f43..992808b73ef 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithEagerCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithEagerCompaction.java @@ -17,10 +17,12 @@ */ package org.apache.hadoop.hbase.client; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.MemoryCompactionPolicy; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.experimental.categories.Category; @@ -29,6 +31,10 @@ import org.junit.experimental.categories.Category; public class TestAsyncTableGetMultiThreadedWithEagerCompaction extends TestAsyncTableGetMultiThreaded { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableGetMultiThreadedWithEagerCompaction.class); + @BeforeClass public static void setUp() throws Exception { setUp(MemoryCompactionPolicy.EAGER); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java index 80e3e4cb390..30085616f8a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java @@ -22,8 +22,8 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.concurrent.ExecutionException; - import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,6 +41,11 @@ import org.junit.rules.TestName; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncTableNoncedRetry { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableNoncedRetry.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScan.java index ebe3a9d20b2..42d2c38376d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScan.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScan.java @@ -20,9 +20,10 @@ package org.apache.hadoop.hbase.client; import java.util.List; import java.util.concurrent.ForkJoinPool; import java.util.function.Supplier; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -33,6 +34,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncTableScan extends AbstractTestAsyncTableScan { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableScan.class); + @Parameter(0) public String scanType; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java index 393ded73e15..b8adbceb50c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java @@ -19,9 +19,10 @@ package org.apache.hadoop.hbase.client; import java.util.List; import java.util.function.Supplier; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -32,6 +33,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncTableScanAll extends AbstractTestAsyncTableScan { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableScanAll.class); + @Parameter(0) public String tableType; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanMetrics.java index 84537a70ce8..7e9f5d9270e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanMetrics.java @@ -25,8 +25,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.ForkJoinPool; - import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -48,6 +49,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncTableScanMetrics { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableScanMetrics.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName TABLE_NAME = TableName.valueOf("ScanMetrics"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanRenewLease.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanRenewLease.java index 1996547bcf6..7cbf89397a9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanRenewLease.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanRenewLease.java @@ -23,7 +23,7 @@ import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -33,12 +33,17 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncTableScanRenewLease { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableScanRenewLease.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanner.java index bae2dbfa254..f832cfd759a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanner.java @@ -21,9 +21,10 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.ForkJoinPool; import java.util.function.Supplier; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -34,6 +35,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ LargeTests.class, ClientTests.class }) public class TestAsyncTableScanner extends AbstractTestAsyncTableScan { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableScanner.class); + @Parameter(0) public String tableType; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScannerCloseWhileSuspending.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScannerCloseWhileSuspending.java index 0e189559d5e..4ac0856ae1f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScannerCloseWhileSuspending.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScannerCloseWhileSuspending.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals; import java.util.concurrent.ForkJoinPool; import java.util.stream.Collectors; import java.util.stream.IntStream; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; @@ -31,12 +31,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncTableScannerCloseWhileSuspending { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncTableScannerCloseWhileSuspending.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncToolAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncToolAdminApi.java index 18d951fd230..bc78aaa0a62 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncToolAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncToolAdminApi.java @@ -19,8 +19,10 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -33,6 +35,10 @@ import org.junit.runners.Parameterized; @Category({ MediumTests.class, ClientTests.class }) public class TestAsyncToolAdminApi extends TestAsyncAdminBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncToolAdminApi.class); + @Test public void testBalancer() throws Exception { boolean initialState = admin.isBalancerEnabled().get(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java index ba61ab46880..0e12ad652d5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java @@ -26,10 +26,10 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -66,6 +67,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; @Category({ LargeTests.class, ClientTests.class }) public class TestAvoidCellReferencesIntoShippedBlocks { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAvoidCellReferencesIntoShippedBlocks.class); + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); static byte[][] ROWS = new byte[2][]; private static byte[] ROW = Bytes.toBytes("testRow"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBatchScanResultCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBatchScanResultCache.java index 31a4594d355..ab81ca01fc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBatchScanResultCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBatchScanResultCache.java @@ -22,20 +22,25 @@ import static org.junit.Assert.assertSame; import java.io.IOException; import java.util.Arrays; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ SmallTests.class, ClientTests.class }) public class TestBatchScanResultCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBatchScanResultCache.class); + private static byte[] CF = Bytes.toBytes("cf"); private BatchScanResultCache resultCache; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java index 7d95a712e6b..fc4c1f92392 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java @@ -33,9 +33,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; @@ -64,6 +64,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -74,6 +75,11 @@ import org.slf4j.LoggerFactory; @Category({ LargeTests.class, ClientTests.class }) @SuppressWarnings("deprecation") public class TestBlockEvictionFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBlockEvictionFromClient.class); + private static final Logger LOG = LoggerFactory.getLogger(TestBlockEvictionFromClient.class); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); static byte[][] ROWS = new byte[2][]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIBadHostname.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIBadHostname.java index 27bcee30ed2..83d4bfaf4bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIBadHostname.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIBadHostname.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.fail; +import java.net.UnknownHostException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; @@ -26,17 +28,21 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.net.UnknownHostException; - /** * Tests that we fail fast when hostname resolution is not working and do not cache * unresolved InetSocketAddresses. */ @Category({MediumTests.class, ClientTests.class}) public class TestCIBadHostname { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCIBadHostname.class); + private static HBaseTestingUtility testUtil; private static ConnectionImplementation conn; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIDeleteOperationTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIDeleteOperationTimeout.java index 8ffca1e956d..c4f6e3bda19 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIDeleteOperationTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIDeleteOperationTimeout.java @@ -18,13 +18,19 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, LargeTests.class }) public class TestCIDeleteOperationTimeout extends AbstractTestCIOperationTimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCIDeleteOperationTimeout.class); + @Override protected void execute(Table table) throws IOException { table.delete(new Delete(FAM_NAM)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIDeleteRpcTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIDeleteRpcTimeout.java index b999db834f7..32c059445f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIDeleteRpcTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIDeleteRpcTimeout.java @@ -18,13 +18,19 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, MediumTests.class }) public class TestCIDeleteRpcTimeout extends AbstractTestCIRpcTimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCIDeleteRpcTimeout.class); + @Override protected void execute(Table table) throws IOException { table.delete(new Delete(FAM_NAM)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIGetOperationTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIGetOperationTimeout.java index b887adeaf11..1fca99f4f35 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIGetOperationTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIGetOperationTimeout.java @@ -18,13 +18,19 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, LargeTests.class }) public class TestCIGetOperationTimeout extends AbstractTestCIOperationTimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCIGetOperationTimeout.class); + @Override protected void execute(Table table) throws IOException { table.get(new Get(FAM_NAM)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIGetRpcTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIGetRpcTimeout.java index 4b1ee62fad0..fb8482a09c5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIGetRpcTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIGetRpcTimeout.java @@ -18,13 +18,19 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, MediumTests.class }) public class TestCIGetRpcTimeout extends AbstractTestCIRpcTimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCIGetRpcTimeout.class); + @Override protected void execute(Table table) throws IOException { table.get(new Get(FAM_NAM)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIIncrementRpcTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIIncrementRpcTimeout.java index 2885d816a61..87b14817181 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIIncrementRpcTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIIncrementRpcTimeout.java @@ -18,13 +18,19 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, MediumTests.class }) public class TestCIIncrementRpcTimeout extends AbstractTestCIRpcTimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCIIncrementRpcTimeout.class); + @Override protected void execute(Table table) throws IOException { table.increment(new Increment(FAM_NAM).addColumn(FAM_NAM, FAM_NAM, 1)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIPutOperationTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIPutOperationTimeout.java index 3a172874c4d..8c36bf6dbec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIPutOperationTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIPutOperationTimeout.java @@ -18,13 +18,19 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, LargeTests.class }) public class TestCIPutOperationTimeout extends AbstractTestCIOperationTimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCIPutOperationTimeout.class); + @Override protected void execute(Table table) throws IOException { table.put(new Put(FAM_NAM).addColumn(FAM_NAM, FAM_NAM, FAM_NAM)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIPutRpcTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIPutRpcTimeout.java index b25ed5a220b..f5921fb05e9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIPutRpcTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIPutRpcTimeout.java @@ -18,13 +18,19 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, MediumTests.class }) public class TestCIPutRpcTimeout extends AbstractTestCIRpcTimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCIPutRpcTimeout.class); + @Override protected void execute(Table table) throws IOException { table.put(new Put(FAM_NAM).addColumn(FAM_NAM, FAM_NAM, FAM_NAM)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java index 5b3af33ed93..761922acf7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java @@ -22,6 +22,7 @@ import static org.junit.Assert.fail; import java.net.SocketTimeoutException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -29,6 +30,7 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -37,6 +39,10 @@ import org.slf4j.LoggerFactory; @Category({ ClientTests.class, MediumTests.class }) public class TestCISleep extends AbstractTestCITimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCISleep.class); + private static Logger LOG = LoggerFactory.getLogger(TestCISleep.class); private TableName tableName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java index d990a4a9bba..15ef065d473 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,6 +17,11 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; @@ -25,18 +29,19 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.IOException; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - @Category(MediumTests.class) public class TestCheckAndMutate { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCheckAndMutate.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] ROWKEY = Bytes.toBytes("12345"); private static final byte[] FAMILY = Bytes.toBytes("cf"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java index cbead73d4fe..bc9b715ed1c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,9 +17,16 @@ */ package org.apache.hadoop.hbase.client; - +import java.io.IOException; +import java.io.InterruptedIOException; +import java.net.SocketTimeoutException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; @@ -36,21 +42,19 @@ import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.net.SocketTimeoutException; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicInteger; - @Category({MediumTests.class, ClientTests.class}) public class TestClientOperationInterrupt { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientOperationInterrupt.class); + private static final Logger LOG = LoggerFactory.getLogger(TestClientOperationInterrupt.class); private static HBaseTestingUtility util; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java index 9103c515cc9..456434f91a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java @@ -17,15 +17,21 @@ */ package org.apache.hadoop.hbase.client; +import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ExponentialClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ServerStatistics; @@ -39,23 +45,22 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - /** * Test that we can actually send and use region metrics to slowdown client writes */ @Category(MediumTests.class) public class TestClientPushback { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientPushback.class); + private static final Logger LOG = LoggerFactory.getLogger(TestClientPushback.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java index 00ebc399b26..888b3d29e77 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java @@ -20,14 +20,12 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -35,22 +33,31 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; + /** * Test the scenario where a HRegionServer#scan() call, while scanning, timeout at client side and * getting retried. This scenario should not result in some data being skipped at RS side. */ @Category({MediumTests.class, ClientTests.class}) public class TestClientScannerRPCTimeout { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientScannerRPCTimeout.class); + private static final Logger LOG = LoggerFactory.getLogger(TestClientScannerRPCTimeout.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] FAMILY = Bytes.toBytes("testFamily"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index cbd1318852f..0488b386a47 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -28,21 +26,22 @@ import java.net.SocketTimeoutException; import java.net.UnknownHostException; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ipc.AbstractRpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.ipc.BlockingRpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -54,6 +53,11 @@ import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; @Category({MediumTests.class, ClientTests.class}) public class TestClientTimeouts { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClientTimeouts.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); protected static int SLAVES = 1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java index 96a123b65c1..f3dd8077459 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -34,6 +34,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +47,11 @@ import org.slf4j.LoggerFactory; */ @Category({LargeTests.class, ClientTests.class}) public class TestCloneSnapshotFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCloneSnapshotFromClient.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCloneSnapshotFromClient.class); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientWithRegionReplicas.java index 5c2eca92087..b67c2883cc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientWithRegionReplicas.java @@ -17,13 +17,20 @@ */ package org.apache.hadoop.hbase.client; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({LargeTests.class, ClientTests.class}) public class TestCloneSnapshotFromClientWithRegionReplicas extends TestCloneSnapshotFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCloneSnapshotFromClientWithRegionReplicas.class); + @Override protected int getNumReplicas() { return 3; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCompleteResultScanResultCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCompleteResultScanResultCache.java index 8759593a5f0..e6b31cb9eaa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCompleteResultScanResultCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCompleteResultScanResultCache.java @@ -22,20 +22,25 @@ import static org.junit.Assert.assertSame; import java.io.IOException; import java.util.Arrays; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ SmallTests.class, ClientTests.class }) public class TestCompleteResultScanResultCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompleteResultScanResultCache.class); + private static byte[] CF = Bytes.toBytes("cf"); private static byte[] CQ1 = Bytes.toBytes("cq1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java index f061fc98cfa..521317d404e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -38,8 +38,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -67,12 +67,12 @@ import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,9 +83,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; */ @Category({LargeTests.class}) public class TestConnectionImplementation { - @Rule - public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()) - .withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConnectionImplementation.class); + private static final Logger LOG = LoggerFactory.getLogger(TestConnectionImplementation.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final TableName TABLE_NAME = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionUtils.java index d34f9fa41c3..1e2dfd46d67 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionUtils.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -19,20 +17,25 @@ */ package org.apache.hadoop.hbase.client; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import static org.junit.Assert.assertTrue; import java.util.Set; import java.util.TreeSet; - -import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; @Category({SmallTests.class, ClientTests.class}) public class TestConnectionUtils { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConnectionUtils.class); + @Test public void testRetryTimeJitter() { long[] retries = new long[200]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestDropTimeoutRequest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestDropTimeoutRequest.java index 141b13f1751..6b59fdeaf61 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestDropTimeoutRequest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestDropTimeoutRequest.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,8 +17,12 @@ */ package org.apache.hadoop.hbase.client; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -30,21 +34,15 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicLong; - /** * Test a drop timeout request. * This test used to be in TestHCM but it has particulare requirements -- i.e. one handler only -- @@ -52,11 +50,11 @@ import java.util.concurrent.atomic.AtomicLong; */ @Category({MediumTests.class}) public class TestDropTimeoutRequest { - @Rule - public final TestRule timeout = CategoryBasedTimeout.builder() - .withTimeout(this.getClass()) - .withLookingForStuckThread(true) - .build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDropTimeoutRequest.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java index eee98945dd6..f996de6f3da 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,12 +17,16 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.concurrent.CountDownLatch; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -38,15 +41,13 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.MasterObserver; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.master.HMaster; -import org.apache.hbase.thirdparty.com.google.common.base.Predicate; -import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -54,12 +55,17 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import org.apache.hbase.thirdparty.com.google.common.base.Predicate; +import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({ MasterTests.class, MediumTests.class }) public class TestEnableTable { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEnableTable.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestEnableTable.class); private static final byte[] FAMILYNAME = Bytes.toBytes("fam"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java index b2eae851e0e..5d088b3257b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java @@ -31,8 +31,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -51,9 +51,10 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; -import org.junit.Ignore; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; @@ -61,6 +62,11 @@ import org.slf4j.LoggerFactory; @Category({MediumTests.class, ClientTests.class}) public class TestFastFail { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFastFail.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFastFail.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte[] FAMILY = Bytes.toBytes("testFamily"); @@ -289,7 +295,7 @@ public class TestFastFail { + "numPreemptiveFastFailExceptions: " + numPreemptiveFastFailExceptions.get(), numPreemptiveFastFailExceptions.get() > 0); - + assertTrue( "Only few thread should ideally be waiting for the dead " + "regionserver to be coming back. numBlockedWorkers:" diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java index 9085fa5e322..dc3f8da3939 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -29,6 +27,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -42,6 +41,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,6 +49,11 @@ import org.junit.rules.TestName; @Category({MediumTests.class, ClientTests.class}) public class TestFlushFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFlushFromClient.class); + private static final Log LOG = LogFactory.getLog(TestFlushFromClient.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static AsyncConnection asyncConn; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 952905ac617..b02888003cd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -104,6 +104,7 @@ import org.apache.hadoop.hbase.util.NonRepeatedEnvironmentEdge; import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -120,6 +121,11 @@ import org.slf4j.LoggerFactory; @Category({LargeTests.class, ClientTests.class}) @SuppressWarnings ("deprecation") public class TestFromClientSide { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFromClientSide.class); + // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide. private static final Logger LOG = LoggerFactory.getLogger(TestFromClientSide.class); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java index 6b031d81249..36b3b9089d3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -36,11 +34,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -67,6 +65,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -79,6 +78,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; @Category({LargeTests.class, ClientTests.class}) public class TestFromClientSide3 { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFromClientSide3.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFromClientSide3.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java index 238f3f7ab54..74bb57de126 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java @@ -20,10 +20,10 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.AbstractRpcClient; @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +44,11 @@ import org.junit.rules.TestName; */ @Category({MediumTests.class, ClientTests.class}) public class TestFromClientSideNoCodec { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFromClientSideNoCodec.class); + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideScanExcpetion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideScanExcpetion.java index 6b16b08d5a8..34481bae669 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideScanExcpetion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideScanExcpetion.java @@ -29,12 +29,12 @@ import java.util.List; import java.util.NavigableSet; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -61,6 +62,10 @@ import org.junit.rules.TestName; @Category({ MediumTests.class, ClientTests.class }) public class TestFromClientSideScanExcpetion { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFromClientSideScanExcpetion.class); + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte[] FAMILY = Bytes.toBytes("testFamily"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideScanExcpetionWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideScanExcpetionWithCoprocessor.java index 3d50ec7eccf..48ad5755693 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideScanExcpetionWithCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideScanExcpetionWithCoprocessor.java @@ -18,12 +18,14 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -33,6 +35,11 @@ import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestFromClientSideScanExcpetionWithCoprocessor extends TestFromClientSideScanExcpetion { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFromClientSideScanExcpetionWithCoprocessor.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java index 447912e8be5..5bf70c4876e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -18,12 +18,14 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -32,6 +34,11 @@ import org.junit.experimental.categories.Category; */ @Category({LargeTests.class, ClientTests.class}) public class TestFromClientSideWithCoprocessor extends TestFromClientSide { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFromClientSideWithCoprocessor.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetProcedureResult.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetProcedureResult.java index 60e18f70ffd..8b1584f9e5a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetProcedureResult.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetProcedureResult.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.concurrent.CountDownLatch; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.TableName; @@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +53,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedu @Category({ MasterTests.class, MediumTests.class }) public class TestGetProcedureResult { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGetProcedureResult.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); public static final class DummyProcedure extends Procedure diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java index 4acaebbe021..bab0fe9ab22 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java @@ -21,10 +21,11 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.protobuf.ServiceException; import java.io.IOException; import java.util.ArrayList; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -35,19 +36,9 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -60,11 +51,26 @@ import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.protobuf.ServiceException; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest; @Category({SmallTests.class, ClientTests.class}) public class TestHBaseAdminNoCluster { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseAdminNoCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHBaseAdminNoCluster.class); @Rule @@ -73,11 +79,6 @@ public class TestHBaseAdminNoCluster { /** * Verify that PleaseHoldException gets retried. * HBASE-8764 - * @throws IOException - * @throws ZooKeeperConnectionException - * @throws MasterNotRunningException - * @throws ServiceException - * @throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException */ //TODO: Clean up, with Procedure V2 and nonce to prevent the same procedure to call mulitple // time, this test is invalid anymore. Just keep the test around for some time before @@ -85,7 +86,7 @@ public class TestHBaseAdminNoCluster { @Ignore @Test public void testMasterMonitorCallableRetries() - throws MasterNotRunningException, ZooKeeperConnectionException, IOException, + throws MasterNotRunningException, ZooKeeperConnectionException, IOException, org.apache.hbase.thirdparty.com.google.protobuf.ServiceException { Configuration configuration = HBaseConfiguration.create(); // Set the pause and retry count way down. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java index f59a0e95044..92468385356 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -24,7 +22,7 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -32,6 +30,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,6 +40,11 @@ import org.slf4j.LoggerFactory; @Category({LargeTests.class, ClientTests.class}) public class TestHTableMultiplexer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHTableMultiplexer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHTableMultiplexer.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte[] FAMILY = Bytes.toBytes("testFamily"); @@ -94,8 +98,8 @@ public class TestHTableMultiplexer { final int VERSION = 3; List failedPuts; boolean success; - - HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), + + HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), PER_REGIONSERVER_QUEUE_SIZE); Table htable1 = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java index 657dfd8ab76..cd605aa2e48 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -19,6 +17,10 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -32,6 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -39,11 +42,13 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - @Category({ LargeTests.class, ClientTests.class }) public class TestHTableMultiplexerFlushCache { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHTableMultiplexerFlushCache.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHTableMultiplexerFlushCache.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte[] FAMILY = Bytes.toBytes("testFamily"); @@ -98,9 +103,9 @@ public class TestHTableMultiplexerFlushCache { Table htable = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, 3, Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), NUM_REGIONS); - HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), + HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), PER_REGIONSERVER_QUEUE_SIZE); - + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName)) { byte[][] startRows = r.getStartKeys(); byte[] row = startRows[1]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java index df39e0988aa..0bb2b3f93e6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -27,8 +26,8 @@ import static org.mockito.Mockito.verify; import java.io.IOException; import java.lang.reflect.Field; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -39,6 +38,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -47,6 +47,11 @@ import org.slf4j.Logger; @Category({LargeTests.class, ClientTests.class}) public class TestIllegalTableDescriptor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIllegalTableDescriptor.class); + // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide. private static final Logger masterLogger; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java index 3f4c4fc827d..bd4cc91c217 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -18,11 +18,13 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -34,6 +36,11 @@ import org.junit.experimental.categories.Category; */ @Category(LargeTests.class) public class TestIncrementFromClientSideWithCoprocessor extends TestIncrementsFromClientSide { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrementFromClientSideWithCoprocessor.class); + @Before public void before() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java index d69d01eeb76..b1aba6a11b9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -29,11 +28,11 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -45,6 +44,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -63,6 +63,11 @@ import org.slf4j.LoggerFactory; */ @Category(LargeTests.class) public class TestIncrementsFromClientSide { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrementsFromClientSide.class); + final Logger LOG = LoggerFactory.getLogger(getClass()); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte [] ROW = Bytes.toBytes("testRow"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java index 43a2e773925..32da9bdfc87 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java @@ -1,25 +1,26 @@ -/* - * 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 +/** + * 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 + * 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. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hbase.client; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,6 +42,10 @@ import org.junit.experimental.categories.Category; @Category({SmallTests.class, ClientTests.class}) public class TestIntraRowPagination { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIntraRowPagination.class); + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java index 661d3f157c8..3e499bd423f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,8 +22,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.util.Arrays; - import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -37,6 +36,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +46,11 @@ import org.slf4j.LoggerFactory; @Category(LargeTests.class) public class TestLeaseRenewal { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLeaseRenewal.class); + public MetricsAssertHelper HELPER = CompatibilityFactory.getInstance(MetricsAssertHelper.class); final Logger LOG = LoggerFactory.getLogger(getClass()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLimitedScanWithFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLimitedScanWithFilter.java index f702e3d55db..9d871d2992b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLimitedScanWithFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLimitedScanWithFilter.java @@ -23,7 +23,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +44,10 @@ import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestLimitedScanWithFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLimitedScanWithFilter.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName TABLE_NAME = TableName.valueOf("TestRegionScanner"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java index 3f618a1e235..6379e5bbe51 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java @@ -17,16 +17,20 @@ */ package org.apache.hadoop.hbase.client; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import static junit.framework.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; import org.apache.hadoop.hbase.exceptions.RegionOpeningException; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.quotas.ThrottlingException; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RSRpcServices; @@ -35,21 +39,24 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import static junit.framework.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; @Category({MediumTests.class, ClientTests.class}) public class TestMetaCache { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetaCache.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final TableName TABLE_NAME = TableName.valueOf("test_table"); private static final byte[] FAMILY = Bytes.toBytes("fam1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java index 08e2922e062..4f38fbedcc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -33,11 +32,10 @@ import java.util.List; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.ClusterMetrics.Option; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -64,12 +62,12 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,10 +76,11 @@ import org.slf4j.LoggerFactory; */ @Category(LargeTests.class) public class TestMetaWithReplicas { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()). - withLookingForStuckThread(true). - build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetaWithReplicas.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMetaWithReplicas.class); private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final int REGIONSERVERS_COUNT = 3; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java index 8cbd2a5cbd3..069ec7affa7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.io.InterruptedIOException; import java.util.Optional; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -52,6 +53,10 @@ import org.junit.rules.TestName; @Category({LargeTests.class, ClientTests.class}) public class TestMobCloneSnapshotFromClient extends TestCloneSnapshotFromClient { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCloneSnapshotFromClient.class); + private static boolean delayFlush = false; @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClient.java index bde6f0f8377..ae8175c3c7d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClient.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.TableName; @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -36,6 +37,10 @@ import org.junit.experimental.categories.Category; @Category({ClientTests.class, LargeTests.class}) public class TestMobRestoreSnapshotFromClient extends TestRestoreSnapshotFromClient { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobRestoreSnapshotFromClient.class); + @BeforeClass public static void setupCluster() throws Exception { setupConf(TEST_UTIL.getConfiguration()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java index e22391ef3e8..ffdd1d1ec99 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java @@ -15,12 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils; @@ -29,20 +27,16 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test to verify that the cloned table is independent of the table from which it was cloned */ @Category(LargeTests.class) public class TestMobSnapshotCloneIndependence extends TestSnapshotCloneIndependence { - private static final Logger LOG = LoggerFactory.getLogger(TestMobSnapshotCloneIndependence.class); @ClassRule - public static final TestRule timeout = - CategoryBasedTimeout.forClass( TestMobSnapshotCloneIndependence.class); + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobSnapshotCloneIndependence.class); /** * Setup the config for the cluster and start it diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java index 827c932f58a..cdc41b01e87 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java @@ -18,16 +18,17 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * Test create/using/deleting snapshots from the client *

@@ -35,6 +36,11 @@ import org.slf4j.LoggerFactory; */ @Category({LargeTests.class, ClientTests.class}) public class TestMobSnapshotFromClient extends TestSnapshotFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobSnapshotFromClient.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMobSnapshotFromClient.class); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index 68659e108f2..f2f01adf32c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -35,6 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -58,6 +58,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -65,6 +66,11 @@ import org.slf4j.LoggerFactory; @Category({MediumTests.class, FlakeyTests.class}) public class TestMultiParallel { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiParallel.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMultiParallel.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java index 43f2c499df7..fbf1eb0bedb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static junit.framework.TestCase.assertEquals; @@ -23,11 +22,11 @@ import static junit.framework.TestCase.assertEquals; import java.util.ArrayList; import java.util.List; import java.util.concurrent.ThreadLocalRandom; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellBuilderFactory; import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -44,6 +43,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,6 +55,11 @@ import org.junit.rules.TestName; */ @Category({MediumTests.class, ClientTests.class}) public class TestMultiRespectsLimits { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiRespectsLimits.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final MetricsAssertHelper METRICS_ASSERT = CompatibilityFactory.getInstance(MetricsAssertHelper.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java index d73ebc0184f..ac6b9d12c13 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,8 +23,8 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; - import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -33,6 +32,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -47,6 +47,11 @@ import org.slf4j.LoggerFactory; */ @Category({LargeTests.class, ClientTests.class}) public class TestMultipleTimestamps { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultipleTimestamps.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMultipleTimestamps.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java index 82f49529adb..23126557658 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,6 +41,10 @@ import org.junit.rules.TestName; @Category({ LargeTests.class, ClientTests.class }) public class TestMvccConsistentScanner { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMvccConsistentScanner.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static Connection CONN; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutDeleteEtcCellIteration.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutDeleteEtcCellIteration.java index 78f030cb06c..b55835875ce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutDeleteEtcCellIteration.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutDeleteEtcCellIteration.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -24,14 +23,15 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Arrays; import java.util.ConcurrentModificationException; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,6 +40,11 @@ import org.junit.experimental.categories.Category; */ @Category({SmallTests.class, ClientTests.class}) public class TestPutDeleteEtcCellIteration { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPutDeleteEtcCellIteration.class); + private static final byte [] ROW = new byte [] {'r'}; private static final long TIMESTAMP = System.currentTimeMillis(); private static final int COUNT = 10; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutWithDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutWithDelete.java index 88df9d0efb4..114fb309fc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutWithDelete.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutWithDelete.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,21 +17,28 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertTrue; + import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import static org.junit.Assert.assertTrue; - @Category({MediumTests.class, ClientTests.class}) public class TestPutWithDelete { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPutWithDelete.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncScanCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncScanCursor.java index 04ffd7b8fb3..40651b8a424 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncScanCursor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncScanCursor.java @@ -25,16 +25,21 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestRawAsyncScanCursor extends AbstractTestScanCursor { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRawAsyncScanCursor.class); + private static AsyncConnection CONN; @BeforeClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableLimitedScanWithFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableLimitedScanWithFilter.java index c40b34e9054..61eb440b91b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableLimitedScanWithFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableLimitedScanWithFilter.java @@ -25,7 +25,7 @@ import java.util.List; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import java.util.stream.IntStream; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -45,6 +46,10 @@ import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestRawAsyncTableLimitedScanWithFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRawAsyncTableLimitedScanWithFilter.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName TABLE_NAME = TableName.valueOf("TestRegionScanner"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTablePartialScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTablePartialScan.java index 3cffdadab9a..4d0cf7e478e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTablePartialScan.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTablePartialScan.java @@ -23,7 +23,7 @@ import java.util.List; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import java.util.stream.IntStream; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -31,12 +31,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestRawAsyncTablePartialScan { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRawAsyncTablePartialScan.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("async"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableScan.java index e25da28333d..f773cd48468 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableScan.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableScan.java @@ -20,9 +20,10 @@ package org.apache.hadoop.hbase.client; import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -33,6 +34,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ MediumTests.class, ClientTests.class }) public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRawAsyncTableScan.class); + @Parameter(0) public String scanType; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index ba4a8c29770..686a8a44a8e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,8 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - package org.apache.hadoop.hbase.client; import java.io.IOException; @@ -29,10 +26,10 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -42,7 +39,6 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; - import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -62,6 +58,7 @@ import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -69,6 +66,11 @@ import org.slf4j.LoggerFactory; @Category({MediumTests.class, ClientTests.class}) public class TestReplicaWithCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicaWithCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicaWithCluster.class); private static final int NB_SERVERS = 3; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java index f5ea10e28b9..c55c6ca7bc9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; +import com.codahale.metrics.Counter; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -33,11 +32,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; - -import com.codahale.metrics.Counter; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -50,9 +47,6 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; @@ -66,11 +60,16 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; + /** * Tests for region replicas. Sad that we cannot isolate these without bringing up a whole * cluster. See {@link org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster}. @@ -78,6 +77,11 @@ import org.slf4j.LoggerFactory; @Category({MediumTests.class, ClientTests.class}) @SuppressWarnings("deprecation") public class TestReplicasClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicasClient.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicasClient.class); private static final int NB_SERVERS = 1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java index fe1cc9c2251..3eb304d9aef 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java @@ -24,10 +24,9 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.HashSet; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -46,21 +45,21 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; /** * Test restore snapshots from the client */ @Category({LargeTests.class, ClientTests.class}) public class TestRestoreSnapshotFromClient { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder() - .withTimeout(this.getClass()) - .withLookingForStuckThread(true) - .build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRestoreSnapshotFromClient.class); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java index 94cf44dfd55..7f5dfa26580 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java @@ -17,13 +17,20 @@ */ package org.apache.hadoop.hbase.client; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({LargeTests.class, ClientTests.class}) public class TestRestoreSnapshotFromClientWithRegionReplicas extends TestRestoreSnapshotFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRestoreSnapshotFromClientWithRegionReplicas.class); + @Override protected int getNumReplicas() { return 3; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java index b494895f045..b38fb6a6c66 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.HBaseTestCase.assertByteEquals; @@ -26,17 +24,17 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.NoSuchElementException; - import junit.framework.TestCase; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +42,10 @@ import org.slf4j.LoggerFactory; @Category({SmallTests.class, ClientTests.class}) public class TestResult extends TestCase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestResult.class); + private static final Logger LOG = LoggerFactory.getLogger(TestResult.class.getName()); static KeyValue[] genKVs(final byte[] row, final byte[] family, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultFromCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultFromCoprocessor.java index 1fb0f6458ea..c5be7cd1bf6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultFromCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultFromCoprocessor.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -19,14 +17,15 @@ */ package org.apache.hadoop.hbase.client; +import static junit.framework.TestCase.assertTrue; + import java.io.IOException; import java.util.Arrays; import java.util.Optional; - -import static junit.framework.TestCase.assertTrue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; @@ -39,11 +38,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MediumTests.class, ClientTests.class}) public class TestResultFromCoprocessor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestResultFromCoprocessor.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] ROW = Bytes.toBytes("normal_row"); private static final byte[] FAMILY = Bytes.toBytes("fm"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerCursor.java index 3b2ef2ce407..568c6fb85b2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerCursor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerCursor.java @@ -18,14 +18,19 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestResultScannerCursor extends AbstractTestResultScannerCursor { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestResultScannerCursor.class); + @Override protected ResultScanner getScanner(Scan scan) throws IOException { return TEST_UTIL.getConnection().getTable(TABLE_NAME).getScanner(scan); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java index 9cd1c69de9b..1a94af013b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,13 +17,16 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertEquals; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; @@ -32,16 +35,19 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import static org.junit.Assert.assertEquals; - @Category(LargeTests.class) public class TestResultSizeEstimation { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestResultSizeEstimation.class); + final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); final static int TAG_DATA_SIZE = 2048; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java index 1044d6b3aa3..f5c217605e3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java @@ -21,26 +21,29 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; + /** * Testcase to make sure that we do not close scanners if ScanRequest.numberOfRows is zero. See * HBASE-18042 for more details. @@ -48,6 +51,10 @@ import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) public class TestScanWithoutFetchingData { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScanWithoutFetchingData.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName TABLE_NAME = TableName.valueOf("test"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java index d2191e18e8a..d7f4464e7c0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -44,6 +45,10 @@ import org.slf4j.LoggerFactory; @Category({LargeTests.class, ClientTests.class}) public class TestScannerTimeout { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerTimeout.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java index 917372654bd..6803a2e0e25 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java @@ -1,24 +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 +/** + * 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 + * 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. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -42,6 +57,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,26 +65,16 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; -import java.util.stream.IntStream; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - - /** * A client-side test, mostly testing scanners with various parameters. */ @Category({MediumTests.class, ClientTests.class}) public class TestScannersFromClientSide { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannersFromClientSide.class); + private static final Logger LOG = LoggerFactory.getLogger(TestScannersFromClientSide.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide2.java index ff57ca0aefd..0a10818de3e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide2.java @@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -47,6 +48,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ MediumTests.class, ClientTests.class }) public class TestScannersFromClientSide2 { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannersFromClientSide2.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("scan"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java index 387253c0da0..0dce2f3565d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -24,10 +23,9 @@ import java.io.IOException; import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -37,17 +35,17 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.ipc.ServerTooBusyException; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; /** * This class is for testing HBaseConnectionManager ServerBusyException. @@ -56,10 +54,11 @@ import org.junit.rules.TestRule; */ @Category({LargeTests.class}) public class TestServerBusyException { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder() - .withTimeout(this.getClass()) - .withLookingForStuckThread(true) - .build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerBusyException.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] FAM_NAM = Bytes.toBytes("f"); private static final byte[] ROW = Bytes.toBytes("bbb"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerLoadDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerLoadDurability.java index ff0dcd661fa..38c3d3a8790 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerLoadDurability.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerLoadDurability.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; @@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,6 +51,11 @@ import org.junit.runners.Parameterized; @RunWith(Parameterized.class) @Category({ MediumTests.class, ClientTests.class }) public class TestServerLoadDurability { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerLoadDurability.class); + private static final byte[] FAMILY = Bytes.toBytes("testFamily"); @Parameterized.Parameter diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java index 9bf3d214d43..beaa59be3e6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,14 +21,12 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -38,14 +35,22 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; + @Category({ MediumTests.class, ClientTests.class }) public class TestShortCircuitConnection { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestShortCircuitConnection.class); + private final static HBaseTestingUtility UTIL = new HBaseTestingUtility(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java index 41e303409f0..a3362749729 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,8 +23,8 @@ import java.util.ArrayList; import java.util.LinkedList; import java.util.List; import java.util.Map.Entry; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -34,14 +33,21 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Maps; @Category(LargeTests.class) public class TestSizeFailures { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSizeFailures.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSizeFailures.class); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte [] FAMILY = Bytes.toBytes("testFamily"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSmallReversedScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSmallReversedScanner.java index 4de6a76262e..6ad251d6681 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSmallReversedScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSmallReversedScanner.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.client; +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -25,15 +27,19 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - @Category(MediumTests.class) public class TestSmallReversedScanner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSmallReversedScanner.class); + public static final Logger LOG = LoggerFactory.getLogger(TestSmallReversedScanner.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java index da422f3f5d5..0562c90347b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java @@ -15,16 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import java.util.List; import java.util.regex.Pattern; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -48,7 +46,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,11 +54,12 @@ import org.slf4j.LoggerFactory; */ @Category({LargeTests.class, ClientTests.class}) public class TestSnapshotCloneIndependence { - private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotCloneIndependence.class); @ClassRule - public static final TestRule timeout = - CategoryBasedTimeout.forClass(TestSnapshotCloneIndependence.class); + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotCloneIndependence.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotCloneIndependence.class); @Rule public TestName testName = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index b077d64164b..7ce130c71ab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -25,10 +25,10 @@ import static org.junit.Assert.fail; import java.util.ArrayList; import java.util.List; import java.util.regex.Pattern; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -36,11 +36,10 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException; -import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.snapshot.SnapshotManifestV1; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -49,14 +48,18 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; + /** * Test create/using/deleting snapshots from the client *

@@ -64,6 +67,11 @@ import org.slf4j.LoggerFactory; */ @Category({LargeTests.class, ClientTests.class}) public class TestSnapshotFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotFromClient.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotFromClient.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClientWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClientWithRegionReplicas.java index 9f8cc3ecc49..6b7f4529350 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClientWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClientWithRegionReplicas.java @@ -17,13 +17,19 @@ */ package org.apache.hadoop.hbase.client; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({LargeTests.class, ClientTests.class}) public class TestSnapshotFromClientWithRegionReplicas extends TestSnapshotFromClient { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotFromClientWithRegionReplicas.class); + @Override protected int getNumReplicas() { return 3; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java index fa3715dfc07..00a5cec6636 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; @@ -24,10 +23,10 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.Collections; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -45,6 +44,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -55,6 +55,11 @@ import org.slf4j.LoggerFactory; */ @Category({MediumTests.class, ClientTests.class}) public class TestSnapshotMetadata { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotMetadata.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotMetadata.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java index 9e543dcf5c3..ce0db30e58f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotWithAcl.java @@ -15,11 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; +import java.io.IOException; +import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -39,15 +41,17 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; -import java.util.UUID; - @Category({ MediumTests.class, ClientTests.class }) public class TestSnapshotWithAcl extends SecureTestUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotWithAcl.class); + public TableName TEST_TABLE = TableName.valueOf(UUID.randomUUID().toString()); private static final int ROW_COUNT = 30000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java index d239eb8e3eb..11f53a9671f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java @@ -27,7 +27,7 @@ import java.util.List; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; @@ -37,15 +37,20 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; -import org.junit.Ignore; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; @Category({MediumTests.class, ClientTests.class}) public class TestSplitOrMergeStatus { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSplitOrMergeStatus.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte [] FAMILY = Bytes.toBytes("testFamily"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java index 8b3a475e626..716b1dc5448 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java @@ -31,8 +31,8 @@ import java.net.InetSocketAddress; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -58,19 +58,24 @@ import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; - @Category({ClientTests.class, MediumTests.class}) public class TestTableFavoredNodes { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableFavoredNodes.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableFavoredNodes.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java index fe5a9ed51a5..408953ad6ec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java @@ -15,18 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; @@ -39,6 +38,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -47,6 +47,10 @@ import org.slf4j.LoggerFactory; @Category({LargeTests.class, ClientTests.class}) public class TestTableSnapshotScanner { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableSnapshotScanner.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableSnapshotScanner.class); private final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final int NUM_REGION_SERVERS = 2; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java index ea60ec2a388..8fc03aeeb5e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,8 +23,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.TimestampsFilter; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -35,6 +34,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,6 +49,11 @@ import org.slf4j.LoggerFactory; */ @Category({MediumTests.class, ClientTests.class}) public class TestTimestampsFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTimestampsFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTimestampsFilter.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -233,7 +238,7 @@ public class TestTimestampsFilter { private void testWithVersionDeletes(boolean flushTables) throws IOException { final byte [] TABLE = Bytes.toBytes(name.getMethodName() + "_" + - (flushTables ? "flush" : "noflush")); + (flushTables ? "flush" : "noflush")); byte [] FAMILY = Bytes.toBytes("event_log"); byte [][] FAMILIES = new byte[][] { FAMILY }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestUpdateConfiguration.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestUpdateConfiguration.java index 3f4029a1779..09dc40c74ae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestUpdateConfiguration.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestUpdateConfiguration.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,12 +24,13 @@ import java.nio.file.FileSystems; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardCopyOption; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -38,9 +38,14 @@ import org.slf4j.LoggerFactory; @Category({MediumTests.class}) public class TestUpdateConfiguration { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestUpdateConfiguration.class); + private static final Logger LOG = LoggerFactory.getLogger(TestUpdateConfiguration.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - + @BeforeClass public static void setup() throws Exception { TEST_UTIL.startMiniCluster(2, 1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java index 4f9a71fdb4a..db7546fc543 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java @@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.IntStream; import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -47,6 +49,11 @@ import org.slf4j.LoggerFactory; @Category({ MediumTests.class, ClientTests.class }) public class TestZKAsyncRegistry { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKAsyncRegistry.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKAsyncRegistry.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java index 7cbccce87ec..7cdb70e5627 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client.locking; import static org.junit.Assert.assertEquals; @@ -32,23 +30,17 @@ import static org.mockito.Mockito.when; import java.util.Random; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.client.PerClientRandomNonceGenerator; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Threads; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.ArgumentCaptor; @@ -56,10 +48,24 @@ import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType; + @Category({ClientTests.class, SmallTests.class}) public class TestEntityLocks { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEntityLocks.class); + private static final Logger LOG = LoggerFactory.getLogger(TestEntityLocks.class); - + private final Configuration conf = HBaseConfiguration.create(); private final LockService.BlockingInterface master = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java index aac57dc5484..ba78e6dde77 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.client.replication; - import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -28,6 +27,7 @@ import java.util.Set; import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ReplicationPeerNotFoundException; @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -68,8 +69,12 @@ import static org.junit.Assert.fail; @Category({MediumTests.class, ClientTests.class}) public class TestReplicationAdmin { - private static final Logger LOG = - LoggerFactory.getLogger(TestReplicationAdmin.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationAdmin.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationAdmin.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java index 82eda2a231c..f67411b8e53 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java @@ -1,12 +1,19 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hbase.client.replication; @@ -19,7 +26,7 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.UUID; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -35,6 +42,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +54,10 @@ import org.junit.rules.TestName; @Category({ MediumTests.class, ClientTests.class }) public class TestReplicationAdminWithClusters extends TestReplicationBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationAdminWithClusters.class); + static Connection connection1; static Connection connection2; static Admin admin1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithTwoDifferentZKClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithTwoDifferentZKClusters.java index f2a5221c3cb..c04ee4b80e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithTwoDifferentZKClusters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithTwoDifferentZKClusters.java @@ -1,19 +1,26 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.hadoop.hbase.client.replication; import static org.junit.Assert.assertTrue; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -26,12 +33,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MediumTests.class, ClientTests.class }) public class TestReplicationAdminWithTwoDifferentZKClusters { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationAdminWithTwoDifferentZKClusters.class); + private static Configuration conf1 = HBaseConfiguration.create(); private static Configuration conf2; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/TestCellMessageCodec.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/TestCellMessageCodec.java index 88a50ea43d6..e4b2b2f8004 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/TestCellMessageCodec.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/TestCellMessageCodec.java @@ -26,22 +26,29 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.io.CountingInputStream; import org.apache.hbase.thirdparty.com.google.common.io.CountingOutputStream; @Category({MiscTests.class, SmallTests.class}) public class TestCellMessageCodec { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellMessageCodec.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCellMessageCodec.class); @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/conf/TestConfigurationManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/conf/TestConfigurationManager.java index 06e1a359ae0..20dd0244263 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/conf/TestConfigurationManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/conf/TestConfigurationManager.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.conf; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -31,6 +32,11 @@ import org.slf4j.LoggerFactory; @Category({SmallTests.class, ClientTests.class}) public class TestConfigurationManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConfigurationManager.class); + private static final Logger LOG = LoggerFactory.getLogger(TestConfigurationManager.class); class DummyConfigurationObserver implements ConfigurationObserver { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java index d7c70da240d..5393b4822d2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java @@ -23,21 +23,22 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -48,6 +49,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, MediumTests.class}) public class TestConstraint { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConstraint.class); + private static final Logger LOG = LoggerFactory .getLogger(TestConstraint.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java index 12a229d250b..5a65447edde 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java @@ -23,16 +23,17 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.constraint.TestConstraint.CheckWasRunConstraint; import org.apache.hadoop.hbase.constraint.WorksConstraint.NameConstraint; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Pair; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +44,11 @@ import org.junit.rules.TestName; */ @Category({MiscTests.class, SmallTests.class}) public class TestConstraints { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConstraints.class); + @Rule public TestName name = new TestName(); @@ -99,7 +105,7 @@ public class TestConstraints { /** * Test that Constraints are properly enabled, disabled, and removed - * + * * @throws Exception */ @SuppressWarnings("unchecked") @@ -135,7 +141,7 @@ public class TestConstraints { /** * Test that when we update a constraint the ordering is not modified. - * + * * @throws Exception */ @SuppressWarnings("unchecked") @@ -161,7 +167,7 @@ public class TestConstraints { /** * Test that if a constraint hasn't been set that there are no problems with * attempting to remove it. - * + * * @throws Throwable * on failure. */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAppendTimeRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAppendTimeRange.java index 88420758012..51f0d7307c0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAppendTimeRange.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAppendTimeRange.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertArrayEquals; @@ -30,6 +28,7 @@ import java.util.Map; import java.util.NavigableMap; import java.util.Optional; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Append; @@ -47,6 +46,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,6 +55,10 @@ import org.junit.rules.TestName; @Category({CoprocessorTests.class, MediumTests.class}) public class TestAppendTimeRange { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAppendTimeRange.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorConfiguration.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorConfiguration.java index 6e6c34b3839..b3012ecf61e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorConfiguration.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorConfiguration.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -16,17 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; - -import static org.mockito.Mockito.*; -import static org.junit.Assert.*; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,6 +51,11 @@ import org.junit.rules.ExpectedException; */ @Category({CoprocessorTests.class, SmallTests.class}) public class TestCoprocessorConfiguration { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorConfiguration.class); + @Rule public ExpectedException thrown = ExpectedException.none(); private static final Configuration CONF = HBaseConfiguration.create(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorHost.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorHost.java index de697f144a0..c541647e195 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorHost.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorHost.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -17,20 +17,28 @@ */ package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({SmallTests.class}) public class TestCoprocessorHost { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorHost.class); + /** * An {@link Abortable} implementation for tests. */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java index 44216ec72cc..6b37db6c09a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; @@ -34,12 +32,12 @@ import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.ConcurrentMap; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -66,6 +64,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -76,6 +75,11 @@ import org.slf4j.LoggerFactory; @Category({CoprocessorTests.class, SmallTests.class}) public class TestCoprocessorInterface { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorInterface.class); + @Rule public TestName name = new TestName(); private static final Logger LOG = LoggerFactory.getLogger(TestCoprocessorInterface.class); private static final HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java index 7f79f06a209..24f74d8aadd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,20 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; -import java.io.IOException; -import java.util.List; -import java.util.Optional; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; - +import java.io.IOException; +import java.util.List; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionLocation; @@ -57,7 +61,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRo import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -66,6 +69,7 @@ import org.apache.hadoop.hbase.wal.WALKey; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -73,11 +77,7 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** * Testing of coprocessor metrics end-to-end. @@ -85,6 +85,10 @@ import static org.junit.Assert.assertTrue; @Category({CoprocessorTests.class, MediumTests.class}) public class TestCoprocessorMetrics { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorMetrics.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCoprocessorMetrics.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java index 58229938e45..c3f7e80ba91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -17,9 +17,12 @@ */ package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.SharedConnection; import org.apache.hadoop.hbase.client.Connection; @@ -28,15 +31,11 @@ import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; - -import java.io.IOException; - -import static org.junit.Assert.assertTrue; /** * Ensure Coprocessors get ShortCircuit Connections when they get a Connection from their @@ -44,10 +43,13 @@ import static org.junit.Assert.assertTrue; */ @Category({CoprocessorTests.class, SmallTests.class}) public class TestCoprocessorShortCircuitRPC { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorShortCircuitRPC.class); + @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); private static final HBaseTestingUtility HTU = HBaseTestingUtility.createLocalHTU(); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorStop.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorStop.java index f96da7009ff..00ca4519d7b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorStop.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorStop.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -16,34 +15,37 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertTrue; + import java.io.IOException; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.*; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.FileSystem; - +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertTrue; - - /** * Tests for master and regionserver coprocessor stop method * */ @Category({CoprocessorTests.class, MediumTests.class}) public class TestCoprocessorStop { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorStop.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCoprocessorStop.class); private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final String MASTER_FILE = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreMasterCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreMasterCoprocessor.java index a0c831e50ce..5afc7b07cfe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreMasterCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreMasterCoprocessor.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -17,7 +17,12 @@ */ package org.apache.hadoop.hbase.coprocessor; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterServices; @@ -26,17 +31,11 @@ import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; - -import java.io.IOException; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; /** * Test CoreCoprocessor Annotation works giving access to facility not usually available. @@ -44,9 +43,12 @@ import static org.junit.Assert.assertTrue; */ @Category({CoprocessorTests.class, SmallTests.class}) public class TestCoreMasterCoprocessor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoreMasterCoprocessor.class); + @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); private static final HBaseTestingUtility HTU = HBaseTestingUtility.createLocalHTU(); private MasterServices ms; private MasterCoprocessorHost mch; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionCoprocessor.java index daf489cc7c1..8e6ebf349b3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionCoprocessor.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -17,7 +17,12 @@ */ package org.apache.hadoop.hbase.coprocessor; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MockRegionServerServices; import org.apache.hadoop.hbase.TableName; @@ -27,7 +32,6 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerServices; @@ -36,17 +40,11 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; - -import java.io.IOException; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; /** * Test CoreCoprocessor Annotation works giving access to facility not usually available. @@ -54,9 +52,12 @@ import static org.junit.Assert.assertTrue; */ @Category({CoprocessorTests.class, SmallTests.class}) public class TestCoreRegionCoprocessor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoreRegionCoprocessor.class); + @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); HBaseTestingUtility HTU = HBaseTestingUtility.createLocalHTU(); private HRegion region = null; private RegionServerServices rss; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionServerCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionServerCoprocessor.java index 68349826f69..4e14867c8df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionServerCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionServerCoprocessor.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -17,27 +17,25 @@ */ package org.apache.hadoop.hbase.coprocessor; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MockRegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; - -import java.io.IOException; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; /** * Test CoreCoprocessor Annotation works giving access to facility not usually available. @@ -45,9 +43,12 @@ import static org.junit.Assert.assertTrue; */ @Category({CoprocessorTests.class, SmallTests.class}) public class TestCoreRegionServerCoprocessor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoreRegionServerCoprocessor.class); + @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); private static final HBaseTestingUtility HTU = HBaseTestingUtility.createLocalHTU(); private RegionServerServices rss; private RegionServerCoprocessorHost rsch; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java index 8805337d88e..a74914f6213 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertArrayEquals; @@ -28,8 +26,8 @@ import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.NavigableMap; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; @@ -49,6 +47,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -61,6 +60,10 @@ import org.junit.experimental.categories.Category; @Category({CoprocessorTests.class, MediumTests.class}) public class TestIncrementTimeRange { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrementTimeRange.class); + private static final HBaseTestingUtility util = new HBaseTestingUtility(); private static ManualEnvironmentEdge mee = new ManualEnvironmentEdge(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java index bd0efd85e35..ce3c726361c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -16,15 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.IOException; import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -43,14 +46,10 @@ import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - /** * Tests unhandled exceptions thrown by coprocessors running on master. * Expected result is that the master will abort with an informative @@ -60,6 +59,10 @@ import static org.junit.Assert.fail; @Category({CoprocessorTests.class, MediumTests.class}) public class TestMasterCoprocessorExceptionWithAbort { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterCoprocessorExceptionWithAbort.class); + public static class MasterTracker extends ZKNodeTracker { public boolean masterZKNodeWasDeleted = false; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java index c4defa2d74e..8c38208d669 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -16,15 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.IOException; import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -42,13 +44,10 @@ import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - /** * Tests unhandled exceptions thrown by coprocessors running on master. * Expected result is that the master will remove the buggy coprocessor from @@ -59,6 +58,10 @@ import static org.junit.Assert.fail; @Category({CoprocessorTests.class, MediumTests.class}) public class TestMasterCoprocessorExceptionWithRemove { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterCoprocessorExceptionWithRemove.class); + public static class MasterTracker extends ZKNodeTracker { public boolean masterZKNodeWasDeleted = false; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java index ea817ff5e1c..fcaed63c98d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertFalse; @@ -31,6 +30,7 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionLocation; @@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -75,13 +76,17 @@ import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; - /** * Tests invocation of the {@link org.apache.hadoop.hbase.coprocessor.MasterObserver} * interface hooks at all appropriate times during normal HMaster operations. */ @Category({CoprocessorTests.class, MediumTests.class}) public class TestMasterObserver { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterObserver.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterObserver.class); public static CountDownLatch tableCreationLatch = new CountDownLatch(1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java index 877265d29bd..336d3428acd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java @@ -1,18 +1,25 @@ /** - * 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. + * 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.coprocessor; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -21,12 +28,13 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.MemStoreSize; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -39,6 +47,11 @@ import org.slf4j.LoggerFactory; */ @Category(LargeTests.class) public class TestNegativeMemStoreSizeWithSlowCoprocessor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNegativeMemStoreSizeWithSlowCoprocessor.class); + static final Logger LOG = LoggerFactory.getLogger(TestNegativeMemStoreSizeWithSlowCoprocessor.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java index b2f7d1ab515..80b0894ca7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,6 +17,17 @@ */ package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -29,33 +39,27 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; -import java.util.Collections; -import java.util.Optional; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * Test that a coprocessor can open a connection and write to another table, inside a hook. */ @Category({CoprocessorTests.class, MediumTests.class}) public class TestOpenTableInCoprocessor { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOpenTableInCoprocessor.class); + private static final TableName otherTable = TableName.valueOf("otherTable"); private static final TableName primaryTable = TableName.valueOf("primary"); private static final byte[] family = new byte[] { 'f' }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java index dbbe1ae3ab4..282d5dc8936 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertFalse; @@ -31,10 +29,10 @@ import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -57,21 +55,18 @@ import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; @Category({ CoprocessorTests.class, MediumTests.class }) public class TestPassCustomCellViaRegionObserver { - private static final Logger LOG = - LoggerFactory.getLogger(TestPassCustomCellViaRegionObserver.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPassCustomCellViaRegionObserver.class); @Rule public TestName testName = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestReadOnlyConfiguration.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestReadOnlyConfiguration.java index d7373564bef..a91c505f175 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestReadOnlyConfiguration.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestReadOnlyConfiguration.java @@ -1,38 +1,41 @@ -/* - * 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 +/** + * 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. + * 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.coprocessor; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; -@Category({RegionServerTests.class, SmallTests.class}) +@Category({ RegionServerTests.class, SmallTests.class }) public class TestReadOnlyConfiguration { - @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()). - withLookingForStuckThread(true). - build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReadOnlyConfiguration.class); + + @Rule + public TestName name = new TestName(); @Test public void testAddDefaultResource() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java index ad45ef630fc..714dfc4f2e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -26,34 +25,40 @@ import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; +import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({CoprocessorTests.class, MediumTests.class}) public class TestRegionObserverBypass { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionObserverBypass.class); + private static HBaseTestingUtility util; private static final TableName tableName = TableName.valueOf("test"); private static final byte[] dummy = Bytes.toBytes("dummy"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java index 136f2032fc9..73e067459f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java @@ -15,16 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -39,7 +41,6 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WALEdit; @@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.wal.WALKey; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -54,12 +56,15 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category(MediumTests.class) public class TestRegionObserverForAddingMutationsFromCoprocessors { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionObserverForAddingMutationsFromCoprocessors.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionObserverForAddingMutationsFromCoprocessors.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java index 5f87d7ded11..6bfd42b7ba5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertArrayEquals; @@ -29,14 +27,13 @@ import java.lang.reflect.Method; import java.util.ArrayList; import java.util.List; import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -66,7 +63,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.NoLimitScannerContext; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.ScannerContext; @@ -83,21 +79,24 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @Category({ CoprocessorTests.class, MediumTests.class }) public class TestRegionObserverInterface { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionObserverInterface.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionObserverInterface.class); - @Rule - public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); public static final TableName TEST_TABLE = TableName.valueOf("TestTable"); public final static byte[] A = Bytes.toBytes("a"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverPreFlushAndPreCompact.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverPreFlushAndPreCompact.java index cea0833e285..734d4e0f913 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverPreFlushAndPreCompact.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverPreFlushAndPreCompact.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,8 +17,12 @@ */ package org.apache.hadoop.hbase.coprocessor; +import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGION_COPROCESSOR_CONF_KEY; + +import java.io.IOException; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -35,20 +39,13 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.mockito.Mockito; -import java.io.IOException; -import java.util.Optional; - -import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGION_COPROCESSOR_CONF_KEY; - - - /** * Test that we fail if a Coprocessor tries to return a null scanner out * {@link RegionObserver#preFlush(ObserverContext, Store, InternalScanner, FlushLifeCycleTracker)} @@ -58,8 +55,11 @@ import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGION_COPROCE */ @Category({CoprocessorTests.class, SmallTests.class}) public class TestRegionObserverPreFlushAndPreCompact { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionObserverPreFlushAndPreCompact.class); + @Rule public TestName name = new TestName(); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index 8e4383480d7..e7a9f61dddd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertEquals; @@ -26,12 +24,12 @@ import java.io.IOException; import java.util.List; import java.util.Optional; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -71,6 +69,7 @@ import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -78,6 +77,11 @@ import org.junit.rules.TestName; @Category({CoprocessorTests.class, MediumTests.class}) public class TestRegionObserverScannerOpenHook { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionObserverScannerOpenHook.class); + private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); static final Path DIR = UTIL.getDataTestDir(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java index 84701167db2..cbc00d3f032 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,17 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import java.io.IOException; import java.util.Optional; - import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -34,22 +31,28 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MockRegionServerServices; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.regionserver.ChunkCreator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerServices; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @Category({CoprocessorTests.class, SmallTests.class}) public class TestRegionObserverStacking extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionObserverStacking.class); + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); static final Path DIR = TEST_UTIL.getDataTestDir(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java index 964193dee86..3e1d41a8942 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -16,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.fail; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -34,11 +32,12 @@ import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -52,6 +51,11 @@ import org.slf4j.LoggerFactory; */ @Category({CoprocessorTests.class, MediumTests.class}) public class TestRegionServerCoprocessorExceptionWithAbort { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionServerCoprocessorExceptionWithAbort.class); + private static final Logger LOG = LoggerFactory.getLogger( TestRegionServerCoprocessorExceptionWithAbort.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java index 5607c3e9dfb..f394e57ca89 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertFalse; @@ -24,20 +22,21 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -50,6 +49,11 @@ import org.junit.experimental.categories.Category; */ @Category({CoprocessorTests.class, MediumTests.class}) public class TestRegionServerCoprocessorExceptionWithRemove { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionServerCoprocessorExceptionWithRemove.class); + public static class BuggyRegionObserver extends SimpleRegionObserver { @SuppressWarnings("null") @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java index 09aa4ff41aa..df80fa0d769 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -35,6 +35,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -67,6 +68,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -81,6 +83,11 @@ import org.slf4j.LoggerFactory; */ @Category({CoprocessorTests.class, MediumTests.class}) public class TestWALObserver { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALObserver.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWALObserver.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java index 8ec1a44f093..8861a69e45d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java @@ -20,8 +20,10 @@ package org.apache.hadoop.hbase.errorhandling; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -34,6 +36,11 @@ import org.slf4j.LoggerFactory; */ @Category({MasterTests.class, SmallTests.class}) public class TestForeignExceptionDispatcher { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestForeignExceptionDispatcher.class); + private static final Logger LOG = LoggerFactory.getLogger(TestForeignExceptionDispatcher.class); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionSerialization.java index 2ab534a4894..127e72f88dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionSerialization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionSerialization.java @@ -23,9 +23,10 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,11 +35,15 @@ import org.junit.experimental.categories.Category; */ @Category({MasterTests.class, SmallTests.class}) public class TestForeignExceptionSerialization { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestForeignExceptionSerialization.class); + private static final String srcName = "someNode"; /** * Verify that we get back similar stack trace information before an after serialization. - * @throws IOException */ @Test public void testSimpleException() throws IOException { @@ -61,7 +66,6 @@ public class TestForeignExceptionSerialization { /** * Compare that a generic exception's stack trace has the same stack trace elements after * serialization and deserialization - * @throws IOException */ @Test public void testRemoteFromLocal() throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java index 27bc6e1c7ae..60362b7c801 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java @@ -19,8 +19,10 @@ package org.apache.hadoop.hbase.errorhandling; import static org.junit.Assert.fail; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -33,6 +35,10 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, SmallTests.class}) public class TestTimeoutExceptionInjector { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTimeoutExceptionInjector.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTimeoutExceptionInjector.class); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java index a3f2f1cecc5..f6e9409a515 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,29 +17,43 @@ */ package org.apache.hadoop.hbase.executor; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.anyObject; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + import java.io.IOException; import java.io.StringWriter; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.executor.ExecutorService.Executor; import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorStatus; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.mockito.Mockito.*; - @Category({MiscTests.class, SmallTests.class}) public class TestExecutorService { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestExecutorService.class); + private static final Logger LOG = LoggerFactory.getLogger(TestExecutorService.class); @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java index 0d8976c0276..948df5f5261 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.favored; import static org.junit.Assert.assertEquals; @@ -32,8 +31,8 @@ import java.util.Map; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -45,6 +44,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Triple; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -57,6 +57,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @Category({MasterTests.class, SmallTests.class}) public class TestFavoredNodeAssignmentHelper { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFavoredNodeAssignmentHelper.class); + private static List servers = new ArrayList<>(); private static Map> rackToServers = new HashMap<>(); private static RackManager rackManager = Mockito.mock(RackManager.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestStartcodeAgnosticServerName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestStartcodeAgnosticServerName.java index 72d734057d5..05e1e0163fd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestStartcodeAgnosticServerName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestStartcodeAgnosticServerName.java @@ -20,15 +20,21 @@ package org.apache.hadoop.hbase.favored; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestStartcodeAgnosticServerName { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStartcodeAgnosticServerName.class); + @Test public void testStartCodeServerName() { ServerName sn = ServerName.valueOf("www.example.org", 1234, 5678); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestBigDecimalComparator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestBigDecimalComparator.java index 76a9721e202..fcff0c1f627 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestBigDecimalComparator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestBigDecimalComparator.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,30 +7,33 @@ * "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 + * 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. + * 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.filter; import java.math.BigDecimal; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ FilterTests.class, SmallTests.class }) public class TestBigDecimalComparator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBigDecimalComparator.class); + @Test public void testObjectEquals() { BigDecimal bd = new BigDecimal(Double.MIN_VALUE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java index 0e5fdb24208..35db7398033 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -6,33 +6,37 @@ * 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. + * + * 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.filter; -import java.nio.ByteBuffer; +import static org.junit.Assert.assertEquals; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; - /** * Tests for the bit comparator */ @Category({FilterTests.class, SmallTests.class}) public class TestBitComparator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBitComparator.class); + private static byte[] zeros = new byte[]{0, 0, 0, 0, 0, 0}; private static ByteBuffer zeros_bb = ByteBuffer.wrap(zeros); private static byte[] ones = new byte[]{1, 1, 1, 1, 1, 1}; @@ -43,13 +47,13 @@ public class TestBitComparator { private static byte[] data2 = new byte[]{0, 0, 0, 0, 0, 15}; private static ByteBuffer data2_bb = ByteBuffer.wrap(data2); private static byte[] data3 = new byte[]{15, 15, 15, 15, 15}; - + // data for testing compareTo method with offset and length parameters private static byte[] data1_2 = new byte[]{15, 15, 0, 0, 0, 0, 0, 15}; private static ByteBuffer data1_2_bb = ByteBuffer.wrap(data1_2); private static byte[] data2_2 = new byte[]{15, 0, 0, 0, 0, 0, 15, 15}; private static ByteBuffer data2_2_bb = ByteBuffer.wrap(data2_2); - + private final int Equal = 0; private final int NotEqual = 1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java index fc012d25f7d..b193260ab43 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -20,16 +19,19 @@ package org.apache.hadoop.hbase.filter; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; + /** * Test for the ColumnPaginationFilter, used mainly to test the successful serialization of the filter. * More test functionality can be found within {@link org.apache.hadoop.hbase.filter.TestFilter#testColumnPaginationFilter()} @@ -37,6 +39,11 @@ import org.junit.experimental.categories.Category; @Category({FilterTests.class, SmallTests.class}) public class TestColumnPaginationFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestColumnPaginationFilter.class); + private static final byte[] ROW = Bytes.toBytes("row_1_test"); private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test"); private static final byte[] VAL_1 = Bytes.toBytes("a"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java index 9b71d454665..136e89ba6eb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java @@ -26,8 +26,8 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +52,10 @@ import org.junit.rules.TestName; @Category({FilterTests.class, SmallTests.class}) public class TestColumnPrefixFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestColumnPrefixFilter.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java index 7bcce946e06..f6452fb7946 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java @@ -25,29 +25,29 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; - import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Rule; -import org.junit.Test; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - class StringRange { private String start = null; private String end = null; @@ -122,6 +122,10 @@ class StringRange { @Category({FilterTests.class, MediumTests.class}) public class TestColumnRangeFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestColumnRangeFilter.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestColumnRangeFilter.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java index fef699bb1ff..6c77bcf61e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -24,17 +22,23 @@ import static org.junit.Assert.assertTrue; import java.math.BigDecimal; import java.util.regex.Pattern; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; + @Category({FilterTests.class, SmallTests.class}) public class TestComparatorSerialization { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestComparatorSerialization.class); + @Test public void testBinaryComparator() throws Exception { BinaryComparator binaryComparator = new BinaryComparator(Bytes.toBytes("binaryComparator")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java index ae90c63b717..3636dd363a6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.filter; import static org.junit.Assert.assertEquals; @@ -27,10 +25,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -48,6 +46,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -55,6 +54,11 @@ import org.slf4j.LoggerFactory; @Category({FilterTests.class, SmallTests.class}) public class TestDependentColumnFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDependentColumnFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(TestDependentColumnFilter.class); private static final byte[][] ROWS = { Bytes.toBytes("test1"),Bytes.toBytes("test2") diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index ec11ce0fd5f..3c5be6303a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.filter; import static org.junit.Assert.assertEquals; @@ -27,11 +25,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -54,21 +52,27 @@ import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import org.apache.hbase.thirdparty.com.google.common.base.Throwables; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Throwables; + /** * Test filters at the HRegion doorstep. */ @Category({FilterTests.class, SmallTests.class}) public class TestFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFilter.class); + private final static Logger LOG = LoggerFactory.getLogger(TestFilter.class); private HRegion region; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java index ad5ee997cf8..8b8b4313c70 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -21,7 +21,11 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -37,19 +41,20 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - /** * To test behavior of filters at server from region side. */ @Category(SmallTests.class) public class TestFilterFromRegionSide { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFilterFromRegionSide.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static HRegion REGION; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java index 2d2a42590b5..b2fe9d0c9f3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -31,6 +30,7 @@ import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -40,15 +40,21 @@ import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @Category({FilterTests.class, SmallTests.class}) public class TestFilterList { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFilterList.class); + static final int MAX_PAGES = 2; @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java index 59c6de29889..7967e0b5cdb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,6 +17,7 @@ */ package org.apache.hadoop.hbase.filter; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.*; @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -42,6 +43,10 @@ import org.slf4j.LoggerFactory; @Category({ MediumTests.class, FilterTests.class }) public class TestFilterListOnMini { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFilterListOnMini.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFilterListOnMini.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOrOperatorWithBlkCnt.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOrOperatorWithBlkCnt.java index 1f1e919feb6..64263da29c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOrOperatorWithBlkCnt.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOrOperatorWithBlkCnt.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -22,8 +22,8 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -52,6 +53,10 @@ import org.slf4j.LoggerFactory; @Category(MediumTests.class) public class TestFilterListOrOperatorWithBlkCnt { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFilterListOrOperatorWithBlkCnt.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestFilterListOrOperatorWithBlkCnt.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java index d470face389..2a0d1f4b96b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -25,21 +23,27 @@ import java.util.ArrayList; import java.util.LinkedList; import java.util.List; import java.util.TreeSet; - import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hadoop.hbase.testclassification.FilterTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.testclassification.FilterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; + @Category({FilterTests.class, SmallTests.class}) public class TestFilterSerialization { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFilterSerialization.class); + @Test public void testColumnCountGetFilter() throws Exception { ColumnCountGetFilter columnCountGetFilter = new ColumnCountGetFilter(1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java index 29845c960d0..b939f081e6b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java @@ -1,6 +1,4 @@ -/* - * Copyright The Apache Software Foundation - * +/** * 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 @@ -26,9 +24,9 @@ import static org.junit.Assert.assertNull; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -39,6 +37,7 @@ import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -49,6 +48,11 @@ import org.slf4j.LoggerFactory; */ @Category({FilterTests.class, MediumTests.class}) public class TestFilterWithScanLimits extends FilterTestingCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFilterWithScanLimits.class); + private static final Logger LOG = LoggerFactory .getLogger(TestFilterWithScanLimits.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java index 45567769617..85e02cb257a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java @@ -1,6 +1,4 @@ -/* - * Copyright The Apache Software Foundation - * +/** * 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 @@ -19,21 +17,23 @@ */ package org.apache.hadoop.hbase.filter; +import static org.junit.Assert.*; + import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -46,12 +46,10 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; - import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; -import static org.junit.Assert.*; - import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +60,11 @@ import org.slf4j.LoggerFactory; */ @Category({FilterTests.class, MediumTests.class}) public class TestFilterWrapper { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFilterWrapper.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFilterWrapper.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java index dfee4146a2f..95dde3632c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java @@ -19,18 +19,23 @@ package org.apache.hadoop.hbase.filter; import java.util.Set; import java.util.TreeSet; - import junit.framework.TestCase; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @SuppressWarnings("deprecation") @Category({FilterTests.class, SmallTests.class}) public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFirstKeyValueMatchingQualifiersFilter.class); + private static final byte[] ROW = Bytes.toBytes("test"); private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test"); private static final byte[] COLUMN_QUALIFIER_1 = Bytes.toBytes("foo"); @@ -41,7 +46,7 @@ public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase { /** * Test the functionality of * {@link FirstKeyValueMatchingQualifiersFilter#filterCell(org.apache.hadoop.hbase.Cell)} - * + * * @throws Exception */ public void testFirstKeyMatchingQualifierFilter() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java index a5baeab817c..a612290208b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,14 +7,13 @@ * "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 + * 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. + * 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.filter; @@ -24,9 +23,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; @@ -43,19 +42,25 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + /** */ @Category({FilterTests.class, MediumTests.class}) public class TestFuzzyRowAndColumnRangeFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFuzzyRowAndColumnRangeFilter.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestFuzzyRowAndColumnRangeFilter.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java index 0d480885e3e..07548a846e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java @@ -17,17 +17,24 @@ */ package org.apache.hadoop.hbase.filter; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({FilterTests.class, SmallTests.class}) public class TestFuzzyRowFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFuzzyRowFilter.class); + @Test public void testSatisfiesNoUnsafeForward() { @@ -224,7 +231,7 @@ public class TestFuzzyRowFilter { 0, 5, new byte[]{0, 1, 2}, new byte[]{1, 0, 0})); - } + } @Test public void testGetNextForFuzzyRuleForward() { assertNext(false, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java index 269b99c8ad9..cf326311ab0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java @@ -17,9 +17,17 @@ */ package org.apache.hadoop.hbase.filter; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -33,7 +41,6 @@ import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -42,6 +49,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,16 +57,15 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import static org.junit.Assert.assertEquals; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({ FilterTests.class, LargeTests.class }) public class TestFuzzyRowFilterEndToEnd { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFuzzyRowFilterEndToEnd.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static byte fuzzyValue = (byte) 63; private static final Logger LOG = LoggerFactory.getLogger(TestFuzzyRowFilterEndToEnd.class); @@ -171,26 +178,26 @@ public class TestFuzzyRowFilterEndToEnd { Table ht = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), Bytes.toBytes(cf), Integer.MAX_VALUE); // Load data - String[] rows = new String[]{ - "\\x9C\\x00\\x044\\x00\\x00\\x00\\x00", - "\\x9C\\x00\\x044\\x01\\x00\\x00\\x00", - "\\x9C\\x00\\x044\\x00\\x01\\x00\\x00", - "\\x9C\\x00\\x044\\x00\\x00\\x01\\x00", - "\\x9C\\x00\\x044\\x00\\x01\\x00\\x01", - "\\x9B\\x00\\x044e\\xBB\\xB2\\xBB", + String[] rows = new String[] { + "\\x9C\\x00\\x044\\x00\\x00\\x00\\x00", + "\\x9C\\x00\\x044\\x01\\x00\\x00\\x00", + "\\x9C\\x00\\x044\\x00\\x01\\x00\\x00", + "\\x9C\\x00\\x044\\x00\\x00\\x01\\x00", + "\\x9C\\x00\\x044\\x00\\x01\\x00\\x01", + "\\x9B\\x00\\x044e\\xBB\\xB2\\xBB", }; - + String badRow = "\\x9C\\x00\\x03\\xE9e\\xBB{X\\x1Fwts\\x1F\\x15vRX"; - + for(int i=0; i < rows.length; i++){ Put p = new Put(Bytes.toBytesBinary(rows[i])); p.addColumn(cf.getBytes(), cq.getBytes(), "value".getBytes()); - ht.put(p); + ht.put(p); } - + Put p = new Put(Bytes.toBytesBinary(badRow)); p.addColumn(cf.getBytes(), cq.getBytes(), "value".getBytes()); - ht.put(p); + ht.put(p); TEST_UTIL.flush(); @@ -199,19 +206,19 @@ public class TestFuzzyRowFilterEndToEnd { byte[] mask = new byte[] { 1,0,0,0}; data.add(new Pair<>(fuzzyKey, mask)); FuzzyRowFilter filter = new FuzzyRowFilter(data); - + Scan scan = new Scan(); scan.setFilter(filter); - + ResultScanner scanner = ht.getScanner(scan); int total = 0; while(scanner.next() != null){ total++; - } + } assertEquals(rows.length, total); TEST_UTIL.deleteTable(TableName.valueOf(name.getMethodName())); } - + @Test public void testEndToEnd() throws Exception { String cf = "f"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java index fe4e5d16a48..d8c8f2739c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,22 +17,29 @@ */ package org.apache.hadoop.hbase.filter; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; - import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; /** * Tests the inclusive stop row filter */ @Category({FilterTests.class, SmallTests.class}) public class TestInclusiveStopFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestInclusiveStopFilter.class); + private final byte [] STOP_ROW = Bytes.toBytes("stop_row"); private final byte [] GOOD_ROW = Bytes.toBytes("good_row"); private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java index 768ab7a03d2..5184b14f753 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.filter; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -41,6 +39,7 @@ import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +50,10 @@ import org.junit.experimental.categories.Category; @Category({FilterTests.class, SmallTests.class}) public class TestInvocationRecordFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestInvocationRecordFilter.class); + private static final byte[] TABLE_NAME_BYTES = Bytes .toBytes("invocationrecord"); private static final byte[] FAMILY_NAME_BYTES = Bytes.toBytes("mycf"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java index ff9db7572c6..695b9f5c394 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -23,8 +23,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValueUtil; @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -50,6 +51,10 @@ import org.slf4j.LoggerFactory; @Category(MediumTests.class) public class TestMultiRowRangeFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiRowRangeFilter.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestMultiRowRangeFilter.class); private byte[] family = Bytes.toBytes("family"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java index d30cb37d097..753c02c0008 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -26,8 +26,8 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +52,10 @@ import org.junit.rules.TestName; @Category({FilterTests.class, SmallTests.class}) public class TestMultipleColumnPrefixFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultipleColumnPrefixFilter.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -108,7 +113,7 @@ public class TestMultipleColumnPrefixFilter { byte [][] filter_prefix = new byte [2][]; filter_prefix[0] = new byte [] {'p'}; filter_prefix[1] = new byte [] {'q'}; - + filter = new MultipleColumnPrefixFilter(filter_prefix); scan.setFilter(filter); List results = new ArrayList<>(); @@ -156,12 +161,11 @@ public class TestMultipleColumnPrefixFilter { for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) { double rand = Math.random(); Cell kv; - if (rand < 0.5) - kv = KeyValueTestUtil.create(row, family1, column, timestamp, - valueString); - else - kv = KeyValueTestUtil.create(row, family2, column, timestamp, - valueString); + if (rand < 0.5) { + kv = KeyValueTestUtil.create(row, family1, column, timestamp, valueString); + } else { + kv = KeyValueTestUtil.create(row, family2, column, timestamp, valueString); + } p.add(kv); kvList.add(kv); for (String s: prefixMap.keySet()) { @@ -180,7 +184,7 @@ public class TestMultipleColumnPrefixFilter { byte [][] filter_prefix = new byte [2][]; filter_prefix[0] = new byte [] {'p'}; filter_prefix[1] = new byte [] {'q'}; - + filter = new MultipleColumnPrefixFilter(filter_prefix); scan.setFilter(filter); List results = new ArrayList<>(); @@ -191,7 +195,7 @@ public class TestMultipleColumnPrefixFilter { HBaseTestingUtility.closeRegionAndWAL(region); } - + @Test public void testMultipleColumnPrefixFilterWithColumnPrefixFilter() throws IOException { String family = "Family"; @@ -225,30 +229,30 @@ public class TestMultipleColumnPrefixFilter { scan1.setMaxVersions(); byte [][] filter_prefix = new byte [1][]; filter_prefix[0] = new byte [] {'p'}; - + multiplePrefixFilter = new MultipleColumnPrefixFilter(filter_prefix); scan1.setFilter(multiplePrefixFilter); List results1 = new ArrayList<>(); InternalScanner scanner1 = region.getScanner(scan1); while (scanner1.next(results1)) ; - + ColumnPrefixFilter singlePrefixFilter; Scan scan2 = new Scan(); scan2.setMaxVersions(); singlePrefixFilter = new ColumnPrefixFilter(Bytes.toBytes("p")); - + scan2.setFilter(singlePrefixFilter); List results2 = new ArrayList<>(); InternalScanner scanner2 = region.getScanner(scan1); while (scanner2.next(results2)) ; - + assertEquals(results1.size(), results2.size()); HBaseTestingUtility.closeRegionAndWAL(region); } - + List generateRandomWords(int numberOfWords, String suffix) { Set wordSet = new HashSet<>(); for (int i = 0; i < numberOfWords; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestNullComparator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestNullComparator.java index 5aacd3f68dc..5417d874eaa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestNullComparator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestNullComparator.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -6,28 +6,32 @@ * 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. + * + * 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.filter; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({FilterTests.class, SmallTests.class}) public class TestNullComparator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNullComparator.class); + @Test public void testNullValue() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java index 139bf6fa39d..5ab0ff971c9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,9 +22,10 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,6 +34,11 @@ import org.junit.experimental.categories.Category; */ @Category({FilterTests.class, SmallTests.class}) public class TestPageFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPageFilter.class); + static final int ROW_LIMIT = 3; /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java index c5200f98aa5..d30958ccb55 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,13 +25,14 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -44,6 +44,10 @@ import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestParseFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestParseFilter.class); + ParseFilter f; Filter filter; @@ -666,11 +670,11 @@ public class TestParseFilter { String filterString = "(FirstKeyOnlyFilter())"; FirstKeyOnlyFilter firstKeyOnlyFilter = doTestFilter(filterString, FirstKeyOnlyFilter.class); } - + @Test public void testRegisterFilter() { ParseFilter.registerFilter("MyFilter", "some.class"); - + assertTrue(f.getSupportedFilters().contains("MyFilter")); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java index 754bdb60445..89f49bd3650 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,21 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.filter; +import static org.junit.Assert.*; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.*; - @Category({FilterTests.class, SmallTests.class}) public class TestPrefixFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPrefixFilter.class); + Filter mainFilter; static final char FIRST_CHAR = 'a'; static final char LAST_CHAR = 'e'; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java index ca4fa0682f7..35ad88748b2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,21 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.filter; +import static org.junit.Assert.*; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.*; - @Category({FilterTests.class, SmallTests.class}) public class TestRandomRowFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRandomRowFilter.class); + protected RandomRowFilter quarterChanceFilter; @Before @@ -40,7 +45,7 @@ public class TestRandomRowFilter { /** * Tests basics - * + * * @throws Exception */ @Test @@ -62,7 +67,7 @@ public class TestRandomRowFilter { /** * Tests serialization - * + * * @throws Exception */ @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRegexComparator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRegexComparator.java index 9dbe432181d..08863bb6d27 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRegexComparator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRegexComparator.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,17 +20,22 @@ package org.apache.hadoop.hbase.filter; import static org.junit.Assert.*; import java.util.regex.Pattern; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.filter.RegexStringComparator.EngineType; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({FilterTests.class, SmallTests.class}) public class TestRegexComparator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegexComparator.class); + @Test public void testSerialization() throws Exception { // Default engine is the Java engine diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java index 495f63fd89e..e3b78ede593 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java @@ -1,6 +1,4 @@ -/* - * Copyright The Apache Software Foundation - * +/** * 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 @@ -19,7 +17,12 @@ */ package org.apache.hadoop.hbase.filter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.commons.codec.binary.Hex; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -29,6 +32,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -36,16 +40,16 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - /** * Test if Scan.setRowPrefixFilter works as intended. */ @Category({FilterTests.class, MediumTests.class}) public class TestScanRowPrefix extends FilterTestingCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScanRowPrefix.class); + private static final Logger LOG = LoggerFactory .getLogger(TestScanRowPrefix.class); @@ -87,7 +91,7 @@ public class TestScanRowPrefix extends FilterTestingCluster { byte[] prefix0 = {}; List expected0 = new ArrayList<>(16); expected0.addAll(Arrays.asList(rowIds)); // Expect all rows - + byte[] prefix1 = {(byte) 0x12, (byte) 0x23}; List expected1 = new ArrayList<>(16); expected1.add(rowIds[2]); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java index c6b1b5fec36..c365237342a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,22 +17,24 @@ */ package org.apache.hadoop.hbase.filter; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.*; - -import java.util.List; -import java.util.ArrayList; - /** * Tests for {@link SingleColumnValueExcludeFilter}. Because this filter * extends {@link SingleColumnValueFilter}, only the added functionality is @@ -42,6 +43,11 @@ import java.util.ArrayList; */ @Category({FilterTests.class, SmallTests.class}) public class TestSingleColumnValueExcludeFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSingleColumnValueExcludeFilter.class); + private static final byte[] ROW = Bytes.toBytes("test"); private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test"); private static final byte[] COLUMN_QUALIFIER = Bytes.toBytes("foo"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java index c1ef1eeec15..05d8045479c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,15 +24,16 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.regex.Pattern; import org.apache.hadoop.hbase.ByteBufferKeyValue; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -42,6 +42,11 @@ import org.junit.experimental.categories.Category; */ @Category({FilterTests.class, SmallTests.class}) public class TestSingleColumnValueFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSingleColumnValueFilter.class); + private static final byte[] ROW = Bytes.toBytes("test"); private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test"); private static final byte [] COLUMN_QUALIFIER = Bytes.toBytes("foo"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java index 5ba7dfa54ae..59d2229dd6b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.fs; import java.io.FileNotFoundException; @@ -27,7 +26,6 @@ import java.net.BindException; import java.net.ServerSocket; import java.util.List; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; @@ -35,6 +33,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -65,6 +64,7 @@ import org.apache.hadoop.ipc.RemoteException; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -77,6 +77,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, LargeTests.class}) public class TestBlockReorder { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBlockReorder.class); + private static final Logger LOG = LoggerFactory.getLogger(TestBlockReorder.class); private Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferOutputStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferOutputStream.java index f5101c79660..ef627532f8b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferOutputStream.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferOutputStream.java @@ -21,14 +21,20 @@ import static org.junit.Assert.*; import java.io.IOException; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestByteBufferOutputStream { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteBufferOutputStream.class); + @Test public void testByteBufferReuse() throws IOException { byte [] someBytes = Bytes.toBytes("some bytes"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFileLink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFileLink.java index eed19a06d8e..879606807c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFileLink.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFileLink.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io; import static org.junit.Assert.assertEquals; @@ -27,12 +26,12 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.ipc.RemoteException; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -50,6 +50,10 @@ import org.junit.experimental.categories.Category; @Category({IOTests.class, MediumTests.class}) public class TestFileLink { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFileLink.class); + @Test public void testEquals() { Path p1 = new Path("/p1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHFileLink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHFileLink.java index 7747bdb09da..c0448b9934a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHFileLink.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHFileLink.java @@ -15,33 +15,38 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.regex.Matcher; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Pair; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.util.regex.Matcher; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * Test that FileLink switches between alternate locations * when the current location moves or gets deleted. */ @Category({IOTests.class, SmallTests.class}) public class TestHFileLink { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileLink.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java index 57aa8775050..9c110707885 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io; import static org.junit.Assert.assertEquals; @@ -27,13 +25,13 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -47,11 +45,17 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ IOTests.class, SmallTests.class }) public class TestHalfStoreFileReader { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHalfStoreFileReader.class); + private static HBaseTestingUtility TEST_UTIL; @BeforeClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java index b3148c71de8..f979397f453 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io; import static org.junit.Assert.assertEquals; @@ -39,7 +37,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantReadWriteLock; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Mutation; @@ -66,6 +64,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.ClassSize; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -77,6 +76,11 @@ import org.slf4j.LoggerFactory; */ @Category({IOTests.class, SmallTests.class}) public class TestHeapSize { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHeapSize.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHeapSize.class); // List of classes implementing HeapSize // BatchOperation, BatchUpdate, BlockIndex, Entry, Entry, HStoreKey diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java index 5716197c878..ee0e13f9fb0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,22 +15,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io; -import junit.framework.TestCase; - -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.experimental.categories.Category; - import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.io.IOException; +import junit.framework.TestCase; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; @Category({IOTests.class, SmallTests.class}) public class TestImmutableBytesWritable extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestImmutableBytesWritable.class); + public void testHash() throws Exception { assertEquals( new ImmutableBytesWritable(Bytes.toBytes("xxabc"), 2, 3).hashCode(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestMetricsIO.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestMetricsIO.java index 3f4f0ff5a50..c6062044a92 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestMetricsIO.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestMetricsIO.java @@ -15,18 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io; import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestMetricsIO { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsIO.class); + public MetricsAssertHelper HELPER = CompatibilityFactory.getInstance(MetricsAssertHelper.class); @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java index 42539c22666..6be44e9158c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java @@ -38,6 +38,7 @@ import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -47,6 +48,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.ipc.RemoteException; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -63,6 +65,10 @@ import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; @Category({ MiscTests.class, MediumTests.class }) public class TestFanOutOneBlockAsyncDFSOutput { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFanOutOneBlockAsyncDFSOutput.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFanOutOneBlockAsyncDFSOutput.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java index 69090275158..f2f91425a7e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java @@ -19,14 +19,15 @@ package org.apache.hadoop.hbase.io.asyncfs; import java.io.IOException; import java.util.concurrent.ExecutionException; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.AfterClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -38,6 +39,10 @@ import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; @Category({ MiscTests.class, SmallTests.class }) public class TestLocalAsyncOutput { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLocalAsyncOutput.class); + private static EventLoopGroup GROUP = new NioEventLoopGroup(); private static Class CHANNEL_CLASS = NioSocketChannel.class; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSaslFanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSaslFanOutOneBlockAsyncDFSOutput.java index 8a77e8147a2..a221a017c52 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSaslFanOutOneBlockAsyncDFSOutput.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSaslFanOutOneBlockAsyncDFSOutput.java @@ -39,13 +39,13 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.ExecutionException; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.crypto.CipherSuite; import org.apache.hadoop.crypto.key.KeyProvider; import org.apache.hadoop.crypto.key.KeyProviderFactory; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.http.ssl.KeyStoreTestUtil; import org.apache.hadoop.hbase.security.HBaseKerberosUtils; @@ -60,6 +60,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -79,6 +80,10 @@ import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; @Category({ MiscTests.class, LargeTests.class }) public class TestSaslFanOutOneBlockAsyncDFSOutput { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSaslFanOutOneBlockAsyncDFSOutput.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static DistributedFileSystem FS; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSendBufSizePredictor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSendBufSizePredictor.java index 6b5b837d593..55ef0b72b52 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSendBufSizePredictor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSendBufSizePredictor.java @@ -19,14 +19,20 @@ package org.apache.hadoop.hbase.io.asyncfs; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MiscTests.class, SmallTests.class }) public class TestSendBufSizePredictor { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSendBufSizePredictor.class); + @Test public void test() { SendBufSizePredictor p = new SendBufSizePredictor(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java index 82a50c481d4..9c078772af3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.encoding; @@ -23,25 +24,30 @@ import static org.junit.Assert.assertTrue; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.nio.ByteBuffer; - import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; -import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; -import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OnheapDecodedCell; import org.apache.hadoop.hbase.codec.Codec.Decoder; import org.apache.hadoop.hbase.codec.Codec.Encoder; +import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; +import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OnheapDecodedCell; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ObjectIntPair; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({IOTests.class, MediumTests.class}) public class TestBufferedDataBlockEncoder { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBufferedDataBlockEncoder.class); + byte[] row1 = Bytes.toBytes("row1"); byte[] row2 = Bytes.toBytes("row2"); byte[] row_1_0 = Bytes.toBytes("row10"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java index b7b0998f9b0..00f335e7507 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java @@ -1,24 +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 +/** + * 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 + * 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. + * 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.io.encoding; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -39,25 +49,22 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Random; - -import static org.junit.Assert.assertTrue; - /** * Tests changing data block encoding settings of a column family. */ @Category({IOTests.class, LargeTests.class}) public class TestChangingEncoding { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestChangingEncoding.class); + private static final Logger LOG = LoggerFactory.getLogger(TestChangingEncoding.class); static final String CF = "EncodingTestCF"; static final byte[] CF_BYTES = Bytes.toBytes(CF); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java index cbbc9dc9cff..5eb350f09da 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.encoding; @@ -29,17 +30,16 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Random; - import org.apache.hadoop.hbase.ArrayBackedTag; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.io.ByteArrayOutputStream; import org.apache.hadoop.hbase.io.compress.Compression; @@ -50,17 +50,15 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.RedundantKVGenerator; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * Test all of the data block encoding algorithms for correctness. Most of the * class generate data which will test different branches in code. @@ -69,10 +67,11 @@ import org.slf4j.LoggerFactory; @RunWith(Parameterized.class) public class TestDataBlockEncoders { - private static final Logger LOG = LoggerFactory.getLogger(TestDataBlockEncoders.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDataBlockEncoders.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + private static final Logger LOG = LoggerFactory.getLogger(TestDataBlockEncoders.class); private static int NUMBER_OF_KV = 10000; private static int NUM_RANDOM_SEEKS = 1000; @@ -99,7 +98,7 @@ public class TestDataBlockEncoders { this.includesTags = includesTag; this.useOffheapData = useOffheapData; } - + private HFileBlockEncodingContext getEncodingContext(Compression.Algorithm algo, DataBlockEncoding encoding) { DataBlockEncoder encoder = encoding.getEncoder(); @@ -117,7 +116,7 @@ public class TestDataBlockEncoders { /** * Test data block encoding of empty KeyValue. - * + * * @throws IOException * On test failure. */ @@ -144,7 +143,7 @@ public class TestDataBlockEncoders { /** * Test KeyValues with negative timestamp. - * + * * @throws IOException * On test failure. */ @@ -385,7 +384,7 @@ public class TestDataBlockEncoders { testAlgorithm(encodedData, unencodedDataBuf, encoder); } } - + @Test public void testZeroByte() throws IOException { List kvList = new ArrayList<>(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoding.java index 58865c948b8..c5a39f54c9e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoding.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoding.java @@ -1,31 +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 +/** + * 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 + * 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. + * 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.io.encoding; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestDataBlockEncoding { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDataBlockEncoding.class); + @Test public void testGetDataBlockEncoder() throws Exception { for (DataBlockEncoding algo : DataBlockEncoding.values()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java index 5c1baca361d..b757e1cdf90 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.encoding; @@ -24,13 +25,13 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; - +import org.apache.hadoop.hbase.ArrayBackedTag; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -44,8 +45,9 @@ import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Strings; import org.apache.hadoop.hbase.util.LoadTestKVGenerator; +import org.apache.hadoop.hbase.util.Strings; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -59,6 +61,10 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) public class TestEncodedSeekers { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEncodedSeekers.class); + private static final String TABLE_NAME = "encodedSeekersTable"; private static final String CF_NAME = "encodedSeekersCF"; private static final byte[] CF_BYTES = Bytes.toBytes(CF_NAME); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java index d46a5535da0..269fa45a931 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.encoding; @@ -20,22 +21,23 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.RegionLocator; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.TestMiniClusterLoadSequential; import org.apache.hadoop.hbase.util.Threads; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runners.Parameterized.Parameters; @@ -47,6 +49,10 @@ import org.junit.runners.Parameterized.Parameters; public class TestLoadAndSwitchEncodeOnDisk extends TestMiniClusterLoadSequential { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLoadAndSwitchEncodeOnDisk.class); + /** We do not alternate the multi-put flag in this test. */ private static final boolean USE_MULTI_PUT = true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java index d304e74bc86..decbb42eee4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -22,9 +22,9 @@ import static org.junit.Assert.assertEquals; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -41,11 +41,17 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ IOTests.class, SmallTests.class }) public class TestSeekBeforeWithReverseScan { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSeekBeforeWithReverseScan.class); + private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); private HRegion region; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java index 8a9e9ddd9c9..e7b14263398 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.encoding; @@ -23,13 +24,13 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; @@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -46,6 +48,11 @@ import org.junit.runners.Parameterized.Parameters; @Category({IOTests.class, SmallTests.class}) @RunWith(Parameterized.class) public class TestSeekToBlockWithEncoders { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSeekToBlockWithEncoders.class); + static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HConstants.HFILEBLOCK_HEADER_SIZE]; private final boolean useOffheapData; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java index 380e6fbbd67..b8b5e883d26 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java @@ -19,23 +19,23 @@ package org.apache.hadoop.hbase.io.hfile; import static org.junit.Assert.*; +import com.fasterxml.jackson.core.JsonGenerationException; +import com.fasterxml.jackson.databind.JsonMappingException; import java.io.IOException; import java.util.Map; import java.util.NavigableSet; import java.util.Objects; - -import com.fasterxml.jackson.core.JsonGenerationException; -import com.fasterxml.jackson.databind.JsonMappingException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.DataCacheEntry; import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.IndexCacheEntry; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -43,6 +43,11 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, SmallTests.class}) public class TestBlockCacheReporting { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBlockCacheReporting.class); + private static final Logger LOG = LoggerFactory.getLogger(TestBlockCacheReporting.class); private Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java index e07d2aa531d..dd301b6ffc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -27,24 +26,25 @@ import java.io.IOException; import java.lang.management.ManagementFactory; import java.lang.management.MemoryUsage; import java.nio.ByteBuffer; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.io.util.MemorySizeUtil; import org.apache.hadoop.hbase.nio.ByteBuff; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Threads; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -59,6 +59,11 @@ import org.slf4j.LoggerFactory; // tests clash on the global variable if this test is run as small sized test. @Category({IOTests.class, LargeTests.class}) public class TestCacheConfig { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCacheConfig.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCacheConfig.class); private Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java index 6d3d4abfcfa..7244e952c8e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io.hfile; import static org.junit.Assert.assertEquals; @@ -33,12 +31,12 @@ import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -64,6 +62,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -71,6 +70,7 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -81,6 +81,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({IOTests.class, MediumTests.class}) public class TestCacheOnWrite { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCacheOnWrite.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCacheOnWrite.class); private static final HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); @@ -421,7 +425,7 @@ public class TestCacheOnWrite { long ts = EnvironmentEdgeManager.currentTime(); for (int iFile = 0; iFile < 5; ++iFile) { for (int iRow = 0; iRow < 500; ++iRow) { - String rowStr = "" + (rowIdx * rowIdx * rowIdx) + "row" + iFile + "_" + + String rowStr = "" + (rowIdx * rowIdx * rowIdx) + "row" + iFile + "_" + iRow; Put p = new Put(Bytes.toBytes(rowStr)); ++rowIdx; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java index fcb179bb696..baa672a5eae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,15 +18,20 @@ package org.apache.hadoop.hbase.io.hfile; import java.nio.ByteBuffer; - import junit.framework.TestCase; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({IOTests.class, SmallTests.class}) public class TestCachedBlockQueue extends TestCase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCachedBlockQueue.class); + public void testQueue() throws Exception { CachedBlock cb1 = new CachedBlock(1000, "cb1", 1); @@ -62,7 +66,7 @@ public class TestCachedBlockQueue extends TestCase { assertEquals(queue.heapSize(), expectedSize); for (int i = 1; i <= 8; i++) { - assertEquals(queue.pollLast().getCacheKey().getHfileName(), "cb"+i); + assertEquals(queue.pollLast().getCacheKey().getHfileName(), "cb"+i); } } @@ -107,7 +111,7 @@ public class TestCachedBlockQueue extends TestCase { assertEquals(queue.heapSize(), expectedSize); for (int i = 0; i <= 8; i++) { - assertEquals(queue.pollLast().getCacheKey().getHfileName(), "cb"+i); + assertEquals(queue.pollLast().getCacheKey().getHfileName(), "cb"+i); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java index d48c5f3b0d6..dd8ebb35677 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java @@ -1,6 +1,4 @@ -/* - * Copyright The Apache Software Foundation - * +/** * 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 @@ -33,11 +31,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; import java.util.List; - import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.fs.HFileSystem; @@ -48,6 +46,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.ChecksumType; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -55,6 +54,11 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, SmallTests.class}) public class TestChecksum { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestChecksum.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHFileBlock.class); static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = { @@ -178,7 +182,7 @@ public class TestChecksum { } os.close(); - // Use hbase checksums. + // Use hbase checksums. assertEquals(true, hfs.useHBaseChecksum()); // Do a read that purposely introduces checksum verification failures. @@ -193,7 +197,7 @@ public class TestChecksum { HFileBlock b = hbr.readBlockData(0, -1, pread, false); b.sanityCheck(); assertEquals(4936, b.getUncompressedSizeWithoutHeader()); - assertEquals(algo == GZ ? 2173 : 4936, + assertEquals(algo == GZ ? 2173 : 4936, b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes()); // read data back from the hfile, exclude header and checksum ByteBuff bb = b.unpack(meta, hbr).getBufferWithoutHeader(); // read back data @@ -209,7 +213,7 @@ public class TestChecksum { // A single instance of hbase checksum failure causes the reader to // switch off hbase checksum verification for the next 100 read // requests. Verify that this is correct. - for (int i = 0; i < + for (int i = 0; i < HFileBlock.CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD + 1; i++) { b = hbr.readBlockData(0, -1, pread, false); assertEquals(0, HFile.getAndResetChecksumFailuresCount()); @@ -225,9 +229,9 @@ public class TestChecksum { assertEquals(0, HFile.getAndResetChecksumFailuresCount()); is.close(); - // Now, use a completely new reader. Switch off hbase checksums in + // Now, use a completely new reader. Switch off hbase checksums in // the configuration. In this case, we should not detect - // any retries within hbase. + // any retries within hbase. HFileSystem newfs = new HFileSystem(TEST_UTIL.getConfiguration(), false); assertEquals(false, newfs.useHBaseChecksum()); is = new FSDataInputStreamWrapper(newfs, path); @@ -237,7 +241,7 @@ public class TestChecksum { b.sanityCheck(); b = b.unpack(meta, hbr); assertEquals(4936, b.getUncompressedSizeWithoutHeader()); - assertEquals(algo == GZ ? 2173 : 4936, + assertEquals(algo == GZ ? 2173 : 4936, b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes()); // read data back from the hfile, exclude header and checksum bb = b.getBufferWithoutHeader(); // read back data @@ -265,7 +269,7 @@ public class TestChecksum { Compression.Algorithm algo = NONE; for (boolean pread : new boolean[] { false, true }) { for (int bytesPerChecksum : BYTES_PER_CHECKSUM) { - Path path = new Path(TEST_UTIL.getDataTestDir(), "checksumChunk_" + + Path path = new Path(TEST_UTIL.getDataTestDir(), "checksumChunk_" + algo + bytesPerChecksum); FSDataOutputStream os = fs.create(path); HFileContext meta = new HFileContextBuilder() @@ -301,7 +305,7 @@ public class TestChecksum { ", dataSize=" + dataSize + ", expectedChunks=" + expectedChunks); - // Verify hbase checksums. + // Verify hbase checksums. assertEquals(true, hfs.useHBaseChecksum()); // Read data back from file. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java index 8041dff5ecd..f4dc38aeae7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,22 +18,30 @@ package org.apache.hadoop.hbase.io.hfile; import static org.junit.Assert.assertEquals; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache.CombinedCacheStats; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({SmallTests.class}) public class TestCombinedBlockCache { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCombinedBlockCache.class); + @Test public void testCombinedCacheStats() { CacheStats lruCacheStats = new CacheStats("lruCacheStats", 2); CacheStats bucketCacheStats = new CacheStats("bucketCacheStats", 2); CombinedCacheStats stats = new CombinedCacheStats(lruCacheStats, bucketCacheStats); - + double delta = 0.01; - + // period 1: // lru cache: 1 hit caching, 1 miss caching // bucket cache: 2 hit non-caching,1 miss non-caching/primary,1 fail insert @@ -43,7 +50,7 @@ public class TestCombinedBlockCache { bucketCacheStats.hit(false,true, BlockType.DATA); bucketCacheStats.hit(false,true, BlockType.DATA); bucketCacheStats.miss(false, true, BlockType.DATA); - + assertEquals(5, stats.getRequestCount()); assertEquals(2, stats.getRequestCachingCount()); assertEquals(2, stats.getMissCount()); @@ -56,8 +63,8 @@ public class TestCombinedBlockCache { assertEquals(0.5, stats.getHitCachingRatio(), delta); assertEquals(0.4, stats.getMissRatio(), delta); assertEquals(0.5, stats.getMissCachingRatio(), delta); - - + + // lru cache: 2 evicted, 1 evict // bucket cache: 1 evict lruCacheStats.evicted(1000, true); @@ -68,11 +75,11 @@ public class TestCombinedBlockCache { assertEquals(2, stats.getEvictedCount()); assertEquals(1, stats.getPrimaryEvictedCount()); assertEquals(1.0, stats.evictedPerEviction(), delta); - + // lru cache: 1 fail insert lruCacheStats.failInsert(); assertEquals(1, stats.getFailedInserts()); - + // rollMetricsPeriod stats.rollMetricsPeriod(); assertEquals(3, stats.getSumHitCountsPastNPeriods()); @@ -81,7 +88,7 @@ public class TestCombinedBlockCache { assertEquals(2, stats.getSumRequestCachingCountsPastNPeriods()); assertEquals(0.6, stats.getHitRatioPastNPeriods(), delta); assertEquals(0.5, stats.getHitCachingRatioPastNPeriods(), delta); - + // period 2: // lru cache: 3 hit caching lruCacheStats.hit(true, true, BlockType.DATA); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java index 8b2a3af4050..7ffff6186c0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -30,16 +29,17 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; - import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -52,6 +52,10 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, SmallTests.class}) public class TestFixedFileTrailer { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFixedFileTrailer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFixedFileTrailer.class); private static final int MAX_COMPARATOR_NAME_LENGTH = 128; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java index 9983e1d6169..a90b572e38e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.hfile; @@ -22,7 +23,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Arrays; import java.util.Collection; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.client.Get; @@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -46,7 +48,7 @@ import org.junit.runners.Parameterized.Parameters; * Make sure we always cache important block types, such as index blocks, as * long as we have a block cache, even though block caching might be disabled * for the column family. - * + * *

TODO: This test writes a lot of data and only tests the most basic of metrics. Cache stats * need to reveal more about what is being cached whether DATA or INDEX blocks and then we could * do more verification in this test. @@ -54,6 +56,11 @@ import org.junit.runners.Parameterized.Parameters; @Category({IOTests.class, MediumTests.class}) @RunWith(Parameterized.class) public class TestForceCacheImportantBlocks { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestForceCacheImportantBlocks.class); + private final HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); private static final String TABLE = "myTable"; @@ -69,7 +76,7 @@ public class TestForceCacheImportantBlocks { /** Extremely small block size, so that we can get some index blocks */ private static final int BLOCK_SIZE = 256; - + private static final Algorithm COMPRESSION_ALGORITHM = Compression.Algorithm.GZ; private static final BloomType BLOOM_TYPE = BloomType.ROW; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java index 30501e2e08b..7053fce1358 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -31,7 +30,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Objects; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -42,13 +40,14 @@ import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFile.Reader; @@ -60,6 +59,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Writable; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -73,6 +73,10 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, SmallTests.class}) public class TestHFile { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFile.class); + @Rule public TestName testName = new TestName(); private static final Logger LOG = LoggerFactory.getLogger(TestHFile.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java index a8b7d1f81fa..829c4b84756 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -39,18 +39,18 @@ import java.util.concurrent.Executor; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.Executors; import java.util.concurrent.Future; - import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.compress.Compressor; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -79,6 +80,11 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, MediumTests.class}) @RunWith(Parameterized.class) public class TestHFileBlock { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileBlock.class); + // change this value to activate more logs private static final boolean detailedLogging = false; private static final boolean[] BOOLEAN_VALUES = new boolean[] { false, true }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java index ad42a66ed78..31407a35712 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io.hfile; import static org.junit.Assert.assertEquals; @@ -34,7 +32,6 @@ import java.util.HashSet; import java.util.List; import java.util.Random; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -42,11 +39,12 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -61,6 +59,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -73,6 +72,10 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, MediumTests.class}) public class TestHFileBlockIndex { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileBlockIndex.class); + @Parameters public static Collection compressionAlgorithms() { return HBaseCommonTestingUtility.COMPRESSION_ALGORITHMS_PARAMETERIZED; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockPositionalRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockPositionalRead.java index a4f233831d1..a13c868d1bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockPositionalRead.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockPositionalRead.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -21,10 +21,11 @@ import static org.junit.Assert.*; import static org.mockito.Mockito.*; import java.io.IOException; - import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -36,6 +37,10 @@ import org.junit.rules.ExpectedException; @Category({IOTests.class, SmallTests.class}) public class TestHFileBlockPositionalRead { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileBlockPositionalRead.class); + @Rule public ExpectedException exception = ExpectedException.none(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java index 2d233766678..b9ff7b2a29b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.hfile; @@ -26,8 +27,8 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.List; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.ByteArrayOutputStream; @@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.RedundantKVGenerator; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -49,6 +51,11 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) @Category({IOTests.class, SmallTests.class}) public class TestHFileDataBlockEncoder { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileDataBlockEncoder.class); + private HFileDataBlockEncoder blockEncoder; private RedundantKVGenerator generator = new RedundantKVGenerator(); private boolean includesMemstoreTS; @@ -192,7 +199,7 @@ public class TestHFileDataBlockEncoder { .withChecksumType(ChecksumType.NULL) .build(); HFileBlock b = new HFileBlock(BlockType.DATA, size, size, -1, buf, - HFileBlock.FILL_HEADER, 0, + HFileBlock.FILL_HEADER, 0, 0, -1, meta); return b; } @@ -243,7 +250,7 @@ public class TestHFileDataBlockEncoder { for (DataBlockEncoding diskAlgo : DataBlockEncoding.values()) { for (boolean includesMemstoreTS : new boolean[] { false, true }) { - HFileDataBlockEncoder dbe = (diskAlgo == DataBlockEncoding.NONE) ? + HFileDataBlockEncoder dbe = (diskAlgo == DataBlockEncoding.NONE) ? NoOpDataBlockEncoder.INSTANCE : new HFileDataBlockEncoderImpl(diskAlgo); configurations.add(new Object[] { dbe, new Boolean(includesMemstoreTS) }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java index a049b329d6c..134bb62a1f7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -29,13 +29,13 @@ import java.io.IOException; import java.security.SecureRandom; import java.util.List; import java.util.UUID; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.RedundantKVGenerator; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -57,6 +58,11 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, SmallTests.class}) public class TestHFileEncryption { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileEncryption.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHFileEncryption.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final SecureRandom RNG = new SecureRandom(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java index d3a890c57ef..8102f111e94 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java @@ -1,32 +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 +/** + * 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 + * 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. + * 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.io.hfile; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -39,13 +41,18 @@ import org.junit.experimental.categories.Category; * the configured chunk size, and split it into a number of intermediate index blocks that should * really be leaf-level blocks. If more keys were added, we would flush the leaf-level block, add * another entry to the root-level block, and that would prevent us from upgrading the leaf-level - * chunk to the root chunk, thus not triggering the bug. + * chunk to the root chunk, thus not triggering the bug. */ @Category({IOTests.class, SmallTests.class}) public class TestHFileInlineToRootChunkConversion { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileInlineToRootChunkConversion.class); + private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); private final Configuration conf = testUtil.getConfiguration(); - + @Test public void testWriteHFile() throws Exception { Path hfPath = new Path(testUtil.getDataTestDir(), @@ -53,7 +60,7 @@ public class TestHFileInlineToRootChunkConversion { int maxChunkSize = 1024; FileSystem fs = FileSystem.get(conf); CacheConfig cacheConf = new CacheConfig(conf); - conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, maxChunkSize); + conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, maxChunkSize); HFileContext context = new HFileContextBuilder().withBlockSize(16).build(); HFile.Writer hfw = new HFile.WriterFactory(conf, cacheConf) .withFileContext(context) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java index 710fe43fe83..6116ba620ce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java @@ -1,29 +1,27 @@ /** - * 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 + * 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 + * 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. + * 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.io.hfile; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Random; import java.util.StringTokenizer; - import junit.framework.TestCase; - import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.GnuParser; @@ -38,15 +36,17 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.io.hfile.HFile.Reader; import org.apache.hadoop.hbase.io.hfile.HFile.Writer; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.io.BytesWritable; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,6 +61,11 @@ import org.slf4j.LoggerFactory; */ @Category({IOTests.class, MediumTests.class}) public class TestHFileSeek extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileSeek.class); + private static final byte[] CF = "f1".getBytes(); private static final byte[] QUAL = "q1".getBytes(); private static final boolean USE_PREAD = true; @@ -82,11 +87,11 @@ public class TestHFileSeek extends TestCase { } conf = new Configuration(); - + if (options.useRawFs) { conf.setClass("fs.file.impl", RawLocalFileSystem.class, FileSystem.class); } - + conf.setInt("tfile.fs.input.buffer.size", options.fsInputBufferSize); conf.setInt("tfile.fs.output.buffer.size", options.fsOutputBufferSize); path = new Path(new Path(options.rootDir), options.file); @@ -378,9 +383,9 @@ public class TestHFileSeek extends TestCase { .withDescription( "specify how many seek operations we perform (requires -x r or -x rw.") .create('n'); - + Option trialCount = - OptionBuilder + OptionBuilder .withLongOpt("trials") .withArgName("n") .hasArg() @@ -393,7 +398,7 @@ public class TestHFileSeek extends TestCase { .withLongOpt("rawfs") .withDescription("use raw instead of checksummed file system") .create(); - + Option help = OptionBuilder.withLongOpt("help").hasArg(false).withDescription( "show this screen").create("h"); @@ -442,7 +447,7 @@ public class TestHFileSeek extends TestCase { if (line.hasOption('n')) { seekCount = Integer.parseInt(line.getOptionValue('n')); } - + if (line.hasOption('t')) { trialCount = Integer.parseInt(line.getOptionValue('t')); } @@ -490,7 +495,7 @@ public class TestHFileSeek extends TestCase { throw new ParseException("Unknown action specifier: " + strOp); } } - + useRawFs = line.hasOption("rawfs"); proceed = true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java index 5dade74a7b5..0a1af8729e9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io.hfile; import static org.junit.Assert.assertEquals; @@ -30,20 +28,20 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; @@ -55,6 +53,7 @@ import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableUtils; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -70,6 +69,10 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, SmallTests.class}) public class TestHFileWriterV3 { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileWriterV3.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHFileWriterV3.class); private static final HBaseTestingUtility TEST_UTIL = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java index f34f32684f2..4542a3cbcaf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java @@ -27,10 +27,10 @@ import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -58,6 +59,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; @Category({IOTests.class, SmallTests.class}) @RunWith(Parameterized.class) public class TestLazyDataBlockDecompression { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLazyDataBlockDecompression.class); + private static final Logger LOG = LoggerFactory.getLogger(TestLazyDataBlockDecompression.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java index 8429ee8566a..fab19a405a5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -29,16 +28,17 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.LruBlockCache.EvictionThread; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.ClassSize; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -52,6 +52,10 @@ import org.junit.experimental.categories.Category; @Category({IOTests.class, SmallTests.class}) public class TestLruBlockCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLruBlockCache.class); + @Test public void testCacheEvictionThreadSafe() throws Exception { long maxSize = 100000; @@ -195,7 +199,7 @@ public class TestLruBlockCache { assertEquals( "Cache should ignore cache requests for blocks already in cache", expectedBlockCount, cache.getBlockCount()); - + // Verify correctly calculated cache heap size assertEquals(expectedCacheSize, cache.heapSize()); @@ -885,7 +889,7 @@ public class TestLruBlockCache { @Override public void serialize(ByteBuffer destination) { } - + @Override public BlockType getBlockType() { return BlockType.DATA; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruCachedBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruCachedBlock.java index 141c95b683a..538e93dca48 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruCachedBlock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruCachedBlock.java @@ -20,9 +20,11 @@ package org.apache.hadoop.hbase.io.hfile; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -30,6 +32,10 @@ import org.mockito.Mockito; @Category({IOTests.class, SmallTests.class}) public class TestLruCachedBlock { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLruCachedBlock.class); + LruCachedBlock block; LruCachedBlock blockEqual; LruCachedBlock blockNotEqual; @@ -38,10 +44,10 @@ public class TestLruCachedBlock { public void setUp() throws Exception { BlockCacheKey cacheKey = new BlockCacheKey("name", 0); BlockCacheKey otherKey = new BlockCacheKey("name2", 1); - + Cacheable cacheable = Mockito.mock(Cacheable.class); Cacheable otheCacheable = Mockito.mock(Cacheable.class); - + block = new LruCachedBlock(cacheKey, cacheable, 0); blockEqual = new LruCachedBlock(otherKey, otheCacheable, 0); blockNotEqual = new LruCachedBlock(cacheKey, cacheable, 1); @@ -51,7 +57,7 @@ public class TestLruCachedBlock { public void testEquality() { assertEquals(block.hashCode(), blockEqual.hashCode()); assertNotEquals(block.hashCode(), blockNotEqual.hashCode()); - + assertEquals(block, blockEqual); assertNotEquals(block, blockNotEqual); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java index 70d283e0da8..de55afa200c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -21,28 +21,32 @@ import static org.junit.Assert.*; import java.io.IOException; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparatorImpl; -import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.fs.HFileSystem; - import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({IOTests.class, SmallTests.class}) public class TestPrefetch { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPrefetch.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final int NUM_VALID_KEY_TYPES = KeyValue.Type.values().length - 2; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java index 8a20af70750..dc2551822ba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,18 +22,19 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -44,6 +44,10 @@ import org.junit.experimental.categories.Category; @Category({IOTests.class, SmallTests.class}) public class TestReseekTo { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReseekTo.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java index 106a8795446..06d617aad4e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,16 +24,16 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ByteBufferKeyValue; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; @@ -50,6 +49,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -59,6 +59,11 @@ import org.slf4j.LoggerFactory; @Category({ RegionServerTests.class, MediumTests.class }) public class TestScannerFromBucketCache { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerFromBucketCache.class); + private static final Logger LOG = LoggerFactory.getLogger(TestScannerFromBucketCache.class); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java index b1ae85513f2..c576329c11a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.hfile; @@ -25,9 +26,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -43,6 +44,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -55,6 +57,11 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) @Category({IOTests.class, SmallTests.class}) public class TestScannerSelectionUsingKeyRange { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerSelectionUsingKeyRange.class); + private static final HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); private static TableName TABLE = TableName.valueOf("myTable"); private static String FAMILY = "myCF"; @@ -85,7 +92,7 @@ public class TestScannerSelectionUsingKeyRange { bloomType = (BloomType)type; expectedCount = (Integer) count; } - + @AfterClass public static void tearDownAfterClass() throws Exception { TEST_UTIL.cleanupTestDir(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java index 5e88b14f033..08a7be2cf2b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.io.hfile; @@ -23,9 +24,9 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -41,6 +42,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Threads; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -57,6 +59,10 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, MediumTests.class}) public class TestScannerSelectionUsingTTL { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerSelectionUsingTTL.class); + private static final Logger LOG = LoggerFactory.getLogger(TestScannerSelectionUsingTTL.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java index 651cf020372..f6b6fbc231f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.io.hfile; import static org.junit.Assert.assertEquals; @@ -24,12 +22,12 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -40,6 +38,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -48,6 +47,10 @@ import org.slf4j.LoggerFactory; @Category({IOTests.class, MediumTests.class}) public class TestSeekBeforeWithInlineBlocks { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSeekBeforeWithInlineBlocks.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSeekBeforeWithInlineBlocks.class); private static final HBaseTestingUtility TEST_UTIL = @@ -67,80 +70,80 @@ public class TestSeekBeforeWithInlineBlocks { private Configuration conf; /** - * Scanner.seekBefore() could fail because when seeking to a previous HFile data block, it needs - * to know the size of that data block, which it calculates using current data block offset and - * the previous data block offset. This fails to work when there are leaf-level index blocks in - * the scannable section of the HFile, i.e. starting in HFileV2. This test will try seekBefore() + * Scanner.seekBefore() could fail because when seeking to a previous HFile data block, it needs + * to know the size of that data block, which it calculates using current data block offset and + * the previous data block offset. This fails to work when there are leaf-level index blocks in + * the scannable section of the HFile, i.e. starting in HFileV2. This test will try seekBefore() * on a flat (single-level) and multi-level (2,3) HFile and confirm this bug is now fixed. This * bug also happens for inline Bloom blocks for the same reasons. */ @Test public void testMultiIndexLevelRandomHFileWithBlooms() throws IOException { conf = TEST_UTIL.getConfiguration(); - + // Try out different HFile versions to ensure reverse scan works on each version - for (int hfileVersion = HFile.MIN_FORMAT_VERSION_WITH_TAGS; + for (int hfileVersion = HFile.MIN_FORMAT_VERSION_WITH_TAGS; hfileVersion <= HFile.MAX_FORMAT_VERSION; hfileVersion++) { conf.setInt(HFile.FORMAT_VERSION_KEY, hfileVersion); fs = HFileSystem.get(conf); - - // Try out different bloom types because inline Bloom blocks break seekBefore() + + // Try out different bloom types because inline Bloom blocks break seekBefore() for (BloomType bloomType : BloomType.values()) { - + // Test out HFile block indices of various sizes/levels for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; testI++) { int indexBlockSize = INDEX_CHUNK_SIZES[testI]; int expectedNumLevels = EXPECTED_NUM_LEVELS[testI]; - - LOG.info(String.format("Testing HFileVersion: %s, BloomType: %s, Index Levels: %s", + + LOG.info(String.format("Testing HFileVersion: %s, BloomType: %s, Index Levels: %s", hfileVersion, bloomType, expectedNumLevels)); - + conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, indexBlockSize); conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, BLOOM_BLOCK_SIZE); - + Cell[] cells = new Cell[NUM_KV]; Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), - String.format("testMultiIndexLevelRandomHFileWithBlooms-%s-%s-%s", + String.format("testMultiIndexLevelRandomHFileWithBlooms-%s-%s-%s", hfileVersion, bloomType, testI)); - + // Disable caching to prevent it from hiding any bugs in block seeks/reads conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f); CacheConfig cacheConf = new CacheConfig(conf); - + // Write the HFile { HFileContext meta = new HFileContextBuilder() .withBlockSize(DATA_BLOCK_SIZE) .build(); - + StoreFileWriter storeFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs) .withFilePath(hfilePath) .withFileContext(meta) .withBloomType(bloomType) .build(); - + for (int i = 0; i < NUM_KV; i++) { byte[] row = RandomKeyValueUtil.randomOrderedKey(RAND, i); byte[] qual = RandomKeyValueUtil.randomRowOrQualifier(RAND); byte[] value = RandomKeyValueUtil.randomValue(RAND); KeyValue kv = new KeyValue(row, FAM, qual, value); - + storeFileWriter.append(kv); cells[i] = kv; } - + storeFileWriter.close(); } - + // Read the HFile HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, true, conf); - + // Sanity check the HFile index level assertEquals(expectedNumLevels, reader.getTrailer().getNumDataIndexLevels()); - + // Check that we can seekBefore in either direction and with both pread // enabled and disabled for (boolean pread : new boolean[] { false, true }) { @@ -158,31 +161,31 @@ public class TestSeekBeforeWithInlineBlocks { checkNoSeekBefore(cells, scanner, 0); scanner.close(); } - + reader.close(); - } + } } } } - + private void checkSeekBefore(Cell[] cells, HFileScanner scanner, int i) throws IOException { assertEquals("Failed to seek to the key before #" + i + " (" - + CellUtil.getCellKeyAsString(cells[i]) + ")", true, + + CellUtil.getCellKeyAsString(cells[i]) + ")", true, scanner.seekBefore(cells[i])); } private void checkNoSeekBefore(Cell[] cells, HFileScanner scanner, int i) throws IOException { assertEquals("Incorrectly succeeded in seeking to before first key (" - + CellUtil.getCellKeyAsString(cells[i]) + ")", false, + + CellUtil.getCellKeyAsString(cells[i]) + ")", false, scanner.seekBefore(cells[i])); } /** Check a key/value pair after it was read by the reader */ private void checkCell(Cell expected, Cell actual) { - assertTrue(String.format("Expected key %s, but was %s", - CellUtil.getCellKeyAsString(expected), CellUtil.getCellKeyAsString(actual)), + assertTrue(String.format("Expected key %s, but was %s", + CellUtil.getCellKeyAsString(expected), CellUtil.getCellKeyAsString(actual)), CellUtil.equals(expected, actual)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java index b08c6ad208c..b6e4a3e71c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -27,7 +26,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -36,6 +34,7 @@ import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.ByteBufferKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -46,6 +45,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -59,6 +59,10 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) public class TestSeekTo { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSeekTo.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final DataBlockEncoding encoding; @Parameters diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java index 3873a6ca169..3c459584d4a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java @@ -1,20 +1,19 @@ /** - * Copyright The Apache Software Foundation + * 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 * - * 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 + * 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. + * 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.io.hfile.bucket; @@ -33,6 +32,7 @@ import java.util.Set; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -61,6 +62,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; @Category({ IOTests.class, SmallTests.class }) public class TestBucketCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBucketCache.class); + private static final Random RAND = new Random(); @Parameterized.Parameters(name = "{index}: blockSize={0}, bucketSizes={1}") diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java index b8949edb4f5..faf259fcd47 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java @@ -1,35 +1,26 @@ /** - * Copyright The Apache Software Foundation + * 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 * - * 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 + * 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. + * 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.io.hfile.bucket; -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; -import org.apache.hadoop.hbase.io.hfile.Cacheable; -import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BucketEntry; -import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.RAMQueueEntry; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import java.io.FileNotFoundException; import java.io.IOException; @@ -37,14 +28,28 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.LongAdder; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; +import org.apache.hadoop.hbase.io.hfile.Cacheable; +import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BucketEntry; +import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.RAMQueueEntry; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.Mockito; @Category({IOTests.class, SmallTests.class}) public class TestBucketWriterThread { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBucketWriterThread.class); + private BucketCache bc; private BucketCache.WriterThread wt; private BlockingQueue q; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java index ab2276a884b..bb58b4e9d30 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java @@ -1,20 +1,19 @@ /** - * Copyright The Apache Software Foundation + * 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 * - * 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 + * 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. + * 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.io.hfile.bucket; @@ -22,13 +21,14 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.hfile.Cacheable; import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType; import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -38,6 +38,10 @@ import org.junit.experimental.categories.Category; @Category({IOTests.class, SmallTests.class}) public class TestByteBufferIOEngine { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteBufferIOEngine.class); + @Test public void testByteBufferIOEngine() throws Exception { int capacity = 32 * 1024 * 1024; // 32 MB diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java index 1bcc02630f6..508626539d2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java @@ -1,20 +1,19 @@ /** - * Copyright The Apache Software Foundation + * 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 * - * 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 + * 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. + * 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.io.hfile.bucket; @@ -25,13 +24,14 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.hfile.bucket.TestByteBufferIOEngine.BufferGrabbingDeserializer; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,6 +41,10 @@ import org.junit.experimental.categories.Category; @Category({IOTests.class, SmallTests.class}) public class TestFileIOEngine { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFileIOEngine.class); + private static final long TOTAL_CAPACITY = 6 * 1024 * 1024; // 6 MB private static final String[] FILE_PATHS = {"testFileIOEngine1", "testFileIOEngine2", "testFileIOEngine3"}; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java index 85bd4a2de77..2748d80bdba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java @@ -1,20 +1,19 @@ /** - * Copyright The Apache Software Foundation + * 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 * - * 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 + * 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. + * 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.io.hfile.bucket; @@ -23,11 +22,12 @@ import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.hfile.bucket.TestByteBufferIOEngine.BufferGrabbingDeserializer; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -36,6 +36,11 @@ import org.junit.experimental.categories.Category; */ @Category({IOTests.class, SmallTests.class}) public class TestFileMmapEngine { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFileMmapEngine.class); + @Test public void testFileMmapEngine() throws IOException { int size = 2 * 1024 * 1024; // 2 MB diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java index 8fbdafd2d36..b9844a2930e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,19 +21,24 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.nio.channels.SocketChannel; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ RPCTests.class, SmallTests.class }) public class TestBlockingIPC extends AbstractTestIPC { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBlockingIPC.class); + @Override protected RpcServer createRpcServer(Server server, String name, List services, InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBufferChain.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBufferChain.java index 5b9b27f5bf1..1280872a6db 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBufferChain.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBufferChain.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -24,12 +24,13 @@ import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -39,6 +40,11 @@ import org.apache.hbase.thirdparty.com.google.common.io.Files; @Category({RPCTests.class, SmallTests.class}) public class TestBufferChain { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBufferChain.class); + private File tmpFile; private static final byte[][] HELLO_WORLD_CHUNKS = new byte[][] { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestCallRunner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestCallRunner.java index 5c8384c6dd1..e614c2b9cd4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestCallRunner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestCallRunner.java @@ -17,15 +17,22 @@ */ package org.apache.hadoop.hbase.ipc; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @Category({RPCTests.class, SmallTests.class}) public class TestCallRunner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCallRunner.class); + /** * Does nothing but exercise a {@link CallRunner} outside of {@link RpcServer} context. */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestFifoRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestFifoRpcScheduler.java index 38caee404ff..896a830d4d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestFifoRpcScheduler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestFifoRpcScheduler.java @@ -17,40 +17,38 @@ */ package org.apache.hadoop.hbase.ipc; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.RPCTests; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.lang.reflect.Field; -import java.net.InetSocketAddress; -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicInteger; - import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import java.io.IOException; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Category({RPCTests.class, LargeTests.class}) public class TestFifoRpcScheduler { - @Rule - public final TestRule timeout = - CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFifoRpcScheduler.class); private static final Logger LOG = LoggerFactory.getLogger(TestFifoRpcScheduler.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseClient.java index b2bccd6e7da..1496b8401c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseClient.java @@ -15,23 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.ipc; +import java.net.InetSocketAddress; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.net.InetSocketAddress; - @Category({RPCTests.class, MediumTests.class}) // Can't be small, we're playing with the EnvironmentEdge public class TestHBaseClient { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseClient.class); + @Test public void testFailedServer(){ ManualEnvironmentEdge ee = new ManualEnvironmentEdge(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java index 0a168ba2c82..b4536b6dee3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java @@ -22,9 +22,9 @@ import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collection; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.nio.ByteBuff; @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.JVM; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -45,11 +46,14 @@ import org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollSocketChannel; import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; - @RunWith(Parameterized.class) @Category({ RPCTests.class, SmallTests.class }) public class TestNettyIPC extends AbstractTestIPC { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNettyIPC.class); + @Parameters(name = "{index}: EventLoop={0}") public static Collection parameters() { List params = new ArrayList<>(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java index 81be74d4a05..f6b4613b3e8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +44,11 @@ import org.junit.rules.TestName; @Category({ RPCTests.class, SmallTests.class }) public class TestNettyRpcServer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNettyRpcServer.class); + @Rule public TestName name = new TestName(); private static HBaseTestingUtility TEST_UTIL; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java index f6f6fc539e6..e0aa4803079 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java @@ -26,20 +26,17 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.Arrays; import java.util.Collection; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -49,6 +46,11 @@ import org.junit.runners.Parameterized.Parameters; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; + /** * Test for testing protocol buffer based RPC mechanism. This test depends on test.proto definition * of types in src/test/protobuf/test.proto and protobuf service definition from @@ -57,6 +59,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @RunWith(Parameterized.class) @Category({ RPCTests.class, MediumTests.class }) public class TestProtoBufRpc { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProtoBufRpc.class); + public final static String ADDRESS = "localhost"; public static int PORT = 0; private InetSocketAddress isa; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java index 9e70c93d6e2..4922fd74fe9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java @@ -6,9 +6,9 @@ * 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 - *

+ * + * 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. @@ -24,9 +24,8 @@ import java.io.IOException; import java.net.Socket; import java.net.SocketAddress; import java.util.List; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -40,12 +39,12 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,8 +52,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category(MediumTests.class) public class TestRpcClientLeaks { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRpcClientLeaks.class); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java index 137fc00ec1e..cddb647fb12 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java @@ -25,16 +25,14 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.Arrays; import java.util.Collection; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; -import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -44,11 +42,19 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; + +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; @RunWith(Parameterized.class) @Category({ RPCTests.class, SmallTests.class }) public class TestRpcHandlerException { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRpcHandlerException.class); + private final static Configuration CONF = HBaseConfiguration.create(); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcMetrics.java index cfcfb21e6f4..74827b99a69 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcMetrics.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -8,7 +7,7 @@ * "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 + * 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, @@ -16,26 +15,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.ipc; +import static org.junit.Assert.*; + import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException; import org.apache.hadoop.hbase.exceptions.RegionMovedException; +import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.test.MetricsAssertHelper; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import static org.junit.Assert.*; - @Category({RPCTests.class, SmallTests.class}) public class TestRpcMetrics { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRpcMetrics.class); + public MetricsAssertHelper HELPER = CompatibilityFactory.getInstance(MetricsAssertHelper.class); @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServer.java index 0f10c49f8aa..560190b0e34 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServer.java @@ -24,7 +24,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.ByteBufferPool; import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup; import org.apache.hadoop.hbase.nio.ByteBuff; @@ -33,12 +33,17 @@ import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Pair; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ RPCTests.class, SmallTests.class }) public class TestRpcServer { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRpcServer.class); + @Test public void testAllocateByteBuffToReadInto() throws Exception { int maxBuffersInPool = 10; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerSlowConnectionSetup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerSlowConnectionSetup.java index e05db33b731..aedf57e72f0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerSlowConnectionSetup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerSlowConnectionSetup.java @@ -28,25 +28,18 @@ import java.net.InetSocketAddress; import java.net.Socket; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.security.AuthMethod; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -56,10 +49,23 @@ import org.junit.runners.Parameterized.Parameters; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader; + @RunWith(Parameterized.class) @Category({ RPCTests.class, MediumTests.class }) public class TestRpcServerSlowConnectionSetup { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRpcServerSlowConnectionSetup.class); + private RpcServer server; private Socket socket; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java index e646c147145..875223dc9df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java @@ -21,19 +21,14 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; -import org.apache.hbase.thirdparty.com.google.common.collect.Maps; - import java.io.IOException; import java.lang.reflect.Field; import java.net.InetSocketAddress; @@ -45,17 +40,12 @@ import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -63,21 +53,30 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Threads; import org.junit.Before; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hbase.thirdparty.com.google.common.collect.Maps; + +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; + @Category({RPCTests.class, SmallTests.class}) public class TestSimpleRpcScheduler { - @Rule - public final TestRule timeout = - CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSimpleRpcScheduler.class); private static final Logger LOG = LoggerFactory.getLogger(TestSimpleRpcScheduler.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java index d813dfb5114..9f68ea59c8f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -24,11 +23,11 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.concurrent.Semaphore; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; @@ -46,6 +45,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -57,6 +57,11 @@ import org.slf4j.LoggerFactory; */ @Category({MasterTests.class, MediumTests.class}) public class TestActiveMasterManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestActiveMasterManager.class); + private final static Logger LOG = LoggerFactory.getLogger(TestActiveMasterManager.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java index 53e80f3dba6..2ebab10821d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,9 +24,9 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -52,6 +51,7 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -62,6 +62,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestAssignmentListener { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAssignmentListener.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAssignmentListener.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java index 3c2b26b8e05..83e350ce69c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java @@ -17,13 +17,15 @@ */ package org.apache.hadoop.hbase.master; -import java.io.IOException; +import static org.junit.Assert.fail; +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -34,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -42,12 +45,14 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.fail; - @Ignore // Disabled temporarily; reenable @Category(MediumTests.class) public class TestAssignmentManagerMetrics { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAssignmentManagerMetrics.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAssignmentManagerMetrics.class); private static final MetricsAssertHelper metricsHelper = CompatibilityFactory .getInstance(MetricsAssertHelper.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java index da501006d1c..b665f327a09 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -33,12 +32,11 @@ import java.util.SortedMap; import java.util.SortedSet; import java.util.TreeMap; import java.util.concurrent.ConcurrentSkipListMap; - import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -65,19 +63,22 @@ import org.apache.hadoop.hbase.util.Triple; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MasterTests.class, SmallTests.class}) public class TestCatalogJanitor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCatalogJanitor.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCatalogJanitor.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); @Rule public final TestName name = new TestName(); private static final HBaseTestingUtility HTU = new HBaseTestingUtility(); private MockMasterServices masterServices; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java index c42d042a43f..5567aba7042 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,9 +24,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaMockingUtil; @@ -49,19 +47,22 @@ import org.apache.hadoop.hbase.util.PairOfSameType; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestCatalogJanitorInMemoryStates { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCatalogJanitorInMemoryStates.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCatalogJanitorInMemoryStates.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); @Rule public final TestName name = new TestName(); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte [] ROW = Bytes.toBytes("testRow"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java index 77f038af066..3df4b3fbcae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,22 +22,29 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.net.InetAddress; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClockOutOfSyncException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; + @Category({MasterTests.class, SmallTests.class}) public class TestClockSkewDetection { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClockSkewDetection.class); + private static final Logger LOG = LoggerFactory.getLogger(TestClockSkewDetection.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterStatusPublisher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterStatusPublisher.java index 68cab5a0ac4..44c6d8e988c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterStatusPublisher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterStatusPublisher.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master; - +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -28,14 +28,17 @@ import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.apache.hadoop.hbase.util.Pair; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.util.ArrayList; -import java.util.List; - @Category({MasterTests.class, MediumTests.class}) // Plays with the ManualEnvironmentEdge public class TestClusterStatusPublisher { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClusterStatusPublisher.class); + private ManualEnvironmentEdge mee = new ManualEnvironmentEdge(); @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java index c4a2fcea1e6..545308a706b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hbase.master; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.experimental.categories.Category; @@ -26,6 +28,10 @@ import org.junit.experimental.categories.Category; @Category({ MasterTests.class, LargeTests.class }) public class TestDLSAsyncFSWAL extends AbstractTestDLS { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDLSAsyncFSWAL.class); + @Override protected String getWalProvider() { return "asyncfs"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java index a7329623a4d..5a661f592c7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hbase.master; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.experimental.categories.Category; @@ -26,6 +28,10 @@ import org.junit.experimental.categories.Category; @Category({ MasterTests.class, LargeTests.class }) public class TestDLSFSHLog extends AbstractTestDLS { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDLSFSHLog.class); + @Override protected String getWalProvider() { return "filesystem"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java index d5f238d9771..d5e657cc44f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java @@ -17,6 +17,12 @@ */ package org.apache.hadoop.hbase.master; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import java.util.Set; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; @@ -31,17 +37,17 @@ import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.util.List; -import java.util.Set; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - @Category({MasterTests.class, MediumTests.class}) public class TestDeadServer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDeadServer.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); final ServerName hostname123 = ServerName.valueOf("127.0.0.1", 123, 3L); @@ -105,7 +111,7 @@ public class TestDeadServer { pExecutor.getEnvironment(), hostname123, false, false); ProcedureTestingUtility.submitAndWait(pExecutor, proc); - + assertFalse(master.getServerManager().getDeadServers().areDeadServersInProgress()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetInfoPort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetInfoPort.java index 418bddc8f0c..ddf2e83596b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetInfoPort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetInfoPort.java @@ -19,12 +19,14 @@ package org.apache.hadoop.hbase.master; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,6 +36,10 @@ import org.junit.experimental.categories.Category; @Category({ MasterTests.class, MediumTests.class }) public class TestGetInfoPort { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGetInfoPort.class); + private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java index 01505b87ca7..c930a1b1e6d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java @@ -23,7 +23,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -39,9 +38,12 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; + /** * Trivial test to confirm that we can get last flushed sequence id by encodedRegionName. See * HBASE-12715. @@ -49,6 +51,10 @@ import org.junit.experimental.categories.Category; @Category(MediumTests.class) public class TestGetLastFlushedSequenceId { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGetLastFlushedSequenceId.class); + private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); private final TableName tableName = TableName.valueOf(getClass().getSimpleName(), "test"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterCommandLine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterCommandLine.java index 2cb42f74176..771edfa6be6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterCommandLine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterCommandLine.java @@ -19,15 +19,21 @@ package org.apache.hadoop.hbase.master; import static org.junit.Assert.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - @Category({MasterTests.class, SmallTests.class}) public class TestHMasterCommandLine { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHMasterCommandLine.class); + private static final HBaseTestingUtility TESTING_UTIL = new HBaseTestingUtility(); @Test public void testRun() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java index 92955e2dcee..8630dbb06dc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,22 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcClientFactory; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -41,16 +36,26 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest; + @Category({ MasterTests.class, MediumTests.class }) public class TestHMasterRPCException { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHMasterRPCException.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHMasterRPCException.class); private final HBaseTestingUtility testUtil = HBaseTestingUtility.createLocalHTU(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java index 59a23cba774..11df31383d2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,7 +24,7 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -49,16 +48,23 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.util.StringUtils; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Joiner; @Category({MasterTests.class, MediumTests.class}) public class TestMaster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMaster.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestMaster.class); private static final TableName TABLENAME = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java index 17b5a63c23c..3bc60eb3b43 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,7 +22,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -34,6 +33,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,6 +41,11 @@ import org.junit.experimental.categories.Category; @Ignore // SimpleLoadBalancer seems borked whether AMv2 or not. Disabling till gets attention. @Category({MasterTests.class, MediumTests.class}) public class TestMasterBalanceThrottling { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterBalanceThrottling.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] FAMILYNAME = Bytes.toBytes("fam"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterCoprocessorServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterCoprocessorServices.java index af550ab2d0c..9a2696f3b26 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterCoprocessorServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterCoprocessorServices.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -22,9 +23,12 @@ import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; import java.util.Arrays; import java.util.Collections; import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.JMXListener; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; import org.apache.hadoop.hbase.coprocessor.MasterObserver; @@ -51,18 +55,21 @@ import org.apache.hadoop.hbase.security.access.AccessController; import org.apache.hadoop.hbase.security.visibility.VisibilityController; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; - /** * Tests that the MasterRpcServices is correctly searching for implementations of the * Coprocessor Service and not just the "default" implementations of those services. */ @Category({SmallTests.class}) public class TestMasterCoprocessorServices { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterCoprocessorServices.class); + private static class MockAccessController implements AccessControlService.Interface, MasterCoprocessor, RegionCoprocessor, MasterObserver, RegionObserver { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java index aa5ad1efab9..b93c0152a54 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -23,8 +22,8 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.util.List; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.ClusterMetrics; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; @@ -34,22 +33,23 @@ import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({FlakeyTests.class, LargeTests.class}) public class TestMasterFailover { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterFailover.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterFailover.class); @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()). - withLookingForStuckThread(true). - build(); /** * Simple test of master failover. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailoverBalancerPersistence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailoverBalancerPersistence.java index 4a451ba1232..43e5481a69c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailoverBalancerPersistence.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailoverBalancerPersistence.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,18 +23,24 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; import org.apache.hadoop.hbase.ClusterMetrics; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MasterTests.class, LargeTests.class}) public class TestMasterFailoverBalancerPersistence { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterFailoverBalancerPersistence.class); + /** * Test that if the master fails, the load balancer maintains its * state (running or not) when the next master takes over diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java index 4d2a885fa23..107d0782ff3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -40,6 +42,11 @@ import org.slf4j.LoggerFactory; */ @Category({MasterTests.class, MediumTests.class}) public class TestMasterFileSystem { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterFileSystem.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterFileSystem.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java index deca5fa46b3..d8f400ee773 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master; import static org.junit.Assert.assertEquals; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -27,6 +28,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -36,6 +38,10 @@ import org.junit.experimental.categories.Category; @Category({MasterTests.class, MediumTests.class}) public class TestMasterFileSystemWithWALDir { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterFileSystemWithWALDir.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); @BeforeClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java index 9b848235146..119039f719f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java @@ -18,30 +18,36 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilityFactory; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos; + @Category({MasterTests.class, MediumTests.class}) public class TestMasterMetrics { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterMetrics.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterMetrics.class); private static final MetricsAssertHelper metricsHelper = CompatibilityFactory .getInstance(MetricsAssertHelper.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java index 1b5175a0cee..a4737e65e9e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.master; import static org.junit.Assert.*; import java.util.AbstractMap.SimpleImmutableEntry; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -37,6 +38,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestMasterMetricsWrapper { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterMetricsWrapper.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterMetricsWrapper.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java index 344b04fdb7a..2e1c979aea8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.master; - import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -26,11 +25,10 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.CoordinatedStateException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -44,8 +42,6 @@ import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.monitoring.MonitoredTask; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.FSUtils; @@ -58,16 +54,19 @@ import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; -import org.junit.Ignore; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; + /** * Standup the master and fake it to test various aspects of master function. * Does NOT spin up a mini hbase nor mini dfs cluster testing master (it does @@ -78,12 +77,14 @@ import org.slf4j.LoggerFactory; */ @Category({MasterTests.class, MediumTests.class}) public class TestMasterNoCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterNoCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterNoCluster.class); private static final HBaseTestingUtility TESTUTIL = new HBaseTestingUtility(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); - @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index e399f2eacdd..f1a45e1dfc0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -34,6 +33,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics.Option; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -68,6 +69,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestMasterOperationsForRegionReplicas { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterOperationsForRegionReplicas.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionPlacement.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Connection CONNECTION = null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java index f5ba325c0ca..5280ed6c974 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java @@ -20,29 +20,36 @@ package org.apache.hadoop.hbase.master; import static org.mockito.Mockito.when; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.QosTestHelper; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos; import org.apache.hadoop.hbase.regionserver.AnnotationReadingPriorityFunction; import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos; + @Category({MasterTests.class, SmallTests.class}) public class TestMasterQosFunction extends QosTestHelper { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterQosFunction.class); + private Configuration conf; private RSRpcServices rpcServices; private AnnotationReadingPriorityFunction qosFunction; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java index 2af6255c0e9..234d563b98f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,8 +22,8 @@ import static org.junit.Assert.assertTrue; import java.util.List; import java.util.NavigableSet; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -47,6 +47,10 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, LargeTests.class}) public class TestMasterRestartAfterDisablingTable { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterRestartAfterDisablingTable.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterRestartAfterDisablingTable.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java index 648ea643bc4..47672b440e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -26,6 +24,7 @@ import static org.junit.Assert.assertTrue; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.LocalHBaseCluster; @@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -43,6 +43,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, LargeTests.class}) public class TestMasterShutdown { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterShutdown.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterShutdown.class); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java index 90dece22ad1..cf80d8e355a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,8 +23,8 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Set; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -44,6 +43,7 @@ import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -56,6 +56,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({MasterTests.class,MediumTests.class}) public class TestMasterStatusServlet { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterStatusServlet.class); + private HMaster master; private Configuration conf; private Admin admin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java index 65351bdc7c2..9b23d49d051 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,7 +18,7 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; @@ -39,6 +38,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +51,11 @@ import org.slf4j.LoggerFactory; */ @Category({MasterTests.class, LargeTests.class}) public class TestMasterTransitions { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterTransitions.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterTransitions.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final TableName TABLENAME = TableName.valueOf("master_transitions"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java index 2e6c6995a22..d063f0a0769 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java @@ -20,17 +20,19 @@ package org.apache.hadoop.hbase.master; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; + import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.master.assignment.RegionStates; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKUtil; @@ -38,6 +40,7 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +49,11 @@ import org.junit.experimental.categories.Category; */ @Category(MediumTests.class) public class TestMetaShutdownHandler { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetaShutdownHandler.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); final static Configuration conf = TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index 279495282e2..a3aa5b9741e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java @@ -33,8 +33,8 @@ import java.util.List; import java.util.Map; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -73,6 +74,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestRegionPlacement { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionPlacement.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionPlacement.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static int SLAVES = 10; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement2.java index 835e274e8d2..6dc371149a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement2.java @@ -24,8 +24,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -50,6 +51,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestRegionPlacement2 { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionPlacement2.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionPlacement2.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static int SLAVES = 7; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlan.java index 3d10c9fc84f..13154cf9567 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlan.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlan.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -22,12 +22,14 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -35,6 +37,11 @@ import org.junit.rules.TestName; @Category({MasterTests.class, SmallTests.class}) public class TestRegionPlan { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionPlan.class); + private final ServerName SRC = ServerName.valueOf("source", 1234, 2345); private final ServerName DEST = ServerName.valueOf("dest", 1234, 2345); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionState.java index fe8b28dfa00..e34e387f610 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionState.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionState.java @@ -17,20 +17,28 @@ */ package org.apache.hadoop.hbase.master; +import static org.junit.Assert.assertEquals; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; @Category({MasterTests.class, SmallTests.class}) public class TestRegionState { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionState.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java index b871bb12ab2..2ea05dfbc7c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,7 +23,7 @@ import static org.junit.Assert.assertTrue; import java.util.List; import java.util.Map; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; @@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; import org.junit.After; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -47,6 +47,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, LargeTests.class}) public class TestRestartCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRestartCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRestartCluster.class); private HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java index 8953147b766..e98f3c9bb47 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,8 +24,8 @@ import java.util.List; import java.util.NavigableSet; import java.util.Set; import java.util.TreeSet; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -40,19 +39,26 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; /** * Tests the restarting of everything as done during rolling restarts. */ @Category({MasterTests.class, LargeTests.class}) -public class TestRollingRestart { +public class TestRollingRestart { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRollingRestart.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRollingRestart.class); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java index 02d7f2f8aab..91108aeed53 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -22,7 +22,7 @@ import static org.junit.Assert.assertNotNull; import java.io.IOException; import java.util.concurrent.CountDownLatch; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -32,10 +32,9 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; /** * Test to confirm that we will not hang when stop a backup master which is trying to become the @@ -43,8 +42,10 @@ import org.junit.rules.TestRule; */ @Category({ MasterTests.class, MediumTests.class }) public class TestShutdownBackupMaster { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestShutdownBackupMaster.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java index 128d7ee4441..20bd6cd2ead 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -39,11 +38,11 @@ import java.io.IOException; import java.util.Map; import java.util.UUID; import java.util.concurrent.atomic.LongAdder; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; @@ -66,6 +65,7 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -74,6 +74,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestSplitLogManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSplitLogManager.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSplitLogManager.class); private final ServerManager sm = Mockito.mock(ServerManager.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java index 5a752973910..fa054b46b05 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java @@ -1,6 +1,4 @@ -/* - * Copyright The Apache Software Foundation - * +/** * 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 @@ -20,12 +18,10 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.TableState; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.zookeeper.ZKUtil; @@ -34,17 +30,25 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; + /** * Tests the default table lock manager */ @Category({ MasterTests.class, LargeTests.class }) public class TestTableStateManager { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableStateManager.class); + private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java index 5cb82a27e22..3babd2ea79f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,8 +21,8 @@ import static org.apache.hadoop.hbase.regionserver.HRegion.warmupHRegion; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -42,6 +41,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -55,6 +55,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, LargeTests.class}) @SuppressWarnings ("deprecation") public class TestWarmupRegion { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWarmupRegion.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWarmupRegion.class); protected TableName TABLENAME = TableName.valueOf("testPurgeFutureDeletes"); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java index 881a46908fb..5003d51e432 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.assignment; import static org.junit.Assert.assertEquals; @@ -37,10 +36,9 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ServerName; @@ -69,15 +67,16 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.ipc.RemoteException; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse; @@ -93,12 +92,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto @Category({MasterTests.class, LargeTests.class}) public class TestAssignmentManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAssignmentManager.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAssignmentManager.class); @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = - CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); @Rule public final ExpectedException exception = ExpectedException.none(); private static final int PROC_NTHREADS = 64; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java index d33eab7cabe..c6939637f18 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.assignment; import static org.junit.Assert.assertEquals; @@ -23,8 +22,8 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -39,6 +38,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -46,6 +46,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, LargeTests.class}) public class TestAssignmentOnRSCrash { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAssignmentOnRSCrash.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAssignmentOnRSCrash.class); private static final TableName TEST_TABLE = TableName.valueOf("testb"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java index 57882dfb4f6..5b70e204ba7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java @@ -15,16 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.assignment; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.util.List; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -47,19 +45,22 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestMergeTableRegionsProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMergeTableRegionsProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMergeTableRegionsProcedure.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); @Rule public final TestName name = new TestName(); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java index 587ebc07860..6f12b95510e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.assignment; import static org.junit.Assert.assertEquals; @@ -27,7 +26,7 @@ import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -41,6 +40,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -48,6 +48,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestRegionStates { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionStates.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionStates.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java index ca1297a00e1..4d0214f4887 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,7 +17,14 @@ */ package org.apache.hadoop.hbase.master.assignment; +import static org.hamcrest.core.Is.isA; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.io.IOException; +import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -31,12 +37,6 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -44,6 +44,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -52,18 +53,24 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.List; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; -import static org.hamcrest.core.Is.isA; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos; /** * Tests to verify master/ assignment manager functionality against rogue RS */ @Category({MasterTests.class, MediumTests.class}) public class TestRogueRSAssignment { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRogueRSAssignment.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRogueRSAssignment.class); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java index d2a4020cc9f..1edb8e5314c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.assignment; import static org.junit.Assert.assertEquals; @@ -24,12 +23,11 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.List; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -56,19 +54,22 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestSplitTableRegionProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSplitTableRegionProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSplitTableRegionProcedure.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java index 17aee4430b5..329d7a17863 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java @@ -38,7 +38,6 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -51,8 +50,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.net.DNSToSwitchMapping; import org.junit.Assert; import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,9 +65,6 @@ public class BalancerTestBase { protected static Configuration conf; protected static StochasticLoadBalancer loadBalancer; - @Rule - public final TestRule timeout = CategoryBasedTimeout.forClass(getClass()); - @BeforeClass public static void beforeAllTests() throws Exception { conf = HBaseConfiguration.create(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java index a1b5391217c..a437fe9501d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java @@ -33,9 +33,9 @@ import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; import java.util.stream.Collectors; - import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.ServerName; @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.net.DNSToSwitchMapping; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -61,11 +62,16 @@ import org.junit.rules.TestName; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({MasterTests.class, MediumTests.class}) public class TestBaseLoadBalancer extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBaseLoadBalancer.class); + private static LoadBalancer loadBalancer; private static final Logger LOG = LoggerFactory.getLogger(TestBaseLoadBalancer.class); private static final ServerName master = ServerName.valueOf("fake-master", 0, 1L); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestDefaultLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestDefaultLoadBalancer.java index 4b500c96198..2f5e2cc1b81 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestDefaultLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestDefaultLoadBalancer.java @@ -24,8 +24,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.net.DNSToSwitchMapping; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,6 +50,11 @@ import org.slf4j.LoggerFactory; */ @Category({MasterTests.class, MediumTests.class}) public class TestDefaultLoadBalancer extends BalancerTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDefaultLoadBalancer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestDefaultLoadBalancer.class); private static LoadBalancer loadBalancer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java index 0178bf390ff..b175bd7218b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java @@ -18,14 +18,14 @@ package org.apache.hadoop.hbase.master.balancer; import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORED_NODES_NUM; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertEquals; import java.util.List; import java.util.Set; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -38,14 +38,15 @@ import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; - -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.junit.After; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; + /* * This case tests a scenario when a cluster with tables is moved from Stochastic Load Balancer * to FavoredStochasticLoadBalancer and the generation of favored nodes after switch. @@ -53,6 +54,10 @@ import org.slf4j.LoggerFactory; @Category(MediumTests.class) public class TestFavoredNodeTableImport { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFavoredNodeTableImport.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFavoredNodeTableImport.class); private static final int SLAVES = 3; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java index 67ed3301837..ece1ee124e9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -30,6 +30,7 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics.Option; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -70,6 +72,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Maps; @Category(LargeTests.class) public class TestFavoredStochasticBalancerPickers extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFavoredStochasticBalancerPickers.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFavoredStochasticBalancerPickers.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java index 4c237714dd0..45f39489b3a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -33,6 +33,7 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics.Option; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -58,6 +59,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -72,6 +74,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @Category(MediumTests.class) public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFavoredStochasticLoadBalancer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFavoredStochasticLoadBalancer.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java index 0479065701d..59454f52326 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -37,11 +38,17 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MasterTests.class, SmallTests.class}) public class TestRegionLocationFinder { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionLocationFinder.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static MiniHBaseCluster cluster; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java index 7b26ae0935f..9f551c8c1ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,8 +17,12 @@ */ package org.apache.hadoop.hbase.master.balancer; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -32,20 +36,15 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * Test options for regions on master; none, system, or any (i.e. master is like any other * regionserver). Checks how regions are deployed when each of the options are enabled. @@ -56,10 +55,13 @@ import static org.junit.Assert.assertTrue; @Ignore @Category({MediumTests.class}) public class TestRegionsOnMasterOptions { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionsOnMasterOptions.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionsOnMasterOptions.class); @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private Configuration c; private String tablesOnMasterOldValue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestServerAndLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestServerAndLoad.java index 2cfaf4e03aa..982321f4c59 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestServerAndLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestServerAndLoad.java @@ -20,24 +20,30 @@ package org.apache.hadoop.hbase.master.balancer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MasterTests.class, SmallTests.class}) public class TestServerAndLoad { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerAndLoad.class); + @Test public void test() { ServerName server = ServerName.valueOf("host", 12345, 112244); int startcode = 12; - + ServerAndLoad sal = new ServerAndLoad(server, startcode); assertEquals(sal.hashCode(), new ServerAndLoad(server, startcode).hashCode()); assertEquals(sal, new ServerAndLoad(server, startcode)); - + assertNotEquals(sal.hashCode(), new ServerAndLoad(server, startcode + 1).hashCode()); assertNotEquals(sal, new ServerAndLoad(server, startcode + 1)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java index 7ddaca2482c..149bf681d09 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java @@ -31,6 +31,7 @@ import java.util.Queue; import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.RegionMetrics; import org.apache.hadoop.hbase.ServerMetrics; @@ -44,11 +45,17 @@ import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer.ServerLoca import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MasterTests.class, MediumTests.class }) public class TestStochasticLoadBalancer extends BalancerTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticLoadBalancer.class); + private static final String REGION_KEY = "testRegion"; // Mapping of locality test -> expected locality diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java index 637c2f6dac1..da573da1094 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,26 +17,21 @@ */ package org.apache.hadoop.hbase.master.balancer; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.After; import org.junit.Before; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; @Category({FlakeyTests.class, LargeTests.class}) public class TestStochasticLoadBalancer2 extends BalancerTestBase { - private static final Logger LOG = LoggerFactory.getLogger(TestStochasticLoadBalancer2.class); - @Rule - public final TestRule timeout = CategoryBasedTimeout.builder() - .withTimeout(this.getClass()) - .withLookingForStuckThread(true) - .build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticLoadBalancer2.class); @Before public void before() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java index bd80c31c404..cbcc6684394 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java @@ -21,11 +21,13 @@ import static org.junit.Assert.assertNull; import java.util.List; import java.util.Map; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -34,6 +36,10 @@ import org.slf4j.LoggerFactory; @Category({ MasterTests.class, LargeTests.class }) public class TestStochasticLoadBalancerBalanceCluster extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticLoadBalancerBalanceCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestStochasticLoadBalancerBalanceCluster.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java index e692eeb51f3..da38187cce6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java @@ -17,14 +17,20 @@ */ package org.apache.hadoop.hbase.master.balancer; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MasterTests.class, MediumTests.class }) public class TestStochasticLoadBalancerLargeCluster extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticLoadBalancerLargeCluster.class); + @Test public void testLargeCluster() { int numNodes = 1000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerMidCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerMidCluster.java index e268c35bbf3..a1289601530 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerMidCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerMidCluster.java @@ -17,14 +17,20 @@ */ package org.apache.hadoop.hbase.master.balancer; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MasterTests.class, LargeTests.class }) public class TestStochasticLoadBalancerMidCluster extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticLoadBalancerMidCluster.class); + @Test public void testMidCluster() { int numNodes = 100; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java index b59c7a838e1..cb48e1d98ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -36,12 +37,17 @@ import org.apache.hadoop.hbase.master.RackManager; import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MasterTests.class, LargeTests.class }) public class TestStochasticLoadBalancerRegionReplica extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticLoadBalancerRegionReplica.class); + @Test public void testReplicaCost() { Configuration conf = HBaseConfiguration.create(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaSameHosts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaSameHosts.java index bdb2a20fa12..b9c3ec8ef2d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaSameHosts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaSameHosts.java @@ -21,16 +21,22 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.TreeMap; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MasterTests.class, LargeTests.class }) public class TestStochasticLoadBalancerRegionReplicaSameHosts extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticLoadBalancerRegionReplicaSameHosts.class); + @Test // Test is flakey. TODO: Fix! public void testRegionReplicationOnMidClusterSameHosts() { conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaWithRacks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaWithRacks.java index ae14d10c425..b7532ebaa95 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaWithRacks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaWithRacks.java @@ -19,17 +19,23 @@ package org.apache.hadoop.hbase.master.balancer; import java.util.List; import java.util.Map; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.master.RackManager; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MasterTests.class, LargeTests.class }) public class TestStochasticLoadBalancerRegionReplicaWithRacks extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticLoadBalancerRegionReplicaWithRacks.class); + private static class ForTestRackManager extends RackManager { int numRacks; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java index d96123334fa..589fa8f9c0f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java @@ -17,14 +17,20 @@ */ package org.apache.hadoop.hbase.master.balancer; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MasterTests.class, MediumTests.class }) public class TestStochasticLoadBalancerSmallCluster extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStochasticLoadBalancerSmallCluster.class); + @Test public void testSmallCluster() { int numNodes = 10; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java index 40a443af901..75557fb0973 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java @@ -23,12 +23,12 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.StoppableImplementation; import org.junit.After; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -47,6 +48,10 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, SmallTests.class}) public class TestCleanerChore { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCleanerChore.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCleanerChore.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java index 45204993212..4e48c56ff43 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -32,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -54,6 +55,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestHFileCleaner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileCleaner.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHFileCleaner.class); private final static HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java index 1ff47d97d9d..667a33e893b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -22,14 +22,13 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; @@ -44,11 +43,11 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; /** * Test the HFileLink Cleaner. @@ -56,8 +55,10 @@ import org.junit.rules.TestRule; */ @Category({MasterTests.class, MediumTests.class}) public class TestHFileLinkCleaner { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileLinkCleaner.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java index 644de6ad9f8..534b123ffa4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; @@ -63,6 +64,7 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -72,6 +74,10 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestLogsCleaner { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLogsCleaner.class); + private static final Logger LOG = LoggerFactory.getLogger(TestLogsCleaner.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java index 85f0d1fb980..260e8d98ab9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java @@ -1,12 +1,19 @@ -/* - * 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. +/** + * 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.master.cleaner; @@ -17,14 +24,11 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; - import java.io.IOException; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Iterator; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -32,6 +36,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; @@ -46,7 +51,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueues; import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments; import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; -import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl; import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner; import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -61,15 +65,23 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + @Category({ MasterTests.class, SmallTests.class }) public class TestReplicationHFileCleaner { - private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationHFileCleaner.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationHFileCleaner.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Server server; private static ReplicationQueues rq; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java index 6aa59cb33cf..b0910e25237 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java @@ -23,8 +23,8 @@ import static org.junit.Assert.assertTrue; import java.util.List; import java.util.Map; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationQueues; @@ -35,11 +35,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MasterTests.class, MediumTests.class }) public class TestReplicationZKNodeCleaner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationZKNodeCleaner.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final String ID_ONE = "1"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java index 6a71df3ecd7..9f2ad924c8d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java @@ -25,29 +25,26 @@ import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.snapshot.DisabledTableSnapshotHandler; import org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; @@ -62,22 +59,32 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptorBuilder; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; + /** * Test the master-related aspects of a snapshot */ @Category({MasterTests.class, MediumTests.class}) public class TestSnapshotFromMaster { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotFromMaster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotFromMaster.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final int NUM_RS = 2; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java index 908504e3285..9f964c6f46c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java @@ -11,11 +11,10 @@ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUTKey WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * 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.master.locking; import static org.junit.Assert.assertEquals; @@ -23,8 +22,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -37,10 +36,11 @@ import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -48,8 +48,13 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({MasterTests.class, SmallTests.class}) +@Category({ MasterTests.class, MediumTests.class }) public class TestLockManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLockManager.class); + @Rule public TestName testName = new TestName(); // crank this up if this test turns out to be flaky. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java index 1f743db7b4a..5e1c9cefdb0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java @@ -11,11 +11,10 @@ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUTKey WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * 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.master.locking; import static org.junit.Assert.assertEquals; @@ -27,10 +26,9 @@ import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -45,7 +43,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.hamcrest.core.IsInstanceOf; import org.hamcrest.core.StringStartsWith; @@ -53,27 +51,30 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse; -@Category({MasterTests.class, SmallTests.class}) +@Category({MasterTests.class, MediumTests.class}) public class TestLockProcedure { - @Rule - public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLockProcedure.class); + @Rule public final ExpectedException exception = ExpectedException.none(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java index 7539b0ea7ef..02a35d8a08f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -28,7 +27,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.RegionLoad; import org.apache.hadoop.hbase.ServerName; @@ -37,12 +36,11 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.master.MasterRpcServices; import org.apache.hadoop.hbase.master.MasterServices; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,12 +49,20 @@ import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse; /** * Tests logic of {@link SimpleRegionNormalizer}. */ @Category({MasterTests.class, SmallTests.class}) public class TestSimpleRegionNormalizer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSimpleRegionNormalizer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSimpleRegionNormalizer.class); private static RegionNormalizer normalizer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java index 5236c0465be..14b8dc4f4a3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,7 +23,7 @@ import java.io.IOException; import java.util.Collections; import java.util.Comparator; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -49,6 +48,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.LoadTestKVGenerator; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -61,6 +61,11 @@ import org.slf4j.LoggerFactory; */ @Category({MasterTests.class, MediumTests.class}) public class TestSimpleRegionNormalizerOnCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSimpleRegionNormalizerOnCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSimpleRegionNormalizerOnCluster.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java index 490501a866f..bb405a5c186 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java @@ -15,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertTrue; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableExistsException; @@ -31,20 +30,27 @@ import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; + @Category({MasterTests.class, MediumTests.class}) public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCloneSnapshotProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCloneSnapshotProcedure.class); protected final byte[] CF = Bytes.toBytes("cf1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java index 32a757e6345..c420a53a9ab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertNotNull; @@ -23,8 +22,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceExistException; @@ -39,6 +38,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -46,6 +46,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestCreateNamespaceProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCreateNamespaceProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCreateNamespaceProcedure.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java index e54eb661623..056155f54df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ModifyRegionUtils; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -44,6 +45,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestCreateTableProcedure extends TestTableDDLProcedureBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCreateTableProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCreateTableProcedure.class); private static final String F1 = "f1"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java index 899d1142157..d5ec62db376 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -24,11 +22,11 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -47,11 +45,17 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MasterTests.class, LargeTests.class}) public class TestDeleteColumnFamilyProcedureFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDeleteColumnFamilyProcedureFromClient.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final TableName TABLENAME = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java index fa2507e926f..10dca9d99e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertNotNull; @@ -23,8 +22,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -40,6 +39,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,6 +49,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestDeleteNamespaceProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDeleteNamespaceProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestDeleteNamespaceProcedure.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java index 9f4c1803f44..bb41ef05b3a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java @@ -15,12 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -31,19 +30,22 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestDeleteTableProcedure extends TestTableDDLProcedureBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDeleteTableProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestDeleteTableProcedure.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); @Rule public TestName name = new TestName(); @Test(timeout=60000, expected=TableNotFoundException.class) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java index e49bfcbbce8..da8878972c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java @@ -15,12 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.procedure2.Procedure; @@ -30,19 +29,22 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestDisableTableProcedure extends TestTableDDLProcedureBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDisableTableProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestDisableTableProcedure.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java index 8364dfea5d4..5aeb665a6ec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java @@ -15,12 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.procedure2.Procedure; @@ -30,19 +29,22 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestEnableTableProcedure extends TestTableDDLProcedureBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEnableTableProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestEnableTableProcedure.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); @Rule public TestName name = new TestName(); @Test(timeout = 60000) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestFastFailOnProcedureNotRegistered.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestFastFailOnProcedureNotRegistered.java index 363eba8935e..69285f19ca6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestFastFailOnProcedureNotRegistered.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestFastFailOnProcedureNotRegistered.java @@ -15,23 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import java.io.IOException; import java.util.HashMap; import java.util.Map; - import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MasterTests.class, MediumTests.class}) public class TestFastFailOnProcedureNotRegistered extends TestTableDDLProcedureBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFastFailOnProcedureNotRegistered.class); + @Test(expected=DoNotRetryIOException.class, timeout = 3000) public void testFastFailOnProcedureNotRegistered() throws IOException { Admin admin = UTIL.getAdmin(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java index 31faa083b9e..48a3237f2a5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertEquals; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -41,9 +40,9 @@ import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableState; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableState; @@ -52,12 +51,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.T @Category({MasterTests.class, LargeTests.class}) public class TestMasterFailoverWithProcedures { - private static final Logger LOG = LoggerFactory.getLogger(TestMasterFailoverWithProcedures.class); - @ClassRule - public static final TestRule timeout = - CategoryBasedTimeout.forClass(TestMasterFailoverWithProcedures.class); + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterFailoverWithProcedures.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestMasterFailoverWithProcedures.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java index 3cb5e646a0e..83e38439898 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertEquals; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -48,6 +49,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestMasterProcedureEvents { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterProcedureEvents.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCreateTableProcedure.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java index 0291165848e..29d94896a95 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java @@ -15,33 +15,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.*; + import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.locking.LockProcedure; -import org.apache.hadoop.hbase.procedure2.Procedure; -import org.apache.hadoop.hbase.procedure2.ProcedureEvent; import org.apache.hadoop.hbase.procedure2.LockType; import org.apache.hadoop.hbase.procedure2.LockedResource; import org.apache.hadoop.hbase.procedure2.LockedResourceType; +import org.apache.hadoop.hbase.procedure2.Procedure; +import org.apache.hadoop.hbase.procedure2.ProcedureEvent; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +52,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, SmallTests.class}) public class TestMasterProcedureScheduler { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterProcedureScheduler.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterProcedureScheduler.class); private MasterProcedureScheduler queue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java index 2e8e52ae24e..2c0db61d537 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java @@ -15,34 +15,38 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.procedure.TestMasterProcedureScheduler.TestTableProcedure; import org.apache.hadoop.hbase.procedure2.Procedure; -import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MasterTests; - +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - @Category({MasterTests.class, MediumTests.class}) public class TestMasterProcedureSchedulerConcurrency { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterProcedureSchedulerConcurrency.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterProcedureSchedulerConcurrency.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureWalLease.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureWalLease.java index ff2303acc30..174033511b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureWalLease.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureWalLease.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertEquals; @@ -24,9 +23,8 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -48,7 +46,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,15 +53,16 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, LargeTests.class}) @Ignore public class TestMasterProcedureWalLease { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterProcedureWalLease.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterProcedureWalLease.class); @Rule public TestName name = new TestName(); - @ClassRule - public static final TestRule timeout = - CategoryBasedTimeout.forClass(TestMasterProcedureWalLease.class); - protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static void setupConf(Configuration conf) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java index 4adab53d00b..a58cc69be37 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -38,6 +37,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -45,6 +45,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestModifyNamespaceProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestModifyNamespaceProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestModifyNamespaceProcedure.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java index 24a6bc5d1bf..930956fdc50 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -34,16 +33,19 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; @Category({MasterTests.class, MediumTests.class}) public class TestModifyTableProcedure extends TestTableDDLProcedureBase { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestModifyTableProcedure.class); + @Rule public TestName name = new TestName(); @Test(timeout=60000) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java index f47654f7ea3..9a1c2d2ba79 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertFalse; @@ -23,9 +22,8 @@ import static org.junit.Assert.assertTrue; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -39,19 +37,22 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestProcedureAdmin { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureAdmin.class); + private static final Logger LOG = LoggerFactory.getLogger(TestProcedureAdmin.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); @Rule public TestName name = new TestName(); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java index 3946ee99276..499531589cd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertEquals; @@ -25,39 +24,42 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; - -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; -import org.apache.hadoop.hbase.client.SnapshotDescription; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; + @Category({MasterTests.class, MediumTests.class}) public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRestoreSnapshotProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRestoreSnapshotProcedure.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); protected final TableName snapshotTableName = TableName.valueOf("testRestoreSnapshot"); protected final byte[] CF1 = Bytes.toBytes("cf1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSafemodeBringsDownMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSafemodeBringsDownMaster.java index 389ed62e6b0..274b4e542e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSafemodeBringsDownMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSafemodeBringsDownMaster.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertTrue; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -40,6 +39,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -47,6 +47,11 @@ import org.slf4j.LoggerFactory; @Category(MediumTests.class) public class TestSafemodeBringsDownMaster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSafemodeBringsDownMaster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSafemodeBringsDownMaster.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java index c1d1812c4c6..f8899e7850d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -44,6 +45,11 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, LargeTests.class}) public class TestServerCrashProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerCrashProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestServerCrashProcedure.class); private HBaseTestingUtility util; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java index 7ea77f641d6..d3b13ad1660 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -24,8 +22,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Set; - import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -43,6 +41,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,6 +54,10 @@ import org.junit.rules.TestName; @Category({MasterTests.class, LargeTests.class}) public class TestTableDescriptorModificationFromClient { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableDescriptorModificationFromClient.class); + @Rule public TestName name = new TestName(); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java index b70bdd554ae..d7f385c655a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java @@ -15,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -32,19 +31,22 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({MasterTests.class, MediumTests.class}) public class TestTruncateTableProcedure extends TestTableDDLProcedureBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTruncateTableProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTruncateTableProcedure.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java index 2140d5eaf85..ac6a3c545ed 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java @@ -15,38 +15,42 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; -import java.util.concurrent.atomic.AtomicInteger; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; -import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.server.datanode.DataNode; - import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - @Category({MasterTests.class, LargeTests.class}) public class TestWALProcedureStoreOnHDFS { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALProcedureStoreOnHDFS.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWALProcedureStoreOnHDFS.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestAssignProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestAssignProcedure.java index ed53c3af56a..8836c9f75e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestAssignProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestAssignProcedure.java @@ -1,20 +1,19 @@ -/* - * Copyright The Apache Software Foundation +/** + * 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 * - * 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 + * 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. + * 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.master.snapshot; @@ -27,9 +26,8 @@ import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -46,24 +44,24 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - @Category({RegionServerTests.class, SmallTests.class}) public class TestAssignProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAssignProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAssignProcedure.class); @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()). - withLookingForStuckThread(true). - build(); /** * An override that opens up the updateTransition method inside in AssignProcedure so can call it diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java index 382625c482a..b8499d5b466 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java @@ -31,10 +31,10 @@ import java.util.HashSet; import java.util.List; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; @@ -46,12 +46,15 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; /** @@ -60,6 +63,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; @Category({MasterTests.class, MediumTests.class}) public class TestSnapshotFileCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotFileCache.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotFileCache.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static long sequenceId = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java index 8f81946753b..0c10a461ee9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java @@ -22,24 +22,25 @@ import static org.junit.Assert.assertFalse; import java.io.IOException; import java.util.Collection; import java.util.HashSet; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -53,6 +54,10 @@ import org.slf4j.LoggerFactory; @Category({MasterTests.class, SmallTests.class}) public class TestSnapshotHFileCleaner { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotHFileCleaner.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotFileCache.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final String TABLE_NAME_STR = "testSnapshotManifest"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java index f716de61fc4..3a6a61f3eff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java @@ -22,14 +22,12 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterServices; @@ -38,7 +36,10 @@ import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner; import org.apache.hadoop.hbase.procedure.ProcedureCoordinator; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.zookeeper.KeeperException; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -50,6 +51,11 @@ import org.mockito.Mockito; */ @Category({MasterTests.class, SmallTests.class}) public class TestSnapshotManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotManager.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java index 65f50c9f7d7..bb194b603e6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,6 +23,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +44,11 @@ import org.slf4j.LoggerFactory; @Category(SmallTests.class) public class TestCachedMobFile { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCachedMobFile.class); + static final Logger LOG = LoggerFactory.getLogger(TestCachedMobFile.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private Configuration conf = TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java index 34546146cd6..9c2a2a4e946 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java @@ -1,27 +1,26 @@ /** -* -* 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. -*/ + * 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.mob; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -32,6 +31,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,6 +40,10 @@ import org.junit.rules.TestName; @Category(LargeTests.class) public class TestDefaultMobStoreFlusher { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDefaultMobStoreFlusher.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static byte [] row1 = Bytes.toBytes("row1"); private final static byte [] row2 = Bytes.toBytes("row2"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java index 5efb5091410..7219db333dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java @@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.mob; import static org.junit.Assert.assertEquals; import java.util.Random; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -35,12 +35,17 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(MediumTests.class) public class TestExpiredMobFileCleaner { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestExpiredMobFileCleaner.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static TableName tableName = TableName.valueOf("TestExpiredMobFileCleaner"); private final static String family = "family"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java index d9fc9f669a8..8aadf008d34 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java @@ -1,25 +1,24 @@ /** -* -* 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. -*/ + * 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.mob; import java.util.Random; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -34,12 +33,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(MediumTests.class) public class TestMobDataBlockEncoding { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobDataBlockEncoding.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static byte [] row1 = Bytes.toBytes("row1"); private final static byte [] family = Bytes.toBytes("family"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java index cc2aa5cef24..c22ca980baa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,6 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -47,6 +48,11 @@ import org.slf4j.LoggerFactory; @Category(SmallTests.class) public class TestMobFile { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobFile.class); + static final Logger LOG = LoggerFactory.getLogger(TestMobFile.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private Configuration conf = TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java index 5077728e1a3..325a580886b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,10 +22,10 @@ import static org.junit.Assert.assertNotNull; import java.io.IOException; import java.util.Date; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -41,6 +40,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -48,6 +48,11 @@ import org.slf4j.LoggerFactory; @Category(SmallTests.class) public class TestMobFileCache { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobFileCache.class); + static final Logger LOG = LoggerFactory.getLogger(TestMobFileCache.class); private HBaseTestingUtility UTIL; private HRegion region; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileLink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileLink.java index 82b51b7f285..57bcf1fd88e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileLink.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileLink.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,10 +18,10 @@ package org.apache.hadoop.hbase.mob; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.HFileLink; @@ -30,6 +29,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -37,6 +37,11 @@ import org.junit.rules.TestName; @Category(SmallTests.class) public class TestMobFileLink { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobFileLink.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java index 5ca73a6d4ec..22a47ba9f8e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,17 +25,22 @@ import static org.junit.Assert.assertTrue; import java.util.Date; import java.util.Random; import java.util.UUID; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.MD5Hash; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestMobFileName { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobFileName.class); + private String uuid; private Date date; private String dateStr; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java index 8c974e676a5..8ef8c7aa67a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -38,15 +37,14 @@ import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; - import javax.crypto.spec.SecretKeySpec; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -98,6 +96,7 @@ import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -107,6 +106,11 @@ import org.slf4j.LoggerFactory; @Category(LargeTests.class) public class TestMobCompactor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompactor.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactor.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Configuration conf = null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java index fabc4e27d8a..27801533dc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -20,16 +19,22 @@ package org.apache.hadoop.hbase.mob.compactions; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestPartitionedMobCompactionRequest { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPartitionedMobCompactionRequest.class); + @Test public void testCompactedPartitionId() { String startKey1 = "startKey1"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java index 34acbe493eb..94c35f485a5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -39,7 +38,6 @@ import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -47,6 +45,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -80,6 +79,7 @@ import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -89,6 +89,11 @@ import org.slf4j.LoggerFactory; @Category(LargeTests.class) public class TestPartitionedMobCompactor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPartitionedMobCompactor.class); + private static final Logger LOG = LoggerFactory.getLogger(TestPartitionedMobCompactor.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static String family = "family"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestMemoryBoundedLogMessageBuffer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestMemoryBoundedLogMessageBuffer.java index f64b297e942..57c86e92eca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestMemoryBoundedLogMessageBuffer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestMemoryBoundedLogMessageBuffer.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,9 +22,10 @@ import static org.junit.Assert.assertTrue; import java.io.PrintWriter; import java.io.StringWriter; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -37,14 +37,18 @@ import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestMemoryBoundedLogMessageBuffer { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMemoryBoundedLogMessageBuffer.class); + private static final long TEN_KB = 10 * 1024; private static final String JP_TEXT = "こんにちは"; - + @Test public void testBuffer() { MemoryBoundedLogMessageBuffer buf = new MemoryBoundedLogMessageBuffer(TEN_KB); - + for (int i = 0; i < 1000; i++) { buf.add("hello " + i); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java index 7abcde8f9c2..a2fe42a263b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,8 +22,8 @@ import static org.junit.Assert.*; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Query; @@ -32,53 +31,58 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestTaskMonitor { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTaskMonitor.class); + @Test public void testTaskMonitorBasics() { TaskMonitor tm = new TaskMonitor(new Configuration()); assertTrue("Task monitor should start empty", tm.getTasks().isEmpty()); - + // Make a task and fetch it back out MonitoredTask task = tm.createStatus("Test task"); MonitoredTask taskFromTm = tm.getTasks().get(0); - + // Make sure the state is reasonable. assertEquals(task.getDescription(), taskFromTm.getDescription()); assertEquals(-1, taskFromTm.getCompletionTimestamp()); assertEquals(MonitoredTask.State.RUNNING, taskFromTm.getState()); - + // Mark it as finished task.markComplete("Finished!"); assertEquals(MonitoredTask.State.COMPLETE, task.getState()); - + // It should still show up in the TaskMonitor list assertEquals(1, tm.getTasks().size()); - + // If we mark its completion time back a few minutes, it should get gced task.expireNow(); assertEquals(0, tm.getTasks().size()); tm.shutdown(); } - + @Test public void testTasksGetAbortedOnLeak() throws InterruptedException { final TaskMonitor tm = new TaskMonitor(new Configuration()); assertTrue("Task monitor should start empty", tm.getTasks().isEmpty()); - + final AtomicBoolean threadSuccess = new AtomicBoolean(false); // Make a task in some other thread and leak it Thread t = new Thread() { @Override public void run() { - MonitoredTask task = tm.createStatus("Test task"); + MonitoredTask task = tm.createStatus("Test task"); assertEquals(MonitoredTask.State.RUNNING, task.getState()); threadSuccess.set(true); } @@ -87,19 +91,19 @@ public class TestTaskMonitor { t.join(); // Make sure the thread saw the correct state assertTrue(threadSuccess.get()); - + // Make sure the leaked reference gets cleared System.gc(); System.gc(); System.gc(); - - // Now it should be aborted + + // Now it should be aborted MonitoredTask taskFromTm = tm.getTasks().get(0); assertEquals(MonitoredTask.State.ABORTED, taskFromTm.getState()); tm.shutdown(); } - + @Test public void testTaskLimit() throws Exception { TaskMonitor tm = new TaskMonitor(new Configuration()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java index 214fe4970ad..d41291e5b4a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,6 +17,13 @@ */ package org.apache.hadoop.hbase.namespace; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.IOException; import java.util.Collections; import java.util.List; @@ -26,14 +32,13 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -79,24 +84,19 @@ import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - @Category(MediumTests.class) public class TestNamespaceAuditor { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNamespaceAuditor.class); + private static final Logger LOG = LoggerFactory.getLogger(TestNamespaceAuditor.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static Admin ADMIN; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestFailedProcCleanup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestFailedProcCleanup.java index 740caea09d3..7b848f26934 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestFailedProcCleanup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestFailedProcCleanup.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.procedure; @@ -21,10 +22,10 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.List; import java.util.Optional; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -36,19 +37,26 @@ import org.apache.hadoop.hbase.coprocessor.MasterObserver; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.security.AccessDeniedException; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; + /** * Check if CompletedProcedureCleaner cleans up failed nonce procedures. */ @Category(MediumTests.class) public class TestFailedProcCleanup { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFailedProcCleanup.class); + private static final Log LOG = LogFactory.getLog(TestFailedProcCleanup.class); protected static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedure.java index 5ceb8e6fd49..a1fccbc9af4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedure.java @@ -28,12 +28,13 @@ import static org.mockito.Mockito.when; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; - -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +44,10 @@ import org.junit.experimental.categories.Category; @Category({MasterTests.class, SmallTests.class}) public class TestProcedure { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedure.class); + ProcedureCoordinator coord; @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureCoordinator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureCoordinator.java index e2b1a79669c..077ed194923 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureCoordinator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureCoordinator.java @@ -40,18 +40,20 @@ import java.util.Arrays; import java.util.List; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.InOrder; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + /** * Test Procedure coordinator operation. *

@@ -60,6 +62,11 @@ import org.mockito.stubbing.Answer; */ @Category({MasterTests.class, SmallTests.class}) public class TestProcedureCoordinator { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureCoordinator.class); + // general test constants private static final long WAKE_FREQUENCY = 1000; private static final long TIMEOUT = 100000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureDescriber.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureDescriber.java index 2e4fb9628e1..836c54d31c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureDescriber.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureDescriber.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,28 +15,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure; import static org.junit.Assert.*; import java.io.IOException; import java.util.Date; - -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.master.procedure.ProcedureDescriber; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; -import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; -import org.apache.hbase.thirdparty.com.google.protobuf.BytesValue; import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; +import org.apache.hbase.thirdparty.com.google.protobuf.BytesValue; + @Category({MasterTests.class, SmallTests.class}) public class TestProcedureDescriber { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureDescriber.class); + public static class TestProcedure extends Procedure { @Override protected Procedure[] execute(Object env) throws ProcedureYieldException, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java index 37827e52711..63273de4bdb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java @@ -22,18 +22,24 @@ import static org.junit.Assert.assertArrayEquals; import java.io.IOException; import java.util.HashMap; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MasterTests.class, MediumTests.class}) public class TestProcedureManager { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureManager.class); + private static final int NUM_RS = 2; private static HBaseTestingUtility util = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureMember.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureMember.java index ea86b400ffd..20406fe9548 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureMember.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureMember.java @@ -32,14 +32,15 @@ import static org.mockito.Mockito.when; import java.io.IOException; import java.util.concurrent.ThreadPoolExecutor; - -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.errorhandling.TimeoutException; import org.apache.hadoop.hbase.procedure.Subprocedure.SubprocedureImpl; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.InOrder; @@ -52,6 +53,11 @@ import org.mockito.stubbing.Answer; */ @Category({MasterTests.class, SmallTests.class}) public class TestProcedureMember { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProcedureMember.class); + private static final long WAKE_FREQUENCY = 100; private static final long TIMEOUT = 100000; private static final long POOL_KEEP_ALIVE = 1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java index c1398626e13..7438bb7c524 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java @@ -33,19 +33,20 @@ import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.errorhandling.TimeoutException; import org.apache.hadoop.hbase.procedure.Subprocedure.SubprocedureImpl; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -55,6 +56,7 @@ import org.mockito.stubbing.Answer; import org.mockito.verification.VerificationMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -63,6 +65,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({MasterTests.class, MediumTests.class}) public class TestZKProcedure { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKProcedure.class); private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final String COORDINATOR_NODE_NAME = "coordinator"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java index df14ead071e..fef77944952 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java @@ -27,10 +27,9 @@ import static org.mockito.Mockito.verify; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Pair; @@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -46,14 +46,21 @@ import org.mockito.stubbing.Answer; import org.mockito.verification.VerificationMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; + /** * Test zookeeper-based, procedure controllers */ @Category({MasterTests.class, MediumTests.class}) public class TestZKProcedureControllers { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKProcedureControllers.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKProcedureControllers.class); private final static HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final String COHORT_NODE_NAME = "expected"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java index 52ae0ab321d..536af716686 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,14 +20,15 @@ package org.apache.hadoop.hbase.protobuf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.protobuf.ByteString; import java.io.IOException; import java.nio.ByteBuffer; - import org.apache.hadoop.hbase.ByteBufferKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; @@ -47,16 +47,20 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.protobuf.ByteString; - /** * Class to test ProtobufUtil. */ @Category({MiscTests.class, SmallTests.class}) public class TestProtobufUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProtobufUtil.class); + @Test public void testException() throws IOException { NameBytesPair.Builder builder = NameBytesPair.newBuilder(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java index 1e3a0c248b8..236c03545be 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java @@ -22,19 +22,24 @@ import static org.junit.Assert.*; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - @Category({MiscTests.class, SmallTests.class}) public class TestReplicationProtobuf { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationProtobuf.class); + /** * Little test to check we can basically convert list of a list of KVs into a CellScanner * @throws IOException diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java index 7c1fd17e4b2..b4711998d06 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestActivePolicyEnforcement.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -26,15 +27,16 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement; -import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; import org.apache.hadoop.hbase.quotas.policies.DefaultViolationPolicyEnforcement; import org.apache.hadoop.hbase.quotas.policies.MissingSnapshotViolationPolicyEnforcement; +import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement; +import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -44,6 +46,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestActivePolicyEnforcement { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestActivePolicyEnforcement.class); + private RegionServerServices rss; @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java index f1e41cdb901..bc2ac788e0c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -31,14 +32,15 @@ import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -51,6 +53,10 @@ import org.mockito.stubbing.Answer; @Category(SmallTests.class) public class TestFileSystemUtilizationChore { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFileSystemUtilizationChore.class); + @SuppressWarnings("unchecked") @Test public void testNoOnlineRegions() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettingsImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettingsImpl.java index 9e5434b33ca..bd8a94a4b69 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettingsImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettingsImpl.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -20,17 +21,23 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; + @Category(SmallTests.class) public class TestGlobalQuotaSettingsImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGlobalQuotaSettingsImpl.class); + QuotaProtos.TimedQuota REQUEST_THROTTLE = QuotaProtos.TimedQuota.newBuilder() .setScope(QuotaProtos.QuotaScope.MACHINE).setSoftLimit(100) .setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java index c024294eb26..5c41c41b41d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterQuotaManager.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -21,16 +22,22 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.mockito.Mockito.mock; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestMasterQuotaManager { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterQuotaManager.class); + @Test public void testUninitializedQuotaManangerDoesNotFail() { MasterServices masterServices = mock(MasterServices.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java index 19c74edf3f8..391b23873b4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserver.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -22,8 +23,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -38,6 +39,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -48,6 +50,11 @@ import org.junit.rules.TestName; */ @Category(MediumTests.class) public class TestMasterSpaceQuotaObserver { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterSpaceQuotaObserver.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static SpaceQuotaHelperForTests helper; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserverWithMocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserverWithMocks.java index 36b5ae17b4d..9dd98545772 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserverWithMocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestMasterSpaceQuotaObserverWithMocks.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -27,13 +28,14 @@ import static org.mockito.Mockito.mock; import java.util.HashSet; import java.util.Set; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.security.access.AccessController; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,6 +45,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestMasterSpaceQuotaObserverWithMocks { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterSpaceQuotaObserverWithMocks.class); + private HMaster master; private Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java index b1883ec7836..ef7af2c2a29 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -27,7 +28,7 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; @@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.invocation.InvocationOnMock; @@ -55,6 +57,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; */ @Category(SmallTests.class) public class TestNamespaceQuotaViolationStore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNamespaceQuotaViolationStore.class); + private static final long ONE_MEGABYTE = 1024L * 1024L; private Connection conn; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java index 0dbc18c5c7e..cffd3be5497 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaAdmin.java @@ -15,14 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.quotas; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.util.Objects; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -32,31 +36,34 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitRequest; /** * minicluster tests that validate that quota entries are properly set in the quota table */ @Category({ClientTests.class, MediumTests.class}) public class TestQuotaAdmin { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaAdmin.class); + private static final Logger LOG = LoggerFactory.getLogger(TestQuotaAdmin.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java index cf564e32d00..035b87573bc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -22,7 +23,7 @@ import static org.mockito.Mockito.when; import java.util.HashMap; import java.util.Map; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; @@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,6 +42,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; */ @Category(SmallTests.class) public class TestQuotaObserverChore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaObserverChore.class); + private Connection conn; private QuotaObserverChore chore; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java index 8451acf5adb..debd54c3b08 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -25,8 +26,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -56,6 +58,11 @@ import org.slf4j.LoggerFactory; */ @Category(LargeTests.class) public class TestQuotaObserverChoreRegionReports { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaObserverChoreRegionReports.class); + private static final Logger LOG = LoggerFactory.getLogger(TestQuotaObserverChoreRegionReports.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java index 16fa09951cc..cc0b988c2a8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreWithMiniCluster.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -31,8 +32,8 @@ import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceNotFoundException; @@ -41,25 +42,33 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.quotas.QuotaObserverChore.TablesWithQuotas; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; + /** * Test class for {@link QuotaObserverChore} that uses a live HBase cluster. */ @Category(LargeTests.class) public class TestQuotaObserverChoreWithMiniCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaObserverChoreWithMiniCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestQuotaObserverChoreWithMiniCluster.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaState.java index 25e99b0c786..c16478b447a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaState.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaState.java @@ -15,30 +15,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.quotas; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle; + @Category({RegionServerTests.class, SmallTests.class}) public class TestQuotaState { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaState.class); + private static final TableName UNKNOWN_TABLE_NAME = TableName.valueOf("unknownTable"); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java index ce0d2f21a27..dffa6374666 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -25,8 +26,8 @@ import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -54,6 +56,11 @@ import org.slf4j.LoggerFactory; */ @Category({MediumTests.class}) public class TestQuotaStatusRPCs { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaStatusRPCs.class); + private static final Logger LOG = LoggerFactory.getLogger(TestQuotaStatusRPCs.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final AtomicLong COUNTER = new AtomicLong(0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java index a839d41a505..6aac054ba8f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaTableUtil.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.quotas; import static org.junit.Assert.assertEquals; @@ -28,9 +27,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -42,28 +41,35 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle; + /** * Test the quota table helpers (e.g. CRUD operations) */ @Category({MasterTests.class, MediumTests.class}) public class TestQuotaTableUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaTableUtil.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private Connection connection; private int tableNameCounter; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java index dfb3484df06..1c3eae4d5d7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.quotas; +import static org.junit.Assert.assertEquals; + import java.util.Objects; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -34,21 +35,24 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; -import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; - +import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; - @Category({RegionServerTests.class, MediumTests.class}) public class TestQuotaThrottle { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestQuotaThrottle.class); + private final static Logger LOG = LoggerFactory.getLogger(TestQuotaThrottle.class); private final static int REFRESH_TIME = 30 * 60000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java index 567577b19b4..96dc9902b67 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.quotas; import static org.junit.Assert.assertEquals; @@ -23,11 +22,13 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -36,6 +37,11 @@ import org.junit.experimental.categories.Category; */ @Category({RegionServerTests.class, SmallTests.class}) public class TestRateLimiter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRateLimiter.class); + @Test public void testWaitIntervalTimeUnitSeconds() { testWaitInterval(TimeUnit.SECONDS, 10, 100); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java index 5f11950b6f3..d47efeba8dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionServerSpaceQuotaManager.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -26,17 +27,18 @@ import static org.mockito.Mockito.when; import java.io.IOException; import java.util.HashMap; import java.util.Map; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; +import org.apache.hadoop.hbase.quotas.policies.DefaultViolationPolicyEnforcement; import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforcement; import org.apache.hadoop.hbase.quotas.policies.NoInsertsViolationPolicyEnforcement; import org.apache.hadoop.hbase.quotas.policies.NoWritesCompactionsViolationPolicyEnforcement; import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement; -import org.apache.hadoop.hbase.quotas.policies.DefaultViolationPolicyEnforcement; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +48,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestRegionServerSpaceQuotaManager { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionServerSpaceQuotaManager.class); + private RegionServerSpaceQuotaManager quotaManager; private RegionServerServices rss; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java index 021ebf23367..7baec50b0c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -24,8 +25,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Random; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -41,6 +42,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -53,6 +55,11 @@ import org.slf4j.LoggerFactory; */ @Category(MediumTests.class) public class TestRegionSizeUse { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionSizeUse.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionSizeUse.class); private static final int SIZE_PER_VALUE = 256; private static final int NUM_SPLITS = 10; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSnapshotQuotaObserverChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSnapshotQuotaObserverChore.java index 793395a7bcc..818d6ff6a36 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSnapshotQuotaObserverChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSnapshotQuotaObserverChore.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -27,6 +28,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -50,6 +52,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -66,6 +69,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; */ @Category(MediumTests.class) public class TestSnapshotQuotaObserverChore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotQuotaObserverChore.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotQuotaObserverChore.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final AtomicLong COUNTER = new AtomicLong(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java index 2ddb5a31ead..58270c344d3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotaViolationPolicyRefresherChore.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -32,9 +33,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; @@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -56,6 +58,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestSpaceQuotaViolationPolicyRefresherChore { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSpaceQuotaViolationPolicyRefresherChore.class); + private RegionServerSpaceQuotaManager manager; private RegionServerServices rss; private SpaceQuotaRefresherChore chore; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java index 98186c57506..f49fa40ed7b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -28,12 +29,12 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -66,6 +67,7 @@ import org.apache.hadoop.util.StringUtils; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -78,6 +80,11 @@ import org.slf4j.LoggerFactory; */ @Category(LargeTests.class) public class TestSpaceQuotas { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSpaceQuotas.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSpaceQuotas.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); // Global for all tests in the class diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java index b5d2fc4037c..e4f212c0e4b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -24,10 +25,10 @@ import java.io.IOException; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.Predicate; @@ -39,25 +40,33 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.SnapshotType; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.quotas.SpaceQuotaHelperForTests.SpaceQuotaSnapshotPredicate; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; /** * Test class to exercise the inclusion of snapshots in space quotas */ @Category({LargeTests.class}) public class TestSpaceQuotasWithSnapshots { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSpaceQuotasWithSnapshots.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSpaceQuotasWithSnapshots.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); // Global for all tests in the class diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java index 9ecde78b930..01f8a2f4bbf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -25,9 +26,9 @@ import java.security.PrivilegedExceptionAction; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -45,6 +46,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -57,6 +59,11 @@ import org.slf4j.LoggerFactory; */ @Category(MediumTests.class) public class TestSuperUserQuotaPermissions { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSuperUserQuotaPermissions.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSuperUserQuotaPermissions.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); // Default to the user running the tests diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java index 0e3f8ffd636..6803ee51267 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -28,7 +29,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; @@ -41,6 +42,7 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.invocation.InvocationOnMock; @@ -56,6 +58,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; */ @Category(SmallTests.class) public class TestTableQuotaViolationStore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableQuotaViolationStore.class); + private static final long ONE_MEGABYTE = 1024L * 1024L; private Connection conn; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java index fd34bdaa1de..8f8db51c210 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableSpaceQuotaViolationNotifier.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -26,28 +27,34 @@ import java.util.List; import java.util.Map.Entry; import java.util.NavigableMap; import java.util.Objects; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.ArgumentMatcher; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; + /** * Test case for {@link TableSpaceQuotaSnapshotNotifier}. */ @Category(SmallTests.class) public class TestTableSpaceQuotaViolationNotifier { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableSpaceQuotaViolationNotifier.class); + private TableSpaceQuotaSnapshotNotifier notifier; private Connection conn; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java index d92e0eeb97b..2298c85e850 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -29,8 +30,8 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.quotas.QuotaObserverChore.TablesWithQuotas; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,6 +51,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; */ @Category(SmallTests.class) public class TestTablesWithQuotas { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTablesWithQuotas.class); + private Connection conn; private Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java index bd0bc8c614c..3628738e394 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestBulkLoadCheckingViolationPolicyEnforcement.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -21,10 +22,10 @@ import static org.mockito.Mockito.when; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.quotas.SpaceLimitingException; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; @@ -33,12 +34,17 @@ import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestBulkLoadCheckingViolationPolicyEnforcement { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBulkLoadCheckingViolationPolicyEnforcement.class); + FileSystem fs; RegionServerServices rss; TableName tableName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java index c42d86628ce..bef60dac49f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestDisableTableViolationPolicyEnforcement.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -16,9 +17,11 @@ */ package org.apache.hadoop.hbase.quotas.policies; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.quotas.SpaceLimitingException; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -28,6 +31,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestDisableTableViolationPolicyEnforcement extends BaseViolationPolicyEnforcement { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDisableTableViolationPolicyEnforcement.class); + private DisableTableViolationPolicyEnforcement enforcement; @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java index 1115fab4f7c..66c30893992 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoInsertsViolationPolicyEnforcement.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -16,9 +17,11 @@ */ package org.apache.hadoop.hbase.quotas.policies; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.quotas.SpaceLimitingException; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -28,6 +31,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestNoInsertsViolationPolicyEnforcement extends BaseViolationPolicyEnforcement { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNoInsertsViolationPolicyEnforcement.class); + private NoInsertsViolationPolicyEnforcement enforcement; @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java index 1348eb73dba..f4d0c4d6183 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesCompactionsViolationPolicyEnforcement.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -16,9 +17,11 @@ */ package org.apache.hadoop.hbase.quotas.policies; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.quotas.SpaceLimitingException; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -29,6 +32,10 @@ import org.junit.experimental.categories.Category; public class TestNoWritesCompactionsViolationPolicyEnforcement extends BaseViolationPolicyEnforcement { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNoWritesCompactionsViolationPolicyEnforcement.class); + private NoWritesCompactionsViolationPolicyEnforcement enforcement; @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java index c032f264fd1..116814596c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/policies/TestNoWritesViolationPolicyEnforcement.java @@ -1,12 +1,13 @@ -/* - * 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 +/** + * 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 + * 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, @@ -16,9 +17,11 @@ */ package org.apache.hadoop.hbase.quotas.policies; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.quotas.SpaceLimitingException; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -28,6 +31,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestNoWritesViolationPolicyEnforcement extends BaseViolationPolicyEnforcement { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNoWritesViolationPolicyEnforcement.class); + private NoWritesViolationPolicyEnforcement enforcement; @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index c8a75f03c77..b14c94f2541 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -32,13 +32,13 @@ import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -83,6 +84,11 @@ import org.slf4j.LoggerFactory; */ @Category({VerySlowRegionServerTests.class, MediumTests.class}) // Starts 100 threads public class TestAtomicOperation { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAtomicOperation.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAtomicOperation.class); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java index 6331e869196..73b596ad579 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,24 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; @@ -43,19 +42,24 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.junit.*; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static junit.framework.TestCase.assertTrue; -import static org.junit.Assert.assertEquals; - @Category({RegionServerTests.class, MediumTests.class}) public class TestBlocksRead { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBlocksRead.class); + private static final Logger LOG = LoggerFactory.getLogger(TestBlocksRead.class); @Rule public TestName testName = new TestName(); @@ -347,7 +351,7 @@ public class TestBlocksRead { // Baseline expected blocks read: 6. [HBASE-4532] kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 6, 7, 7); assertEquals(0, kvs.length); - + // File 7: Put back new data putData(FAMILY, "row", "col1", 11); putData(FAMILY, "row", "col2", 12); @@ -407,7 +411,7 @@ public class TestBlocksRead { assertEquals(2 * BLOOM_TYPE.length, result.size()); rs.close(); blocksEnd = getBlkCount(); - + assertEquals(2 * BLOOM_TYPE.length, blocksEnd - blocksStart); } finally { HBaseTestingUtility.closeRegionAndWAL(this.region); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java index a0babe8ceee..7db34ac2cbb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java @@ -16,10 +16,11 @@ * limitations under the License. */ package org.apache.hadoop.hbase.regionserver; + import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -36,12 +37,18 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @SuppressWarnings("deprecation") @Category({RegionServerTests.class, SmallTests.class}) public class TestBlocksScanned extends HBaseTestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBlocksScanned.class); + private static byte [] FAMILY = Bytes.toBytes("family"); private static byte [] COL = Bytes.toBytes("col"); private static byte [] START_KEY = Bytes.toBytes("aaa"); @@ -110,7 +117,7 @@ public class TestBlocksScanned extends HBaseTestCase { int expectResultSize = 'z' - 'a'; assertEquals(expectResultSize, results.size()); - int kvPerBlock = (int) Math.ceil(BLOCK_SIZE / + int kvPerBlock = (int) Math.ceil(BLOCK_SIZE / (double) KeyValueUtil.ensureKeyValue(results.get(0)).getLength()); Assert.assertEquals(2, kvPerBlock); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java index 0121aef9128..2eb73f5d44a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static java.util.Arrays.asList; @@ -43,6 +42,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -52,15 +52,15 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; @@ -68,18 +68,23 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; -import org.hamcrest.Description; -import org.hamcrest.Matcher; -import org.hamcrest.TypeSafeMatcher; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; + /** * This class attempts to unit test bulk HLog loading. */ @Category(SmallTests.class) public class TestBulkLoad { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBulkLoad.class); + @ClassRule public static TemporaryFolder testFolder = new TemporaryFolder(); private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java index 0f4ece6c2a3..579210697c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertTrue; @@ -29,6 +27,7 @@ import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; @@ -56,6 +55,7 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -74,6 +74,10 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) public class TestCacheOnWriteInSchema { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCacheOnWriteInSchema.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCacheOnWriteInSchema.class); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java index 734b930adf9..55d4947f378 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -29,11 +28,11 @@ import java.util.Iterator; import java.util.NavigableMap; import java.util.NavigableSet; import java.util.SortedSet; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.io.util.MemorySizeUtil; @@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -51,6 +51,11 @@ import org.junit.runners.Parameterized; @Category({RegionServerTests.class, SmallTests.class}) @RunWith(Parameterized.class) public class TestCellFlatSet { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellFlatSet.class); + @Parameterized.Parameters public static Object[] data() { return new Object[] { "SMALL_CHUNKS", "NORMAL_CHUNKS" }; // test with different chunk sizes diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java index a58a22e701c..4f46bcf3d95 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,21 +17,27 @@ */ package org.apache.hadoop.hbase.regionserver; +import java.util.Iterator; +import java.util.SortedSet; import junit.framework.TestCase; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import java.util.Iterator; -import java.util.SortedSet; - @Category({RegionServerTests.class, SmallTests.class}) public class TestCellSkipListSet extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellSkipListSet.class); + private final CellSet csls = new CellSet(CellComparatorImpl.COMPARATOR); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java index 96671683386..c20c42ec522 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,9 +17,14 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.ArrayList; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CacheEvictionStats; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -35,6 +39,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -42,14 +47,14 @@ import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; - -import static org.junit.Assert.assertEquals; - @Category(MediumTests.class) @RunWith(Parameterized.class) public class TestClearRegionBlockCache { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClearRegionBlockCache.class); + private static final Logger LOG = LoggerFactory.getLogger(TestClearRegionBlockCache.class); private static final TableName TABLE_NAME = TableName.valueOf("testClearRegionBlockCache"); private static final byte[] FAMILY = Bytes.toBytes("family"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java index 46fd7029fb7..21e7ac19ceb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java @@ -1,20 +1,19 @@ -/* - * Copyright The Apache Software Foundation - * - * 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 +/** + * 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. + * 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.regionserver; @@ -22,32 +21,36 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import java.util.UUID; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.LoadBalancer; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - /** * Test metrics incremented on region server operations. */ @Category({RegionServerTests.class, MediumTests.class}) public class TestClusterId { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestClusterId.class); + private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java index f6cd4e29d4f..9b63500f604 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; @@ -29,9 +27,8 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Set; - -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -45,19 +42,22 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) public class TestColumnSeeking { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestColumnSeeking.class); + @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); private static final Logger LOG = LoggerFactory.getLogger(TestColumnSeeking.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java index 4df21e0bf17..ca746c3a44c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java @@ -17,12 +17,14 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.util.Collection; +import static org.junit.Assert.assertEquals; +import java.util.Collection; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.*; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -32,16 +34,19 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; - import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; - @Category(MediumTests.class) public class TestCompactSplitThread { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactSplitThread.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCompactSplitThread.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final TableName tableName = TableName.valueOf(getClass().getSimpleName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java index 87e4affabb7..d68191cd4f2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,15 +17,19 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.lang.management.ManagementFactory; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -50,21 +53,22 @@ import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - /** * compacted memstore test case */ @Category({RegionServerTests.class, MediumTests.class}) public class TestCompactingMemStore extends TestDefaultMemStore { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactingMemStore.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCompactingMemStore.class); protected static ChunkCreator chunkCreator; protected HRegion region; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java index 030658e5647..0036426ff94 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,8 +17,14 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -29,9 +34,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.Threads; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -39,15 +42,17 @@ import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.List; - /** * compacted memstore test case */ @Category({RegionServerTests.class, MediumTests.class}) @RunWith(Parameterized.class) public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactingToCellFlatMapMemStore.class); + @Parameterized.Parameters public static Object[] data() { return new Object[] { "CHUNK_MAP", "ARRAY_MAP" }; // test different immutable indexes diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java index f017617f86d..75110fd9a0f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -39,13 +38,13 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ChoreService; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -73,6 +72,7 @@ import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Assume; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -81,12 +81,16 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; - /** * Test compaction framework and common functions */ @Category({RegionServerTests.class, MediumTests.class}) public class TestCompaction { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompaction.class); + @Rule public TestName name = new TestName(); private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU(); protected Configuration conf = UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java index e9f381e6b21..aa5365c37a3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java @@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; @@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -59,6 +61,11 @@ import org.mockito.Mockito; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestCompactionArchiveConcurrentClose { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactionArchiveConcurrentClose.class); + public HBaseTestingUtility testUtil; private Path testDir; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java index 9c085e5eed2..b8780af5dbf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java @@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; @@ -51,6 +52,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -64,6 +66,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; */ @Category(MediumTests.class) public class TestCompactionArchiveIOException { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactionArchiveIOException.class); + private static final String ERROR_FILE = "fffffffffffffffffdeadbeef"; public HBaseTestingUtility testUtil; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java index 0dafd8053cc..22a888de6e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,8 +22,8 @@ import static org.junit.Assert.assertEquals; import java.io.FileNotFoundException; import java.io.IOException; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -39,6 +38,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -51,6 +51,11 @@ import org.slf4j.LoggerFactory; */ @Category(MediumTests.class) public class TestCompactionFileNotFound { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactionFileNotFound.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCompactionFileNotFound.class); private static final HBaseTestingUtility util = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java index 457f0029872..78042cc976f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java @@ -23,8 +23,8 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -61,6 +62,11 @@ import org.slf4j.LoggerFactory; @RunWith(Parameterized.class) @Category({ RegionServerTests.class, LargeTests.class }) public class TestCompactionInDeadRegionServer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactionInDeadRegionServer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCompactionInDeadRegionServer.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionLifeCycleTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionLifeCycleTracker.java index 7f4ecca0b24..6a6e9076872 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionLifeCycleTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionLifeCycleTracker.java @@ -27,11 +27,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Optional; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell.Type; import org.apache.hadoop.hbase.CellBuilderFactory; import org.apache.hadoop.hbase.CellBuilderType; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -57,6 +57,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -67,6 +68,10 @@ import org.junit.experimental.categories.Category; @Category({ CoprocessorTests.class, MediumTests.class }) public class TestCompactionLifeCycleTracker { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactionLifeCycleTracker.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName NAME = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java index 241d0648dec..e0f87a608ab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; @@ -25,9 +24,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Random; - -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.CompactionState; import org.apache.hadoop.hbase.client.Put; @@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -47,6 +47,11 @@ import org.slf4j.LoggerFactory; /** Unit tests to test retrieving table/region compaction state*/ @Category({VerySlowRegionServerTests.class, LargeTests.class}) public class TestCompactionState { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactionState.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCompactionState.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static Random random = new Random(); @@ -235,5 +240,5 @@ public class TestCompactionState { TEST_UTIL.flush(); puts.clear(); } - } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionWithCoprocessor.java index 4ad92a33c3f..572a0baad12 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionWithCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionWithCoprocessor.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,9 +17,11 @@ */ package org.apache.hadoop.hbase.regionserver; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -28,6 +30,11 @@ import org.junit.experimental.categories.Category; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestCompactionWithCoprocessor extends TestCompaction { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactionWithCoprocessor.class); + /** constructor */ public TestCompactionWithCoprocessor() throws Exception { super(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java index 12763f9fff7..25eb700ee0a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; @@ -31,12 +29,12 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -56,6 +54,7 @@ import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.BloomFilterUtil; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -68,6 +67,10 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) public class TestCompoundBloomFilter { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompoundBloomFilter.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -121,7 +124,9 @@ public class TestCompoundBloomFilter { private FileSystem fs; private BlockCache blockCache; - /** A message of the form "in test#:" to include in logging. */ + /** + * A message of the form in test#<number>" to include in logging. + */ private String testIdMsg; private static final int GENERATION_SEED = 2319; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicy.java index 4f02d8a162e..37635295f25 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicy.java @@ -18,18 +18,23 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; import org.apache.hadoop.hbase.regionserver.compactions.ExponentialCompactionWindowFactory; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, SmallTests.class }) public class TestDateTieredCompactionPolicy extends AbstractTestDateTieredCompactionPolicy { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDateTieredCompactionPolicy.class); + @Override protected void config() { super.config(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicyOverflow.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicyOverflow.java index 530d2845abd..29f9ea74414 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicyOverflow.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicyOverflow.java @@ -18,18 +18,23 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; import org.apache.hadoop.hbase.regionserver.compactions.ExponentialCompactionWindowFactory; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, SmallTests.class }) public class TestDateTieredCompactionPolicyOverflow extends AbstractTestDateTieredCompactionPolicy { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDateTieredCompactionPolicyOverflow.class); + @Override protected void config() { super.config(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java index 4dea3154c10..3c3bbd8e33c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; @@ -28,12 +28,17 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.TimeOffsetEnvironmentEdge; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestDefaultCompactSelection extends TestCompactionPolicy { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDefaultCompactSelection.class); + @Override protected void config() { super.config(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index 53e04e049a3..cd8539a9c9e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -32,10 +31,10 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -61,11 +60,11 @@ import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.AfterClass; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,10 +75,13 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** memstore test case */ @Category({RegionServerTests.class, MediumTests.class}) public class TestDefaultMemStore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDefaultMemStore.class); + private static final Logger LOG = LoggerFactory.getLogger(TestDefaultMemStore.class); @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); protected AbstractMemStore memstore; protected static final int ROW_COUNT = 10; protected static final int QUALIFIER_COUNT = ROW_COUNT; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java index a1a602279c8..98f972989e7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,23 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @Category({RegionServerTests.class, SmallTests.class}) public class TestDefaultStoreEngine { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDefaultStoreEngine.class); + public static class DummyStoreFlusher extends DefaultStoreFlusher { public DummyStoreFlusher(Configuration conf, HStore store) { super(conf, store); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java index 9dd688599d2..859f0695a68 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java @@ -19,9 +19,9 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.Random; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -45,6 +46,10 @@ import org.junit.rules.TestName; @Category(MediumTests.class) public class TestDeleteMobTable { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDeleteMobTable.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static byte[] FAMILY = Bytes.toBytes("family"); private final static byte[] QF = Bytes.toBytes("qualifier"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java index ec714be7ae8..52d6aae81dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -26,11 +26,10 @@ import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.List; - import javax.crypto.spec.SecretKeySpec; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -60,6 +60,11 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) public class TestEncryptionKeyRotation { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEncryptionKeyRotation.class); + private static final Logger LOG = LoggerFactory.getLogger(TestEncryptionKeyRotation.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Configuration conf = TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java index 42ef5338681..eef0b901adc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -23,9 +23,9 @@ import static org.junit.Assert.assertTrue; import java.security.Key; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -42,11 +42,17 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, MediumTests.class}) public class TestEncryptionRandomKeying { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEncryptionRandomKeying.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Configuration conf = TEST_UTIL.getConfiguration(); private static HTableDescriptor htd; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index a770f8caa9c..2c123411a6c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -28,10 +28,10 @@ import java.util.Random; import java.util.Set; import java.util.TreeSet; import java.util.stream.Collectors; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ChoreService; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -57,16 +57,23 @@ import org.apache.hadoop.hbase.util.StoppableImplementation; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Iterators; @Category(LargeTests.class) public class TestEndToEndSplitTransaction { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEndToEndSplitTransaction.class); + private static final Logger LOG = LoggerFactory.getLogger(TestEndToEndSplitTransaction.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Configuration CONF = TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java index bbf06fdf959..a7e959f20aa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -28,12 +27,12 @@ import java.lang.ref.SoftReference; import java.util.ArrayList; import java.util.Collections; import java.util.List; - import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PositionedReadable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -52,6 +51,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assume; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -65,6 +65,11 @@ import org.slf4j.LoggerFactory; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestFSErrorsExposed { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSErrorsExposed.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFSErrorsExposed.class); HBaseTestingUtility util = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java index 84e424eb84f..e27a9862784 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,7 +17,6 @@ */ package org.apache.hadoop.hbase.regionserver; - import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; @@ -26,11 +24,11 @@ import static org.mockito.Mockito.when; import java.io.IOException; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DroppedSnapshotException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; @@ -47,6 +45,7 @@ import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -62,6 +61,11 @@ import org.slf4j.LoggerFactory; */ @Category({MediumTests.class}) public class TestFailedAppendAndSync { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFailedAppendAndSync.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFailedAppendAndSync.class); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFlushLifeCycleTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFlushLifeCycleTracker.java index 8550bdc88d6..278afe029ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFlushLifeCycleTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFlushLifeCycleTracker.java @@ -27,10 +27,10 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.util.Optional; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.hbase.Cell.Type; import org.apache.hadoop.hbase.CellBuilderFactory; import org.apache.hadoop.hbase.CellBuilderType; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -49,6 +49,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,6 +59,10 @@ import org.junit.experimental.categories.Category; @Category({ CoprocessorTests.class, MediumTests.class }) public class TestFlushLifeCycleTracker { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFlushLifeCycleTracker.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final TableName NAME = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFlushRegionEntry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFlushRegionEntry.java index 12fdb77ed73..8034365e376 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFlushRegionEntry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFlushRegionEntry.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -38,6 +40,11 @@ import org.junit.rules.TestName; @Category({ RegionServerTests.class, MediumTests.class }) public class TestFlushRegionEntry { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFlushRegionEntry.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index aa38b7aacad..0db73ba7d91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,17 +17,20 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Objects; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -45,6 +47,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -52,15 +55,17 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - /** * TestGet is a medley of tests of get all done up as a single test. * This class */ @Category({RegionServerTests.class, MediumTests.class}) public class TestGetClosestAtOrBefore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGetClosestAtOrBefore.class); + @Rule public TestName testName = new TestName(); private static final Logger LOG = LoggerFactory.getLogger(TestGetClosestAtOrBefore.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java index 9845d8a4998..e8dbdacbab3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -32,9 +31,7 @@ import java.util.List; import java.util.NavigableSet; import java.util.Optional; import java.util.concurrent.ConcurrentSkipListSet; - import javax.crypto.spec.SecretKeySpec; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -42,6 +39,7 @@ import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -71,6 +69,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -81,6 +80,11 @@ import org.slf4j.LoggerFactory; @Category(MediumTests.class) public class TestHMobStore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHMobStore.class); + public static final Logger LOG = LoggerFactory.getLogger(TestHMobStore.class); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index ee11075d780..e443503674d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -71,7 +70,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ArrayBackedTag; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell.Type; import org.apache.hadoop.hbase.CellBuilderFactory; @@ -80,6 +78,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; import org.apache.hadoop.hbase.DroppedSnapshotException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -173,7 +172,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatcher; import org.mockito.Mockito; @@ -202,14 +200,16 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescript @Category({VerySlowRegionServerTests.class, LargeTests.class}) @SuppressWarnings("deprecation") public class TestHRegion { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegion.class); + // Do not spin up clusters in here. If you need to spin up a cluster, do it // over in TestHRegionOnCluster. private static final Logger LOG = LoggerFactory.getLogger(TestHRegion.class); @Rule public TestName name = new TestName(); - @ClassRule - public static final TestRule timeout = - CategoryBasedTimeout.forClass(TestHRegion.class); @Rule public final ExpectedException thrown = ExpectedException.none(); private static final String COLUMN_FAMILY = "MyCF"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java index 0514005b603..a4e8e19f67a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; @@ -29,7 +27,6 @@ import java.io.IOException; import java.net.URI; import java.util.Collection; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -37,6 +34,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.TableName; @@ -46,11 +44,12 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.fs.HFileSystem; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.util.Progressable; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,8 +57,13 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({RegionServerTests.class, SmallTests.class}) +@Category({RegionServerTests.class, MediumTests.class}) public class TestHRegionFileSystem { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionFileSystem.class); + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestHRegionFileSystem.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java index ae54505a166..50b675dfb0c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,32 +25,40 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.RegionState; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.MD5Hash; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo; + @Category({RegionServerTests.class, SmallTests.class}) public class TestHRegionInfo { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionInfo.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java index 9b89768dbfb..fd448023b4e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java @@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,7 +50,6 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * Tests that need to spin up a cluster testing an {@link HRegion}. Use * {@link TestHRegion} if you don't need a cluster, if you can test w/ a @@ -58,6 +58,10 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) public class TestHRegionOnCluster { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionOnCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHRegionOnCluster.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java index dcd7ff7b9d4..31eea48b5b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java @@ -47,6 +47,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -81,6 +82,7 @@ import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay; import org.apache.hadoop.util.StringUtils; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -110,6 +112,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescript @Category(MediumTests.class) public class TestHRegionReplayEvents { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionReplayEvents.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHRegion.class); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java index 034d9d7391d..51d2f0230ae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java @@ -29,11 +29,11 @@ import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread; import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.client.ClientServiceCallable; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Result; @@ -56,6 +55,7 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.SecureBulkLoadClient; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.io.compress.Compression; @@ -76,6 +76,7 @@ import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -83,7 +84,9 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest; @@ -95,6 +98,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegi @RunWith(Parameterized.class) @Category({RegionServerTests.class, LargeTests.class}) public class TestHRegionServerBulkLoad { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionServerBulkLoad.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHRegionServerBulkLoad.class); protected static HBaseTestingUtility UTIL = new HBaseTestingUtility(); protected final static Configuration conf = UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java index 3e0b72733ce..de01401d262 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java @@ -21,9 +21,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread; import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext; @@ -33,21 +33,24 @@ import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.RpcRetryingCaller; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest; + /** * Tests bulk loading of HFiles with old non-secure client for backward compatibility. Will be * removed when old non-secure client for backward compatibility is not supported. @@ -55,6 +58,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @RunWith(Parameterized.class) @Category({RegionServerTests.class, LargeTests.class}) public class TestHRegionServerBulkLoadWithOldClient extends TestHRegionServerBulkLoad { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionServerBulkLoadWithOldClient.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHRegionServerBulkLoadWithOldClient.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java index ac10a35292b..dfe52d06001 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,8 +18,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; - -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; @@ -29,7 +27,6 @@ import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.apache.hadoop.hbase.wal.WAL; import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,12 +37,14 @@ import org.slf4j.LoggerFactory; @Category({VerySlowRegionServerTests.class, LargeTests.class}) @SuppressWarnings("deprecation") public class TestHRegionWithInMemoryFlush extends TestHRegion{ + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHRegionWithInMemoryFlush.class); + // Do not spin up clusters in here. If you need to spin up a cluster, do it // over in TestHRegionOnCluster. private static final Logger LOG = LoggerFactory.getLogger(TestHRegionWithInMemoryFlush.class); - @ClassRule - public static final TestRule timeout = - CategoryBasedTimeout.forClass(TestHRegionWithInMemoryFlush.class); /** * @return A region on which you must call diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java index 288333b3432..9d6aedb4eec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java @@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -105,6 +106,7 @@ import org.apache.hadoop.util.Progressable; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -120,6 +122,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; */ @Category({ RegionServerTests.class, MediumTests.class }) public class TestHStore { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHStore.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHStore.class); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java index 1a38d7f7105..72da1a3f919 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,6 +17,9 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -29,20 +31,20 @@ import java.util.Map; import java.util.OptionalLong; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -65,23 +67,27 @@ import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Joiner; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - /** * Test HStoreFile */ @Category({RegionServerTests.class, SmallTests.class}) public class TestHStoreFile extends HBaseTestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHStoreFile.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHStoreFile.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHdfsSnapshotHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHdfsSnapshotHRegion.java index 17e698f8c13..feea0865a05 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHdfsSnapshotHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHdfsSnapshotHRegion.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,6 +21,7 @@ import java.io.IOException; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -34,12 +34,17 @@ import org.apache.hadoop.hdfs.DFSClient; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, MediumTests.class}) public class TestHdfsSnapshotHRegion { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHdfsSnapshotHRegion.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final String SNAPSHOT_NAME = "foo_snapshot"; private Table table; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java index c71e0c77393..41bfd48c0ad 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -26,11 +25,11 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.lang.management.ManagementFactory; import java.util.Iterator; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -52,12 +51,17 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestHeapMemoryManager { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHeapMemoryManager.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private long maxHeapSize = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java index d218389c899..8e84f8a64be 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,13 +21,13 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Random; - import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -45,6 +44,7 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,6 +58,11 @@ import org.slf4j.LoggerFactory; */ @Category({RegionServerTests.class, LargeTests.class}) public class TestJoinedScanners { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestJoinedScanners.class); + private static final Logger LOG = LoggerFactory.getLogger(TestJoinedScanners.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -188,24 +193,24 @@ public class TestJoinedScanners { "Data block encoding; Default: FAST_DIFF"); encodingOption.setRequired(false); options.addOption(encodingOption); - + Option ratioOption = new Option("r", "selectionRatio", true, "Ratio of selected rows using essential column family"); ratioOption.setRequired(false); options.addOption(ratioOption); - + Option widthOption = new Option("w", "valueWidth", true, "Width of value for non-essential column family"); widthOption.setRequired(false); options.addOption(widthOption); - + CommandLineParser parser = new GnuParser(); CommandLine cmd = parser.parse(options, args); if (args.length < 1) { HelpFormatter formatter = new HelpFormatter(); formatter.printHelp("TestJoinedScanners", options, true); } - + if (cmd.hasOption("e")) { blockEncoding = DataBlockEncoding.valueOf(cmd.getOptionValue("e")); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java index ee29ef257e5..12a53124f73 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java @@ -27,14 +27,14 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeepDeletedCells; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,6 +56,11 @@ import org.junit.rules.TestName; @Category({RegionServerTests.class, SmallTests.class}) public class TestKeepDeletes { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeepDeletes.class); + HBaseTestingUtility hbu = HBaseTestingUtility.createLocalHTU(); private final byte[] T0 = Bytes.toBytes("0"); private final byte[] T1 = Bytes.toBytes("1"); @@ -68,7 +74,7 @@ public class TestKeepDeletes { private final byte[] c1 = COLUMNS[1]; @Rule public TestName name = new TestName(); - + @Before public void setUp() throws Exception { /* HBASE-6832: [WINDOWS] Tests should use explicit timestamp for Puts, and not rely on diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java index d9956de8c0e..839bcaaff1c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,16 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -33,11 +31,17 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CollectionBackedScanner; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestKeyValueHeap extends HBaseTestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeyValueHeap.class); + private byte[] row1 = Bytes.toBytes("row1"); private byte[] fam1 = Bytes.toBytes("fam1"); private byte[] col1 = Bytes.toBytes("col1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java index 0c071a6b00b..da46ef54a3e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,26 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import java.io.IOException; - import junit.framework.TestCase; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestKeyValueScanFixture extends TestCase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestKeyValueScanFixture.class); + public void testKeyValueScanFixture() throws IOException { KeyValue kvs[] = new KeyValue[]{ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java index 0840a63167d..e261886b518 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -34,10 +33,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -61,6 +60,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -76,6 +76,11 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) @RunWith(Parameterized.class) public class TestMajorCompaction { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMajorCompaction.class); + @Parameterized.Parameters public static Object[] data() { return new Object[] { "NONE", "BASIC", "EAGER" }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java index e4b37541ae3..2a89b99ed56 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,8 +23,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.util.concurrent.Semaphore; - import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; @@ -34,15 +33,21 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.ClassRule; import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, MediumTests.class}) public class TestMasterAddressTracker { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterAddressTracker.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMasterAddressTracker.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java index 69d7589078d..891e8358407 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java @@ -1,25 +1,33 @@ /** - * Copyright The Apache Software Foundation + * 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 * - * 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 + * 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. + * 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.regionserver; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ByteBufferKeyValue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.UnexpectedStateException; @@ -30,23 +38,20 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; -import java.lang.management.ManagementFactory; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Random; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * Test the {@link MemStoreChunkPool} class */ @Category({RegionServerTests.class, SmallTests.class}) public class TestMemStoreChunkPool { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMemStoreChunkPool.class); + private final static Configuration conf = new Configuration(); private static ChunkCreator chunkCreator; private static boolean chunkPoolDisabledBeforeTest; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java index 795fa2f8c6a..52491e77c89 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -27,10 +26,10 @@ import java.util.List; import java.util.Map; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ByteBufferKeyValue; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -42,18 +41,22 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.primitives.Ints; -import org.junit.experimental.categories.Category; - @Category({RegionServerTests.class, SmallTests.class}) public class TestMemStoreLAB { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMemStoreLAB.class); + private final static Configuration conf = new Configuration(); private static final byte[] rk = Bytes.toBytes("r1"); @@ -126,16 +129,16 @@ public class TestMemStoreLAB { Configuration conf = new Configuration(); MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf); - + final AtomicInteger totalAllocated = new AtomicInteger(); - + final MemStoreLAB mslab = new MemStoreLABImpl(); List> allocations = Lists.newArrayList(); - + for (int i = 0; i < 10; i++) { final List allocsByThisThread = Lists.newLinkedList(); allocations.add(allocsByThisThread); - + TestThread t = new MultithreadedTestUtil.RepeatingTestThread(ctx) { private Random r = new Random(); @Override @@ -176,7 +179,7 @@ public class TestMemStoreLAB { oldVal); } assertEquals("Sanity check test", sizeCounted, totalAllocated.get()); - + // Now check each byte array to make sure allocations don't overlap for (Map allocsInChunk : mapsByChunk.values()) { // since we add the chunkID at the 0th offset of the chunk and the @@ -304,7 +307,7 @@ public class TestMemStoreLAB { } return Ints.compare(this.offset, e.offset); } - + @Override public String toString() { return "AllocRecord(offset=" + this.offset + ", size=" + size + ")"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java index 13780877326..0813a8c47ae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,10 +25,10 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ByteBufferKeyValue; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -37,14 +36,19 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; - @Ignore // See HBASE-19742 for issue on reenabling. @Category({RegionServerTests.class, SmallTests.class}) public class TestMemstoreLABWithoutPool { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMemstoreLABWithoutPool.class); + private final static Configuration conf = new Configuration(); private static final byte[] rk = Bytes.toBytes("r1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsHeapMemoryManager.java index 44f49619d18..9958ca664bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsHeapMemoryManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsHeapMemoryManager.java @@ -1,30 +1,31 @@ -/* - * Copyright The Apache Software Foundation +/** + * 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 * - * 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 + * 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. + * 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.regionserver; import static org.junit.Assert.assertNotNull; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -33,6 +34,11 @@ import org.junit.experimental.categories.Category; */ @Category({ RegionServerTests.class, SmallTests.class }) public class TestMetricsHeapMemoryManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsHeapMemoryManager.class); + public static MetricsAssertHelper HELPER = CompatibilitySingletonFactory .getInstance(MetricsAssertHelper.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java index e86f1e56455..2119b911e7d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java @@ -15,19 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.test.MetricsAssertHelper; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestMetricsRegion { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsRegion.class); + public MetricsAssertHelper HELPER = CompatibilityFactory.getInstance(MetricsAssertHelper.class); @@ -37,7 +42,7 @@ public class TestMetricsRegion { MetricsRegionAggregateSource agg = mr.getSource().getAggregateSource(); HELPER.assertGauge( - "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_storeCount", + "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_storeCount", 101, agg); HELPER.assertGauge( "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_storeFileCount", @@ -62,7 +67,7 @@ public class TestMetricsRegion { "filteredReadRequestCount", 107, agg); HELPER.assertCounter( - "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_replicaid", + "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_replicaid", 0, agg); mr.close(); @@ -70,7 +75,7 @@ public class TestMetricsRegion { mr = new MetricsRegion(new MetricsRegionWrapperStub(1)); agg = mr.getSource().getAggregateSource(); HELPER.assertGauge( - "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_storeCount", + "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_storeCount", 101, agg); HELPER.assertGauge( "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_storeFileCount", diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java index 67b186e4a93..bfc0b63572f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java @@ -17,24 +17,31 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.junit.Assert.assertNotNull; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.util.JvmPauseMonitor; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertNotNull; - /** * Unit test version of rs metrics tests. */ @Category({RegionServerTests.class, SmallTests.class}) public class TestMetricsRegionServer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsRegionServer.class); + public static MetricsAssertHelper HELPER = CompatibilityFactory.getInstance(MetricsAssertHelper.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableAggregate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableAggregate.java index 5130e57c4bd..e3bac65b1e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableAggregate.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableAggregate.java @@ -20,15 +20,21 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import org.apache.hadoop.hbase.CompatibilityFactory; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestMetricsTableAggregate { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsTableAggregate.class); + public static MetricsAssertHelper HELPER = CompatibilityFactory.getInstance(MetricsAssertHelper.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableLatencies.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableLatencies.java index a845acde9c6..43e8d58564f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableLatencies.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableLatencies.java @@ -22,16 +22,22 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import org.apache.hadoop.hbase.CompatibilityFactory; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestMetricsTableLatencies { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsTableLatencies.class); + public static MetricsAssertHelper HELPER = CompatibilityFactory.getInstance(MetricsAssertHelper.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java index 4ffe5d23604..13ad7f89e07 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,21 +23,22 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeepDeletedCells; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.filter.TimestampsFilter; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,6 +49,11 @@ import org.junit.rules.TestName; */ @Category({RegionServerTests.class, SmallTests.class}) public class TestMinVersions { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMinVersions.class); + HBaseTestingUtility hbu = HBaseTestingUtility.createLocalHTU(); private final byte[] T0 = Bytes.toBytes("0"); private final byte[] T1 = Bytes.toBytes("1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMiniBatchOperationInProgress.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMiniBatchOperationInProgress.java index c3472b511ba..87d41853755 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMiniBatchOperationInProgress.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMiniBatchOperationInProgress.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -21,19 +21,25 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestMiniBatchOperationInProgress { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMiniBatchOperationInProgress.class); + @Test public void testMiniBatchOperationInProgressMethods() { Pair[] operations = new Pair[10]; @@ -42,7 +48,7 @@ public class TestMiniBatchOperationInProgress { for (int i = 0; i < 10; i++) { operations[i] = new Pair<>(new Put(Bytes.toBytes(i)), null); } - MiniBatchOperationInProgress> miniBatch = + MiniBatchOperationInProgress> miniBatch = new MiniBatchOperationInProgress<>(operations, retCodeDetails, walEditsFromCoprocessors, 0, 5, 5); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java index 4ae92a4c24b..e53ed363e5d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,6 +24,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,11 +52,16 @@ import org.slf4j.LoggerFactory; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestMinorCompaction { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMinorCompaction.class); + @Rule public TestName name = new TestName(); private static final Logger LOG = LoggerFactory.getLogger(TestMinorCompaction.class.getName()); private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU(); protected Configuration conf = UTIL.getConfiguration(); - + private HRegion r = null; private HTableDescriptor htd = null; private int compactionThreshold; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java index c6a300fe119..814c86fa70e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -32,7 +31,6 @@ import java.util.List; import java.util.Map; import java.util.Random; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -40,6 +38,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -67,6 +66,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -79,6 +79,11 @@ import org.slf4j.LoggerFactory; */ @Category(MediumTests.class) public class TestMobStoreCompaction { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobStoreCompaction.class); + @Rule public TestName name = new TestName(); static final Logger LOG = LoggerFactory.getLogger(TestMobStoreCompaction.class.getName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java index 59e4ded1262..7c6c6fe786e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,13 +20,13 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -62,6 +62,10 @@ import org.junit.rules.TestName; @Category(MediumTests.class) public class TestMobStoreScanner { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobStoreScanner.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static byte [] row1 = Bytes.toBytes("row1"); private final static byte [] row2 = Bytes.toBytes("row2"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java index 19685ea395e..20d28b3e4e3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -33,16 +32,16 @@ import java.util.Map; import java.util.Random; import java.util.Set; import java.util.TreeSet; - import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; @@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -66,6 +66,10 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) public class TestMultiColumnScanner { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiColumnScanner.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMultiColumnScanner.class); private static final String TABLE_NAME = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java index 39f76203fc3..8e11ed58d20 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,25 +20,28 @@ package org.apache.hadoop.hbase.regionserver; import static org.mockito.Mockito.verify; import java.io.IOException; - -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest; +import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction; -import org.apache.hadoop.hbase.util.Bytes; + import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; + /** * Tests logging of large batch commands via Multi. Tests are fast, but uses a mini-cluster (to test * via "Multi" commands) so classified as MediumTests @@ -47,6 +49,10 @@ import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; @Category(MediumTests.class) public class TestMultiLogThreshold { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiLogThreshold.class); + private static RSRpcServices SERVICES; private static HBaseTestingUtility TEST_UTIL; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java index a09c8cd0b78..919224b7094 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java @@ -20,10 +20,11 @@ package org.apache.hadoop.hbase.regionserver; import java.util.Random; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -32,6 +33,11 @@ import org.junit.experimental.categories.Category; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestMultiVersionConcurrencyControl { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiVersionConcurrencyControl.class); + static class Writer implements Runnable { final AtomicBoolean finished; final MultiVersionConcurrencyControl mvcc; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControlBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControlBasic.java index 920440ee59a..6ac8aaf814e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControlBasic.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControlBasic.java @@ -19,8 +19,10 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -30,6 +32,11 @@ import org.junit.experimental.categories.Category; */ @Category({RegionServerTests.class, SmallTests.class}) public class TestMultiVersionConcurrencyControlBasic { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiVersionConcurrencyControlBasic.class); + @Test public void testSimpleMvccOps() { MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMutateRowsRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMutateRowsRecovery.java index 02b8e5e137d..328de3261e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMutateRowsRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMutateRowsRecovery.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -45,11 +45,17 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) public class TestMutateRowsRecovery { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMutateRowsRecovery.class); + private MiniHBaseCluster cluster = null; private Connection connection = null; private static final int NB_SERVERS = 3; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java index ebd2f01d131..897fc3c00d5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java @@ -17,8 +17,11 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.io.IOException; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -36,17 +39,19 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; - @Category({ RegionServerTests.class, MediumTests.class }) public class TestNewVersionBehaviorFromClientSide { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNewVersionBehaviorFromClientSide.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] ROW = Bytes.toBytes("r1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestObservedExceptionsInBatch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestObservedExceptionsInBatch.java index 64237fdd574..a84a9a9c8f5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestObservedExceptionsInBatch.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestObservedExceptionsInBatch.java @@ -20,9 +20,11 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.regionserver.HRegion.ObservedExceptionsInBatch; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -32,6 +34,10 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestObservedExceptionsInBatch { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestObservedExceptionsInBatch.class); + private ObservedExceptionsInBatch observedExceptions; @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java index ec161dc37e7..3a3b6db9c26 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java @@ -23,26 +23,27 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Random; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -50,16 +51,20 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * Testing of multiPut in parallel. * */ @Category({RegionServerTests.class, MediumTests.class}) public class TestParallelPut { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestParallelPut.class); + private static final Logger LOG = LoggerFactory.getLogger(TestParallelPut.class); - @Rule public TestName name = new TestName(); - + @Rule public TestName name = new TestName(); + private HRegion region = null; private static HBaseTestingUtility HBTU = new HBaseTestingUtility(); private static final int THREADS100 = 100; @@ -94,14 +99,14 @@ public class TestParallelPut { EnvironmentEdgeManagerTestHelper.reset(); if (region != null) region.close(true); } - + public String getName() { return name.getMethodName(); } ////////////////////////////////////////////////////////////////////////////// // New tests that don't spin up a mini cluster but rather just test the - // individual code pieces in the HRegion. + // individual code pieces in the HRegion. ////////////////////////////////////////////////////////////////////////////// /** @@ -154,7 +159,7 @@ public class TestParallelPut { " Ignoring....", e); } } - LOG.info("testParallelPuts successfully verified " + + LOG.info("testParallelPuts successfully verified " + (numOps * THREADS100) + " put operations."); } @@ -210,7 +215,7 @@ public class TestParallelPut { // iterate for the specified number of operations for (int i=0; i - * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * 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. @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hbase.regionserver; +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -31,21 +33,23 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.LargeTests; - import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.IOException; - @Category({RegionServerTests.class, LargeTests.class}) public class TestRemoveRegionMetrics { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRemoveRegionMetrics.class); + private static MiniHBaseCluster cluster; private static Configuration conf; private static HBaseTestingUtility TEST_UTIL; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java index 570422ee862..0b68754a684 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -22,17 +21,18 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,6 +40,11 @@ import org.junit.rules.TestName; @Category({RegionServerTests.class, SmallTests.class}) public class TestResettingCounters { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestResettingCounters.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java index 2b94b0dd28b..66e18472ea0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java @@ -1,20 +1,19 @@ /** - * Copyright The Apache Software Foundation + * 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 * - * 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 + * 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. + * 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.regionserver; @@ -28,11 +27,11 @@ import java.util.Collections; import java.util.List; import java.util.Map; import java.util.NavigableSet; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -59,12 +58,14 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -72,6 +73,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestReversibleScanners { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReversibleScanners.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReversibleScanners.class); HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowTooBig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowTooBig.java index f9d0d1a4d47..904c2b9409c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowTooBig.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowTooBig.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; +import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RowTooBigException; @@ -29,17 +29,21 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; - /** * Test case to check HRS throws {@link org.apache.hadoop.hbase.client.RowTooBigException} * when row size exceeds configured limits. */ @Category({RegionServerTests.class, MediumTests.class}) public class TestRowTooBig { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowTooBig.class); + private final static HBaseTestingUtility HTU = HBaseTestingUtility.createLocalHTU(); private static Path rootRegionDir; private static final HTableDescriptor TEST_HTD = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java index 47dca514f4f..879d459822f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertTrue; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; import org.apache.hadoop.hbase.ipc.RWQueueRpcExecutor; @@ -34,7 +34,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; /** * A silly test that does nothing but make sure an rpcscheduler factory makes what it says @@ -42,9 +41,12 @@ import org.junit.rules.TestRule; */ @Category(SmallTests.class) public class TestRpcSchedulerFactory { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRpcSchedulerFactory.class); + @Rule public TestName testName = new TestName(); - @ClassRule public static TestRule timeout = - CategoryBasedTimeout.forClass(TestRpcSchedulerFactory.class); private Configuration conf; @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java index 4cf2964e7af..5b78be2e02f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java @@ -1,6 +1,4 @@ -/* - * Copyright The Apache Software Foundation - * +/** * 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 @@ -25,12 +23,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; @@ -46,9 +42,12 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,6 +57,11 @@ import org.junit.experimental.categories.Category; * w.r.t. essential column family optimization */ public class TestSCVFWithMiniCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSCVFWithMiniCluster.class); + private static final TableName HBASE_TABLE_NAME = TableName.valueOf("TestSCVFWithMiniCluster"); private static final byte[] FAMILY_A = Bytes.toBytes("a"); @@ -122,7 +126,7 @@ public class TestSCVFWithMiniCluster { new BinaryComparator(Bytes.toBytes("false"))); ((SingleColumnValueFilter) scanFilter).setFilterIfMissing(true); } - + @AfterClass public static void tearDown() throws Exception { htable.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java index 00e573ff554..ae692e96c69 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -30,12 +29,12 @@ import java.util.Comparator; import java.util.List; import java.util.NavigableSet; import java.util.TreeSet; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.KeyValue; @@ -49,6 +48,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -65,6 +65,10 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, SmallTests.class}) public class TestScanWithBloomError { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScanWithBloomError.class); + private static final Logger LOG = LoggerFactory.getLogger(TestScanWithBloomError.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java index afbb48d6a18..a403ece7b3a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -30,10 +29,9 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; import java.util.List; - -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -56,11 +54,11 @@ import org.apache.hadoop.hbase.filter.WhileMatchFilter; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,9 +67,12 @@ import org.slf4j.LoggerFactory; */ @Category({RegionServerTests.class, SmallTests.class}) public class TestScanner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScanner.class); + @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); private static final Logger LOG = LoggerFactory.getLogger(TestScanner.class); private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java index 11cc365a766..5aec32aa3d2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java @@ -25,7 +25,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; - import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -34,6 +33,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTestConst; @@ -57,11 +57,13 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; @@ -76,6 +78,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRespon @Category(MediumTests.class) public class TestScannerHeartbeatMessages { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerHeartbeatMessages.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Table TABLE = null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerRetriableFailure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerRetriableFailure.java index a6aea6a3dfc..aaee8cb7b7b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerRetriableFailure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerRetriableFailure.java @@ -15,48 +15,52 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.TestTableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.TestTableName; - import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - @Category({RegionServerTests.class, LargeTests.class}) public class TestScannerRetriableFailure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerRetriableFailure.class); + private static final Logger LOG = LoggerFactory.getLogger(TestScannerRetriableFailure.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java index 63ea993f0ec..f30f0842738 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java @@ -23,11 +23,11 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY; import java.io.IOException; import java.util.List; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -57,6 +58,11 @@ import org.junit.rules.TestName; @Category({RegionServerTests.class, MediumTests.class}) public class TestScannerWithBulkload { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerWithBulkload.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithCorruptHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithCorruptHFile.java index 51fff854cfb..a6ec9fc6fbf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithCorruptHFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithCorruptHFile.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,8 +20,8 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.List; import java.util.Optional; - import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -52,6 +52,11 @@ import org.junit.rules.TestName; */ @Category(MediumTests.class) public class TestScannerWithCorruptHFile { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScannerWithCorruptHFile.class); + @Rule public TestName name = new TestName(); private static final byte[] FAMILY_NAME = Bytes.toBytes("f"); private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java index 287f65e6c06..a6a6510b3b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -31,15 +30,15 @@ import java.util.List; import java.util.Map; import java.util.Random; import java.util.Set; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; @@ -49,6 +48,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -65,6 +65,10 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) public class TestSeekOptimizations { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSeekOptimizations.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSeekOptimizations.class); @@ -120,7 +124,7 @@ public class TestSeekOptimizations { private BloomType bloomType; private long totalSeekDiligent, totalSeekLazy; - + private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); @Parameters diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java index 23296d039a7..e2525db73f0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -26,8 +25,8 @@ import static org.junit.Assert.fail; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; @@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.apache.hadoop.hbase.util.Threads; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -45,6 +45,10 @@ import org.mockito.stubbing.Answer; @Category({RegionServerTests.class, SmallTests.class}) public class TestServerNonceManager { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestServerNonceManager.class); + @Test public void testMvcc() throws Exception { ServerNonceManager nm = createManager(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSettingTimeoutOnBlockingPoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSettingTimeoutOnBlockingPoint.java index 1cb6bfc2da4..130b6514ac0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSettingTimeoutOnBlockingPoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSettingTimeoutOnBlockingPoint.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -20,7 +19,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.Optional; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -48,6 +48,10 @@ import org.junit.rules.TestName; @Category({LargeTests.class}) public class TestSettingTimeoutOnBlockingPoint { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSettingTimeoutOnBlockingPoint.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] FAM = Bytes.toBytes("f"); private static final byte[] ROW1 = Bytes.toBytes("row1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSimpleTimeRangeTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSimpleTimeRangeTracker.java index b660366c293..6aa676848b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSimpleTimeRangeTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSimpleTimeRangeTracker.java @@ -24,17 +24,22 @@ import static org.junit.Assert.assertTrue; import java.io.DataOutputStream; import java.io.IOException; import java.util.Optional; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.ByteArrayOutputStream; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestSimpleTimeRangeTracker { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSimpleTimeRangeTracker.class); + protected TimeRangeTracker getTimeRangeTracker() { return TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java index 8519c3c6a50..5e3fa39bc3c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -30,11 +29,11 @@ import java.io.IOException; import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.LongAdder; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.Server; @@ -59,6 +58,7 @@ import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs.Ids; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -66,6 +66,11 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) public class TestSplitLogWorker { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSplitLogWorker.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSplitLogWorker.class); private static final int WAIT_TIME = 15000; private final ServerName MANAGER = ServerName.valueOf("manager,1,1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index c80fc2ed3b6..8d91ce1f5b6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -36,9 +35,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -93,15 +92,17 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; @@ -113,9 +114,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto @Category({RegionServerTests.class, LargeTests.class}) @SuppressWarnings("deprecation") public class TestSplitTransactionOnCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSplitTransactionOnCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSplitTransactionOnCluster.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); private Admin admin = null; private MiniHBaseCluster cluster = null; private static final int NB_SERVERS = 3; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java index 073845e06e8..5596fc9d678 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java @@ -25,9 +25,9 @@ import static org.mockito.Mockito.spy; import java.io.IOException; import java.util.Collection; import java.util.Map; - import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.hadoop.hbase.DroppedSnapshotException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Matchers; @@ -63,6 +64,10 @@ import org.slf4j.LoggerFactory; @Category({ MediumTests.class }) public class TestSplitWalDataLoss { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSplitWalDataLoss.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSplitWalDataLoss.class); private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java index 988316f3bf7..24b73a520a6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,12 +20,13 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.*; import java.io.IOException; - import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.io.HFileLink; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -35,6 +35,11 @@ import org.junit.experimental.categories.Category; */ @Category({RegionServerTests.class, SmallTests.class}) public class TestStoreFileInfo { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStoreFileInfo.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java index 35c744f705a..1e3bdcdb9e6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; @@ -34,6 +33,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Stoppable; @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.StoppableImplementation; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -62,6 +63,10 @@ import org.junit.rules.TestName; @Category({RegionServerTests.class, SmallTests.class}) public class TestStoreFileRefresherChore { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStoreFileRefresherChore.class); + private HBaseTestingUtility TEST_UTIL; private Path testDir; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java index 3efbdad32f0..264638d6970 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -24,12 +23,12 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -43,12 +42,17 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestStoreFileScannerWithTagCompression { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStoreFileScannerWithTagCompression.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Configuration conf = TEST_UTIL.getConfiguration(); private static CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java index 2c679d8d43f..104d294e5fc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.CellUtil.createCell; @@ -36,17 +34,16 @@ import java.util.NavigableSet; import java.util.OptionalInt; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; @@ -55,22 +52,25 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; // Can't be small as it plays with EnvironmentEdgeManager @Category({RegionServerTests.class, MediumTests.class}) public class TestStoreScanner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStoreScanner.class); + private static final Logger LOG = LoggerFactory.getLogger(TestStoreScanner.class); @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); private static final String CF_STR = "cf"; private static final byte[] CF = Bytes.toBytes(CF_STR); static Configuration CONF = HBaseConfiguration.create(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java index 6b5e2f2b883..9371ce04eb3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java @@ -31,10 +31,10 @@ import static org.mockito.Mockito.when; import java.util.ArrayList; import java.util.Arrays; import java.util.OptionalLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; @@ -43,12 +43,17 @@ import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestStripeStoreEngine { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStripeStoreEngine.class); + @Test public void testCreateBasedOnConfig() throws Exception { Configuration conf = HBaseConfiguration.create(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java index b76cd1a29cb..2185b83944c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -32,11 +31,11 @@ import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -46,12 +45,18 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @Category({RegionServerTests.class, SmallTests.class}) public class TestStripeStoreFileManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStripeStoreFileManager.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Path BASEDIR = TEST_UTIL.getDataTestDir(TestStripeStoreFileManager.class.getSimpleName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java index cab6aadc806..6c8e45966c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java @@ -25,8 +25,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.ThreadLocalRandom; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -40,12 +40,17 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) public class TestSwitchToStreamRead { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSwitchToStreamRead.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static TableName TABLE_NAME = TableName.valueOf("stream"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSyncTimeRangeTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSyncTimeRangeTracker.java index 273535b399c..272e468e34f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSyncTimeRangeTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSyncTimeRangeTracker.java @@ -20,15 +20,20 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertTrue; import java.util.concurrent.ThreadLocalRandom; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestSyncTimeRangeTracker extends TestSimpleTimeRangeTracker { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncTimeRangeTracker.class); + private static final int NUM_KEYS = 10000000; private static final int NUM_OF_THREADS = 20; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java index 52e6b601295..682d951d790 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -24,12 +24,12 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -73,6 +74,11 @@ import org.junit.rules.TestName; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestTags { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTags.class); + static boolean useFilter = false; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimestampFilterSeekHint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimestampFilterSeekHint.java index 7900bd66a41..e97ed6ba10d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimestampFilterSeekHint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimestampFilterSeekHint.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,10 +17,11 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.io.IOException; +import static org.junit.Assert.assertTrue; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; +import java.io.IOException; import org.apache.commons.lang3.RandomStringUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.client.Get; @@ -31,14 +31,19 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertTrue; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; @Category({RegionServerTests.class, LargeTests.class}) public class TestTimestampFilterSeekHint { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTimestampFilterSeekHint.class); + private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); private final static String RK = "myRK"; private final static byte[] RK_BYTES = Bytes.toBytes(RK); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java index 70bdc4956e1..7abba4c7008 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -25,13 +24,13 @@ import java.util.NavigableMap; import java.util.TreeMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -46,12 +45,12 @@ import org.apache.hadoop.hbase.regionserver.wal.DamagedWALException; import org.apache.hadoop.hbase.regionserver.wal.FSHLog; import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.hbase.wal.WALProvider.Writer; @@ -59,6 +58,7 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -73,6 +73,11 @@ import org.slf4j.LoggerFactory; */ @Category({MediumTests.class}) public class TestWALLockup { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALLockup.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWALLockup.class); @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java index a5148b3f3fc..cdc157254a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -25,10 +24,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -76,6 +76,11 @@ import org.slf4j.LoggerFactory; @RunWith(Parameterized.class) @Category({ RegionServerTests.class, SmallTests.class }) public class TestWALMonotonicallyIncreasingSeqId { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALMonotonicallyIncreasingSeqId.class); + private final Logger LOG = LoggerFactory.getLogger(getClass()); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Path testDir = TEST_UTIL.getDataTestDir("TestWALMonotonicallyIncreasingSeqId"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java index 290f71a4ad3..ab56991b596 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java @@ -17,12 +17,16 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.util.Arrays; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -32,13 +36,10 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.WAL; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - /** * This test verifies the correctness of the Per Column Family flushing strategy * when part of the memstores are compacted memstores @@ -46,6 +47,10 @@ import static org.junit.Assert.assertTrue; @Category({ RegionServerTests.class, LargeTests.class }) public class TestWalAndCompactingMemStoreFlush { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWalAndCompactingMemStoreFlush.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Path DIR = TEST_UTIL.getDataTestDir("TestHRegion"); public static final TableName TABLENAME = TableName.valueOf("TestWalAndCompactingMemStoreFlush", diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java index 380de9e92e1..9fc8bfcf869 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import java.io.IOException; @@ -25,9 +23,9 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Random; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -39,6 +37,7 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -46,6 +45,11 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, SmallTests.class}) public class TestWideScanner extends HBaseTestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWideScanner.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWideScanner.class); static final byte[] A = Bytes.toBytes("A"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java index 44e97e26718..8547e650c98 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -28,10 +28,10 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -51,12 +51,18 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @Category({ MediumTests.class, RegionServerTests.class }) public class TestCompactedHFilesDischarger { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactedHFilesDischarger.class); + private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); private HRegion region; private final static byte[] fam = Bytes.toBytes("cf_1"); @@ -153,7 +159,7 @@ public class TestCompactedHFilesDischarger { } compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); assertTrue(compactedfiles.isEmpty()); - + } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java index 834373ce088..698dc816c1f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java @@ -32,11 +32,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.OptionalLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -66,6 +67,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ RegionServerTests.class, SmallTests.class }) public class TestDateTieredCompactor { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDateTieredCompactor.class); + private static final byte[] NAME_OF_THINGS = Bytes.toBytes("foo"); private static final TableName TABLE_NAME = TableName.valueOf(NAME_OF_THINGS, NAME_OF_THINGS); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java index d06855bfa7f..7ea879e165a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java @@ -22,9 +22,9 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.List; import java.util.concurrent.ThreadLocalRandom; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.TimeOffsetEnvironmentEdge; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,6 +59,10 @@ import org.junit.rules.ExpectedException; @Category({ RegionServerTests.class, MediumTests.class }) public class TestFIFOCompactionPolicy { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFIFOCompactionPolicy.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final TableName tableName = TableName.valueOf(getClass().getSimpleName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestOffPeakHours.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestOffPeakHours.java index f43c29a9ff6..c2ddf8617f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestOffPeakHours.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestOffPeakHours.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -21,16 +21,23 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestOffPeakHours { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOffPeakHours.class); + private static HBaseTestingUtility testUtil; @BeforeClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java index fea2ae9e58b..aee3dc64361 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java @@ -43,11 +43,11 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.OptionalLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -76,6 +76,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ConcatenatedLists; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -90,6 +91,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @RunWith(Parameterized.class) @Category({RegionServerTests.class, SmallTests.class}) public class TestStripeCompactionPolicy { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStripeCompactionPolicy.class); + private static final byte[] KEY_A = Bytes.toBytes("aaa"); private static final byte[] KEY_B = Bytes.toBytes("bbb"); private static final byte[] KEY_C = Bytes.toBytes("ccc"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java index 302a4eb4c9d..8b5df7262ac 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java @@ -30,12 +30,12 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -62,6 +63,11 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) @Category({ RegionServerTests.class, SmallTests.class }) public class TestStripeCompactor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStripeCompactor.class); + private static final byte[] NAME_OF_THINGS = Bytes.toBytes("foo"); private static final TableName TABLE_NAME = TableName.valueOf(NAME_OF_THINGS, NAME_OF_THINGS); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestCompactionScanQueryMatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestCompactionScanQueryMatcher.java index 30ab007f694..223508f2a97 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestCompactionScanQueryMatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestCompactionScanQueryMatcher.java @@ -24,7 +24,7 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -45,6 +46,10 @@ import org.slf4j.LoggerFactory; @Category({ RegionServerTests.class, SmallTests.class }) public class TestCompactionScanQueryMatcher extends AbstractTestScanQueryMatcher { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactionScanQueryMatcher.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCompactionScanQueryMatcher.class); @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestExplicitColumnTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestExplicitColumnTracker.java index 4e07f802faf..e87dfd813a9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestExplicitColumnTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestExplicitColumnTracker.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.querymatcher; import static org.junit.Assert.assertEquals; @@ -26,18 +24,23 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.TreeSet; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, SmallTests.class }) public class TestExplicitColumnTracker { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestExplicitColumnTracker.class); + private final byte[] col1 = Bytes.toBytes("col1"); private final byte[] col2 = Bytes.toBytes("col2"); private final byte[] col3 = Bytes.toBytes("col3"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestNewVersionBehaviorTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestNewVersionBehaviorTracker.java index 81a14a75bc4..098c5ff260f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestNewVersionBehaviorTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestNewVersionBehaviorTracker.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,24 +17,29 @@ */ package org.apache.hadoop.hbase.regionserver.querymatcher; -import java.io.IOException; +import static org.junit.Assert.assertEquals; +import java.io.IOException; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker.DeleteResult; import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; - @Category({ RegionServerTests.class, SmallTests.class }) public class TestNewVersionBehaviorTracker { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNewVersionBehaviorTracker.class); + private final byte[] col1 = Bytes.toBytes("col1"); private final byte[] col2 = Bytes.toBytes("col2"); private final byte[] row = Bytes.toBytes("row"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestScanDeleteTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestScanDeleteTracker.java index 972343250c2..7cfc323eadf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestScanDeleteTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestScanDeleteTracker.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,24 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.querymatcher; import static org.junit.Assert.*; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker.DeleteResult; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, SmallTests.class }) public class TestScanDeleteTracker { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScanDeleteTracker.class); + private ScanDeleteTracker sdt; private long timestamp = 10L; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestScanWildcardColumnTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestScanWildcardColumnTracker.java index 8a17095446f..b8130c3587b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestScanWildcardColumnTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestScanWildcardColumnTracker.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.querymatcher; import static org.junit.Assert.assertEquals; @@ -25,19 +23,24 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, SmallTests.class }) public class TestScanWildcardColumnTracker { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScanWildcardColumnTracker.class); + final static int VERSIONS = 2; @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java index b43a89d65c5..861b83e2fe5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java @@ -24,12 +24,12 @@ import static org.junit.Assert.assertFalse; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchC import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -45,6 +46,10 @@ import org.slf4j.LoggerFactory; @Category({ RegionServerTests.class, SmallTests.class }) public class TestUserScanQueryMatcher extends AbstractTestScanQueryMatcher { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestUserScanQueryMatcher.class); + private static final Logger LOG = LoggerFactory.getLogger(TestUserScanQueryMatcher.class); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java index 1d3834d870e..a7cdfa5b389 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java @@ -23,8 +23,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; import java.util.concurrent.ThreadLocalRandom; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -55,6 +56,11 @@ import org.slf4j.LoggerFactory; @Category({ RegionServerTests.class, MediumTests.class }) public class TestCompactionWithThroughputController { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactionWithThroughputController.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCompactionWithThroughputController.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java index cb6155845cc..83533d9969d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java @@ -1,12 +1,19 @@ /** - * 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. + * 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.regionserver.throttle; @@ -17,8 +24,8 @@ import java.io.IOException; import java.util.List; import java.util.Random; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -42,6 +49,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Pair; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +59,11 @@ import org.slf4j.LoggerFactory; @Category(MediumTests.class) public class TestFlushWithThroughputController { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFlushWithThroughputController.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFlushWithThroughputController.class); private static final double EPSILON = 1E-6; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java index 396b2681bcf..450c01b5b53 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java @@ -17,30 +17,35 @@ */ package org.apache.hadoop.hbase.regionserver.wal; -import org.apache.hbase.thirdparty.io.netty.channel.Channel; -import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; - import java.io.IOException; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; + /** * Provides AsyncFSWAL test cases. */ @Category({ RegionServerTests.class, MediumTests.class }) public class TestAsyncFSWAL extends AbstractTestFSWAL { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncFSWAL.class); + private static EventLoopGroup GROUP; private static Class CHANNEL_CLASS; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java index af022fc5215..c0a61625403 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java @@ -18,15 +18,21 @@ package org.apache.hadoop.hbase.regionserver.wal; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) public class TestAsyncLogRollPeriod extends AbstractTestLogRollPeriod { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncLogRollPeriod.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = AbstractTestLogRollPeriod.TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java index 01e44fb275e..aecb27384ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java @@ -20,9 +20,8 @@ package org.apache.hadoop.hbase.regionserver.wal; import static org.junit.Assert.assertEquals; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper; @@ -33,15 +32,16 @@ import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; @Category({ VerySlowRegionServerTests.class, LargeTests.class }) public class TestAsyncLogRolling extends AbstractTestLogRolling { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncLogRolling.class); @BeforeClass public static void setUpBeforeClass() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java index 86b4c89f294..0ea75b6fc40 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java @@ -17,18 +17,11 @@ */ package org.apache.hadoop.hbase.regionserver.wal; -import org.apache.hbase.thirdparty.com.google.common.base.Throwables; - -import org.apache.hbase.thirdparty.io.netty.channel.Channel; -import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; - import java.io.IOException; import java.io.InterruptedIOException; import java.util.concurrent.ExecutionException; - import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.wal.AsyncFSWALProvider; @@ -37,11 +30,22 @@ import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.com.google.common.base.Throwables; +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; + @Category({ RegionServerTests.class, MediumTests.class }) public class TestAsyncProtobufLog extends AbstractTestProtobufLog { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncProtobufLog.class); + private static EventLoopGroup EVENT_LOOP_GROUP; private static Class CHANNEL_CLASS; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java index 1157a6188e8..80b7477de13 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java @@ -17,16 +17,11 @@ */ package org.apache.hadoop.hbase.regionserver.wal; -import org.apache.hbase.thirdparty.io.netty.channel.Channel; -import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; - import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -35,11 +30,21 @@ import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; + @Category({ RegionServerTests.class, MediumTests.class }) public class TestAsyncWALReplay extends AbstractTestWALReplay { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncWALReplay.class); + private static EventLoopGroup GROUP; private static Class CHANNEL_CLASS; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java index c088891a4ce..347b81258c0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java @@ -18,15 +18,21 @@ package org.apache.hadoop.hbase.regionserver.wal; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) public class TestAsyncWALReplayCompressed extends TestAsyncWALReplay { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncWALReplayCompressed.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = AbstractTestWALReplay.TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCompressor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCompressor.java index 03baf48012d..d8a5099f84c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCompressor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCompressor.java @@ -13,11 +13,10 @@ * 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 + * limitations under the License. */ package org.apache.hadoop.hbase.regionserver.wal; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -26,13 +25,14 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; - -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.util.Dictionary; import org.apache.hadoop.hbase.io.util.LRUDictionary; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,6 +41,11 @@ import org.junit.experimental.categories.Category; */ @Category({RegionServerTests.class, SmallTests.class}) public class TestCompressor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompressor.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java index c3931531f80..9391a85f4af 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java @@ -20,8 +20,10 @@ package org.apache.hadoop.hbase.regionserver.wal; import static org.junit.Assert.assertEquals; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -31,6 +33,10 @@ import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestCustomWALCellCodec { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCustomWALCellCodec.class); + public static class CustomWALCellCodec extends WALCellCodec { public Configuration conf; public CompressionContext context; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java index 68fa33f47ee..b7aa0e3b1fe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java @@ -25,6 +25,7 @@ import java.util.Arrays; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -52,6 +53,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -67,6 +69,11 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) @Category({ RegionServerTests.class, MediumTests.class }) public class TestDurability { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDurability.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static FileSystem FS; private static MiniDFSCluster CLUSTER; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java index cb45f49ce1e..9c789ba90e7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,6 +17,8 @@ */ package org.apache.hadoop.hbase.regionserver.wal; +import static org.junit.Assert.assertEquals; + import java.io.IOException; import java.lang.reflect.Field; import java.util.List; @@ -27,10 +28,10 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -48,18 +49,22 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import static org.junit.Assert.assertEquals; - /** * Provides FSHLog test cases. */ @Category({ RegionServerTests.class, MediumTests.class }) public class TestFSHLog extends AbstractTestFSWAL { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSHLog.class); + @Rule public TestName name = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSWALEntry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSWALEntry.java index cddecd305e4..993934d2384 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSWALEntry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSWALEntry.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,25 +15,30 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.wal; +import static org.junit.Assert.assertEquals; + import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; -import static org.junit.Assert.assertEquals; - import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - @Category({ RegionServerTests.class, SmallTests.class }) public class TestFSWALEntry { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSWALEntry.class); + @Test public void testCollectFamilies() { byte[] family0 = Bytes.toBytes("family0"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 57679c40585..e27fb9718f8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -24,6 +24,7 @@ import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -56,6 +57,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -67,6 +69,11 @@ import org.slf4j.LoggerFactory; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestLogRollAbort { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLogRollAbort.class); + private static final Logger LOG = LoggerFactory.getLogger(AbstractTestLogRolling.class); private static MiniDFSCluster dfsCluster; private static Admin admin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java index 24499427cf9..0d6a1831b0b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java @@ -18,15 +18,21 @@ package org.apache.hadoop.hbase.regionserver.wal; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) public class TestLogRollPeriod extends AbstractTestLogRollPeriod { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLogRollPeriod.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = AbstractTestLogRollPeriod.TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 53b15714d31..55f27260334 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -29,11 +29,10 @@ import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -58,19 +57,20 @@ import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({ VerySlowRegionServerTests.class, LargeTests.class }) public class TestLogRolling extends AbstractTestLogRolling { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLogRolling.class); + private static final Logger LOG = LoggerFactory.getLogger(TestLogRolling.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); @BeforeClass public static void setUpBeforeClass() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index 2766b319802..c83e4e7307c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -25,7 +25,7 @@ import java.util.TreeMap; import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -43,10 +43,9 @@ import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALKeyImpl; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,8 +54,11 @@ import org.slf4j.LoggerFactory; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestLogRollingNoCluster { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLogRollingNoCluster.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static byte [] EMPTY_1K_ARRAY = new byte[1024]; private static final int NUM_THREADS = 100; // Spin up this many threads diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWAL.java index 8841bf2cb8d..c0d34166de4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestMetricsWAL.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,23 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.wal; -import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import java.util.concurrent.TimeUnit; - import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + @Category({MiscTests.class, SmallTests.class}) public class TestMetricsWAL { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsWAL.class); + @Test public void testLogRollRequested() throws Exception { MetricsWALSource source = mock(MetricsWALSourceImpl.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java index 0add8525c13..080b5be807c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java @@ -18,19 +18,24 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.io.IOException; - import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.FSHLogProvider; +import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.hbase.wal.WALProvider.Writer; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) public class TestProtobufLog extends AbstractTestProtobufLog { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestProtobufLog.class); + @Override protected Writer createWriter(Path path) throws IOException { return FSHLogProvider.createWriter(TEST_UTIL.getConfiguration(), fs, path, false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureAsyncWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureAsyncWALReplay.java index d31d8cb07f0..d63ac7716bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureAsyncWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureAsyncWALReplay.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.regionserver.wal; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -26,14 +26,15 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.wal.AsyncFSWALProvider.AsyncWriter; import org.apache.hadoop.hbase.wal.WAL.Reader; import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; @Category({ RegionServerTests.class, MediumTests.class }) public class TestSecureAsyncWALReplay extends TestAsyncWALReplay { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecureAsyncWALReplay.class); @BeforeClass public static void setUpBeforeClass() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java index 91172ab5212..6a2f81e9779 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.regionserver.wal; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -25,11 +26,16 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.wal.WAL.Reader; import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) public class TestSecureWALReplay extends TestWALReplay { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecureWALReplay.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = AbstractTestWALReplay.TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSequenceIdAccounting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSequenceIdAccounting.java index 4e90b335bcb..754aedb2c22 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSequenceIdAccounting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSequenceIdAccounting.java @@ -25,15 +25,21 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestSequenceIdAccounting { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSequenceIdAccounting.class); + private static final byte [] ENCODED_REGION_NAME = Bytes.toBytes("r"); private static final byte [] FAMILY_NAME = Bytes.toBytes("cf"); private static final Set FAMILIES; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSyncFuture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSyncFuture.java index 809263659de..1b2477c044a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSyncFuture.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSyncFuture.java @@ -19,15 +19,21 @@ package org.apache.hadoop.hbase.regionserver.wal; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.exceptions.TimeoutIOException; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, SmallTests.class }) public class TestSyncFuture { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncFuture.class); + @Test(expected = TimeoutIOException.class) public void testGet() throws Exception { long timeout = 5000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java index 380ea7429d3..d7d31669894 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -27,6 +26,7 @@ import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -54,6 +55,10 @@ import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestWALActionsListener { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALActionsListener.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java index 5b09ca1bd87..dd9ee697867 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,10 +25,10 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.ByteBufferKeyValue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; @@ -40,12 +39,17 @@ import org.apache.hadoop.hbase.io.util.LRUDictionary; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, SmallTests.class}) public class TestWALCellCodecWithCompression { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALCellCodecWithCompression.class); + @Test public void testEncodeDecodeKVsWithTags() throws Exception { doTest(false, false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index 2758d4d29e9..649e98141a9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -18,26 +18,25 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; @Category({ RegionServerTests.class, MediumTests.class }) public class TestWALReplay extends AbstractTestWALReplay { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALReplay.class); @BeforeClass public static void setUpBeforeClass() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayBoundedLogWriterCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayBoundedLogWriterCreation.java index 55bbeafad32..1c245af054a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayBoundedLogWriterCreation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayBoundedLogWriterCreation.java @@ -15,17 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.wal; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.wal.WALSplitter; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; - @Category(MediumTests.class) public class TestWALReplayBoundedLogWriterCreation extends TestWALReplay { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALReplayBoundedLogWriterCreation.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { TestWALReplay.setUpBeforeClass(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java index 6237c8d0db4..0ffaa88d1f2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java @@ -18,10 +18,12 @@ package org.apache.hadoop.hbase.regionserver.wal; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -30,6 +32,10 @@ import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) public class TestWALReplayCompressed extends TestWALReplay { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALReplayCompressed.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = AbstractTestWALReplay.TEST_UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index 101272017a5..cd0b2bf90fe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.ClusterMetrics; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -76,6 +77,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -84,6 +86,10 @@ import org.slf4j.LoggerFactory; @Category({ReplicationTests.class, LargeTests.class}) public class TestMasterReplication { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterReplication.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationBase.class); private Configuration baseConfiguration; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index c57d9bb2b70..7b2e73fa651 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -1,6 +1,4 @@ -/* - * Copyright The Apache Software Foundation - * +/** * 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 @@ -29,13 +27,13 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.Arrays; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; @@ -49,6 +47,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -57,6 +56,10 @@ import org.slf4j.LoggerFactory; @Category({ReplicationTests.class, LargeTests.class}) public class TestMultiSlaveReplication { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMultiSlaveReplication.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationBase.class); private static Configuration conf1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java index 0f5fda0be14..4c1b94f5013 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication; import static org.junit.Assert.assertArrayEquals; @@ -29,7 +27,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -47,6 +45,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -55,6 +54,10 @@ import org.slf4j.LoggerFactory; @Category({MediumTests.class}) public class TestNamespaceReplication extends TestReplicationBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNamespaceReplication.class); + private static final Logger LOG = LoggerFactory.getLogger(TestNamespaceReplication.class); private static String ns1 = "ns1"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java index 98b3fdade31..1dd0c19f960 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java @@ -1,6 +1,4 @@ -/* - * Copyright The Apache Software Foundation - * +/** * 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 @@ -19,13 +17,15 @@ */ package org.apache.hadoop.hbase.replication; +import static org.junit.Assert.*; + import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,11 +58,15 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.*; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; @Category({FlakeyTests.class, LargeTests.class}) public class TestPerTableCFReplication { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPerTableCFReplication.class); + private static final Logger LOG = LoggerFactory.getLogger(TestPerTableCFReplication.class); private static Configuration conf1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java index abf8d261b6e..96e0edefe8a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -23,7 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -35,6 +34,7 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -45,6 +45,11 @@ import org.slf4j.LoggerFactory; */ @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationChangingPeerRegionservers extends TestReplicationBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationChangingPeerRegionservers.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationChangingPeerRegionservers.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java index 7b9dea49478..5bdcd459fdd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java @@ -18,22 +18,29 @@ */ package org.apache.hadoop.hbase.replication; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.fail; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.fail; - @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationDisableInactivePeer extends TestReplicationBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationDisableInactivePeer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationDisableInactivePeer.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java index 044e55d7ef0..6de4a4269b4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,7 +17,12 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.hadoop.hbase.testclassification.LargeTests; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -33,22 +37,23 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.fail; - -import java.util.ArrayList; -import java.util.List; - @Category(LargeTests.class) public class TestReplicationDroppedTables extends TestReplicationBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationDroppedTables.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationDroppedTables.class); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java index 4a43aa78d41..fe773cb548b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -33,12 +34,17 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WAL; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ReplicationTests.class, LargeTests.class }) public class TestReplicationEmptyWALRecovery extends TestReplicationBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationEmptyWALRecovery.class); + @Before public void setUp() throws IOException, InterruptedException { cleanUp(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java index c5f83419bff..25cc3c43a10 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication; import static org.mockito.Mockito.mock; @@ -29,8 +28,8 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; - import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -56,6 +55,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -66,6 +66,11 @@ import org.slf4j.LoggerFactory; */ @Category({ReplicationTests.class, MediumTests.class}) public class TestReplicationEndpoint extends TestReplicationBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationEndpoint.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationEndpoint.class); static int numRegionServers; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRS.java index 5487c048576..c92c4d4219f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRS.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hbase.replication; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -32,6 +34,10 @@ import org.junit.runners.Parameterized; @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationKillMasterRS extends TestReplicationKillRS { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationKillMasterRS.class); + @Test(timeout=300000) public void killOneMasterRS() throws Exception { loadTableAndKillRS(utility1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java index 8deffd98394..90b9bd8a49c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -31,6 +32,10 @@ import org.junit.experimental.categories.Category; @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationKillMasterRSCompressed extends TestReplicationKillMasterRS { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationKillMasterRSCompressed.class); + /** * @throws java.lang.Exception */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java index 283704539a3..643b629a200 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,6 +17,9 @@ */ package org.apache.hadoop.hbase.replication; +import static org.junit.Assert.fail; + +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.client.Result; @@ -25,15 +27,18 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.fail; - @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationKillRS extends TestReplicationBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationKillRS.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationKillRS.class); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java index 6a824d0f01c..15f667bc8dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hbase.replication; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -32,6 +34,10 @@ import org.junit.runners.Parameterized; @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationKillSlaveRS extends TestReplicationKillRS { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationKillSlaveRS.class); + @Test(timeout=300000) public void killOneSlaveRS() throws Exception { loadTableAndKillRS(utility2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index f46a7b17b23..f4d4d71752b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; @@ -53,6 +54,7 @@ import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -63,6 +65,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; @Category({ ReplicationTests.class, LargeTests.class }) public class TestReplicationSmallTests extends TestReplicationBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSmallTests.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSmallTests.class); private static final String PEER_ID = "2"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index 48d8924377b..ed181ddb914 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -28,44 +27,48 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.MiniHBaseCluster; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.Waiter.Predicate; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.replication.regionserver.Replication; -import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager; -import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALKeyImpl; -import org.apache.hadoop.hbase.wal.WALProvider; -import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint; +import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource; +import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.apache.hadoop.hbase.wal.WALProvider; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - @Category({ReplicationTests.class, MediumTests.class}) public class TestReplicationSource { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSource.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSource.class); private final static HBaseTestingUtility TEST_UTIL = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java index 231d655e2c2..d51d3c357c5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java @@ -22,12 +22,12 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; @@ -47,6 +47,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -55,6 +56,10 @@ import org.slf4j.LoggerFactory; @Category({ReplicationTests.class, MediumTests.class}) public class TestReplicationStateZKImpl extends TestReplicationStateBasic { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationStateZKImpl.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateZKImpl.class); private static Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java index 7a624b21ec5..0bd0260bf74 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java @@ -24,6 +24,7 @@ import java.util.EnumSet; import java.util.List; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.ClusterStatus; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Admin; @@ -32,6 +33,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -39,6 +41,11 @@ import org.slf4j.LoggerFactory; @Category({ReplicationTests.class, MediumTests.class}) public class TestReplicationStatus extends TestReplicationBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationStatus.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStatus.class); private static final String PEER_ID = "2"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java index b4732771e20..1672390a24e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java @@ -17,6 +17,11 @@ */ package org.apache.hadoop.hbase.replication; +import static org.junit.Assert.assertEquals; + +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -34,19 +39,19 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.List; - -import static org.junit.Assert.assertEquals; - @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationSyncUpTool extends TestReplicationBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSyncUpTool.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSyncUpTool.class); private static final TableName t1_su = TableName.valueOf("t1_syncup"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java index a04d5247d80..69500a1730e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java @@ -1,5 +1,4 @@ -/* - +/** * 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 @@ -26,12 +25,12 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; @@ -48,6 +47,7 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -63,6 +63,10 @@ import org.slf4j.LoggerFactory; @Category({ReplicationTests.class, MediumTests.class}) public class TestReplicationTrackerZKImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationTrackerZKImpl.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationTrackerZKImpl.class); private static Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java index 8127222528b..67a25512534 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication; import static org.junit.Assert.assertNull; @@ -29,20 +28,21 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +51,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({ReplicationTests.class, SmallTests.class}) public class TestReplicationWALEntryFilters { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationWALEntryFilters.class); + static byte[] a = new byte[] {'a'}; static byte[] b = new byte[] {'b'}; static byte[] c = new byte[] {'c'}; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java index b2ecb679315..0a477375b41 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java @@ -25,11 +25,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -70,6 +71,10 @@ import org.slf4j.LoggerFactory; @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationWithTags { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationWithTags.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationWithTags.class); private static final byte TAG_TYPE = 1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java index 3185a5be32e..88513ead2a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java @@ -1,6 +1,4 @@ -/* - * Copyright The Apache Software Foundation - * +/** * 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 @@ -25,9 +23,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -55,6 +53,7 @@ import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -64,6 +63,11 @@ import org.slf4j.LoggerFactory; @Category({ ReplicationTests.class, LargeTests.class }) public class TestSerialReplication { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSerialReplication.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSerialReplication.class); private static Configuration conf1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java index 2993043b5b5..b1fe2e9ee94 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -17,11 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication.master; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; @@ -35,6 +39,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -42,16 +47,13 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - @Category({ReplicationTests.class, SmallTests.class}) public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTableCFsUpdater.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTableCFsUpdater.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationEndpointWithMultipleAsyncWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationEndpointWithMultipleAsyncWAL.java index debe8a1de4e..594aac0b5c9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationEndpointWithMultipleAsyncWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationEndpointWithMultipleAsyncWAL.java @@ -17,16 +17,23 @@ */ package org.apache.hadoop.hbase.replication.multiwal; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.replication.TestReplicationEndpoint; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.wal.RegionGroupingProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ReplicationTests.class, MediumTests.class }) public class TestReplicationEndpointWithMultipleAsyncWAL extends TestReplicationEndpoint { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationEndpointWithMultipleAsyncWAL.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { conf1.set(WALFactory.WAL_PROVIDER, "multiwal"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationEndpointWithMultipleWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationEndpointWithMultipleWAL.java index 8f350f297a6..68b41be457b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationEndpointWithMultipleWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationEndpointWithMultipleWAL.java @@ -17,16 +17,23 @@ */ package org.apache.hadoop.hbase.replication.multiwal; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.replication.TestReplicationEndpoint; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.wal.RegionGroupingProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ReplicationTests.class, MediumTests.class }) public class TestReplicationEndpointWithMultipleWAL extends TestReplicationEndpoint { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationEndpointWithMultipleWAL.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { conf1.set(WALFactory.WAL_PROVIDER, "multiwal"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationKillMasterRSCompressedWithMultipleAsyncWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationKillMasterRSCompressedWithMultipleAsyncWAL.java index 416cda7f2f6..4685f24c0de 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationKillMasterRSCompressedWithMultipleAsyncWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationKillMasterRSCompressedWithMultipleAsyncWAL.java @@ -17,17 +17,24 @@ */ package org.apache.hadoop.hbase.replication.multiwal; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.replication.TestReplicationKillMasterRSCompressed; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.wal.RegionGroupingProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationKillMasterRSCompressedWithMultipleAsyncWAL extends TestReplicationKillMasterRSCompressed { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationKillMasterRSCompressedWithMultipleAsyncWAL.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { conf1.set(WALFactory.WAL_PROVIDER, "multiwal"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationKillMasterRSCompressedWithMultipleWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationKillMasterRSCompressedWithMultipleWAL.java index 18067762cc0..82fef3aa58b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationKillMasterRSCompressedWithMultipleWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationKillMasterRSCompressedWithMultipleWAL.java @@ -17,17 +17,24 @@ */ package org.apache.hadoop.hbase.replication.multiwal; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.replication.TestReplicationKillMasterRSCompressed; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.wal.RegionGroupingProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationKillMasterRSCompressedWithMultipleWAL extends TestReplicationKillMasterRSCompressed { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationKillMasterRSCompressedWithMultipleWAL.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { conf1.set(WALFactory.WAL_PROVIDER, "multiwal"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationSyncUpToolWithMultipleAsyncWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationSyncUpToolWithMultipleAsyncWAL.java index bb91aaf7d17..14514993470 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationSyncUpToolWithMultipleAsyncWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationSyncUpToolWithMultipleAsyncWAL.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.replication.multiwal; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.replication.TestReplicationBase; import org.apache.hadoop.hbase.replication.TestReplicationSyncUpTool; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -24,10 +25,16 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.wal.RegionGroupingProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ReplicationTests.class, LargeTests.class }) public class TestReplicationSyncUpToolWithMultipleAsyncWAL extends TestReplicationSyncUpTool { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSyncUpToolWithMultipleAsyncWAL.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { conf1.set(WALFactory.WAL_PROVIDER, "multiwal"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationSyncUpToolWithMultipleWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationSyncUpToolWithMultipleWAL.java index 7329a902e9e..e487039dcd9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationSyncUpToolWithMultipleWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/multiwal/TestReplicationSyncUpToolWithMultipleWAL.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.replication.multiwal; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.replication.TestReplicationBase; import org.apache.hadoop.hbase.replication.TestReplicationSyncUpTool; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -24,10 +25,16 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.wal.RegionGroupingProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ ReplicationTests.class, LargeTests.class }) public class TestReplicationSyncUpToolWithMultipleWAL extends TestReplicationSyncUpTool { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSyncUpToolWithMultipleWAL.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { conf1.set(WALFactory.WAL_PROVIDER, "multiwal"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java index fab125b8bcc..e714bf81d50 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,16 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication.regionserver; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -50,6 +48,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -59,6 +58,11 @@ import org.slf4j.LoggerFactory; @Category({ ReplicationTests.class, LargeTests.class }) public class TestGlobalThrottler { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGlobalThrottler.class); + private static final Logger LOG = LoggerFactory.getLogger(TestGlobalThrottler.class); private static Configuration conf1; private static Configuration conf2; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java index 38590c7d099..80eed960d5c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java @@ -28,8 +28,8 @@ import java.io.IOException; import java.util.List; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -46,32 +46,38 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.wal.WAL.Entry; -import org.apache.hadoop.hbase.wal.WALKeyImpl; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + /** * Tests RegionReplicaReplicationEndpoint class by setting up region replicas and verifying * async wal replication replays the edits to the secondary region in various scenarios. */ @Category({FlakeyTests.class, LargeTests.class}) public class TestRegionReplicaReplicationEndpoint { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionReplicaReplicationEndpoint.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionReplicaReplicationEndpoint.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java index 66351fb79e8..33ea8d750f0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,14 +17,20 @@ */ package org.apache.hadoop.hbase.replication.regionserver; +import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion; +import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + import java.io.IOException; import java.util.Optional; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -49,8 +55,6 @@ import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationEndpoint.ReplicateContext; import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint.RegionReplicaReplayCallable; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; @@ -63,14 +67,13 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion; -import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion; -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; /** * Tests RegionReplicaReplicationEndpoint. Unlike TestRegionReplicaReplicationEndpoint this @@ -79,6 +82,10 @@ import static org.mockito.Mockito.when; @Category({ReplicationTests.class, MediumTests.class}) public class TestRegionReplicaReplicationEndpointNoMaster { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionReplicaReplicationEndpointNoMaster.class); + private static final int NB_SERVERS = 2; private static TableName tableName = TableName.valueOf( TestRegionReplicaReplicationEndpointNoMaster.class.getSimpleName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java index b389efc535d..4629c68e352 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -32,13 +31,13 @@ import java.util.Map; import java.util.Random; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -53,12 +52,6 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; @@ -67,13 +60,27 @@ import org.apache.hadoop.hbase.util.HFileTestUtil; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey; + @Category({ReplicationTests.class, MediumTests.class}) public class TestReplicationSink { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSink.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSink.class); private static final int BATCH_SIZE = 10; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java index e29a3337a09..3be3bfb55d1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -22,25 +22,31 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.ClusterConnection; -import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; + @Category({ReplicationTests.class, SmallTests.class}) public class TestReplicationSinkManager { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSinkManager.class); + private static final String PEER_CLUSTER_ID = "PEER_CLUSTER_ID"; private ReplicationPeers replicationPeers; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index 057c64d2664..c03aa7c9e2e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -47,6 +47,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -89,6 +90,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -108,6 +110,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescr @Category({ReplicationTests.class, MediumTests.class}) public abstract class TestReplicationSourceManager { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSourceManager.class); + protected static final Logger LOG = LoggerFactory.getLogger(TestReplicationSourceManager.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java index b47a8d3f628..64db7a884c3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java @@ -1,26 +1,30 @@ -/* -* -* 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. -*/ +/** + * 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.replication.regionserver; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.replication.ReplicationFactory; @@ -33,14 +37,10 @@ import org.apache.hadoop.hbase.replication.ReplicationSourceDummy; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * Tests the ReplicationSourceManager with ReplicationQueueZkImpl's and * ReplicationQueuesClientZkImpl. Also includes extra tests outside of those in @@ -48,6 +48,11 @@ import static org.junit.Assert.assertTrue; */ @Category({ReplicationTests.class, MediumTests.class}) public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSourceManagerZkImpl.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { conf = HBaseConfiguration.create(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationThrottler.java index eb6b9cf2874..5f7be5c0487 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationThrottler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationThrottler.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.testclassification.SmallTests; - +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -32,6 +32,10 @@ import org.slf4j.LoggerFactory; @Category({ReplicationTests.class, SmallTests.class}) public class TestReplicationThrottler { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationThrottler.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationThrottler.class); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java index 375e64e6982..d8db3b15e76 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -23,8 +23,8 @@ import java.io.IOException; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,6 +47,7 @@ import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse; @@ -86,11 +88,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegio import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse; - @Category(MediumTests.class) @Ignore("Flaky, needs to be rewritten, see HBASE-19125") public class TestReplicator extends TestReplicationBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicator.class); + static final Logger LOG = LoggerFactory.getLogger(TestReplicator.class); static final int NUM_ROWS = 10; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java index b34d6f171bd..62990656195 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java @@ -28,15 +28,14 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellBuilder; import org.apache.hadoop.hbase.CellBuilderFactory; import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Stoppable; @@ -67,15 +66,16 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; /** @@ -83,12 +83,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; */ @Category({ReplicationTests.class, SmallTests.class}) public class TestWALEntrySinkFilter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALEntrySinkFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSink.class); @Rule public TestName name = new TestName(); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()). - withLookingForStuckThread(true). - build(); static final int BOUNDARY = 5; static final AtomicInteger UNFILTERED = new AtomicInteger(); static final AtomicInteger FILTERED = new AtomicInteger(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java index 44c76f8988e..d89c9c25f91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java @@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -61,6 +62,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -70,6 +72,10 @@ import org.mockito.Mockito; @Category({ ReplicationTests.class, LargeTests.class }) public class TestWALEntryStream { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALEntryStream.class); + private static HBaseTestingUtility TEST_UTIL; private static Configuration conf; private static FileSystem fs; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java index 1399b21e68f..a53b489565c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -28,8 +27,6 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertSame; import static org.junit.Assert.fail; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; - import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; @@ -38,12 +35,11 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; - import javax.security.sasl.SaslException; - import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ipc.BlockingRpcClient; @@ -56,9 +52,6 @@ import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.RpcServerFactory; import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.ipc.SimpleRpcServer; -import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; -import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.minikdc.MiniKdc; @@ -67,6 +60,7 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -77,10 +71,20 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; import org.mockito.Mockito; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; + +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; + @RunWith(Parameterized.class) @Category({ SecurityTests.class, MediumTests.class }) public class TestSecureIPC { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecureIPC.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final File KEYTAB_FILE = new File( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestUser.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestUser.java index d5eed3ce437..28c13cfba3f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestUser.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestUser.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,27 +17,34 @@ */ package org.apache.hadoop.hbase.security; +import static org.junit.Assert.*; + import java.io.IOException; import java.security.PrivilegedAction; import java.security.PrivilegedExceptionAction; - import org.apache.commons.lang3.SystemUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.security.UserGroupInformation; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; -import static org.junit.Assert.*; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; @Category({SecurityTests.class, SmallTests.class}) public class TestUser { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestUser.class); + private static final Logger LOG = LoggerFactory.getLogger(TestUser.class); @Test @@ -232,7 +238,7 @@ public class TestUser { conf = HBaseConfiguration.create(); conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - assertFalse("HBase security should not be enabled if " + assertFalse("HBase security should not be enabled if " + User.HBASE_SECURITY_CONF_KEY + " is not set accordingly", User.isHBaseSecurityEnabled(conf)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestUsersOperationsWithSecureHadoop.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestUsersOperationsWithSecureHadoop.java index 6a21639fd9a..b69c5d9d6ba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestUsersOperationsWithSecureHadoop.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestUsersOperationsWithSecureHadoop.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -28,8 +27,8 @@ import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -37,12 +36,17 @@ import org.apache.hadoop.minikdc.MiniKdc; import org.apache.hadoop.security.UserGroupInformation; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ SecurityTests.class, SmallTests.class }) public class TestUsersOperationsWithSecureHadoop { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestUsersOperationsWithSecureHadoop.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final File KEYTAB_FILE = new File(TEST_UTIL.getDataTestDir("keytab").toUri() .getPath()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java index 8a1af20d7fe..e62c76e1183 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.security.access; import static org.junit.Assert.assertEquals; @@ -26,8 +25,8 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; import java.util.UUID; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -45,6 +44,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -52,6 +52,11 @@ import org.junit.rules.TestName; @Category({SecurityTests.class, LargeTests.class}) public class TestAccessControlFilter extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAccessControlFilter.class); + @Rule public TestName name = new TestName(); private static HBaseTestingUtility TEST_UTIL; @@ -64,11 +69,11 @@ public class TestAccessControlFilter extends SecureTestUtil { private static byte[] PRIVATE_COL = Bytes.toBytes("private"); private static byte[] PUBLIC_COL = Bytes.toBytes("public"); - @Before + @Before public void setup () { TABLE = TableName.valueOf(name.getMethodName()); } - + @BeforeClass public static void setupBeforeClass() throws Exception { TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 14e94bee8a0..ba781d6ca17 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -45,6 +45,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -128,6 +129,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -144,6 +146,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedu */ @Category({SecurityTests.class, LargeTests.class}) public class TestAccessController extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAccessController.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAccessController.class); private static TableName TEST_TABLE = TableName.valueOf("testtable1"); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java index 7b7a150fefd..17de97bc135 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -25,9 +25,9 @@ import static org.junit.Assert.assertTrue; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.TestTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -52,13 +53,13 @@ import org.apache.hadoop.hbase.security.access.Permission.Action; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.TestTableName; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -67,6 +68,11 @@ import org.slf4j.LoggerFactory; @Category({SecurityTests.class, LargeTests.class}) public class TestAccessController2 extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAccessController2.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAccessController2.class); private static final byte[] TEST_ROW = Bytes.toBytes("test"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java index babba153bf9..6ca2ef8a505 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -63,6 +65,11 @@ import org.slf4j.LoggerFactory; */ @Category({SecurityTests.class, MediumTests.class}) public class TestAccessController3 extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAccessController3.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAccessController.class); private static TableName TEST_TABLE = TableName.valueOf("testtable1"); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java index c9a1e6f1382..88d03143ec6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -24,10 +24,10 @@ import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.HashMap; import java.util.Map; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.AuthUtil; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -54,6 +54,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -62,6 +63,11 @@ import org.slf4j.LoggerFactory; @Category({SecurityTests.class, MediumTests.class}) public class TestCellACLWithMultipleVersions extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellACLWithMultipleVersions.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCellACLWithMultipleVersions.class); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java index bf3699f24d2..54cf79da6e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -22,11 +22,11 @@ import static org.junit.Assert.assertEquals; import java.util.HashMap; import java.util.List; import java.util.Map; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.AuthUtil; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -56,6 +56,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -66,6 +67,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({SecurityTests.class, LargeTests.class}) public class TestCellACLs extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellACLs.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCellACLs.class); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java index a357c1fc4c7..1686ba37aa4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -22,11 +22,10 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -47,7 +46,6 @@ import org.junit.After; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +54,11 @@ import org.slf4j.LoggerFactory; */ @Category({SecurityTests.class, MediumTests.class}) public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorWhitelistMasterObserver.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCoprocessorWhitelistMasterObserver.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); @@ -80,10 +83,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil { UTIL.shutdownMiniCluster(); } - @ClassRule - public static TestRule timeout = - CategoryBasedTimeout.forClass(TestCoprocessorWhitelistMasterObserver.class); - /** * Test a table modification adding a coprocessor path * which is not whitelisted diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java index df06f7e30fd..328024c58fa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -21,12 +21,13 @@ import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.protobuf.BlockingRpcChannel; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -50,15 +51,21 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; -import com.google.protobuf.BlockingRpcChannel; @Category({SecurityTests.class, MediumTests.class}) public class TestNamespaceCommands extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNamespaceCommands.class); + private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestNamespaceCommands.class); private static String TEST_NAMESPACE = "ns1"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java index fcf989b67a5..96d6b994ffa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -22,9 +22,9 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.UUID; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -49,6 +49,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -57,6 +58,11 @@ import org.slf4j.LoggerFactory; @Category({SecurityTests.class, MediumTests.class}) public class TestScanEarlyTermination extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestScanEarlyTermination.class); + private static final Logger LOG = LoggerFactory.getLogger(TestScanEarlyTermination.class); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java index 3980e443c69..7243690e5bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -28,16 +28,16 @@ import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; @@ -46,10 +46,12 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; @@ -58,6 +60,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; */ @Category({SecurityTests.class, LargeTests.class}) public class TestTablePermissions { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTablePermissions.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTablePermissions.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static ZKWatcher ZKW; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java index 5273dc17657..57d9e4bf860 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -20,11 +20,11 @@ package org.apache.hadoop.hbase.security.access; import static org.junit.Assert.assertEquals; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -72,6 +72,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -82,6 +83,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({SecurityTests.class, LargeTests.class}) public class TestWithDisabledAuthorization extends SecureTestUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWithDisabledAuthorization.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWithDisabledAuthorization.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java index 1bb86f4f5e6..c8ab86307e8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -23,11 +23,11 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -45,6 +46,11 @@ import org.slf4j.LoggerFactory; */ @Category({SecurityTests.class, LargeTests.class}) public class TestZKPermissionWatcher { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKPermissionWatcher.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKPermissionWatcher.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static TableAuthManager AUTH_A; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestAuthenticationKey.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestAuthenticationKey.java index 97341592ae5..cdd89e0d6b4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestAuthenticationKey.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestAuthenticationKey.java @@ -21,11 +21,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import java.io.UnsupportedEncodingException; - import javax.crypto.SecretKey; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -33,6 +33,10 @@ import org.mockito.Mockito; @Category({SecurityTests.class, SmallTests.class}) public class TestAuthenticationKey { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAuthenticationKey.class); + @Test public void test() throws UnsupportedEncodingException { SecretKey secret = Mockito.mock(SecretKey.class); @@ -41,15 +45,15 @@ public class TestAuthenticationKey { AuthenticationKey key = new AuthenticationKey(0, 1234, secret); assertEquals(key.hashCode(), new AuthenticationKey(0, 1234, secret).hashCode()); assertEquals(key, new AuthenticationKey(0, 1234, secret)); - + AuthenticationKey otherID = new AuthenticationKey(1, 1234, secret); assertNotEquals(key.hashCode(), otherID.hashCode()); assertNotEquals(key, otherID); - + AuthenticationKey otherExpiry = new AuthenticationKey(0, 8765, secret); assertNotEquals(key.hashCode(), otherExpiry.hashCode()); assertNotEquals(key, otherExpiry); - + SecretKey other = Mockito.mock(SecretKey.class); Mockito.when(secret.getEncoded()).thenReturn("other".getBytes("UTF-8")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java index ab8026a8ae8..6a8b0eb6056 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java @@ -21,7 +21,7 @@ import static org.junit.Assert.assertArrayEquals; import java.util.Arrays; import java.util.Collection; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -43,6 +43,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -56,6 +57,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ SecurityTests.class, MediumTests.class }) public class TestDelegationTokenWithEncryption extends SecureTestCluster { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDelegationTokenWithEncryption.class); + @BeforeClass public static void setUp() throws Exception { // enable rpc encryption diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java index d34b64fd2ae..76a8d44c778 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java @@ -23,11 +23,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import com.google.protobuf.ServiceException; - import java.io.IOException; import java.util.Arrays; import java.util.Collection; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; @@ -51,6 +50,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -62,6 +62,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ SecurityTests.class, MediumTests.class }) public class TestGenerateDelegationToken extends SecureTestCluster { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestGenerateDelegationToken.class); + @BeforeClass public static void setUp() throws Exception { SecureTestCluster.setUp(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java index 67887e681e2..afde03790e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -25,6 +25,9 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.protobuf.BlockingService; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; import java.io.IOException; import java.io.InterruptedIOException; import java.net.InetSocketAddress; @@ -32,13 +35,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.CoordinatedStateManager; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; @@ -79,10 +81,9 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.junit.After; import org.junit.Before; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; @@ -95,10 +96,6 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescrip import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor; import org.apache.hbase.thirdparty.com.google.protobuf.Message; -import com.google.protobuf.BlockingService; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; - /** * Tests for authentication token creation and usage */ @@ -110,6 +107,11 @@ import com.google.protobuf.ServiceException; @RunWith(Parameterized.class) @Category({SecurityTests.class, MediumTests.class}) public class TestTokenAuthentication { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestTokenAuthentication.class); + static { // Setting whatever system properties after recommendation from // http://docs.oracle.com/javase/6/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html @@ -117,10 +119,6 @@ public class TestTokenAuthentication { System.setProperty("java.security.krb5.kdc", "blah"); } - @Rule - public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()) - .withLookingForStuckThread(true).build(); - /** * Basic server process for RPC authentication testing */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java index 252de3a3cb5..6d5ad263011 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.security.token; import static org.junit.Assert.assertEquals; @@ -25,9 +24,9 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -47,6 +47,11 @@ import org.slf4j.LoggerFactory; */ @Category({SecurityTests.class, LargeTests.class}) public class TestZKSecretWatcher { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKSecretWatcher.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKSecretWatcher.class); private static HBaseTestingUtility TEST_UTIL; private static AuthenticationTokenSecretManager KEY_MASTER; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java index 9a249c39f07..54e25a23f1d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.security.token; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.SecurityTests; @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -41,6 +42,11 @@ import org.slf4j.LoggerFactory; */ @Category({ SecurityTests.class, SmallTests.class }) public class TestZKSecretWatcherRefreshKeys { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKSecretWatcherRefreshKeys.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKSecretWatcherRefreshKeys.class); private static HBaseTestingUtility TEST_UTIL; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestDefaultScanLabelGeneratorStack.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestDefaultScanLabelGeneratorStack.java index 6dacf02fc8a..322c1c07564 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestDefaultScanLabelGeneratorStack.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestDefaultScanLabelGeneratorStack.java @@ -23,10 +23,10 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.security.PrivilegedExceptionAction; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,6 +52,10 @@ import org.junit.rules.TestName; @Category({SecurityTests.class, MediumTests.class}) public class TestDefaultScanLabelGeneratorStack { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDefaultScanLabelGeneratorStack.class); + public static final String CONFIDENTIAL = "confidential"; private static final String SECRET = "secret"; public static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -245,7 +250,7 @@ public class TestDefaultScanLabelGeneratorStack { }); } - + private static Result [] getResult(Table table, Scan scan) throws IOException { ResultScanner scanner = table.getScanner(scan); Result[] next = scanner.next(1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java index c7075e35a48..0d2b466f3f1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java @@ -23,8 +23,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.security.PrivilegedExceptionAction; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -48,6 +49,10 @@ import org.junit.rules.TestName; @Category({SecurityTests.class, MediumTests.class}) public class TestEnforcingScanLabelGenerator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEnforcingScanLabelGenerator.class); + public static final String CONFIDENTIAL = "confidential"; private static final String SECRET = "secret"; public static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionExpander.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionExpander.java index e0c0b984811..ba788b25267 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionExpander.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionExpander.java @@ -20,18 +20,24 @@ package org.apache.hadoop.hbase.security.visibility; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.hbase.testclassification.SecurityTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.Operator; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({SecurityTests.class, SmallTests.class}) public class TestExpressionExpander { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestExpressionExpander.class); + @Test public void testPositiveCases() throws Exception { ExpressionExpander expander = new ExpressionExpander(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionParser.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionParser.java index 7c7f54bb08f..5a1e37a86d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionParser.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestExpressionParser.java @@ -21,18 +21,24 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import org.apache.hadoop.hbase.testclassification.SecurityTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.Operator; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({SecurityTests.class, SmallTests.class}) public class TestExpressionParser { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestExpressionParser.class); + private ExpressionParser parser = new ExpressionParser(); @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java index 76bba48bedc..ba0dee3643e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java @@ -24,11 +24,11 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.security.PrivilegedExceptionAction; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -51,12 +51,18 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.Before; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Category({ SecurityTests.class, MediumTests.class }) public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilityLabelsReplication { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityLabelReplicationWithExpAsString.class); + private static final Logger LOG = LoggerFactory .getLogger(TestVisibilityLabelReplicationWithExpAsString.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOpWithDifferentUsersNoACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOpWithDifferentUsersNoACL.java index 8d75156a05e..3e47cc13184 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOpWithDifferentUsersNoACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOpWithDifferentUsersNoACL.java @@ -21,12 +21,13 @@ import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LA import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.protobuf.ByteString; import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -38,15 +39,19 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import com.google.protobuf.ByteString; - @Category({SecurityTests.class, MediumTests.class}) public class TestVisibilityLabelsOpWithDifferentUsersNoACL { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityLabelsOpWithDifferentUsersNoACL.class); + private static final String PRIVATE = "private"; private static final String CONFIDENTIAL = "confidential"; private static final String SECRET = "secret"; @@ -97,7 +102,7 @@ public class TestVisibilityLabelsOpWithDifferentUsersNoACL { VisibilityLabelsResponse response = SUPERUSER.runAs(action); assertTrue(response.getResult(0).getException().getValue().isEmpty()); assertTrue(response.getResult(1).getException().getValue().isEmpty()); - + // Ideally this should not be allowed. this operation should fail or do nothing. action = new PrivilegedExceptionAction() { @Override @@ -139,7 +144,7 @@ public class TestVisibilityLabelsOpWithDifferentUsersNoACL { assertTrue(authsList.contains(CONFIDENTIAL)); assertTrue(authsList.contains(PRIVATE)); - PrivilegedExceptionAction action2 = + PrivilegedExceptionAction action2 = new PrivilegedExceptionAction() { @Override public VisibilityLabelsResponse run() throws Exception { @@ -164,7 +169,7 @@ public class TestVisibilityLabelsOpWithDifferentUsersNoACL { } private static void addLabels() throws Exception { - PrivilegedExceptionAction action = + PrivilegedExceptionAction action = new PrivilegedExceptionAction() { @Override public VisibilityLabelsResponse run() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java index dce8591ba40..cea037b4b80 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java @@ -28,12 +28,12 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -74,6 +74,7 @@ import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -83,6 +84,11 @@ import org.slf4j.LoggerFactory; @Category({ SecurityTests.class, MediumTests.class }) public class TestVisibilityLabelsReplication { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityLabelsReplication.class); + private static final Logger LOG = LoggerFactory.getLogger(TestVisibilityLabelsReplication.class); protected static final int NON_VIS_TAG_TYPE = 100; protected static final String TEMP = "temp"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java index ef1ae98dbe8..e36e4519e11 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java @@ -22,12 +22,13 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import com.google.protobuf.ByteString; import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -51,16 +52,19 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import com.google.protobuf.ByteString; - @Category({SecurityTests.class, MediumTests.class}) public class TestVisibilityLabelsWithACL { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityLabelsWithACL.class); + private static final String PRIVATE = "private"; private static final String CONFIDENTIAL = "confidential"; private static final String SECRET = "secret"; @@ -224,7 +228,7 @@ public class TestVisibilityLabelsWithACL { @Test public void testLabelsTableOpsWithDifferentUsers() throws Throwable { - PrivilegedExceptionAction action = + PrivilegedExceptionAction action = new PrivilegedExceptionAction() { @Override public VisibilityLabelsResponse run() throws Exception { @@ -295,7 +299,7 @@ public class TestVisibilityLabelsWithACL { VisibilityClient.setAuths(TEST_UTIL.getConnection(), new String[] { CONFIDENTIAL, PRIVATE }, "user3"); - PrivilegedExceptionAction action1 = + PrivilegedExceptionAction action1 = new PrivilegedExceptionAction() { @Override public GetAuthsResponse run() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithCustomVisLabService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithCustomVisLabService.java index ea1ed100ee8..7233f9c658b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithCustomVisLabService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithCustomVisLabService.java @@ -23,7 +23,7 @@ import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LA import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.security.User; @@ -31,12 +31,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({SecurityTests.class, MediumTests.class}) public class TestVisibilityLabelsWithCustomVisLabService extends TestVisibilityLabels { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityLabelsWithCustomVisLabService.class); + @BeforeClass public static void setupBeforeClass() throws Exception { // setup configuration diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java index 2adcf9efecb..fdf665a0be0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java @@ -20,15 +20,16 @@ package org.apache.hadoop.hbase.security.visibility; import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME; import static org.apache.hadoop.hbase.security.visibility.VisibilityUtils.SYSTEM_LABEL; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.protobuf.ByteString; import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -49,15 +50,19 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.Threads; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.protobuf.ByteString; - @Category({SecurityTests.class, MediumTests.class}) public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibilityLabels { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityLabelsWithDefaultVisLabelService.class); + private static final Logger LOG = LoggerFactory.getLogger( TestVisibilityLabelsWithDefaultVisLabelService.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java index 9d536fe2b8a..0e69a3a2e82 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java @@ -27,11 +27,11 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -69,6 +70,11 @@ import org.slf4j.LoggerFactory; */ @Category({SecurityTests.class, MediumTests.class}) public class TestVisibilityLabelsWithDeletes { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityLabelsWithDeletes.class); + private static final Logger LOG = LoggerFactory.getLogger(TestVisibilityLabelsWithDeletes.class); private static final String TOPSECRET = "TOPSECRET"; private static final String PUBLIC = "PUBLIC"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java index b82d5036f51..7cdf2f36ec0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java @@ -23,8 +23,8 @@ import static org.junit.Assert.assertNull; import java.io.IOException; import java.security.PrivilegedExceptionAction; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -50,6 +51,10 @@ import org.junit.rules.TestName; @Category({SecurityTests.class, MediumTests.class}) public class TestVisibilityLabelsWithSLGStack { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityLabelsWithSLGStack.class); + public static final String CONFIDENTIAL = "confidential"; private static final String SECRET = "secret"; public static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -105,7 +110,7 @@ public class TestVisibilityLabelsWithSLGStack { } private static void addLabels() throws Exception { - PrivilegedExceptionAction action = + PrivilegedExceptionAction action = new PrivilegedExceptionAction() { @Override public VisibilityLabelsResponse run() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLablesWithGroups.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLablesWithGroups.java index 906b9d5d75f..a7afab73dad 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLablesWithGroups.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLablesWithGroups.java @@ -23,14 +23,15 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.protobuf.ByteString; import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -49,16 +50,19 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import com.google.protobuf.ByteString; - @Category({SecurityTests.class, MediumTests.class}) public class TestVisibilityLablesWithGroups { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityLablesWithGroups.class); + public static final String CONFIDENTIAL = "confidential"; private static final String SECRET = "secret"; public static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java index ebf38a52b0a..153a892f114 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java @@ -22,8 +22,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.security.PrivilegedExceptionAction; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -53,6 +54,11 @@ import org.junit.rules.TestName; * Test visibility by setting 'hbase.security.visibility.mutations.checkauths' to true */ public class TestVisibilityWithCheckAuths { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVisibilityWithCheckAuths.class); + private static final String TOPSECRET = "TOPSECRET"; private static final String PUBLIC = "PUBLIC"; public static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java index 7a555844276..833590887a2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java @@ -22,11 +22,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.protobuf.ByteString; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -45,16 +46,19 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import com.google.protobuf.ByteString; - @Category({SecurityTests.class, LargeTests.class}) public class TestWithDisabledAuthorization { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWithDisabledAuthorization.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final String CONFIDENTIAL = "confidential"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java index 9b0209652ef..74ae8303f5e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java @@ -28,26 +28,22 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.client.SnapshotType; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -58,10 +54,12 @@ import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; + /** * Test creating/using/deleting snapshots from the client *

@@ -72,10 +70,12 @@ import org.slf4j.LoggerFactory; */ @Category({RegionServerTests.class, LargeTests.class}) public class TestFlushSnapshotFromClient { - private static final Logger LOG = LoggerFactory.getLogger(TestFlushSnapshotFromClient.class); + @ClassRule - public static final TestRule timeout = - CategoryBasedTimeout.forClass(TestFlushSnapshotFromClient.class); + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFlushSnapshotFromClient.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestFlushSnapshotFromClient.class); protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); protected static final int NUM_RS = 2; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java index 44c14aa0aef..5941676bf56 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java @@ -18,9 +18,8 @@ package org.apache.hadoop.hbase.snapshot; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Table; @@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,11 +42,12 @@ import org.slf4j.LoggerFactory; */ @Category({ClientTests.class, LargeTests.class}) public class TestMobFlushSnapshotFromClient extends TestFlushSnapshotFromClient { - private static final Logger LOG = LoggerFactory.getLogger(TestFlushSnapshotFromClient.class); @ClassRule - public static final TestRule timeout = - CategoryBasedTimeout.forClass(TestMobFlushSnapshotFromClient.class); + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobFlushSnapshotFromClient.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestFlushSnapshotFromClient.class); @BeforeClass public static void setupCluster() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreFlushSnapshotFromClient.java index e86115065d6..20d0da6df2f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreFlushSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreFlushSnapshotFromClient.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hbase.snapshot; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Table; @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +40,11 @@ import org.slf4j.LoggerFactory; */ @Category({ClientTests.class,LargeTests.class}) public class TestMobRestoreFlushSnapshotFromClient extends TestRestoreFlushSnapshotFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobRestoreFlushSnapshotFromClient.class); + final Logger LOG = LoggerFactory.getLogger(getClass()); @BeforeClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreSnapshotHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreSnapshotHelper.java index f4e9870dde7..df8eb68a4a2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreSnapshotHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreSnapshotHelper.java @@ -18,11 +18,12 @@ package org.apache.hadoop.hbase.snapshot; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils.SnapshotMock; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +33,11 @@ import org.slf4j.LoggerFactory; */ @Category(SmallTests.class) public class TestMobRestoreSnapshotHelper extends TestRestoreSnapshotHelper { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobRestoreSnapshotHelper.class); + final Logger LOG = LoggerFactory.getLogger(getClass()); @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java index 380beba3318..d84297fc286 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java @@ -17,9 +17,21 @@ */ package org.apache.hadoop.hbase.snapshot; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Table; @@ -30,30 +42,19 @@ import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.snapshot.FlushSnapshotSubprocedure; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; - import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.spy; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; /** * Testing the region snapshot task on a cluster. @@ -61,6 +62,11 @@ import static org.mockito.Mockito.spy; */ @Category({MediumTests.class, RegionServerTests.class}) public class TestRegionSnapshotTask { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionSnapshotTask.class); + private final Logger LOG = LoggerFactory.getLogger(getClass()); private static HBaseTestingUtility TEST_UTIL; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java index 160f5099e08..cfe3b15d2a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hbase.snapshot; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -35,6 +35,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -48,6 +49,11 @@ import org.slf4j.LoggerFactory; */ @Category({RegionServerTests.class, LargeTests.class}) public class TestRestoreFlushSnapshotFromClient { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRestoreFlushSnapshotFromClient.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRestoreFlushSnapshotFromClient.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java index 02bdae1a74b..08c5088e704 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java @@ -22,36 +22,43 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; + /** * Test the restore/clone operation from a file-system point of view. */ @Category({RegionServerTests.class, SmallTests.class}) public class TestRestoreSnapshotHelper { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRestoreSnapshotHelper.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRestoreSnapshotHelper.class); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java index bb492bad2f6..9899b7bbe1c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,20 +22,21 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TestTableName; import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; -import org.apache.hadoop.hbase.coprocessor.MasterObserver; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.MasterObserver; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.TestTableName; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -45,6 +45,11 @@ import org.slf4j.LoggerFactory; @Category({ MediumTests.class }) public class TestSnapshotClientRetries { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotClientRetries.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotClientRetries.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java index 73e0560ef1b..8369f12c218 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java @@ -21,28 +21,35 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.junit.After; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; + /** * Test that the {@link SnapshotDescription} helper is helping correctly. */ @Category({RegionServerTests.class, MediumTests.class}) public class TestSnapshotDescriptionUtils { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotDescriptionUtils.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static FileSystem fs; private static Path root; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java index 1f67b60fe02..926a7e5f783 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java @@ -20,32 +20,40 @@ package org.apache.hadoop.hbase.snapshot; import static org.junit.Assert.fail; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; + @Category({MasterTests.class, SmallTests.class}) public class TestSnapshotManifest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotManifest.class); + private final Logger LOG = LoggerFactory.getLogger(getClass()); private static final String TABLE_NAME_STR = "testSnapshotManifest"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java index 965b399a8bc..e713a5af81e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java @@ -15,10 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.tool; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isA; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; @@ -28,11 +42,11 @@ import org.apache.hadoop.util.ToolRunner; import org.apache.log4j.Appender; import org.apache.log4j.LogManager; import org.apache.log4j.spi.LoggingEvent; -import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.junit.After; import org.junit.Before; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.Ignore; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; @@ -41,25 +55,16 @@ import org.mockito.ArgumentMatcher; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; - -import static org.junit.Assert.assertNotEquals; -import static org.mockito.ArgumentMatchers.argThat; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.eq; -import static org.mockito.Matchers.isA; -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; +import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; @RunWith(MockitoJUnitRunner.class) @Category({MediumTests.class}) public class TestCanaryTool { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCanaryTool.class); + private HBaseTestingUtility testingUtility; private static final byte[] FAMILY = Bytes.toBytes("f"); private static final byte[] COLUMN = Bytes.toBytes("col"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java index a3ca3230f86..e744394460b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,6 +21,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; + import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -51,7 +52,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem; @@ -60,17 +60,25 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + /** * Test cases for the "load" half of the HFileOutputFormat bulk load functionality. These tests run * faster than the full MR cluster tests in TestHFileOutputFormat */ @Category({ MiscTests.class, LargeTests.class }) public class TestLoadIncrementalHFiles { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLoadIncrementalHFiles.class); + @Rule public TestName tn = new TestName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java index 2d32a3c8b7d..b9c2d18e183 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java @@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.tool; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -33,10 +33,10 @@ import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.IntStream; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -78,9 +79,11 @@ import org.junit.rules.TestName; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest; @@ -90,6 +93,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHF */ @Category({ MiscTests.class, LargeTests.class }) public class TestLoadIncrementalHFilesSplitRecovery { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLoadIncrementalHFilesSplitRecovery.class); + private static final Logger LOG = LoggerFactory.getLogger(TestHRegionServerBulkLoad.class); static HBaseTestingUtility util; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFiles.java index 3d4f4c62485..4e10f01fa58 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFiles.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFiles.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -19,6 +17,7 @@ */ package org.apache.hadoop.hbase.tool; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; import org.apache.hadoop.hbase.security.HadoopSecurityEnabledUserProviderForTesting; @@ -28,6 +27,7 @@ import org.apache.hadoop.hbase.security.access.SecureTestUtil; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** @@ -42,6 +42,10 @@ import org.junit.experimental.categories.Category; @Category({ MiscTests.class, LargeTests.class }) public class TestSecureLoadIncrementalHFiles extends TestLoadIncrementalHFiles { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecureLoadIncrementalHFiles.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { // set the always on security provider diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFilesSplitRecovery.java index 9fa60fa2a04..20e76beb6bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFilesSplitRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestSecureLoadIncrementalHFilesSplitRecovery.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.tool; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.security.HadoopSecurityEnabledUserProviderForTesting; import org.apache.hadoop.hbase.security.UserProvider; @@ -25,6 +26,7 @@ import org.apache.hadoop.hbase.security.access.SecureTestUtil; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,6 +43,10 @@ import org.junit.experimental.categories.Category; public class TestSecureLoadIncrementalHFilesSplitRecovery extends TestLoadIncrementalHFilesSplitRecovery { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecureLoadIncrementalHFilesSplitRecovery.class); + // This "overrides" the parent static method // make sure they are in sync @BeforeClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java index 860ec4ab2a7..c39afddaf7d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java @@ -20,12 +20,15 @@ package org.apache.hadoop.hbase.trace; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.htrace.core.POJOSpanReceiver; @@ -34,18 +37,21 @@ import org.apache.htrace.core.Span; import org.apache.htrace.core.TraceScope; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.util.Collection; -import java.util.LinkedList; -import java.util.List; +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @Category({MiscTests.class, MediumTests.class}) public class TestHTraceHooks { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHTraceHooks.class); + private static final byte[] FAMILY_BYTES = "family".getBytes(); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static POJOSpanReceiver rcvr; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java index 9b9f69a452f..a59d3475bd8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,23 +15,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.nio.ByteBuffer; - import junit.framework.TestCase; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.nio.MultiByteBuff; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestBloomFilterChunk extends TestCase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBloomFilterChunk.class); + public void testBasicBloom() throws Exception { BloomFilterChunk bf1 = new BloomFilterChunk(1000, (float)0.01, Hash.MURMUR_HASH, 0); BloomFilterChunk bf2 = new BloomFilterChunk(1000, (float)0.01, Hash.MURMUR_HASH, 0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBoundedPriorityBlockingQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBoundedPriorityBlockingQueue.java index 7746bea2ac8..9639af0f19c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBoundedPriorityBlockingQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBoundedPriorityBlockingQueue.java @@ -1,27 +1,27 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.util; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; import java.util.Comparator; import java.util.PriorityQueue; @@ -29,17 +29,22 @@ import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; - import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestBoundedPriorityBlockingQueue { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBoundedPriorityBlockingQueue.class); + private final static int CAPACITY = 16; static class TestObject { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBuffUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBuffUtils.java index ecc66112d4a..7ed8891e03a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBuffUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBuffUtils.java @@ -1,35 +1,42 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.util; import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.MultiByteBuff; import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ MiscTests.class, SmallTests.class }) public class TestByteBuffUtils { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteBuffUtils.class); + @Test public void testCopyAndCompare() throws Exception { ByteBuffer bb1 = ByteBuffer.allocate(50); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCompressionTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCompressionTest.java index a90a47fc75f..729160f1988 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCompressionTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCompressionTest.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,31 +15,36 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; +import static org.junit.Assert.*; + +import java.io.BufferedOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionOutputStream; import org.apache.hadoop.util.NativeCodeLoader; import org.apache.hadoop.util.ReflectionUtils; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedOutputStream; -import java.io.DataOutputStream; -import java.io.IOException; - -import static org.junit.Assert.*; - @Category({MiscTests.class, SmallTests.class}) public class TestCompressionTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompressionTest.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCompressionTest.class); @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestConfigurationUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestConfigurationUtil.java index f8cbb978c3f..583b6921f3e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestConfigurationUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestConfigurationUtil.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -15,26 +15,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import java.util.List; -import java.util.Map; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + @Category({ SmallTests.class }) public class TestConfigurationUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConfigurationUtil.class); + private Configuration conf; private Map keyValues; private String key; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestConnectionCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestConnectionCache.java index 88df7f4d200..bb74b296758 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestConnectionCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestConnectionCache.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -19,15 +18,22 @@ package org.apache.hadoop.hbase.util; import junit.framework.TestCase; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.security.UserProvider; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, MediumTests.class}) public class TestConnectionCache extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConnectionCache.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index a96a32d3baa..568a1ba4e55 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -16,6 +16,10 @@ * limitations under the License. */ package org.apache.hadoop.hbase.util; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.junit.ClassRule; + // this is deliberately not in the o.a.h.h.regionserver package // in order to make sure all required classes/method are available @@ -71,6 +75,11 @@ import org.junit.runners.Parameterized.Parameters; @Category({ MiscTests.class, MediumTests.class }) @RunWith(Parameterized.class) public class TestCoprocessorScanPolicy { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCoprocessorScanPolicy.class); + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] F = Bytes.toBytes("fam"); private static final byte[] Q = Bytes.toBytes("qual"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestDefaultEnvironmentEdge.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestDefaultEnvironmentEdge.java index 1b9c2601a35..4d2c8939702 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestDefaultEnvironmentEdge.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestDefaultEnvironmentEdge.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,14 +17,16 @@ */ package org.apache.hadoop.hbase.util; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import static junit.framework.Assert.assertTrue; import static junit.framework.Assert.fail; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + /** * Tests to make sure that the default environment edge conforms to appropriate * behaviour. @@ -33,6 +34,10 @@ import static junit.framework.Assert.fail; @Category({MiscTests.class, MediumTests.class}) public class TestDefaultEnvironmentEdge { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDefaultEnvironmentEdge.class); + @Test public void testGetCurrentTimeUsesSystemClock() { DefaultEnvironmentEdge edge = new DefaultEnvironmentEdge(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestEncryptionTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestEncryptionTest.java index 5d2f04f543a..a254eb3f358 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestEncryptionTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestEncryptionTest.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import static org.junit.Assert.fail; import java.security.Key; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.crypto.Cipher; @@ -33,12 +31,17 @@ import org.apache.hadoop.hbase.io.crypto.KeyProvider; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestEncryptionTest { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestEncryptionTest.class); + @Test public void testTestKeyProvider() { Configuration conf = HBaseConfiguration.create(); @@ -137,6 +140,6 @@ public class TestEncryptionTest { public Cipher getCipher(String name) { return null; } - + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java index a554e996938..d500cf9ec18 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java @@ -20,16 +20,17 @@ package org.apache.hadoop.hbase.util; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -41,6 +42,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, MediumTests.class}) public class TestFSHDFSUtils { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSHDFSUtils.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFSHDFSUtils.class); private static final HBaseTestingUtility HTU = new HBaseTestingUtility(); static { @@ -59,9 +65,8 @@ public class TestFSHDFSUtils { /** * Test recover lease eventually succeeding. - * @throws IOException */ - @Test (timeout = 30000) + @Test public void testRecoverLease() throws IOException { HTU.getConfiguration().setInt("hbase.lease.recovery.dfs.timeout", 1000); CancelableProgressable reporter = Mockito.mock(CancelableProgressable.class); @@ -80,9 +85,8 @@ public class TestFSHDFSUtils { /** * Test that isFileClosed makes us recover lease faster. - * @throws IOException */ - @Test (timeout = 30000) + @Test public void testIsFileClosed() throws IOException { // Make this time long so it is plain we broke out because of the isFileClosed invocation. HTU.getConfiguration().setInt("hbase.lease.recovery.dfs.timeout", 100000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java index 2c27a9313ec..0ddc0c30551 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java @@ -29,24 +29,25 @@ import java.io.IOException; import java.util.Arrays; import java.util.Comparator; import java.util.Map; - +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -60,6 +61,11 @@ import org.slf4j.LoggerFactory; // Do not support to be executed in he same JVM as other tests @Category({MiscTests.class, MediumTests.class}) public class TestFSTableDescriptors { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSTableDescriptors.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestFSTableDescriptors.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java index 3d1f10f20da..edd35c7f824 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -29,7 +28,6 @@ import java.io.File; import java.io.IOException; import java.util.Random; import java.util.UUID; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -37,6 +35,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HDFSBlocksDistribution; @@ -48,6 +47,7 @@ import org.apache.hadoop.hdfs.DFSHedgedReadMetrics; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -58,6 +58,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, MediumTests.class}) public class TestFSUtils { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSUtils.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFSUtils.class); private HBaseTestingUtility htu; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java index be302d59812..aa2a7453755 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,16 +20,17 @@ package org.apache.hadoop.hbase.util; import static org.junit.Assert.assertEquals; import java.io.IOException; -import java.util.UUID; -import java.util.Set; import java.util.HashSet; - +import java.util.Set; +import java.util.UUID; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.*; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +40,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, MediumTests.class}) public class TestFSVisitor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSVisitor.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFSVisitor.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFromClientSide3WoUnsafe.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFromClientSide3WoUnsafe.java index c04e76bd9fe..58943594823 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFromClientSide3WoUnsafe.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFromClientSide3WoUnsafe.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -19,16 +17,22 @@ */ package org.apache.hadoop.hbase.util; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.TestFromClientSide3; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({LargeTests.class, ClientTests.class}) public class TestFromClientSide3WoUnsafe extends TestFromClientSide3 { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFromClientSide3WoUnsafe.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { TestByteBufferUtils.disableUnsafe(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java index acd62b1906f..99c8e236955 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,12 +20,14 @@ package org.apache.hadoop.hbase.util; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo; import org.apache.hadoop.hbase.util.HBaseFsck.MetaEntry; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -36,6 +37,10 @@ import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestHBaseFsckComparator { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseFsckComparator.class); + TableName table = TableName.valueOf("table1"); TableName table2 = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java index 31f1909f104..608039e1f29 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -25,11 +25,10 @@ import java.security.Key; import java.security.SecureRandom; import java.util.ArrayList; import java.util.List; - import javax.crypto.spec.SecretKeySpec; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -54,12 +53,17 @@ import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker; import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, LargeTests.class}) public class TestHBaseFsckEncryption { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseFsckEncryption.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java index 2548ecfebfb..3c526589ad8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,11 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; +import static org.junit.Assert.assertEquals; + +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ClusterConnection; @@ -35,18 +40,17 @@ import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import static org.junit.Assert.assertEquals; - @Category({MiscTests.class, LargeTests.class}) public class TestHBaseFsckMOB extends BaseTestHBaseFsck { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseFsckMOB.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java index 4657171b0fb..239b22aa2f8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java @@ -19,24 +19,30 @@ package org.apache.hadoop.hbase.util; import static org.junit.Assert.*; +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.IOException; - /** * Test that the utility works as expected */ @Category({MiscTests.class, SmallTests.class}) public class TestHFileArchiveUtil { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileArchiveUtil.class); + private Path rootDir = new Path("./"); @Rule @@ -54,14 +60,14 @@ public class TestHFileArchiveUtil { FSUtils.setRootDir(conf, new Path("root")); assertNotNull(HFileArchiveUtil.getArchivePath(conf)); } - + @Test public void testRegionArchiveDir() { Path regionDir = new Path("region"); assertNotNull(HFileArchiveUtil.getRegionArchiveDir(rootDir, TableName.valueOf(name.getMethodName()), regionDir)); } - + @Test public void testGetStoreArchivePath() throws IOException { byte[] family = Bytes.toBytes("Family"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java index 769f8ef6db3..4eb44e27a4c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import static org.junit.Assert.assertTrue; @@ -30,9 +28,10 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -42,6 +41,10 @@ import org.slf4j.LoggerFactory; // Medium as it creates 100 threads; seems better to run it isolated public class TestIdLock { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIdLock.class); + private static final Logger LOG = LoggerFactory.getLogger(TestIdLock.class); private static final int NUM_IDS = 16; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLock.java index af89d357d84..51f518a746e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLock.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,11 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import java.util.Arrays; import java.util.Map; @@ -34,10 +32,11 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.IdReadWriteLock.ReferenceType; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -50,6 +49,10 @@ import org.slf4j.LoggerFactory; // Medium as it creates 100 threads; seems better to run it isolated public class TestIdReadWriteLock { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIdReadWriteLock.class); + private static final Logger LOG = LoggerFactory.getLogger(TestIdReadWriteLock.class); private static final int NUM_IDS = 16; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIncrementingEnvironmentEdge.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIncrementingEnvironmentEdge.java index 44894a6e029..9f4339f2223 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIncrementingEnvironmentEdge.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIncrementingEnvironmentEdge.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -20,8 +19,10 @@ package org.apache.hadoop.hbase.util; import static junit.framework.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -32,6 +33,10 @@ import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestIncrementingEnvironmentEdge { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrementingEnvironmentEdge.class); + @Test public void testGetCurrentTimeUsesSystemClock() { IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge(1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java index bb68898171d..02747614c67 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java @@ -17,26 +17,25 @@ */ package org.apache.hadoop.hbase.util; -import java.io.IOException; -import java.lang.management.GarbageCollectorMXBean; -import java.lang.management.ManagementFactory; -import java.util.Hashtable; -import java.util.List; -import java.util.Map; - -import javax.management.MalformedObjectNameException; -import javax.management.ObjectName; -import javax.management.openmbean.CompositeData; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; - +import java.io.IOException; +import java.lang.management.GarbageCollectorMXBean; +import java.lang.management.ManagementFactory; +import java.util.Hashtable; +import java.util.List; +import java.util.Map; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; +import javax.management.openmbean.CompositeData; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -45,6 +44,10 @@ import org.slf4j.LoggerFactory; @Category({MiscTests.class, SmallTests.class}) public class TestJSONMetricUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestJSONMetricUtil.class); + private static final Logger LOG = LoggerFactory.getLogger(TestJSONMetricUtil.class); @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadEncoded.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadEncoded.java index 865cd110054..a114a2c2fc3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadEncoded.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadEncoded.java @@ -1,28 +1,30 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.util; import java.util.ArrayList; import java.util.Collection; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.runners.Parameterized.Parameters; @@ -35,6 +37,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({MiscTests.class, LargeTests.class}) public class TestMiniClusterLoadEncoded extends TestMiniClusterLoadParallel { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMiniClusterLoadEncoded.class); + /** We do not alternate the multi-put flag in this test. */ private static final boolean USE_MULTI_PUT = true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadParallel.java index e6b05e9a739..d318eec2086 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadParallel.java @@ -1,26 +1,29 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.util; import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -35,6 +38,10 @@ import org.junit.runners.Parameterized; public class TestMiniClusterLoadParallel extends TestMiniClusterLoadSequential { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMiniClusterLoadParallel.class); + public TestMiniClusterLoadParallel(boolean isMultiPut, DataBlockEncoding encoding) { super(isMultiPut, encoding); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java index 3230e27aed1..9bcad721915 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java @@ -1,18 +1,19 @@ -/* - * 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 +/** + * 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 + * 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. + * 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.util; @@ -25,6 +26,7 @@ import java.util.EnumSet; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics.Option; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -39,6 +41,7 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -55,6 +58,10 @@ import org.slf4j.LoggerFactory; @RunWith(Parameterized.class) public class TestMiniClusterLoadSequential { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMiniClusterLoadSequential.class); + private static final Logger LOG = LoggerFactory.getLogger( TestMiniClusterLoadSequential.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestPoolMap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestPoolMap.java index 7fc09d2dfc8..dcca330f6e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestPoolMap.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestPoolMap.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -23,12 +22,12 @@ import java.util.List; import java.util.Random; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; - import junit.framework.TestCase; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.PoolMap.PoolType; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Suite; @@ -38,6 +37,11 @@ import org.junit.runners.Suite; TestPoolMap.TestReusablePoolType.class}) @Category({MiscTests.class, SmallTests.class}) public class TestPoolMap { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPoolMap.class); + public abstract static class TestPoolType extends TestCase { protected PoolMap poolMap; protected Random random = new Random(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover.java index 7ab0dd38cfa..e92b027f561 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import static org.junit.Assert.assertEquals; import java.io.File; import java.io.FileWriter; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -33,10 +32,10 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.RegionMover.RegionMoverBuilder; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -49,6 +48,10 @@ import org.slf4j.LoggerFactory; @Category(MediumTests.class) public class TestRegionMover { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionMover.class); + final Logger LOG = LoggerFactory.getLogger(getClass()); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitCalculator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitCalculator.java index eee3030c339..ece83bc4d43 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitCalculator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitCalculator.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -27,20 +26,25 @@ import java.util.Comparator; import java.util.List; import java.util.SortedSet; import java.util.UUID; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; - -import org.apache.hbase.thirdparty.com.google.common.collect.ComparisonChain; -import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; - import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ComparisonChain; +import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; + @Category({MiscTests.class, SmallTests.class}) public class TestRegionSplitCalculator { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionSplitCalculator.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionSplitCalculator.class); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java index 40564ee07fa..f8f538a4797 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,9 +25,9 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.List; - import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; @@ -37,12 +36,13 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.util.RegionSplitter.HexStringSplit; import org.apache.hadoop.hbase.util.RegionSplitter.DecimalStringSplit; +import org.apache.hadoop.hbase.util.RegionSplitter.HexStringSplit; import org.apache.hadoop.hbase.util.RegionSplitter.SplitAlgorithm; import org.apache.hadoop.hbase.util.RegionSplitter.UniformSplit; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -56,6 +56,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, MediumTests.class}) public class TestRegionSplitter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionSplitter.class); + private final static Logger LOG = LoggerFactory.getLogger(TestRegionSplitter.class); private final static HBaseTestingUtility UTIL = new HBaseTestingUtility(); private final static String CF_NAME = "SPLIT_TEST_CF"; @@ -418,7 +423,7 @@ public class TestRegionSplitter { } /** - * List.indexOf() doesn't really work for a List, because byte[] + * List.indexOf() doesn't really work for a List <byte[]>, because byte[] * doesn't override equals(). This method checks whether a list contains * a given element by checking each element using the byte array * comparator. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRootPath.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRootPath.java index 61d3aea43d5..c5dc1b2cab5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRootPath.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRootPath.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -16,17 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; -import junit.framework.TestCase; - import java.io.IOException; - +import junit.framework.TestCase; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,6 +34,11 @@ import org.slf4j.LoggerFactory; */ @Category({MiscTests.class, SmallTests.class}) public class TestRootPath extends TestCase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRootPath.class); + private static final Logger LOG = LoggerFactory.getLogger(TestRootPath.class); /** The test */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestSortedList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestSortedList.java index bd2130556b2..343c10bb8da 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestSortedList.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestSortedList.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import static org.junit.Assert.*; @@ -25,17 +23,22 @@ import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.ListIterator; - -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + @Category({MiscTests.class, SmallTests.class}) public class TestSortedList { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSortedList.class); + static class StringComparator implements Comparator { @Override public int compare(String o1, String o2) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java index 48dde68f6c3..4d69f8e99c9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,25 +17,30 @@ */ package org.apache.hadoop.hbase.util; -import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.StealJobQueue; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - -import static org.junit.Assert.*; - +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestStealJobQueue { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStealJobQueue.class); + StealJobQueue stealJobQueue; BlockingQueue stealFromQueue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java index 3cf65c59799..3a39ee94e19 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -33,6 +32,7 @@ import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -44,6 +44,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -56,6 +57,11 @@ import org.slf4j.LoggerFactory; @RunWith(Parameterized.class) @Category({ RegionServerTests.class, LargeTests.class }) public class TestBoundedRegionGroupingStrategy { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBoundedRegionGroupingStrategy.class); + private static final Logger LOG = LoggerFactory.getLogger(TestBoundedRegionGroupingStrategy.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java index d9ee9eb3922..5aea0cf282e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -33,6 +32,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; -// imports for things that haven't moved from regionserver.wal yet. import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -60,6 +60,11 @@ import org.slf4j.LoggerFactory; @Category({RegionServerTests.class, MediumTests.class}) public class TestFSHLogProvider { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSHLogProvider.class); + private static final Logger LOG = LoggerFactory.getLogger(TestFSHLogProvider.class); private static Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java index b4160e99aa2..814320aa3fa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -62,6 +64,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ RegionServerTests.class, MediumTests.class }) public class TestSecureWAL { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecureWAL.class); + static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java index ceb43d5ea3e..5679d96cd03 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -65,6 +66,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -77,6 +79,11 @@ import org.slf4j.LoggerFactory; */ @Category({RegionServerTests.class, MediumTests.class}) public class TestWALFactory { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALFactory.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWALFactory.class); protected static Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFiltering.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFiltering.java index 9260d8e751d..3739d68a6b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFiltering.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFiltering.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -20,36 +19,43 @@ package org.apache.hadoop.hbase.wal; import static org.junit.Assert.assertEquals; +import com.google.protobuf.ServiceException; import java.io.IOException; import java.util.List; import java.util.Random; import java.util.SortedMap; import java.util.TreeMap; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest; @Category({RegionServerTests.class, MediumTests.class}) public class TestWALFiltering { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALFiltering.class); + private static final int NUM_MASTERS = 1; private static final int NUM_RS = 4; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java index 994f008b5d4..9e88f6e2cda 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -26,17 +25,16 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.NavigableSet; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -// imports for things that haven't moved from regionserver.wal yet. import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -44,6 +42,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers; import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController; import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -52,7 +51,12 @@ import org.junit.experimental.categories.Category; */ @Category({RegionServerTests.class, SmallTests.class}) public class TestWALMethods { - private static final byte[] TEST_REGION = Bytes.toBytes("test_region");; + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALMethods.class); + + private static final byte[] TEST_REGION = Bytes.toBytes("test_region"); private static final TableName TEST_TABLE = TableName.valueOf("test_table"); @@ -60,14 +64,17 @@ public class TestWALMethods { @Test public void testServerNameFromWAL() throws Exception { - Path walPath = new Path("/hbase/WALs/regionserver-2.example.com,22101,1487767381290", "regionserver-2.example.com%2C22101%2C1487767381290.null0.1487785392316"); + Path walPath = new Path("/hbase/WALs/regionserver-2.example.com,22101,1487767381290", + "regionserver-2.example.com%2C22101%2C1487767381290.null0.1487785392316"); ServerName name = AbstractFSWALProvider.getServerNameFromWALDirectoryName(walPath); assertEquals(ServerName.valueOf("regionserver-2.example.com", 22101, 1487767381290L), name); } @Test public void testServerNameFromTestWAL() throws Exception { - Path walPath = new Path("/user/example/test-data/12ff1404-68c6-4715-a4b9-775e763842bc/WALs/TestWALRecordReader", "TestWALRecordReader.default.1487787939118"); + Path walPath = new Path( + "/user/example/test-data/12ff1404-68c6-4715-a4b9-775e763842bc/WALs/TestWALRecordReader", + "TestWALRecordReader.default.1487787939118"); ServerName name = AbstractFSWALProvider.getServerNameFromWALDirectoryName(walPath); assertNull(name); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALOpenAfterDNRollingStart.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALOpenAfterDNRollingStart.java index a612aff0350..d9775a535dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALOpenAfterDNRollingStart.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALOpenAfterDNRollingStart.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,8 +20,8 @@ package org.apache.hadoop.hbase.wal; import java.io.IOException; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -34,6 +33,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -45,6 +45,10 @@ import org.junit.runners.Parameterized.Parameters; @Category({ RegionServerTests.class, LargeTests.class }) public class TestWALOpenAfterDNRollingStart { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALOpenAfterDNRollingStart.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); // Sleep time before restart next dn, we need to wait the current dn to finish start up private static long DN_RESTART_INTERVAL = 15000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java index dcb6f5a46e9..3cbd37ef10d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ByteBufferKeyValue; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -55,16 +56,22 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -/* +/** * Test that verifies WAL written by SecureProtobufLogWriter is not readable by ProtobufLogReader */ @Category({RegionServerTests.class, MediumTests.class}) public class TestWALReaderOnSecureWAL { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALReaderOnSecureWAL.class); + static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); final byte[] value = Bytes.toBytes("Test value"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java index e46dc21c4a8..412acb68f8d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -47,6 +49,11 @@ import org.slf4j.LoggerFactory; @Category(MediumTests.class) public class TestWALRootDir { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALRootDir.class); + private static final Logger LOG = LoggerFactory.getLogger(TestWALRootDir.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Configuration conf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java index dded5062ca4..9b988593a6c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -50,6 +49,7 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -81,6 +81,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -104,6 +105,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; */ @Category({RegionServerTests.class, LargeTests.class}) public class TestWALSplit { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALSplit.class); + { // Uncomment the following lines if more verbosity is needed for // debugging (see HBASE-12285 for details). diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitBoundedLogWriterCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitBoundedLogWriterCreation.java index 400d12b3094..eed09c72717 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitBoundedLogWriterCreation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitBoundedLogWriterCreation.java @@ -15,17 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.wal; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(LargeTests.class) public class TestWALSplitBoundedLogWriterCreation extends TestWALSplit{ + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALSplitBoundedLogWriterCreation.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { TestWALSplit.setUpBeforeClass(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitCompressed.java index 1a0c883c4c3..4926ec6c9c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitCompressed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitCompressed.java @@ -15,19 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.wal; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({RegionServerTests.class, LargeTests.class}) public class TestWALSplitCompressed extends TestWALSplit { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALSplitCompressed.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { TestWALSplit.setUpBeforeClass(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java index 3e9e650b378..d668466f6bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java @@ -25,10 +25,9 @@ import java.io.FileWriter; import java.io.IOException; import java.util.ArrayList; import java.util.List; - import javax.security.auth.login.AppConfigurationEntry; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -42,6 +41,7 @@ import org.apache.zookeeper.data.Stat; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -49,6 +49,11 @@ import org.slf4j.LoggerFactory; @Category({ ZKTests.class, MediumTests.class }) public class TestZooKeeperACL { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZooKeeperACL.class); + private final static Logger LOG = LoggerFactory.getLogger(TestZooKeeperACL.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java index 645119a5ac1..ca371e145b9 100644 --- a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java +++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java @@ -18,15 +18,21 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.jruby.embed.PathType; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, LargeTests.class }) public class TestReplicationShell extends AbstractTestShell { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationShell.class); + @Test public void testRunShellTests() throws IOException { System.setProperty("shell.test.include", "replication_admin_test.rb"); diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java index 882b811cf55..e3de94a2781 100644 --- a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java +++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,16 +18,21 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.jruby.embed.PathType; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, LargeTests.class }) public class TestShell extends AbstractTestShell { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestShell.class); + @Test public void testRunShellTests() throws IOException { System.setProperty("shell.test.exclude", "replication_admin_test.rb,rsgroup_shell_test.rb"); diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShellNoCluster.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShellNoCluster.java index a7c811f945a..3dfd991501d 100644 --- a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShellNoCluster.java +++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShellNoCluster.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -21,18 +20,23 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.ArrayList; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.jruby.embed.PathType; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ClientTests.class, LargeTests.class }) public class TestShellNoCluster extends AbstractTestShell { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestShellNoCluster.class); + @BeforeClass public static void setUpBeforeClass() throws Exception { // no cluster diff --git a/hbase-shell/src/test/rsgroup/org/apache/hadoop/hbase/client/rsgroup/TestShellRSGroups.java b/hbase-shell/src/test/rsgroup/org/apache/hadoop/hbase/client/rsgroup/TestShellRSGroups.java index 70b215c452d..55260696fdb 100644 --- a/hbase-shell/src/test/rsgroup/org/apache/hadoop/hbase/client/rsgroup/TestShellRSGroups.java +++ b/hbase-shell/src/test/rsgroup/org/apache/hadoop/hbase/client/rsgroup/TestShellRSGroups.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -18,6 +17,10 @@ */ package org.apache.hadoop.hbase.client.rsgroup; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -31,19 +34,21 @@ import org.jruby.embed.PathType; import org.jruby.embed.ScriptingContainer; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - //Separate Shell test class for Groups //Since we need to use a different balancer and run more than 1 RS @Category({ClientTests.class, LargeTests.class}) public class TestShellRSGroups { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestShellRSGroups.class); + final Logger LOG = LoggerFactory.getLogger(getClass()); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static ScriptingContainer jruby = new ScriptingContainer(); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestCallQueue.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestCallQueue.java index 526b2e00e10..7e01029fac1 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestCallQueue.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestCallQueue.java @@ -1,33 +1,33 @@ -/* - * Copyright The Apache Software Foundation +/** + * 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 * - * 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 + * 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. + * 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.thrift; import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.LinkedBlockingQueue; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -44,6 +44,10 @@ import org.slf4j.LoggerFactory; @RunWith(Parameterized.class) public class TestCallQueue { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCallQueue.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCallQueue.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); @@ -107,8 +111,8 @@ public class TestCallQueue { private static void verifyMetrics(ThriftMetrics metrics, String name, int expectValue) - throws Exception { - metricsHelper.assertCounter(name, expectValue, metrics.getSource()); + throws Exception { + metricsHelper.assertCounter(name, expectValue, metrics.getSource()); } private static Runnable createDummyRunnable() { @@ -118,6 +122,5 @@ public class TestCallQueue { } }; } - } diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java index 9c113d04e60..8a9f9032794 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java @@ -1,20 +1,19 @@ -/* - * Copyright The Apache Software Foundation +/** + * 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 * - * 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 + * 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. + * 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.thrift; @@ -24,8 +23,8 @@ import static org.junit.Assert.fail; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -40,12 +39,14 @@ import org.apache.thrift.transport.THttpClient; import org.apache.thrift.transport.TTransportException; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Joiner; /** @@ -56,6 +57,10 @@ import org.apache.hbase.thirdparty.com.google.common.base.Joiner; public class TestThriftHttpServer { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestThriftHttpServer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestThriftHttpServer.class); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java index fc474eef829..01cffe46126 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -31,9 +30,9 @@ import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -63,6 +62,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -76,6 +76,11 @@ import org.slf4j.LoggerFactory; */ @Category({ClientTests.class, LargeTests.class}) public class TestThriftServer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestThriftServer.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestThriftServer.class); private static final MetricsAssertHelper metricsHelper = CompatibilityFactory diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java index 9cfb0fe109b..26546fd9c53 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java @@ -1,20 +1,19 @@ -/* - * Copyright The Apache Software Foundation +/** + * 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 * - * 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 + * 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. + * 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.thrift; @@ -25,7 +24,7 @@ import java.net.InetAddress; import java.util.ArrayList; import java.util.Collection; import java.util.List; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -43,6 +42,7 @@ import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -50,6 +50,7 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Joiner; /** @@ -60,6 +61,10 @@ import org.apache.hbase.thirdparty.com.google.common.base.Joiner; @RunWith(Parameterized.class) public class TestThriftServerCmdLine { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestThriftServerCmdLine.class); + private static final Logger LOG = LoggerFactory.getLogger(TestThriftServerCmdLine.class); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java index e98c2cf5856..dc0522778f0 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,13 +18,11 @@ package org.apache.hadoop.hbase.thrift2; import static java.nio.ByteBuffer.wrap; - import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deleteFromThrift; import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getFromThrift; import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.incrementFromThrift; import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.putFromThrift; import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.scanFromThrift; - import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -45,12 +42,12 @@ import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.TimeUnit; - import org.apache.commons.collections4.CollectionUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompatibilityFactory; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -99,12 +96,14 @@ import org.apache.thrift.TException; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -114,6 +113,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({ClientTests.class, MediumTests.class}) public class TestThriftHBaseServiceHandler { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestThriftHBaseServiceHandler.class); + private static final Logger LOG = LoggerFactory.getLogger(TestThriftHBaseServiceHandler.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java index 2ed312472bd..ee5ecdef5e5 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.thrift2; import static java.nio.ByteBuffer.wrap; - import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -31,8 +30,8 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.List; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -68,6 +67,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -76,6 +76,10 @@ import org.slf4j.LoggerFactory; @Category({ClientTests.class, MediumTests.class}) public class TestThriftHBaseServiceHandlerWithLabels { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestThriftHBaseServiceHandlerWithLabels.class); + private static final Logger LOG = LoggerFactory .getLogger(TestThriftHBaseServiceHandlerWithLabels.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithReadOnly.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithReadOnly.java index fb6407cb8be..a8b306a3bf2 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithReadOnly.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithReadOnly.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -19,7 +18,6 @@ package org.apache.hadoop.hbase.thrift2; import static java.nio.ByteBuffer.wrap; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -29,9 +27,9 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; @@ -57,11 +55,17 @@ import org.apache.thrift.TException; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ClientTests.class, MediumTests.class}) public class TestThriftHBaseServiceHandlerWithReadOnly { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestThriftHBaseServiceHandlerWithReadOnly.class); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); // Static names for tables, columns, rows, and values diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java index 89bb034fa66..6deba3fa0a9 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java @@ -22,16 +22,17 @@ import static org.junit.Assert.assertNotNull; import java.io.IOException; import java.util.Properties; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ZKTests; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,6 +41,11 @@ import org.junit.experimental.categories.Category; */ @Category({ ZKTests.class, MediumTests.class }) public class TestHQuorumPeer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHQuorumPeer.class); + private static final HBaseZKTestingUtility TEST_UTIL = new HBaseZKTestingUtility(); private static int PORT_NO = 21818; private Path dataDir; diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java index 5c984a55f16..99379532ed1 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java @@ -18,15 +18,21 @@ package org.apache.hadoop.hbase.zookeeper; import java.util.concurrent.atomic.AtomicReference; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.ZKTests; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ZKTests.class, SmallTests.class }) public class TestInstancePending { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestInstancePending.class); + @Test(timeout = 1000) public void test() throws Exception { final InstancePending pending = new InstancePending<>(); diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java index 47f0e0b91f8..a97a7c61746 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java @@ -43,6 +43,7 @@ import java.util.concurrent.Exchanger; import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; @@ -56,12 +57,17 @@ import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ZKTests.class, MediumTests.class }) public class TestReadOnlyZKClient { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReadOnlyZKClient.class); + private static HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility(); private static int PORT; diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java index bcba906a05b..1e7db058463 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java @@ -21,9 +21,9 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.lang.reflect.Field; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -37,12 +37,17 @@ import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ZKTests.class, MediumTests.class }) public class TestRecoverableZooKeeper { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRecoverableZooKeeper.class); + private final static HBaseZKTestingUtility TEST_UTIL = new HBaseZKTestingUtility(); Abortable abortable = new Abortable() { diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java index 5971a18b039..8371f91c318 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java @@ -24,9 +24,9 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.Stoppable; @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -44,6 +45,11 @@ import org.slf4j.LoggerFactory; @Category({ ZKTests.class, MediumTests.class }) public class TestZKLeaderManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKLeaderManager.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKLeaderManager.class); private static final String LEADER_ZNODE = diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java index d8db8aedacd..f8052603c1d 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -22,18 +21,24 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.security.Permission; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.ZKTests; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ZKTests.class, SmallTests.class }) public class TestZKMainServer { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKMainServer.class); + // ZKMS calls System.exit. Catch the call and prevent exit using trick described up in // http://stackoverflow.com/questions/309396/java-how-to-test-methods-that-call-system-exit protected static class ExitException extends SecurityException { diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java index e6fe8108000..7be8b2f7159 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -25,9 +23,9 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ZKTests; @@ -39,6 +37,7 @@ import org.apache.zookeeper.Op; import org.apache.zookeeper.ZooDefs.Ids; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -49,6 +48,11 @@ import org.slf4j.LoggerFactory; */ @Category({ ZKTests.class, MediumTests.class }) public class TestZKMulti { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKMulti.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKMulti.class); private final static HBaseZKTestingUtility TEST_UTIL = new HBaseZKTestingUtility(); private static ZKWatcher zkw = null; diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java index 9afa9d1595e..9e90177ea34 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -27,8 +26,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.concurrent.Semaphore; import java.util.concurrent.ThreadLocalRandom; - import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; @@ -41,6 +40,7 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -48,6 +48,11 @@ import org.slf4j.LoggerFactory; @Category({ ZKTests.class, MediumTests.class }) public class TestZKNodeTracker { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKNodeTracker.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKNodeTracker.class); private final static HBaseZKTestingUtility TEST_UTIL = new HBaseZKTestingUtility(); diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java index 95dd270ff5c..6c3279a8182 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ZKTests; @@ -39,6 +40,7 @@ import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -49,6 +51,10 @@ import org.apache.hbase.thirdparty.com.google.common.io.Closeables; @Category({ ZKTests.class, MediumTests.class }) public class TestZKUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKUtil.class); + private static final Logger LOG = LoggerFactory.getLogger(TestZKUtil.class); private static HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility(); diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtilNoServer.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtilNoServer.java index cc2517b163f..99ec35ebab7 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtilNoServer.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtilNoServer.java @@ -7,14 +7,13 @@ * "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 + * 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. + * 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.zookeeper; @@ -25,6 +24,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.security.Superusers; @@ -36,6 +36,7 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.Perms; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Id; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -43,6 +44,10 @@ import org.mockito.Mockito; @Category({ ZKTests.class, SmallTests.class }) public class TestZKUtilNoServer { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKUtilNoServer.class); + @Test public void testUnsecure() throws ZooKeeperConnectionException, IOException { Configuration conf = HBaseConfiguration.create(); diff --git a/pom.xml b/pom.xml index bf5abb8a47a..a7b08dbac44 100755 --- a/pom.xml +++ b/pom.xml @@ -611,6 +611,12 @@ --> ${test.exclude.pattern} + + + listener + org.apache.hadoop.hbase.HBaseClassTestRuleChecker,org.apache.hadoop.hbase.ResourceCheckerJUnitListener + +