You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2020/08/04 02:33:05 UTC
[hbase] branch master updated: HBAE-24507 Remove HTableDescriptor
and HColumnDescriptor (#2186)
This is an automated email from the ASF dual-hosted git repository.
zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new d2f5a5f HBAE-24507 Remove HTableDescriptor and HColumnDescriptor (#2186)
d2f5a5f is described below
commit d2f5a5f27bed9753d5dbb2dbf30e1b1aa1aba8ca
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Tue Aug 4 10:31:42 2020 +0800
HBAE-24507 Remove HTableDescriptor and HColumnDescriptor (#2186)
Signed-off-by: stack <st...@apache.org>
Signed-off-by: Viraj Jasani <vj...@apache.org>
Signed-off-by: tedyu <yu...@gmail.com>
---
.../exemplars/shaded_client/HelloHBase.java | 6 +-
.../apache/hadoop/hbase/backup/TestBackupBase.java | 15 +-
.../hadoop/hbase/backup/TestIncrementalBackup.java | 26 +-
.../backup/TestIncrementalBackupWithFailures.java | 10 +-
.../hadoop/hbase/backup/TestRemoteBackup.java | 21 +-
.../org/apache/hadoop/hbase/HColumnDescriptor.java | 792 ----------------
.../org/apache/hadoop/hbase/HTableDescriptor.java | 992 ---------------------
.../client/ColumnFamilyDescriptorBuilder.java | 13 +-
.../hbase/client/TableDescriptorBuilder.java | 32 +-
.../apache/hadoop/hbase/TestHColumnDescriptor.java | 148 ---
.../apache/hadoop/hbase/TestHTableDescriptor.java | 369 --------
.../client/TestColumnFamilyDescriptorBuilder.java | 64 +-
.../hbase/client/TestTableDescriptorBuilder.java | 170 ++--
.../coprocessor/TestBatchCoprocessorEndpoint.java | 9 +-
.../hbase/coprocessor/TestCoprocessorEndpoint.java | 9 +-
.../coprocessor/TestCoprocessorTableEndpoint.java | 32 +-
.../IntegrationTestIngestStripeCompactions.java | 16 +-
.../hadoop/hbase/IntegrationTestIngestWithMOB.java | 8 +-
.../hadoop/hbase/IntegrationTestLazyCfLoading.java | 13 +-
.../hadoop/hbase/IntegrationTestMobCompaction.java | 20 +-
.../StripeCompactionsPerformanceEvaluation.java | 38 +-
.../hbase/test/IntegrationTestBigLinkedList.java | 42 +-
...IntegrationTestBigLinkedListWithVisibility.java | 12 +-
.../hbase/test/IntegrationTestLoadAndVerify.java | 14 +-
...grationTestWithCellVisibilityLoadAndVerify.java | 6 +-
.../hadoop/hbase/mapred/TableRecordReader.java | 2 +-
.../hadoop/hbase/mapred/TableRecordReaderImpl.java | 6 +-
.../apache/hadoop/hbase/mapreduce/ImportTsv.java | 25 +-
.../hbase/mapreduce/TableRecordReaderImpl.java | 6 +-
.../apache/hadoop/hbase/PerformanceEvaluation.java | 30 +-
.../hbase/mapreduce/TestTimeRangeMapRed.java | 12 +-
.../apache/hadoop/hbase/rest/SchemaResource.java | 5 +-
.../hadoop/hbase/rest/model/ColumnSchemaModel.java | 42 +-
.../hadoop/hbase/rest/model/TableSchemaModel.java | 30 +-
.../hadoop/hbase/rest/PerformanceEvaluation.java | 20 +-
.../hadoop/hbase/rest/TestScannersWithFilters.java | 10 +-
.../hbase/rest/client/TestRemoteAdminRetries.java | 10 +-
.../hadoop/hbase/rest/client/TestRemoteTable.java | 16 +-
.../hbase/tmpl/master/MasterStatusTmpl.jamon | 6 +-
.../org/apache/hadoop/hbase/LocalHBaseCluster.java | 31 +-
.../hadoop/hbase/constraint/Constraints.java | 1 -
.../hadoop/hbase/io/hfile/LruBlockCache.java | 73 +-
.../org/apache/hadoop/hbase/quotas/QuotaUtil.java | 27 +-
.../security/visibility/VisibilityController.java | 22 +-
.../resources/hbase-webapps/master/rsgroup.jsp | 12 +-
.../main/resources/hbase-webapps/master/table.jsp | 2 +-
.../apache/hadoop/hbase/HBaseTestingUtility.java | 319 +++----
...TestColumnFamilyDescriptorDefaultVersions.java} | 57 +-
.../org/apache/hadoop/hbase/TestMultiVersions.java | 43 +-
.../apache/hadoop/hbase/TestRegionRebalancing.java | 9 +-
.../org/apache/hadoop/hbase/TestSerialization.java | 31 +-
.../org/apache/hadoop/hbase/client/TestAdmin2.java | 87 +-
.../hbase/client/TestAsyncTableAdminApi3.java | 6 +-
.../hbase/client/TestBlockEvictionFromClient.java | 9 +-
.../hadoop/hbase/client/TestEnableTable.java | 24 +-
.../hadoop/hbase/client/TestFromClientSide.java | 25 +-
.../hadoop/hbase/client/TestFromClientSide3.java | 58 +-
.../hadoop/hbase/client/TestFromClientSide4.java | 19 +-
.../hadoop/hbase/client/TestFromClientSide5.java | 19 +-
.../hbase/client/TestIllegalTableDescriptor.java | 142 ++-
.../hbase/client/TestIncrementsFromClientSide.java | 12 +-
.../hbase/client/TestIntraRowPagination.java | 11 +-
.../hbase/client/TestReplicaWithCluster.java | 89 +-
.../hadoop/hbase/client/TestReplicasClient.java | 11 +-
.../hbase/client/TestScannersFromClientSide.java | 3 +-
.../hbase/client/TestServerBusyException.java | 6 +-
.../hadoop/hbase/client/TestSizeFailures.java | 10 +-
.../client/TestSnapshotCloneIndependence.java | 18 +-
.../hbase/client/TestSnapshotFromClient.java | 11 +-
.../hadoop/hbase/client/TestSnapshotMetadata.java | 59 +-
.../client/TestSnapshotTemporaryDirectory.java | 7 +-
.../coprocessor/TestCoprocessorConfiguration.java | 8 +-
.../coprocessor/TestCoprocessorInterface.java | 13 +-
.../hbase/coprocessor/TestCoprocessorMetrics.java | 73 +-
.../TestMasterCoprocessorExceptionWithAbort.java | 9 +-
.../TestMasterCoprocessorExceptionWithRemove.java | 15 +-
.../hbase/coprocessor/TestMasterObserver.java | 30 +-
.../coprocessor/TestOpenTableInCoprocessor.java | 20 +-
...ObserverForAddingMutationsFromCoprocessors.java | 10 +-
.../coprocessor/TestRegionObserverInterface.java | 9 +-
.../TestRegionObserverScannerOpenHook.java | 37 +-
.../coprocessor/TestRegionObserverStacking.java | 15 +-
.../hadoop/hbase/filter/FilterTestingCluster.java | 10 +-
.../hbase/filter/TestDependentColumnFilter.java | 20 +-
.../org/apache/hadoop/hbase/filter/TestFilter.java | 45 +-
.../hbase/filter/TestFilterFromRegionSide.java | 11 +-
.../hadoop/hbase/filter/TestFilterWrapper.java | 16 +-
.../hbase/filter/TestInvocationRecordFilter.java | 10 +-
.../io/encoding/TestSeekBeforeWithReverseScan.java | 10 +-
.../hfile/TestScannerSelectionUsingKeyRange.java | 12 +-
.../org/apache/hadoop/hbase/master/TestMaster.java | 8 +-
.../hbase/master/TestMasterMetricsWrapper.java | 8 +-
.../hadoop/hbase/master/TestRegionPlacement.java | 8 +-
.../hbase/master/TestRegionPlansWithThrottle.java | 6 +-
.../balancer/TestFavoredNodeTableImport.java | 13 +-
.../TestFavoredStochasticLoadBalancer.java | 80 +-
.../TestDeleteColumnFamilyProcedureFromClient.java | 16 +-
.../master/procedure/TestModifyTableProcedure.java | 83 +-
.../procedure/TestRestoreSnapshotProcedure.java | 55 +-
.../TestTableDescriptorModificationFromClient.java | 94 +-
.../hadoop/hbase/mob/MobStressToolRunner.java | 21 +-
.../hbase/mob/TestDefaultMobStoreFlusher.java | 83 +-
.../hadoop/hbase/mob/TestMobCompactionOptMode.java | 7 +-
.../mob/TestMobCompactionOptRegionBatchMode.java | 1 -
.../TestMobCompactionRegularRegionBatchMode.java | 1 -
.../hbase/mob/TestMobCompactionWithDefaults.java | 17 +-
.../hadoop/hbase/mob/TestMobDataBlockEncoding.java | 20 +-
.../hadoop/hbase/mob/TestMobFileCleanerChore.java | 19 +-
.../hadoop/hbase/mob/TestMobStoreCompaction.java | 15 +-
.../hadoop/hbase/mob/TestMobStoreScanner.java | 19 +-
.../hbase/namespace/TestNamespaceAuditor.java | 13 +-
.../hbase/regionserver/TestAtomicOperation.java | 23 +-
.../hbase/regionserver/TestBlocksScanned.java | 37 +-
.../hadoop/hbase/regionserver/TestBulkLoad.java | 9 +-
.../hbase/regionserver/TestColumnSeeking.java | 13 +-
.../hbase/regionserver/TestCompactSplitThread.java | 14 +-
.../hbase/regionserver/TestCompactingMemStore.java | 11 +-
.../hadoop/hbase/regionserver/TestCompaction.java | 19 +-
.../hbase/regionserver/TestCompactionPolicy.java | 13 +-
.../hbase/regionserver/TestDeleteMobTable.java | 23 +-
.../hbase/regionserver/TestFSErrorsExposed.java | 11 +-
.../hadoop/hbase/regionserver/TestHRegion.java | 102 +--
.../hbase/regionserver/TestHRegionOnCluster.java | 7 +-
.../hbase/regionserver/TestIsDeleteFailure.java | 6 +-
.../hbase/regionserver/TestJoinedScanners.java | 15 +-
.../hadoop/hbase/regionserver/TestKeepDeletes.java | 29 +-
.../hbase/regionserver/TestMajorCompaction.java | 10 +-
.../hbase/regionserver/TestMinorCompaction.java | 13 +-
.../hbase/regionserver/TestMutateRowsRecovery.java | 7 +-
.../TestNewVersionBehaviorFromClientSide.java | 11 +-
.../hadoop/hbase/regionserver/TestParallelPut.java | 10 +-
.../regionserver/TestPerColumnFamilyFlush.java | 40 +-
.../hbase/regionserver/TestRSStatusServlet.java | 15 +-
.../hadoop/hbase/regionserver/TestRegionOpen.java | 16 +-
.../regionserver/TestRegionReplicaFailover.java | 30 +-
.../TestRegionReplicasAreDistributed.java | 7 +-
.../TestRegionReplicasWithModifyTable.java | 8 +-
.../hbase/regionserver/TestResettingCounters.java | 11 +-
.../hbase/regionserver/TestReversibleScanners.java | 10 +-
.../hadoop/hbase/regionserver/TestRowTooBig.java | 27 +-
.../regionserver/TestSCVFWithMiniCluster.java | 16 +-
.../hadoop/hbase/regionserver/TestScanner.java | 23 +-
.../regionserver/TestScannerWithCorruptHFile.java | 9 +-
.../TestSettingTimeoutOnBlockingPoint.java | 11 +-
.../TestSplitTransactionOnCluster.java | 12 +-
.../regionserver/TestStoreScannerClosure.java | 7 +-
.../apache/hadoop/hbase/regionserver/TestTags.java | 42 +-
.../TestWalAndCompactingMemStoreFlush.java | 18 +-
.../compactions/TestCompactedHFilesDischarger.java | 7 +-
.../compactions/TestDateTieredCompactor.java | 4 +-
.../compactions/TestStripeCompactionPolicy.java | 5 +-
.../compactions/TestStripeCompactor.java | 4 +-
.../regionserver/wal/AbstractTestWALReplay.java | 355 +++-----
.../replication/TestMultiSlaveReplication.java | 15 +-
.../replication/TestPerTableCFReplication.java | 83 +-
.../hbase/replication/TestReplicationWithTags.java | 14 +-
.../TestGlobalReplicationThrottler.java | 10 +-
.../TestRegionReplicaReplicationEndpoint.java | 73 +-
...stRegionReplicaReplicationEndpointNoMaster.java | 10 +-
.../security/access/TestAccessController.java | 84 +-
.../security/access/TestAccessController2.java | 22 +-
.../security/access/TestAccessController3.java | 22 +-
.../access/TestCellACLWithMultipleVersions.java | 20 +-
.../hadoop/hbase/security/access/TestCellACLs.java | 14 +-
.../TestCoprocessorWhitelistMasterObserver.java | 48 +-
.../security/access/TestNamespaceCommands.java | 8 +-
.../security/access/TestScanEarlyTermination.java | 26 +-
.../access/TestWithDisabledAuthorization.java | 38 +-
.../token/TestDelegationTokenWithEncryption.java | 8 +-
...tVisibilityLabelReplicationWithExpAsString.java | 10 +-
.../security/visibility/TestVisibilityLabels.java | 35 +-
.../TestVisibilityLabelsReplication.java | 10 +-
.../visibility/TestVisibilityWithCheckAuths.java | 10 +-
.../hadoop/hbase/util/BaseTestHBaseFsck.java | 5 +-
.../apache/hadoop/hbase/util/HFileTestUtil.java | 5 +-
.../hbase/util/TestMiniClusterLoadSequential.java | 9 +-
.../hbase/thrift/ThriftHBaseServiceHandler.java | 21 +-
.../hadoop/hbase/thrift/ThriftUtilities.java | 30 +-
.../hadoop/hbase/thrift2/ThriftUtilities.java | 18 +-
.../thrift2/TestThriftHBaseServiceHandler.java | 36 +-
.../TestThriftHBaseServiceHandlerWithLabels.java | 19 +-
.../TestThriftHBaseServiceHandlerWithReadOnly.java | 19 +-
182 files changed, 1960 insertions(+), 5132 deletions(-)
diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
index 4179955..053275a 100644
--- a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
+++ b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
@@ -110,10 +110,8 @@ public final class HelloHBase {
+ "], with one Column Family ["
+ Bytes.toString(MY_COLUMN_FAMILY_NAME) + "].");
- admin.createTable(new TableDescriptorBuilder.ModifyableTableDescriptor(MY_TABLE_NAME)
- .setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- MY_COLUMN_FAMILY_NAME)));
+ admin.createTable(TableDescriptorBuilder.newBuilder(MY_TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(MY_COLUMN_FAMILY_NAME)).build());
}
}
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
index 94a8bde..12060dc 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
@@ -47,13 +47,13 @@ import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
import org.apache.hadoop.hbase.backup.util.BackupUtils;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
import org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner;
@@ -84,7 +84,7 @@ public class TestBackupBase {
protected static Configuration conf2;
protected static TableName table1 = TableName.valueOf("table1");
- protected static TableDescriptorBuilder.ModifyableTableDescriptor table1Desc;
+ protected static TableDescriptor table1Desc;
protected static TableName table2 = TableName.valueOf("table2");
protected static TableName table3 = TableName.valueOf("table3");
protected static TableName table4 = TableName.valueOf("table4");
@@ -428,11 +428,8 @@ public class TestBackupBase {
ha.createNamespace(desc3);
ha.createNamespace(desc4);
- TableDescriptorBuilder.ModifyableTableDescriptor desc =
- new TableDescriptorBuilder.ModifyableTableDescriptor(table1);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(famName);
- desc.setColumnFamily(familyDescriptor);
+ TableDescriptor desc = TableDescriptorBuilder.newBuilder(table1)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(famName)).build();
ha.createTable(desc);
table1Desc = desc;
Connection conn = ConnectionFactory.createConnection(conf1);
@@ -440,8 +437,8 @@ public class TestBackupBase {
loadTable(table);
table.close();
table2 = TableName.valueOf("ns2:test-" + tid + 1);
- desc = new TableDescriptorBuilder.ModifyableTableDescriptor(table2);
- desc.setColumnFamily(familyDescriptor);
+ desc = TableDescriptorBuilder.newBuilder(table2)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(famName)).build();
ha.createTable(desc);
table = conn.getTable(table2);
loadTable(table);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
index b53c3e6..5ed4025 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
@@ -34,6 +34,8 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -82,14 +84,11 @@ public class TestIncrementalBackup extends TestBackupBase {
final byte[] fam3Name = Bytes.toBytes("f3");
final byte[] mobName = Bytes.toBytes("mob");
- table1Desc.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3Name));
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor mobHcd =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(mobName);
- mobHcd.setMobEnabled(true);
- mobHcd.setMobThreshold(5L);
- table1Desc.setColumnFamily(mobHcd);
- HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
+ TableDescriptor newTable1Desc = TableDescriptorBuilder.newBuilder(table1Desc)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam3Name))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(mobName).setMobEnabled(true)
+ .setMobThreshold(5L).build()).build();
+ TEST_UTIL.getAdmin().modifyTable(newTable1Desc);
try (Connection conn = ConnectionFactory.createConnection(conf1)) {
int NB_ROWS_FAM3 = 6;
@@ -150,13 +149,12 @@ public class TestIncrementalBackup extends TestBackupBase {
assertTrue(checkSucceeded(backupIdIncMultiple));
// add column family f2 to table1
- final byte[] fam2Name = Bytes.toBytes("f2");
- table1Desc.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2Name));
-
// drop column family f3
- table1Desc.removeColumnFamily(fam3Name);
- HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
+ final byte[] fam2Name = Bytes.toBytes("f2");
+ newTable1Desc = TableDescriptorBuilder.newBuilder(newTable1Desc)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam2Name)).removeColumnFamily(fam3Name)
+ .build();
+ TEST_UTIL.getAdmin().modifyTable(newTable1Desc);
int NB_ROWS_FAM2 = 7;
Table t3 = insertIntoTable(conn, table1, fam2Name, 2, NB_ROWS_FAM2);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
index aeb1edb..00b13ba 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
@@ -24,7 +24,6 @@ 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.TableName;
import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
@@ -37,6 +36,8 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.ToolRunner;
@@ -76,16 +77,15 @@ public class TestIncrementalBackupWithFailures extends TestBackupBase {
// implement all test cases in 1 test since incremental backup/restore has dependencies
@Test
public void testIncBackupRestore() throws Exception {
-
int ADD_ROWS = 99;
// #1 - create full backup for all tables
LOG.info("create full backup image for all tables");
List<TableName> tables = Lists.newArrayList(table1, table2);
final byte[] fam3Name = Bytes.toBytes("f3");
- table1Desc.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3Name));
- HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
+ TableDescriptor newTable1Desc = TableDescriptorBuilder.newBuilder(table1Desc)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam3Name)).build();
+ TEST_UTIL.getAdmin().modifyTable(newTable1Desc);
Connection conn = ConnectionFactory.createConnection(conf1);
int NB_ROWS_FAM3 = 6;
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
index 3ef37c4..136c4b2 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.concurrent.CountDownLatch;
-
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
@@ -32,11 +31,12 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.junit.Assert;
import org.junit.ClassRule;
import org.junit.Test;
@@ -44,6 +44,8 @@ import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
@Category(LargeTests.class)
public class TestRemoteBackup extends TestBackupBase {
@@ -93,16 +95,13 @@ public class TestRemoteBackup extends TestBackupBase {
}
});
t.start();
-
- table1Desc.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3Name));
// family 2 is MOB enabled
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2Name);
- familyDescriptor.setMobEnabled(true);
- familyDescriptor.setMobThreshold(0L);
- table1Desc.setColumnFamily(familyDescriptor);
- TEST_UTIL.getAdmin().modifyTable(table1Desc);
+ TableDescriptor newTable1Desc = TableDescriptorBuilder.newBuilder(table1Desc)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam3Name))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam2Name).setMobEnabled(true)
+ .setMobThreshold(0L).build())
+ .build();
+ TEST_UTIL.getAdmin().modifyTable(newTable1Desc);
SnapshotTestingUtils.loadData(TEST_UTIL, table1, 50, fam2Name);
Table t1 = conn.getTable(table1);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
deleted file mode 100644
index 4b94929..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ /dev/null
@@ -1,792 +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.util.Map;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.exceptions.HBaseException;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.PrettyPrinter.Unit;
-
-/**
- * An HColumnDescriptor contains information about a column family such as the
- * number of versions, compression settings, etc.
- *
- * It is used as input when creating a table or adding a column.
- */
-@InterfaceAudience.Public
-@Deprecated // remove it in 3.0
-public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HColumnDescriptor> {
- public static final String IN_MEMORY_COMPACTION = ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION;
- public static final String COMPRESSION = ColumnFamilyDescriptorBuilder.COMPRESSION;
- public static final String COMPRESSION_COMPACT = ColumnFamilyDescriptorBuilder.COMPRESSION_COMPACT;
- public static final String ENCODE_ON_DISK = "ENCODE_ON_DISK";
- public static final String DATA_BLOCK_ENCODING = ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING;
- public static final String BLOCKCACHE = ColumnFamilyDescriptorBuilder.BLOCKCACHE;
- public static final String CACHE_DATA_ON_WRITE = ColumnFamilyDescriptorBuilder.CACHE_DATA_ON_WRITE;
- public static final String CACHE_INDEX_ON_WRITE = ColumnFamilyDescriptorBuilder.CACHE_INDEX_ON_WRITE;
- public static final String CACHE_BLOOMS_ON_WRITE = ColumnFamilyDescriptorBuilder.CACHE_BLOOMS_ON_WRITE;
- public static final String EVICT_BLOCKS_ON_CLOSE = ColumnFamilyDescriptorBuilder.EVICT_BLOCKS_ON_CLOSE;
- public static final String CACHE_DATA_IN_L1 = "CACHE_DATA_IN_L1";
- public static final String PREFETCH_BLOCKS_ON_OPEN = ColumnFamilyDescriptorBuilder.PREFETCH_BLOCKS_ON_OPEN;
- public static final String BLOCKSIZE = ColumnFamilyDescriptorBuilder.BLOCKSIZE;
- public static final String LENGTH = "LENGTH";
- public static final String TTL = ColumnFamilyDescriptorBuilder.TTL;
- public static final String BLOOMFILTER = ColumnFamilyDescriptorBuilder.BLOOMFILTER;
- public static final String FOREVER = "FOREVER";
- public static final String REPLICATION_SCOPE = ColumnFamilyDescriptorBuilder.REPLICATION_SCOPE;
- public static final byte[] REPLICATION_SCOPE_BYTES = Bytes.toBytes(REPLICATION_SCOPE);
- public static final String MIN_VERSIONS = ColumnFamilyDescriptorBuilder.MIN_VERSIONS;
- public static final String KEEP_DELETED_CELLS = ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS;
- public static final String COMPRESS_TAGS = ColumnFamilyDescriptorBuilder.COMPRESS_TAGS;
- public static final String ENCRYPTION = ColumnFamilyDescriptorBuilder.ENCRYPTION;
- public static final String ENCRYPTION_KEY = ColumnFamilyDescriptorBuilder.ENCRYPTION_KEY;
- public static final String IS_MOB = ColumnFamilyDescriptorBuilder.IS_MOB;
- public static final byte[] IS_MOB_BYTES = Bytes.toBytes(IS_MOB);
- public static final String MOB_THRESHOLD = ColumnFamilyDescriptorBuilder.MOB_THRESHOLD;
- public static final byte[] MOB_THRESHOLD_BYTES = Bytes.toBytes(MOB_THRESHOLD);
- public static final long DEFAULT_MOB_THRESHOLD = ColumnFamilyDescriptorBuilder.DEFAULT_MOB_THRESHOLD;
- public static final String MOB_COMPACT_PARTITION_POLICY = ColumnFamilyDescriptorBuilder.MOB_COMPACT_PARTITION_POLICY;
- public static final byte[] MOB_COMPACT_PARTITION_POLICY_BYTES = Bytes.toBytes(MOB_COMPACT_PARTITION_POLICY);
- public static final MobCompactPartitionPolicy DEFAULT_MOB_COMPACT_PARTITION_POLICY
- = ColumnFamilyDescriptorBuilder.DEFAULT_MOB_COMPACT_PARTITION_POLICY;
- public static final String DFS_REPLICATION = ColumnFamilyDescriptorBuilder.DFS_REPLICATION;
- public static final short DEFAULT_DFS_REPLICATION = ColumnFamilyDescriptorBuilder.DEFAULT_DFS_REPLICATION;
- public static final String STORAGE_POLICY = ColumnFamilyDescriptorBuilder.STORAGE_POLICY;
- public static final String DEFAULT_COMPRESSION = ColumnFamilyDescriptorBuilder.DEFAULT_COMPRESSION.name();
- public static final boolean DEFAULT_ENCODE_ON_DISK = true;
- public static final String DEFAULT_DATA_BLOCK_ENCODING = ColumnFamilyDescriptorBuilder.DEFAULT_DATA_BLOCK_ENCODING.name();
- public static final int DEFAULT_VERSIONS = ColumnFamilyDescriptorBuilder.DEFAULT_MAX_VERSIONS;
- public static final int DEFAULT_MIN_VERSIONS = ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS;
- public static final boolean DEFAULT_IN_MEMORY = ColumnFamilyDescriptorBuilder.DEFAULT_IN_MEMORY;
- public static final KeepDeletedCells DEFAULT_KEEP_DELETED = ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED;
- public static final boolean DEFAULT_BLOCKCACHE = ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKCACHE;
- public static final boolean DEFAULT_CACHE_DATA_ON_WRITE = ColumnFamilyDescriptorBuilder.DEFAULT_CACHE_DATA_ON_WRITE;
- public static final boolean DEFAULT_CACHE_DATA_IN_L1 = false;
- public static final boolean DEFAULT_CACHE_INDEX_ON_WRITE = ColumnFamilyDescriptorBuilder.DEFAULT_CACHE_INDEX_ON_WRITE;
- public static final int DEFAULT_BLOCKSIZE = ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKSIZE;
- public static final String DEFAULT_BLOOMFILTER = ColumnFamilyDescriptorBuilder.DEFAULT_BLOOMFILTER.name();
- public static final boolean DEFAULT_CACHE_BLOOMS_ON_WRITE = ColumnFamilyDescriptorBuilder.DEFAULT_CACHE_BLOOMS_ON_WRITE;
- public static final int DEFAULT_TTL = ColumnFamilyDescriptorBuilder.DEFAULT_TTL;
- public static final int DEFAULT_REPLICATION_SCOPE = ColumnFamilyDescriptorBuilder.DEFAULT_REPLICATION_SCOPE;
- public static final boolean DEFAULT_EVICT_BLOCKS_ON_CLOSE = ColumnFamilyDescriptorBuilder.DEFAULT_EVICT_BLOCKS_ON_CLOSE;
- public static final boolean DEFAULT_COMPRESS_TAGS = ColumnFamilyDescriptorBuilder.DEFAULT_COMPRESS_TAGS;
- public static final boolean DEFAULT_PREFETCH_BLOCKS_ON_OPEN = ColumnFamilyDescriptorBuilder.DEFAULT_PREFETCH_BLOCKS_ON_OPEN;
- public static final String NEW_VERSION_BEHAVIOR = ColumnFamilyDescriptorBuilder.NEW_VERSION_BEHAVIOR;
- public static final boolean DEFAULT_NEW_VERSION_BEHAVIOR = ColumnFamilyDescriptorBuilder.DEFAULT_NEW_VERSION_BEHAVIOR;
- protected final ModifyableColumnFamilyDescriptor delegatee;
-
- /**
- * Construct a column descriptor specifying only the family name
- * The other attributes are defaulted.
- *
- * @param familyName Column family name. Must be 'printable' -- digit or
- * letter -- and may not contain a <code>:</code>
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (<a href="https://issues.apache.org/jira/browse/HBASE-18433">HBASE-18433</a>).
- * Use {@link ColumnFamilyDescriptorBuilder#of(String)}.
- */
- @Deprecated
- public HColumnDescriptor(final String familyName) {
- this(new ModifyableColumnFamilyDescriptor(Bytes.toBytes(familyName)));
- }
-
- /**
- * Construct a column descriptor specifying only the family name
- * The other attributes are defaulted.
- *
- * @param familyName Column family name. Must be 'printable' -- digit or
- * letter -- and may not contain a <code>:</code>
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (<a href="https://issues.apache.org/jira/browse/HBASE-18433">HBASE-18433</a>).
- * Use {@link ColumnFamilyDescriptorBuilder#of(byte[])}.
- */
- @Deprecated
- public HColumnDescriptor(final byte [] familyName) {
- this(new ModifyableColumnFamilyDescriptor(familyName));
- }
-
- /**
- * Constructor.
- * Makes a deep copy of the supplied descriptor.
- * Can make a modifiable descriptor from an UnmodifyableHColumnDescriptor.
- *
- * @param desc The descriptor.
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (<a href="https://issues.apache.org/jira/browse/HBASE-18433">HBASE-18433</a>).
- * Use {@link ColumnFamilyDescriptorBuilder#copy(ColumnFamilyDescriptor)}.
- */
- @Deprecated
- public HColumnDescriptor(HColumnDescriptor desc) {
- this(desc, true);
- }
-
- protected HColumnDescriptor(HColumnDescriptor desc, boolean deepClone) {
- this(deepClone ? new ModifyableColumnFamilyDescriptor(desc)
- : desc.delegatee);
- }
-
- protected HColumnDescriptor(ModifyableColumnFamilyDescriptor delegate) {
- this.delegatee = delegate;
- }
-
- /**
- * @return Name of this column family
- */
- @Override
- public byte [] getName() {
- return delegatee.getName();
- }
-
- /**
- * @return The name string of this column family
- */
- @Override
- public String getNameAsString() {
- return delegatee.getNameAsString();
- }
-
- /**
- * @param key The key.
- * @return The value.
- */
- @Override
- public byte[] getValue(byte[] key) {
- return delegatee.getValue(key);
- }
-
- /**
- * @param key The key.
- * @return The value as a string.
- */
- public String getValue(String key) {
- byte[] value = getValue(Bytes.toBytes(key));
- return value == null ? null : Bytes.toString(value);
- }
-
- @Override
- public Map<Bytes, Bytes> getValues() {
- return delegatee.getValues();
- }
-
- /**
- * @param key The key.
- * @param value The value.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setValue(byte[] key, byte[] value) {
- getDelegateeForModification().setValue(key, value);
- return this;
- }
-
- /**
- * @param key Key whose key and value we're to remove from HCD parameters.
- */
- public void remove(final byte [] key) {
- getDelegateeForModification().removeValue(new Bytes(key));
- }
-
- /**
- * @param key The key.
- * @param value The value.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setValue(String key, String value) {
- getDelegateeForModification().setValue(key, value);
- return this;
- }
-
- /**
- * @return compression type being used for the column family
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (<a href="https://issues.apache.org/jira/browse/HBASE-13655">HBASE-13655</a>).
- * Use {@link #getCompressionType()}.
- */
- @Deprecated
- public Compression.Algorithm getCompression() {
- return getCompressionType();
- }
-
- /**
- * @return compression type being used for the column family for major compaction
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (<a href="https://issues.apache.org/jira/browse/HBASE-13655">HBASE-13655</a>).
- * Use {@link #getCompactionCompressionType()}.
- */
- @Deprecated
- public Compression.Algorithm getCompactionCompression() {
- return getCompactionCompressionType();
- }
-
- @Override
- public int getMaxVersions() {
- return delegatee.getMaxVersions();
- }
-
- /**
- * @param value maximum number of versions
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setMaxVersions(int value) {
- getDelegateeForModification().setMaxVersions(value);
- return this;
- }
-
- /**
- * Set minimum and maximum versions to keep
- *
- * @param minVersions minimal number of versions
- * @param maxVersions maximum number of versions
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setVersions(int minVersions, int maxVersions) {
- if (minVersions <= 0) {
- // TODO: Allow minVersion and maxVersion of 0 to be the way you say "Keep all versions".
- // Until there is support, consider 0 or < 0 -- a configuration error.
- throw new IllegalArgumentException("Minimum versions must be positive");
- }
-
- if (maxVersions < minVersions) {
- throw new IllegalArgumentException("Unable to set MaxVersion to " + maxVersions
- + " and set MinVersion to " + minVersions
- + ", as maximum versions must be >= minimum versions.");
- }
- setMinVersions(minVersions);
- setMaxVersions(maxVersions);
- return this;
- }
-
- @Override
- public int getBlocksize() {
- return delegatee.getBlocksize();
- }
-
- /**
- * @param value Blocksize to use when writing out storefiles/hfiles on this
- * column family.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setBlocksize(int value) {
- getDelegateeForModification().setBlocksize(value);
- return this;
- }
-
- @Override
- public Compression.Algorithm getCompressionType() {
- return delegatee.getCompressionType();
- }
-
- /**
- * Compression types supported in hbase.
- * LZO is not bundled as part of the hbase distribution.
- * See <a href="http://hbase.apache.org/book.html#lzo.compression">LZO Compression</a>
- * for how to enable it.
- * @param value Compression type setting.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setCompressionType(Compression.Algorithm value) {
- getDelegateeForModification().setCompressionType(value);
- return this;
- }
-
- @Override
- public DataBlockEncoding getDataBlockEncoding() {
- return delegatee.getDataBlockEncoding();
- }
-
- /**
- * Set data block encoding algorithm used in block cache.
- * @param value What kind of data block encoding will be used.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setDataBlockEncoding(DataBlockEncoding value) {
- getDelegateeForModification().setDataBlockEncoding(value);
- return this;
- }
-
- /**
- * Set whether the tags should be compressed along with DataBlockEncoding. When no
- * DataBlockEncoding is been used, this is having no effect.
- *
- * @param value
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setCompressTags(boolean value) {
- getDelegateeForModification().setCompressTags(value);
- return this;
- }
-
- @Override
- public boolean isCompressTags() {
- return delegatee.isCompressTags();
- }
-
- @Override
- public Compression.Algorithm getCompactionCompressionType() {
- return delegatee.getCompactionCompressionType();
- }
-
- /**
- * Compression types supported in hbase.
- * LZO is not bundled as part of the hbase distribution.
- * See <a href="http://hbase.apache.org/book.html#lzo.compression">LZO Compression</a>
- * for how to enable it.
- * @param value Compression type setting.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setCompactionCompressionType(Compression.Algorithm value) {
- getDelegateeForModification().setCompactionCompressionType(value);
- return this;
- }
-
- @Override
- public boolean isInMemory() {
- return delegatee.isInMemory();
- }
-
- /**
- * @param value True if we are to favor keeping all values for this column family in the
- * HRegionServer cache
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setInMemory(boolean value) {
- getDelegateeForModification().setInMemory(value);
- return this;
- }
-
- @Override
- public MemoryCompactionPolicy getInMemoryCompaction() {
- return delegatee.getInMemoryCompaction();
- }
-
- /**
- * @param value the prefered in-memory compaction policy
- * for this column family
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setInMemoryCompaction(MemoryCompactionPolicy value) {
- getDelegateeForModification().setInMemoryCompaction(value);
- return this;
- }
-
- @Override
- public KeepDeletedCells getKeepDeletedCells() {
- return delegatee.getKeepDeletedCells();
- }
-
- /**
- * @param value True if deleted rows should not be collected
- * immediately.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setKeepDeletedCells(KeepDeletedCells value) {
- getDelegateeForModification().setKeepDeletedCells(value);
- return this;
- }
-
- /**
- * By default, HBase only consider timestamp in versions. So a previous Delete with higher ts
- * will mask a later Put with lower ts. Set this to true to enable new semantics of versions.
- * We will also consider mvcc in versions. See HBASE-15968 for details.
- */
- @Override
- public boolean isNewVersionBehavior() {
- return delegatee.isNewVersionBehavior();
- }
-
- public HColumnDescriptor setNewVersionBehavior(boolean newVersionBehavior) {
- getDelegateeForModification().setNewVersionBehavior(newVersionBehavior);
- return this;
- }
-
-
- @Override
- public int getTimeToLive() {
- return delegatee.getTimeToLive();
- }
-
- /**
- * @param value Time-to-live of cell contents, in seconds.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setTimeToLive(int value) {
- getDelegateeForModification().setTimeToLive(value);
- return this;
- }
-
- /**
- * @param value Time to live of cell contents, in human readable format
- * @see org.apache.hadoop.hbase.util.PrettyPrinter#format(String, Unit)
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setTimeToLive(String value) throws HBaseException {
- getDelegateeForModification().setTimeToLive(value);
- return this;
- }
-
- @Override
- public int getMinVersions() {
- return delegatee.getMinVersions();
- }
-
- /**
- * @param value The minimum number of versions to keep.
- * (used when timeToLive is set)
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setMinVersions(int value) {
- getDelegateeForModification().setMinVersions(value);
- return this;
- }
-
- @Override
- public boolean isBlockCacheEnabled() {
- return delegatee.isBlockCacheEnabled();
- }
-
- /**
- * @param value True if hfile DATA type blocks should be cached (We always cache
- * INDEX and BLOOM blocks; you cannot turn this off).
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setBlockCacheEnabled(boolean value) {
- getDelegateeForModification().setBlockCacheEnabled(value);
- return this;
- }
-
- @Override
- public BloomType getBloomFilterType() {
- return delegatee.getBloomFilterType();
- }
-
- /**
- * @param value bloom filter type
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setBloomFilterType(final BloomType value) {
- getDelegateeForModification().setBloomFilterType(value);
- return this;
- }
-
- @Override
- public int getScope() {
- return delegatee.getScope();
- }
-
- /**
- * @param value the scope tag
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setScope(int value) {
- getDelegateeForModification().setScope(value);
- return this;
- }
-
- @Override
- public boolean isCacheDataOnWrite() {
- return delegatee.isCacheDataOnWrite();
- }
-
- /**
- * @param value true if we should cache data blocks on write
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setCacheDataOnWrite(boolean value) {
- getDelegateeForModification().setCacheDataOnWrite(value);
- return this;
- }
-
- @Override
- public boolean isCacheIndexesOnWrite() {
- return delegatee.isCacheIndexesOnWrite();
- }
-
- /**
- * @param value true if we should cache index blocks on write
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setCacheIndexesOnWrite(boolean value) {
- getDelegateeForModification().setCacheIndexesOnWrite(value);
- return this;
- }
-
- @Override
- public boolean isCacheBloomsOnWrite() {
- return delegatee.isCacheBloomsOnWrite();
- }
-
- /**
- * @param value true if we should cache bloomfilter blocks on write
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setCacheBloomsOnWrite(boolean value) {
- getDelegateeForModification().setCacheBloomsOnWrite(value);
- return this;
- }
-
- @Override
- public boolean isEvictBlocksOnClose() {
- return delegatee.isEvictBlocksOnClose();
- }
-
- /**
- * @param value true if we should evict cached blocks from the blockcache on
- * close
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setEvictBlocksOnClose(boolean value) {
- getDelegateeForModification().setEvictBlocksOnClose(value);
- return this;
- }
-
- @Override
- public boolean isPrefetchBlocksOnOpen() {
- return delegatee.isPrefetchBlocksOnOpen();
- }
-
- /**
- * @param value true if we should prefetch blocks into the blockcache on open
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setPrefetchBlocksOnOpen(boolean value) {
- getDelegateeForModification().setPrefetchBlocksOnOpen(value);
- return this;
- }
-
- /**
- * @see java.lang.Object#toString()
- */
- @Override
- public String toString() {
- return delegatee.toString();
- }
-
- /**
- * @return Column family descriptor with only the customized attributes.
- */
- @Override
- public String toStringCustomizedValues() {
- return delegatee.toStringCustomizedValues();
- }
-
- public static Unit getUnit(String key) {
- return ColumnFamilyDescriptorBuilder.getUnit(key);
- }
-
- public static Map<String, String> getDefaultValues() {
- return ColumnFamilyDescriptorBuilder.getDefaultValues();
- }
-
- /**
- * @see java.lang.Object#equals(java.lang.Object)
- */
- @Override
- public boolean equals(Object obj) {
- if (this == obj) {
- return true;
- }
- if (obj instanceof HColumnDescriptor) {
- return delegatee.equals(((HColumnDescriptor) obj).delegatee);
- }
- return false;
- }
-
- /**
- * @see java.lang.Object#hashCode()
- */
- @Override
- public int hashCode() {
- return delegatee.hashCode();
- }
-
- @Override
- public int compareTo(HColumnDescriptor other) {
- return COMPARATOR.compare(this, other);
- }
-
- /**
- * @return This instance serialized with pb with pb magic prefix
- * @see #parseFrom(byte[])
- */
- public byte[] toByteArray() {
- return ColumnFamilyDescriptorBuilder.toByteArray(delegatee);
- }
-
- /**
- * @param bytes A pb serialized {@link HColumnDescriptor} instance with pb magic prefix
- * @return An instance of {@link HColumnDescriptor} made from <code>bytes</code>
- * @throws DeserializationException
- * @see #toByteArray()
- */
- public static HColumnDescriptor parseFrom(final byte [] bytes) throws DeserializationException {
- ColumnFamilyDescriptor desc = ColumnFamilyDescriptorBuilder.parseFrom(bytes);
- if (desc instanceof ModifyableColumnFamilyDescriptor) {
- return new HColumnDescriptor((ModifyableColumnFamilyDescriptor) desc);
- } else {
- return new HColumnDescriptor(new ModifyableColumnFamilyDescriptor(desc));
- }
- }
-
- @Override
- public String getConfigurationValue(String key) {
- return delegatee.getConfigurationValue(key);
- }
-
- @Override
- public Map<String, String> getConfiguration() {
- return delegatee.getConfiguration();
- }
-
- /**
- * Setter for storing a configuration setting.
- * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
- * @param value String value. If null, removes the configuration.
- */
- public HColumnDescriptor setConfiguration(String key, String value) {
- getDelegateeForModification().setConfiguration(key, value);
- return this;
- }
-
- /**
- * Remove a configuration setting represented by the key.
- */
- public void removeConfiguration(final String key) {
- getDelegateeForModification().removeConfiguration(key);
- }
-
- @Override
- public String getEncryptionType() {
- return delegatee.getEncryptionType();
- }
-
- /**
- * Set the encryption algorithm for use with this family
- * @param value
- */
- public HColumnDescriptor setEncryptionType(String value) {
- getDelegateeForModification().setEncryptionType(value);
- return this;
- }
-
- @Override
- public byte[] getEncryptionKey() {
- return delegatee.getEncryptionKey();
- }
-
- /** Set the raw crypto key attribute for the family */
- public HColumnDescriptor setEncryptionKey(byte[] value) {
- getDelegateeForModification().setEncryptionKey(value);
- return this;
- }
-
- @Override
- public long getMobThreshold() {
- return delegatee.getMobThreshold();
- }
-
- /**
- * Sets the mob threshold of the family.
- * @param value The mob threshold.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setMobThreshold(long value) {
- getDelegateeForModification().setMobThreshold(value);
- return this;
- }
-
- @Override
- public boolean isMobEnabled() {
- return delegatee.isMobEnabled();
- }
-
- /**
- * Enables the mob for the family.
- * @param value Whether to enable the mob for the family.
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setMobEnabled(boolean value) {
- getDelegateeForModification().setMobEnabled(value);
- return this;
- }
-
- @Override
- public MobCompactPartitionPolicy getMobCompactPartitionPolicy() {
- return delegatee.getMobCompactPartitionPolicy();
- }
-
- /**
- * Set the mob compact partition policy for the family.
- * @param value policy type
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setMobCompactPartitionPolicy(MobCompactPartitionPolicy value) {
- getDelegateeForModification().setMobCompactPartitionPolicy(value);
- return this;
- }
-
- @Override
- public short getDFSReplication() {
- return delegatee.getDFSReplication();
- }
-
- /**
- * Set the replication factor to hfile(s) belonging to this family
- * @param value number of replicas the blocks(s) belonging to this CF should have, or
- * {@link #DEFAULT_DFS_REPLICATION} for the default replication factor set in the
- * filesystem
- * @return this (for chained invocation)
- */
- public HColumnDescriptor setDFSReplication(short value) {
- getDelegateeForModification().setDFSReplication(value);
- return this;
- }
-
- @Override
- public String getStoragePolicy() {
- return delegatee.getStoragePolicy();
- }
-
- /**
- * Set the storage policy for use with this family
- * @param value the policy to set, valid setting includes: <i>"LAZY_PERSIST"</i>,
- * <i>"ALL_SSD"</i>, <i>"ONE_SSD"</i>, <i>"HOT"</i>, <i>"WARM"</i>, <i>"COLD"</i>
- */
- public HColumnDescriptor setStoragePolicy(String value) {
- getDelegateeForModification().setStoragePolicy(value);
- return this;
- }
-
- @Override
- public Bytes getValue(Bytes key) {
- return delegatee.getValue(key);
- }
-
- protected ModifyableColumnFamilyDescriptor getDelegateeForModification() {
- return delegatee;
- }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
deleted file mode 100644
index 02597ce..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ /dev/null
@@ -1,992 +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.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.CoprocessorDescriptor;
-import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * HTableDescriptor contains the details about an HBase table such as the descriptors of
- * all the column families, is the table a catalog table, <code> hbase:meta </code>,
- * if the table is read only, the maximum size of the memstore,
- * when the region split should occur, coprocessors associated with it etc...
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
- * Use {@link TableDescriptorBuilder} to build {@link HTableDescriptor}.
- */
-@Deprecated
-@InterfaceAudience.Public
-public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescriptor> {
- public static final String SPLIT_POLICY = TableDescriptorBuilder.SPLIT_POLICY;
- public static final String MAX_FILESIZE = TableDescriptorBuilder.MAX_FILESIZE;
- public static final String OWNER = TableDescriptorBuilder.OWNER;
- public static final Bytes OWNER_KEY = TableDescriptorBuilder.OWNER_KEY;
- public static final String READONLY = TableDescriptorBuilder.READONLY;
- public static final String COMPACTION_ENABLED = TableDescriptorBuilder.COMPACTION_ENABLED;
- public static final String SPLIT_ENABLED = TableDescriptorBuilder.SPLIT_ENABLED;
- public static final String MERGE_ENABLED = TableDescriptorBuilder.MERGE_ENABLED;
- public static final String MEMSTORE_FLUSHSIZE = TableDescriptorBuilder.MEMSTORE_FLUSHSIZE;
- public static final String FLUSH_POLICY = TableDescriptorBuilder.FLUSH_POLICY;
- public static final String IS_ROOT = "IS_ROOT";
- public static final String IS_META = TableDescriptorBuilder.IS_META;
- public static final String DURABILITY = TableDescriptorBuilder.DURABILITY;
- public static final String REGION_REPLICATION = TableDescriptorBuilder.REGION_REPLICATION;
- public static final String REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.REGION_MEMSTORE_REPLICATION;
- public static final String NORMALIZATION_ENABLED = TableDescriptorBuilder.NORMALIZATION_ENABLED;
- public static final String NORMALIZER_TARGET_REGION_COUNT =
- TableDescriptorBuilder.NORMALIZER_TARGET_REGION_COUNT;
- public static final String NORMALIZER_TARGET_REGION_SIZE =
- TableDescriptorBuilder.NORMALIZER_TARGET_REGION_SIZE;
- public static final String PRIORITY = TableDescriptorBuilder.PRIORITY;
- public static final boolean DEFAULT_READONLY = TableDescriptorBuilder.DEFAULT_READONLY;
- public static final boolean DEFAULT_COMPACTION_ENABLED = TableDescriptorBuilder.DEFAULT_COMPACTION_ENABLED;
- public static final boolean DEFAULT_NORMALIZATION_ENABLED = TableDescriptorBuilder.DEFAULT_NORMALIZATION_ENABLED;
- public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE;
- public static final int DEFAULT_REGION_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_REPLICATION;
- public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_MEMSTORE_REPLICATION;
- protected final ModifyableTableDescriptor delegatee;
-
- /**
- * Construct a table descriptor specifying a TableName object
- * @param name Table name.
- * @see <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581 HBASE: (HBASE-174) Un-openable tablename bug</a>
- */
- public HTableDescriptor(final TableName name) {
- this(new ModifyableTableDescriptor(name));
- }
-
- /**
- * Construct a table descriptor by cloning the descriptor passed as a parameter.
- * <p>
- * Makes a deep copy of the supplied descriptor.
- * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
- * @param desc The descriptor.
- */
- public HTableDescriptor(final HTableDescriptor desc) {
- this(desc, true);
- }
-
- protected HTableDescriptor(final HTableDescriptor desc, boolean deepClone) {
- this(deepClone ? new ModifyableTableDescriptor(desc.getTableName(), desc)
- : desc.delegatee);
- }
-
- public HTableDescriptor(final TableDescriptor desc) {
- this(new ModifyableTableDescriptor(desc.getTableName(), desc));
- }
-
- /**
- * Construct a table descriptor by cloning the descriptor passed as a parameter
- * but using a different table name.
- * <p>
- * Makes a deep copy of the supplied descriptor.
- * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
- * @param name Table name.
- * @param desc The descriptor.
- */
- public HTableDescriptor(final TableName name, final HTableDescriptor desc) {
- this(new ModifyableTableDescriptor(name, desc));
- }
-
- protected HTableDescriptor(ModifyableTableDescriptor delegatee) {
- this.delegatee = delegatee;
- }
-
- /**
- * This is vestigial API. It will be removed in 3.0.
- *
- * @return always return the false
- */
- public boolean isRootRegion() {
- return false;
- }
-
- /**
- * Checks if this table is <code> hbase:meta </code>
- * region.
- *
- * @return true if this table is <code> hbase:meta </code>
- * region
- */
- @Override
- public boolean isMetaRegion() {
- return delegatee.isMetaRegion();
- }
-
- /**
- * Checks if the table is a <code>hbase:meta</code> table
- *
- * @return true if table is <code> hbase:meta </code> region.
- */
- @Override
- public boolean isMetaTable() {
- return delegatee.isMetaTable();
- }
-
- /**
- * @return Getter for fetching an unmodifiable map.
- */
- @Override
- public Map<Bytes, Bytes> getValues() {
- return delegatee.getValues();
- }
-
- /**
- * Setter for storing metadata as a (key, value) pair in map
- *
- * @param key The key.
- * @param value The value. If null, removes the setting.
- */
- public HTableDescriptor setValue(byte[] key, byte[] value) {
- getDelegateeForModification().setValue(key, value);
- return this;
- }
-
- /*
- * Setter for storing metadata as a (key, value) pair in map
- *
- * @param key The key.
- * @param value The value. If null, removes the setting.
- */
- public HTableDescriptor setValue(final Bytes key, final Bytes value) {
- getDelegateeForModification().setValue(key, value);
- return this;
- }
-
- /**
- * Setter for storing metadata as a (key, value) pair in map
- *
- * @param key The key.
- * @param value The value. If null, removes the setting.
- */
- public HTableDescriptor setValue(String key, String value) {
- getDelegateeForModification().setValue(key, value);
- return this;
- }
-
- /**
- * Remove metadata represented by the key from the map
- *
- * @param key Key whose key and value we're to remove from HTableDescriptor
- * parameters.
- */
- public void remove(final String key) {
- getDelegateeForModification().removeValue(Bytes.toBytes(key));
- }
-
- /**
- * Remove metadata represented by the key from the map
- *
- * @param key Key whose key and value we're to remove from HTableDescriptor
- * parameters.
- */
- public void remove(Bytes key) {
- getDelegateeForModification().removeValue(key);
- }
-
- /**
- * Remove metadata represented by the key from the map
- *
- * @param key Key whose key and value we're to remove from HTableDescriptor
- * parameters.
- */
- public void remove(final byte [] key) {
- getDelegateeForModification().removeValue(key);
- }
-
- /**
- * Check if the readOnly flag of the table is set. If the readOnly flag is
- * set then the contents of the table can only be read from but not modified.
- *
- * @return true if all columns in the table should be read only
- */
- @Override
- public boolean isReadOnly() {
- return delegatee.isReadOnly();
- }
-
- /**
- * Setting the table as read only sets all the columns in the table as read
- * only. By default all tables are modifiable, but if the readOnly flag is
- * set to true then the contents of the table can only be read but not modified.
- *
- * @param readOnly True if all of the columns in the table should be read
- * only.
- */
- public HTableDescriptor setReadOnly(final boolean readOnly) {
- getDelegateeForModification().setReadOnly(readOnly);
- return this;
- }
-
- /**
- * Check if the compaction enable flag of the table is true. If flag is
- * false then no minor/major compactions will be done in real.
- *
- * @return true if table compaction enabled
- */
- @Override
- public boolean isCompactionEnabled() {
- return delegatee.isCompactionEnabled();
- }
-
- /**
- * Setting the table compaction enable flag.
- *
- * @param isEnable True if enable compaction.
- */
- public HTableDescriptor setCompactionEnabled(final boolean isEnable) {
- getDelegateeForModification().setCompactionEnabled(isEnable);
- return this;
- }
-
- /**
- * Check if the region split enable flag of the table is true. If flag is
- * false then no split will be done.
- *
- * @return true if table region split enabled
- */
- @Override
- public boolean isSplitEnabled() {
- return delegatee.isSplitEnabled();
- }
-
- /**
- * Setting the table region split enable flag.
- *
- * @param isEnable True if enable split.
- */
- public HTableDescriptor setSplitEnabled(final boolean isEnable) {
- getDelegateeForModification().setSplitEnabled(isEnable);
- return this;
- }
-
-
- /**
- * Check if the region merge enable flag of the table is true. If flag is
- * false then no merge will be done.
- *
- * @return true if table region merge enabled
- */
- @Override
- public boolean isMergeEnabled() {
- return delegatee.isMergeEnabled();
- }
-
- /**
- * Setting the table region merge enable flag.
- *
- * @param isEnable True if enable merge.
- */
- public HTableDescriptor setMergeEnabled(final boolean isEnable) {
- getDelegateeForModification().setMergeEnabled(isEnable);
- return this;
- }
-
- /**
- * Check if normalization enable flag of the table is true. If flag is
- * false then no region normalizer won't attempt to normalize this table.
- *
- * @return true if region normalization is enabled for this table
- */
- @Override
- public boolean isNormalizationEnabled() {
- return delegatee.isNormalizationEnabled();
- }
-
- /**
- * Setting the table normalization enable flag.
- *
- * @param isEnable True if enable normalization.
- */
- public HTableDescriptor setNormalizationEnabled(final boolean isEnable) {
- getDelegateeForModification().setNormalizationEnabled(isEnable);
- return this;
- }
-
- @Override
- public int getNormalizerTargetRegionCount() {
- return getDelegateeForModification().getNormalizerTargetRegionCount();
- }
-
- public HTableDescriptor setNormalizerTargetRegionCount(final int regionCount) {
- getDelegateeForModification().setNormalizerTargetRegionCount(regionCount);
- return this;
- }
-
- @Override
- public long getNormalizerTargetRegionSize() {
- return getDelegateeForModification().getNormalizerTargetRegionSize();
- }
-
- public HTableDescriptor setNormalizerTargetRegionSize(final long regionSize) {
- getDelegateeForModification().setNormalizerTargetRegionSize(regionSize);
- return this;
- }
-
- /**
- * Sets the {@link Durability} setting for the table. This defaults to Durability.USE_DEFAULT.
- * @param durability enum value
- */
- public HTableDescriptor setDurability(Durability durability) {
- getDelegateeForModification().setDurability(durability);
- return this;
- }
-
- /**
- * Returns the durability setting for the table.
- * @return durability setting for the table.
- */
- @Override
- public Durability getDurability() {
- return delegatee.getDurability();
- }
-
- /**
- * Get the name of the table
- *
- * @return TableName
- */
- @Override
- public TableName getTableName() {
- return delegatee.getTableName();
- }
-
- /**
- * Get the name of the table as a String
- *
- * @return name of table as a String
- */
- public String getNameAsString() {
- return delegatee.getTableName().getNameAsString();
- }
-
- /**
- * This sets the class associated with the region split policy which
- * determines when a region split should occur. The class used by
- * default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
- * @param clazz the class name
- */
- public HTableDescriptor setRegionSplitPolicyClassName(String clazz) {
- getDelegateeForModification().setRegionSplitPolicyClassName(clazz);
- return this;
- }
-
- /**
- * This gets the class associated with the region split policy which
- * determines when a region split should occur. The class used by
- * default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
- *
- * @return the class name of the region split policy for this table.
- * If this returns null, the default split policy is used.
- */
- @Override
- public String getRegionSplitPolicyClassName() {
- return delegatee.getRegionSplitPolicyClassName();
- }
-
- /**
- * Returns the maximum size upto which a region can grow to after which a region
- * split is triggered. The region size is represented by the size of the biggest
- * store file in that region.
- *
- * @return max hregion size for table, -1 if not set.
- *
- * @see #setMaxFileSize(long)
- */
- @Override
- public long getMaxFileSize() {
- return delegatee.getMaxFileSize();
- }
-
- /**
- * Sets the maximum size upto which a region can grow to after which a region
- * split is triggered. The region size is represented by the size of the biggest
- * store file in that region, i.e. If the biggest store file grows beyond the
- * maxFileSize, then the region split is triggered. This defaults to a value of
- * 256 MB.
- * <p>
- * This is not an absolute value and might vary. Assume that a single row exceeds
- * the maxFileSize then the storeFileSize will be greater than maxFileSize since
- * a single row cannot be split across multiple regions
- * </p>
- *
- * @param maxFileSize The maximum file size that a store file can grow to
- * before a split is triggered.
- */
- public HTableDescriptor setMaxFileSize(long maxFileSize) {
- getDelegateeForModification().setMaxFileSize(maxFileSize);
- return this;
- }
-
- /**
- * Returns the size of the memstore after which a flush to filesystem is triggered.
- *
- * @return memory cache flush size for each hregion, -1 if not set.
- *
- * @see #setMemStoreFlushSize(long)
- */
- @Override
- public long getMemStoreFlushSize() {
- return delegatee.getMemStoreFlushSize();
- }
-
- /**
- * Represents the maximum size of the memstore after which the contents of the
- * memstore are flushed to the filesystem. This defaults to a size of 64 MB.
- *
- * @param memstoreFlushSize memory cache flush size for each hregion
- */
- public HTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
- getDelegateeForModification().setMemStoreFlushSize(memstoreFlushSize);
- return this;
- }
-
- /**
- * This sets the class associated with the flush policy which determines determines the stores
- * need to be flushed when flushing a region. The class used by default is defined in
- * org.apache.hadoop.hbase.regionserver.FlushPolicy.
- * @param clazz the class name
- */
- public HTableDescriptor setFlushPolicyClassName(String clazz) {
- getDelegateeForModification().setFlushPolicyClassName(clazz);
- return this;
- }
-
- /**
- * This gets the class associated with the flush policy which determines the stores need to be
- * flushed when flushing a region. The class used by default is defined in
- * org.apache.hadoop.hbase.regionserver.FlushPolicy.
- * @return the class name of the flush policy for this table. If this returns null, the default
- * flush policy is used.
- */
- @Override
- public String getFlushPolicyClassName() {
- return delegatee.getFlushPolicyClassName();
- }
-
- /**
- * Adds a column family.
- * For the updating purpose please use {@link #modifyFamily(HColumnDescriptor)} instead.
- * @param family HColumnDescriptor of family to add.
- */
- public HTableDescriptor addFamily(final HColumnDescriptor family) {
- getDelegateeForModification().setColumnFamily(family);
- return this;
- }
-
- /**
- * Modifies the existing column family.
- * @param family HColumnDescriptor of family to update
- * @return this (for chained invocation)
- */
- public HTableDescriptor modifyFamily(final HColumnDescriptor family) {
- getDelegateeForModification().modifyColumnFamily(family);
- return this;
- }
-
- /**
- * Checks to see if this table contains the given column family
- * @param familyName Family name or column name.
- * @return true if the table contains the specified family name
- */
- public boolean hasFamily(final byte [] familyName) {
- return delegatee.hasColumnFamily(familyName);
- }
-
- /**
- * @return Name of this table and then a map of all of the column family
- * descriptors.
- * @see #getNameAsString()
- */
- @Override
- public String toString() {
- return delegatee.toString();
- }
-
- /**
- * @return Name of this table and then a map of all of the column family
- * descriptors (with only the non-default column family attributes)
- */
- @Override
- public String toStringCustomizedValues() {
- return delegatee.toStringCustomizedValues();
- }
-
- /**
- * @return map of all table attributes formatted into string.
- */
- public String toStringTableAttributes() {
- return delegatee.toStringTableAttributes();
- }
-
- /**
- * Compare the contents of the descriptor with another one passed as a parameter.
- * Checks if the obj passed is an instance of HTableDescriptor, if yes then the
- * contents of the descriptors are compared.
- *
- * @return true if the contents of the the two descriptors exactly match
- *
- * @see java.lang.Object#equals(java.lang.Object)
- */
- @Override
- public boolean equals(Object obj) {
- if (this == obj) {
- return true;
- }
- if (obj instanceof HTableDescriptor) {
- return delegatee.equals(((HTableDescriptor) obj).delegatee);
- }
- return false;
- }
-
- /**
- * @see java.lang.Object#hashCode()
- */
- @Override
- public int hashCode() {
- return delegatee.hashCode();
- }
-
- // Comparable
-
- /**
- * Compares the descriptor with another descriptor which is passed as a parameter.
- * This compares the content of the two descriptors and not the reference.
- *
- * @return 0 if the contents of the descriptors are exactly matching,
- * 1 if there is a mismatch in the contents
- */
- @Override
- public int compareTo(final HTableDescriptor other) {
- return TableDescriptor.COMPARATOR.compare(this, other);
- }
-
- /**
- * Returns an unmodifiable collection of all the {@link HColumnDescriptor}
- * of all the column families of the table.
- * @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getColumnFamilies()} instead.
- * @return Immutable collection of {@link HColumnDescriptor} of all the
- * column families.
- * @see #getColumnFamilies()
- * @see <a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>
- */
- @Deprecated
- public Collection<HColumnDescriptor> getFamilies() {
- return Stream.of(delegatee.getColumnFamilies())
- .map(this::toHColumnDescriptor)
- .collect(Collectors.toList());
- }
-
- /**
- * Returns the configured replicas per region
- */
- @Override
- public int getRegionReplication() {
- return delegatee.getRegionReplication();
- }
-
- /**
- * Sets the number of replicas per region.
- * @param regionReplication the replication factor per region
- */
- public HTableDescriptor setRegionReplication(int regionReplication) {
- getDelegateeForModification().setRegionReplication(regionReplication);
- return this;
- }
-
- /**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
- * Use {@link #hasRegionMemStoreReplication()} instead
- */
- @Deprecated
- public boolean hasRegionMemstoreReplication() {
- return hasRegionMemStoreReplication();
- }
-
- /**
- * @return true if the read-replicas memstore replication is enabled.
- */
- @Override
- public boolean hasRegionMemStoreReplication() {
- return delegatee.hasRegionMemStoreReplication();
- }
-
- /**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
- * Use {@link #setRegionMemStoreReplication(boolean)} instead
- */
- @Deprecated
- public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
- return setRegionMemStoreReplication(memstoreReplication);
- }
-
- /**
- * Enable or Disable the memstore replication from the primary region to the replicas.
- * The replication will be used only for meta operations (e.g. flush, compaction, ...)
- *
- * @param memstoreReplication true if the new data written to the primary region
- * should be replicated.
- * false if the secondaries can tollerate to have new
- * data only when the primary flushes the memstore.
- */
- public HTableDescriptor setRegionMemStoreReplication(boolean memstoreReplication) {
- getDelegateeForModification().setRegionMemStoreReplication(memstoreReplication);
- return this;
- }
-
- public HTableDescriptor setPriority(int priority) {
- getDelegateeForModification().setPriority(priority);
- return this;
- }
-
- @Override
- public int getPriority() {
- return delegatee.getPriority();
- }
-
- /**
- * Returns all the column family names of the current table. The map of
- * HTableDescriptor contains mapping of family name to HColumnDescriptors.
- * This returns all the keys of the family map which represents the column
- * family names of the table.
- *
- * @return Immutable sorted set of the keys of the families.
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (<a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>).
- * Use {@link #getColumnFamilyNames()}.
- */
- @Deprecated
- public Set<byte[]> getFamiliesKeys() {
- return delegatee.getColumnFamilyNames();
- }
-
- /**
- * Returns the count of the column families of the table.
- *
- * @return Count of column families of the table
- */
- @Override
- public int getColumnFamilyCount() {
- return delegatee.getColumnFamilyCount();
- }
-
- /**
- * Returns an array all the {@link HColumnDescriptor} of the column families
- * of the table.
- *
- * @return Array of all the HColumnDescriptors of the current table
- * @deprecated since 2.0.0 and will be removed in 3.0.0.
- * @see #getFamilies()
- * @see <a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>
- */
- @Deprecated
- @Override
- public HColumnDescriptor[] getColumnFamilies() {
- return Stream.of(delegatee.getColumnFamilies())
- .map(this::toHColumnDescriptor)
- .toArray(size -> new HColumnDescriptor[size]);
- }
-
- /**
- * Returns the HColumnDescriptor for a specific column family with name as
- * specified by the parameter column.
- * @param column Column family name
- * @return Column descriptor for the passed family name or the family on
- * passed in column.
- * @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getColumnFamily(byte[])}
- * instead.
- * @see #getColumnFamily(byte[])
- * @see <a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>
- */
- @Deprecated
- public HColumnDescriptor getFamily(final byte[] column) {
- return toHColumnDescriptor(delegatee.getColumnFamily(column));
- }
-
-
- /**
- * Removes the HColumnDescriptor with name specified by the parameter column
- * from the table descriptor
- *
- * @param column Name of the column family to be removed.
- * @return Column descriptor for the passed family name or the family on
- * passed in column.
- */
- public HColumnDescriptor removeFamily(final byte [] column) {
- return toHColumnDescriptor(getDelegateeForModification().removeColumnFamily(column));
- }
-
- /**
- * Return a HColumnDescriptor for user to keep the compatibility as much as possible.
- * @param desc read-only ColumnFamilyDescriptor
- * @return The older implementation of ColumnFamilyDescriptor
- */
- protected HColumnDescriptor toHColumnDescriptor(ColumnFamilyDescriptor desc) {
- if (desc == null) {
- return null;
- } else if (desc instanceof ModifyableColumnFamilyDescriptor) {
- return new HColumnDescriptor((ModifyableColumnFamilyDescriptor) desc);
- } else if (desc instanceof HColumnDescriptor) {
- return (HColumnDescriptor) desc;
- } else {
- return new HColumnDescriptor(new ModifyableColumnFamilyDescriptor(desc));
- }
- }
-
- /**
- * Add a table coprocessor to this table. The coprocessor
- * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
- * It won't check if the class can be loaded or not.
- * Whether a coprocessor is loadable or not will be determined when
- * a region is opened.
- * @param className Full class name.
- * @throws IOException
- */
- public HTableDescriptor addCoprocessor(String className) throws IOException {
- getDelegateeForModification().setCoprocessor(className);
- return this;
- }
-
- /**
- * Add a table coprocessor to this table. The coprocessor
- * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
- * It won't check if the class can be loaded or not.
- * Whether a coprocessor is loadable or not will be determined when
- * a region is opened.
- * @param jarFilePath Path of the jar file. If it's null, the class will be
- * loaded from default classloader.
- * @param className Full class name.
- * @param priority Priority
- * @param kvs Arbitrary key-value parameter pairs passed into the coprocessor.
- * @throws IOException
- */
- public HTableDescriptor addCoprocessor(String className, Path jarFilePath,
- int priority, final Map<String, String> kvs)
- throws IOException {
- getDelegateeForModification().setCoprocessor(
- CoprocessorDescriptorBuilder.newBuilder(className)
- .setJarPath(jarFilePath == null ? null : jarFilePath.toString())
- .setPriority(priority)
- .setProperties(kvs == null ? Collections.emptyMap() : kvs)
- .build());
- return this;
- }
-
- /**
- * Add a table coprocessor to this table. The coprocessor
- * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
- * It won't check if the class can be loaded or not.
- * Whether a coprocessor is loadable or not will be determined when
- * a region is opened.
- * @param specStr The Coprocessor specification all in in one String formatted so matches
- * {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
- * @throws IOException
- */
- public HTableDescriptor addCoprocessorWithSpec(final String specStr) throws IOException {
- getDelegateeForModification().setCoprocessorWithSpec(specStr);
- return this;
- }
-
- /**
- * Check if the table has an attached co-processor represented by the name className
- *
- * @param classNameToMatch - Class name of the co-processor
- * @return true of the table has a co-processor className
- */
- @Override
- public boolean hasCoprocessor(String classNameToMatch) {
- return delegatee.hasCoprocessor(classNameToMatch);
- }
-
- @Override
- public Collection<CoprocessorDescriptor> getCoprocessorDescriptors() {
- return delegatee.getCoprocessorDescriptors();
- }
-
- /**
- * Return the list of attached co-processor represented by their name className
- *
- * @return The list of co-processors classNames
- */
- public List<String> getCoprocessors() {
- return getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
- .collect(Collectors.toList());
- }
-
- /**
- * Remove a coprocessor from those set on the table
- * @param className Class name of the co-processor
- */
- public void removeCoprocessor(String className) {
- getDelegateeForModification().removeCoprocessor(className);
- }
-
- public final static String NAMESPACE_FAMILY_INFO = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO;
- public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES;
- public final static byte[] NAMESPACE_COL_DESC_BYTES = TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES;
-
- /** Table descriptor for namespace table */
- public static final HTableDescriptor NAMESPACE_TABLEDESC
- = new HTableDescriptor(TableDescriptorBuilder.NAMESPACE_TABLEDESC);
-
- /**
- * @deprecated since 0.94.1
- * @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
- */
- @Deprecated
- public HTableDescriptor setOwner(User owner) {
- getDelegateeForModification().setOwner(owner);
- return this;
- }
-
- /**
- * @deprecated since 0.94.1
- * @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
- */
- // used by admin.rb:alter(table_name,*args) to update owner.
- @Deprecated
- public HTableDescriptor setOwnerString(String ownerString) {
- getDelegateeForModification().setOwnerString(ownerString);
- return this;
- }
-
- /**
- * @deprecated since 0.94.1
- * @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
- */
- @Override
- @Deprecated
- public String getOwnerString() {
- return delegatee.getOwnerString();
- }
-
- /**
- * @return This instance serialized with pb with pb magic prefix
- * @see #parseFrom(byte[])
- */
- public byte[] toByteArray() {
- return TableDescriptorBuilder.toByteArray(delegatee);
- }
-
- /**
- * @param bytes A pb serialized {@link HTableDescriptor} instance with pb magic prefix
- * @return An instance of {@link HTableDescriptor} made from <code>bytes</code>
- * @throws DeserializationException
- * @throws IOException
- * @see #toByteArray()
- */
- public static HTableDescriptor parseFrom(final byte [] bytes)
- throws DeserializationException, IOException {
- TableDescriptor desc = TableDescriptorBuilder.parseFrom(bytes);
- if (desc instanceof ModifyableTableDescriptor) {
- return new HTableDescriptor((ModifyableTableDescriptor) desc);
- } else {
- return new HTableDescriptor(desc);
- }
- }
-
- /**
- * Getter for accessing the configuration value by key
- */
- public String getConfigurationValue(String key) {
- return delegatee.getValue(key);
- }
-
- /**
- * Getter for fetching an unmodifiable map.
- */
- public Map<String, String> getConfiguration() {
- return delegatee.getConfiguration();
- }
-
- /**
- * Setter for storing a configuration setting in map.
- * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
- * @param value String value. If null, removes the setting.
- */
- public HTableDescriptor setConfiguration(String key, String value) {
- getDelegateeForModification().setValue(key, value);
- return this;
- }
-
- /**
- * Remove a config setting represented by the key from the map
- */
- public void removeConfiguration(final String key) {
- getDelegateeForModification().removeValue(Bytes.toBytes(key));
- }
-
- @Override
- public Bytes getValue(Bytes key) {
- return delegatee.getValue(key);
- }
-
- @Override
- public String getValue(String key) {
- return delegatee.getValue(key);
- }
-
- @Override
- public byte[] getValue(byte[] key) {
- return delegatee.getValue(key);
- }
-
- @Override
- public Set<byte[]> getColumnFamilyNames() {
- return delegatee.getColumnFamilyNames();
- }
-
- @Override
- public boolean hasColumnFamily(byte[] name) {
- return delegatee.hasColumnFamily(name);
- }
-
- @Override
- public ColumnFamilyDescriptor getColumnFamily(byte[] name) {
- return delegatee.getColumnFamily(name);
- }
-
- protected ModifyableTableDescriptor getDelegateeForModification() {
- return delegatee;
- }
-
- @Override
- public Optional<String> getRegionServerGroup() {
- return delegatee.getRegionServerGroup();
- }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
index 87579bf..3889d32 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
@@ -587,10 +587,8 @@ public class ColumnFamilyDescriptorBuilder {
* number of versions, compression settings, etc.
*
* It is used as input when creating a table or adding a column.
- * TODO: make this package-private after removing the HColumnDescriptor
*/
- @InterfaceAudience.Private
- public static class ModifyableColumnFamilyDescriptor
+ private static final class ModifyableColumnFamilyDescriptor
implements ColumnFamilyDescriptor, Comparable<ModifyableColumnFamilyDescriptor> {
// Column family name
@@ -692,15 +690,6 @@ public class ColumnFamilyDescriptorBuilder {
return this;
}
- /**
- *
- * @param key Key whose key and value we're to remove from HCD parameters.
- * @return this (for chained invocation)
- */
- public ModifyableColumnFamilyDescriptor removeValue(final Bytes key) {
- return setValue(key, (Bytes) null);
- }
-
private static <T> Bytes toBytesOrNull(T t, Function<T, byte[]> f) {
if (t == null) {
return null;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index bf591a1..f5ec540 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -576,12 +576,8 @@ public class TableDescriptorBuilder {
return new ModifyableTableDescriptor(desc);
}
- /**
- * TODO: make this private after removing the HTableDescriptor
- */
- @InterfaceAudience.Private
- public static class ModifyableTableDescriptor
- implements TableDescriptor, Comparable<ModifyableTableDescriptor> {
+ private static final class ModifyableTableDescriptor
+ implements TableDescriptor, Comparable<ModifyableTableDescriptor> {
private final TableName name;
@@ -602,11 +598,9 @@ public class TableDescriptorBuilder {
* Construct a table descriptor specifying a TableName object
*
* @param name Table name.
- * TODO: make this private after removing the HTableDescriptor
*/
- @InterfaceAudience.Private
- public ModifyableTableDescriptor(final TableName name) {
- this(name, Collections.EMPTY_LIST, Collections.EMPTY_MAP);
+ private ModifyableTableDescriptor(final TableName name) {
+ this(name, Collections.emptyList(), Collections.emptyMap());
}
private ModifyableTableDescriptor(final TableDescriptor desc) {
@@ -620,11 +614,8 @@ public class TableDescriptorBuilder {
* Makes a deep copy of the supplied descriptor.
* @param name The new name
* @param desc The descriptor.
- * TODO: make this private after removing the HTableDescriptor
*/
- @InterfaceAudience.Private
- @Deprecated // only used by HTableDescriptor. remove this method if HTD is removed
- public ModifyableTableDescriptor(final TableName name, final TableDescriptor desc) {
+ private ModifyableTableDescriptor(final TableName name, final TableDescriptor desc) {
this(name, Arrays.asList(desc.getColumnFamilies()), desc.getValues());
}
@@ -697,19 +688,6 @@ public class TableDescriptorBuilder {
}
/**
- * Getter for fetching an unmodifiable map.
- */
- public Map<String, String> getConfiguration() {
- return getValues().entrySet().stream()
- .collect(Collectors.toMap(
- e -> Bytes.toString(e.getKey().get(), e.getKey().getOffset(),
- e.getKey().getLength()),
- e -> Bytes.toString(e.getValue().get(), e.getValue().getOffset(),
- e.getValue().getLength())
- ));
- }
-
- /**
* Setter for storing metadata as a (key, value) pair in {@link #values} map
*
* @param key The key.
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
deleted file mode 100644
index 8149ba0..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java
+++ /dev/null
@@ -1,148 +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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-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.apache.hadoop.hbase.util.PrettyPrinter;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
-
-/**
- * Tests the HColumnDescriptor with appropriate arguments.
- *
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 together with
- * {@link HColumnDescriptor}.
- */
-@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
- public void testPb() throws DeserializationException {
- HColumnDescriptor hcd = new HColumnDescriptor(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(HConstants.CATALOG_FAMILY)
- .setInMemory(true)
- .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
- .setBloomFilterType(BloomType.NONE));
- final int v = 123;
- hcd.setBlocksize(v);
- hcd.setTimeToLive(v);
- hcd.setBlockCacheEnabled(!HColumnDescriptor.DEFAULT_BLOCKCACHE);
- hcd.setValue("a", "b");
- hcd.setMaxVersions(v);
- assertEquals(v, hcd.getMaxVersions());
- hcd.setMinVersions(v);
- assertEquals(v, hcd.getMinVersions());
- hcd.setKeepDeletedCells(KeepDeletedCells.TRUE);
- hcd.setInMemory(!HColumnDescriptor.DEFAULT_IN_MEMORY);
- boolean inmemory = hcd.isInMemory();
- hcd.setScope(v);
- hcd.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
- hcd.setBloomFilterType(BloomType.ROW);
- hcd.setCompressionType(Algorithm.SNAPPY);
- hcd.setMobEnabled(true);
- hcd.setMobThreshold(1000L);
- hcd.setDFSReplication((short) v);
-
- byte [] bytes = hcd.toByteArray();
- HColumnDescriptor deserializedHcd = HColumnDescriptor.parseFrom(bytes);
- assertTrue(hcd.equals(deserializedHcd));
- assertEquals(v, hcd.getBlocksize());
- assertEquals(v, hcd.getTimeToLive());
- assertEquals(v, hcd.getScope());
- assertEquals(hcd.getValue("a"), deserializedHcd.getValue("a"));
- assertEquals(hcd.getMaxVersions(), deserializedHcd.getMaxVersions());
- assertEquals(hcd.getMinVersions(), deserializedHcd.getMinVersions());
- assertEquals(hcd.getKeepDeletedCells(), deserializedHcd.getKeepDeletedCells());
- assertEquals(inmemory, deserializedHcd.isInMemory());
- assertEquals(hcd.getScope(), deserializedHcd.getScope());
- assertTrue(deserializedHcd.getCompressionType().equals(Compression.Algorithm.SNAPPY));
- assertTrue(deserializedHcd.getDataBlockEncoding().equals(DataBlockEncoding.FAST_DIFF));
- assertTrue(deserializedHcd.getBloomFilterType().equals(BloomType.ROW));
- assertEquals(hcd.isMobEnabled(), deserializedHcd.isMobEnabled());
- assertEquals(hcd.getMobThreshold(), deserializedHcd.getMobThreshold());
- assertEquals(v, deserializedHcd.getDFSReplication());
- }
-
- /**
- * Tests HColumnDescriptor with empty familyName
- */
- @Test
- public void testHColumnDescriptorShouldThrowIAEWhenFamilyNameEmpty() throws Exception {
- expectedEx.expect(IllegalArgumentException.class);
- expectedEx.expectMessage("Column Family name can not be empty");
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes(""));
- }
-
- @Test
- public void testMobValuesInHColumnDescriptorShouldReadable() {
- boolean isMob = true;
- long threshold = 1000;
- String policy = "weekly";
- // We unify the format of all values saved in the descriptor.
- // Each value is stored as bytes of string.
- String isMobString = PrettyPrinter.format(String.valueOf(isMob),
- HColumnDescriptor.getUnit(HColumnDescriptor.IS_MOB));
- String thresholdString = PrettyPrinter.format(String.valueOf(threshold),
- HColumnDescriptor.getUnit(HColumnDescriptor.MOB_THRESHOLD));
- String policyString = PrettyPrinter.format(Bytes.toStringBinary(Bytes.toBytes(policy)),
- HColumnDescriptor.getUnit(HColumnDescriptor.MOB_COMPACT_PARTITION_POLICY));
- assertEquals(String.valueOf(isMob), isMobString);
- assertEquals(String.valueOf(threshold), thresholdString);
- assertEquals(String.valueOf(policy), policyString);
- }
-
- @Test
- public void testClassMethodsAreBuilderStyle() {
- /* HColumnDescriptor should have a builder style setup where setXXX/addXXX methods
- * can be chainable together:
- * . For example:
- * HColumnDescriptor hcd
- * = new HColumnDescriptor()
- * .setFoo(foo)
- * .setBar(bar)
- * .setBuz(buz)
- *
- * This test ensures that all methods starting with "set" returns the declaring object
- */
-
- BuilderStyleTest.assertClassesAreBuilderStyle(HColumnDescriptor.class);
- }
-
-}
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
deleted file mode 100644
index 4e05a82..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
+++ /dev/null
@@ -1,369 +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 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.Arrays;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-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;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test setting values in the descriptor
- *
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 together with
- * {@link HTableDescriptor}.
- */
-@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
- public TestName name = new TestName();
-
- @Test (expected=IOException.class)
- public void testAddCoprocessorTwice() throws IOException {
- HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
- String cpName = "a.b.c.d";
- htd.addCoprocessor(cpName);
- htd.addCoprocessor(cpName);
- }
-
- @Test
- public void testAddCoprocessorWithSpecStr() throws IOException {
- HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
- String cpName = "a.b.c.d";
- try {
- htd.addCoprocessorWithSpec(cpName);
- fail();
- } catch (IllegalArgumentException iae) {
- // Expected as cpName is invalid
- }
-
- // Try minimal spec.
- try {
- htd.addCoprocessorWithSpec("file:///some/path" + "|" + cpName);
- fail();
- } catch (IllegalArgumentException iae) {
- // Expected to be invalid
- }
-
- // Try more spec.
- String spec = "hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2";
- try {
- htd.addCoprocessorWithSpec(spec);
- } catch (IllegalArgumentException iae) {
- fail();
- }
-
- // Try double add of same coprocessor
- try {
- htd.addCoprocessorWithSpec(spec);
- fail();
- } catch (IOException ioe) {
- // Expect that the coprocessor already exists
- }
- }
-
- @Test
- public void testPb() throws DeserializationException, IOException {
- HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
- final int v = 123;
- htd.setMaxFileSize(v);
- htd.setDurability(Durability.ASYNC_WAL);
- htd.setReadOnly(true);
- htd.setRegionReplication(2);
- byte [] bytes = htd.toByteArray();
- HTableDescriptor deserializedHtd = HTableDescriptor.parseFrom(bytes);
- assertEquals(htd, deserializedHtd);
- assertEquals(v, deserializedHtd.getMaxFileSize());
- assertTrue(deserializedHtd.isReadOnly());
- assertEquals(Durability.ASYNC_WAL, deserializedHtd.getDurability());
- assertEquals(2, deserializedHtd.getRegionReplication());
- }
-
- /**
- * Test cps in the table description.
- *
- * @throws Exception if adding a coprocessor fails
- */
- @Test
- public void testGetSetRemoveCP() throws Exception {
- HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
- // simple CP
- String className = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
- // add and check that it is present
- desc.addCoprocessor(className);
- assertTrue(desc.hasCoprocessor(className));
- // remove it and check that it is gone
- desc.removeCoprocessor(className);
- assertFalse(desc.hasCoprocessor(className));
- }
-
- /**
- * Test cps in the table description.
- *
- * @throws Exception if adding a coprocessor fails
- */
- @Test
- public void testSetListRemoveCP() throws Exception {
- HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
- // simple CP
- String className1 = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
- String className2 = "org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver";
- // Check that any coprocessor is present.
- assertTrue(desc.getCoprocessors().isEmpty());
-
- // Add the 1 coprocessor and check if present.
- desc.addCoprocessor(className1);
- assertTrue(desc.getCoprocessors().size() == 1);
- assertTrue(desc.getCoprocessors().contains(className1));
-
- // Add the 2nd coprocessor and check if present.
- // remove it and check that it is gone
- desc.addCoprocessor(className2);
- assertTrue(desc.getCoprocessors().size() == 2);
- assertTrue(desc.getCoprocessors().contains(className2));
-
- // Remove one and check
- desc.removeCoprocessor(className1);
- assertTrue(desc.getCoprocessors().size() == 1);
- assertFalse(desc.getCoprocessors().contains(className1));
- assertTrue(desc.getCoprocessors().contains(className2));
-
- // Remove the last and check
- desc.removeCoprocessor(className2);
- assertTrue(desc.getCoprocessors().isEmpty());
- assertFalse(desc.getCoprocessors().contains(className1));
- assertFalse(desc.getCoprocessors().contains(className2));
- }
-
- /**
- * Test that we add and remove strings from settings properly.
- */
- @Test
- public void testAddGetRemoveString() {
- HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
- String key = "Some";
- String value = "value";
- desc.setValue(key, value);
- assertEquals(value, desc.getValue(key));
- desc.remove(key);
- assertEquals(null, desc.getValue(key));
- String keyShouldNotNull = "Some2";
- String valueIsNull = null;
- desc.setValue(keyShouldNotNull, valueIsNull);
- assertEquals(valueIsNull, desc.getValue(keyShouldNotNull));
- desc.remove(keyShouldNotNull);
- assertEquals(null, desc.getValue(keyShouldNotNull));
- }
-
- String[] legalTableNames = { "foo", "with-dash_under.dot", "_under_start_ok",
- "with-dash.with_underscore", "02-01-2012.my_table_01-02", "xyz._mytable_", "9_9_0.table_02",
- "dot1.dot2.table", "new.-mytable", "with-dash.with.dot", "legal..t2", "legal..legal.t2",
- "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02",
- "汉", "汉:å—", "_å—_", "foo:å—", "foo.å—", "å—.foo"};
- // Avoiding "zookeeper" in here as it's tough to encode in regex
- String[] illegalTableNames = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok",
- "-dash-.start_illegal", "new.table with space", "01 .table", "ns:-illegaldash",
- "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2", String.valueOf((char)130),
- String.valueOf((char)5), String.valueOf((char)65530)};
-
- @Test
- public void testLegalHTableNames() {
- for (String tn : legalTableNames) {
- TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
- }
- }
-
- @Test
- public void testIllegalHTableNames() {
- for (String tn : illegalTableNames) {
- try {
- TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
- fail("invalid tablename " + tn + " should have failed");
- } catch (Exception e) {
- // expected
- }
- }
- }
-
- @Test
- public void testIllegalZooKeeperName() {
- for (String name : Arrays.asList("zookeeper", "ns:zookeeper", "zookeeper:table")) {
- try {
- TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(name));
- fail("invalid tablename " + name + " should have failed");
- } catch (Exception e) {
- // expected
- }
- }
- }
-
- @Test
- public void testLegalHTableNamesRegex() {
- for (String tn : legalTableNames) {
- TableName tName = TableName.valueOf(tn);
- assertTrue("Testing: '" + tn + "'", Pattern.matches(TableName.VALID_USER_TABLE_REGEX,
- tName.getNameAsString()));
- }
- }
-
- @Test
- public void testIllegalHTableNamesRegex() {
- for (String tn : illegalTableNames) {
- LOG.info("Testing: '" + tn + "'");
- assertFalse(Pattern.matches(TableName.VALID_USER_TABLE_REGEX, tn));
- }
- }
-
- /**
- * Test default value handling for maxFileSize
- */
- @Test
- public void testGetMaxFileSize() {
- HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
- assertEquals(-1, desc.getMaxFileSize());
- desc.setMaxFileSize(1111L);
- assertEquals(1111L, desc.getMaxFileSize());
- }
-
- /**
- * Test default value handling for memStoreFlushSize
- */
- @Test
- public void testGetMemStoreFlushSize() {
- HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
- assertEquals(-1, desc.getMemStoreFlushSize());
- desc.setMemStoreFlushSize(1111L);
- assertEquals(1111L, desc.getMemStoreFlushSize());
- }
-
- /**
- * Test that we add and remove strings from configuration properly.
- */
- @Test
- public void testAddGetRemoveConfiguration() throws Exception {
- HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
- String key = "Some";
- String value = "value";
- desc.setConfiguration(key, value);
- assertEquals(value, desc.getConfigurationValue(key));
- desc.removeConfiguration(key);
- assertEquals(null, desc.getConfigurationValue(key));
- }
-
- @Test
- public void testClassMethodsAreBuilderStyle() {
- /* HTableDescriptor should have a builder style setup where setXXX/addXXX methods
- * can be chainable together:
- * . For example:
- * HTableDescriptor htd
- * = new HTableDescriptor()
- * .setFoo(foo)
- * .setBar(bar)
- * .setBuz(buz)
- *
- * This test ensures that all methods starting with "set" returns the declaring object
- */
-
- BuilderStyleTest.assertClassesAreBuilderStyle(HTableDescriptor.class);
- }
-
- @Test
- public void testModifyFamily() {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
- byte[] familyName = Bytes.toBytes("cf");
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
- .setBlocksize(1000)
- .setDFSReplication((short) 3);
- tableDescriptor.setColumnFamily(familyDescriptor);
- assertEquals(1000, tableDescriptor.getColumnFamily(familyName).getBlocksize());
- assertEquals(3, tableDescriptor.getColumnFamily(familyName).getDFSReplication());
- familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
- .setBlocksize(2000)
- .setDFSReplication((short) 1);
- tableDescriptor.modifyColumnFamily(familyDescriptor);
- assertEquals(2000, tableDescriptor.getColumnFamily(familyName).getBlocksize());
- assertEquals(1, tableDescriptor.getColumnFamily(familyName).getDFSReplication());
- }
-
- @Test(expected=IllegalArgumentException.class)
- public void testModifyInexistentFamily() {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
- byte[] familyName = Bytes.toBytes("cf");
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName);
- tableDescriptor.modifyColumnFamily(familyDescriptor);
- }
-
- @Test(expected=IllegalArgumentException.class)
- public void testAddDuplicateFamilies() {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
- byte[] familyName = Bytes.toBytes("cf");
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
- .setBlocksize(1000);
- tableDescriptor.setColumnFamily(familyDescriptor);
- assertEquals(1000, tableDescriptor.getColumnFamily(familyName).getBlocksize());
- familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
- .setBlocksize(2000);
- tableDescriptor.setColumnFamily(familyDescriptor);
- }
-
- @Test
- public void testPriority() {
- HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
- htd.setPriority(42);
- assertEquals(42, htd.getPriority());
- }
-}
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 82479ed..d6ea1b3 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
@@ -18,10 +18,10 @@
package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
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;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -37,38 +37,31 @@ 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;
-import org.junit.rules.ExpectedException;
-@Category({MiscTests.class, SmallTests.class})
+@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();
+ HBaseClassTestRule.forClass(TestColumnFamilyDescriptorBuilder.class);
@Test
public void testBuilder() throws DeserializationException {
- ColumnFamilyDescriptorBuilder builder
- = ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY)
- .setInMemory(true)
- .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
- .setBloomFilterType(BloomType.NONE);
+ ColumnFamilyDescriptorBuilder builder =
+ ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY).setInMemory(true)
+ .setScope(HConstants.REPLICATION_SCOPE_LOCAL).setBloomFilterType(BloomType.NONE);
final int v = 123;
builder.setBlocksize(v);
builder.setTimeToLive(v);
- builder.setBlockCacheEnabled(!HColumnDescriptor.DEFAULT_BLOCKCACHE);
+ builder.setBlockCacheEnabled(!ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKCACHE);
builder.setValue(Bytes.toBytes("a"), Bytes.toBytes("b"));
builder.setMaxVersions(v);
assertEquals(v, builder.build().getMaxVersions());
builder.setMinVersions(v);
assertEquals(v, builder.build().getMinVersions());
builder.setKeepDeletedCells(KeepDeletedCells.TRUE);
- builder.setInMemory(!HColumnDescriptor.DEFAULT_IN_MEMORY);
+ builder.setInMemory(!ColumnFamilyDescriptorBuilder.DEFAULT_IN_MEMORY);
boolean inmemory = builder.build().isInMemory();
builder.setScope(v);
builder.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
@@ -79,13 +72,13 @@ public class TestColumnFamilyDescriptorBuilder {
builder.setDFSReplication((short) v);
ColumnFamilyDescriptor hcd = builder.build();
- byte [] bytes = ColumnFamilyDescriptorBuilder.toByteArray(hcd);
+ byte[] bytes = ColumnFamilyDescriptorBuilder.toByteArray(hcd);
ColumnFamilyDescriptor deserializedHcd = ColumnFamilyDescriptorBuilder.parseFrom(bytes);
assertTrue(hcd.equals(deserializedHcd));
assertEquals(v, hcd.getBlocksize());
assertEquals(v, hcd.getTimeToLive());
- assertTrue(Bytes.equals(hcd.getValue(Bytes.toBytes("a")),
- deserializedHcd.getValue(Bytes.toBytes("a"))));
+ assertTrue(
+ Bytes.equals(hcd.getValue(Bytes.toBytes("a")), deserializedHcd.getValue(Bytes.toBytes("a"))));
assertEquals(hcd.getMaxVersions(), deserializedHcd.getMaxVersions());
assertEquals(hcd.getMinVersions(), deserializedHcd.getMinVersions());
assertEquals(hcd.getKeepDeletedCells(), deserializedHcd.getKeepDeletedCells());
@@ -104,9 +97,8 @@ public class TestColumnFamilyDescriptorBuilder {
*/
@Test
public void testHColumnDescriptorShouldThrowIAEWhenFamilyNameEmpty() throws Exception {
- expectedEx.expect(IllegalArgumentException.class);
- expectedEx.expectMessage("Column Family name can not be empty");
- ColumnFamilyDescriptorBuilder.of("");
+ assertThrows("Column Family name can not be empty", IllegalArgumentException.class,
+ () -> ColumnFamilyDescriptorBuilder.of(""));
}
/**
@@ -114,8 +106,8 @@ public class TestColumnFamilyDescriptorBuilder {
*/
@Test
public void testAddGetRemoveConfiguration() {
- ColumnFamilyDescriptorBuilder builder
- = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo"));
+ ColumnFamilyDescriptorBuilder builder =
+ ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo"));
String key = "Some";
String value = "value";
builder.setConfiguration(key, value);
@@ -132,11 +124,12 @@ public class TestColumnFamilyDescriptorBuilder {
// We unify the format of all values saved in the descriptor.
// Each value is stored as bytes of string.
String isMobString = PrettyPrinter.format(String.valueOf(isMob),
- HColumnDescriptor.getUnit(HColumnDescriptor.IS_MOB));
+ ColumnFamilyDescriptorBuilder.getUnit(ColumnFamilyDescriptorBuilder.IS_MOB));
String thresholdString = PrettyPrinter.format(String.valueOf(threshold),
- HColumnDescriptor.getUnit(HColumnDescriptor.MOB_THRESHOLD));
+ ColumnFamilyDescriptorBuilder.getUnit(ColumnFamilyDescriptorBuilder.MOB_THRESHOLD));
String policyString = PrettyPrinter.format(Bytes.toStringBinary(Bytes.toBytes(policy)),
- HColumnDescriptor.getUnit(HColumnDescriptor.MOB_COMPACT_PARTITION_POLICY));
+ ColumnFamilyDescriptorBuilder
+ .getUnit(ColumnFamilyDescriptorBuilder.MOB_COMPACT_PARTITION_POLICY));
assertEquals(String.valueOf(isMob), isMobString);
assertEquals(String.valueOf(threshold), thresholdString);
assertEquals(String.valueOf(policy), policyString);
@@ -144,16 +137,11 @@ public class TestColumnFamilyDescriptorBuilder {
@Test
public void testClassMethodsAreBuilderStyle() {
- /* HColumnDescriptor should have a builder style setup where setXXX/addXXX methods
- * can be chainable together:
- * . For example:
- * HColumnDescriptor hcd
- * = new HColumnDescriptor()
- * .setFoo(foo)
- * .setBar(bar)
- * .setBuz(buz)
- *
- * This test ensures that all methods starting with "set" returns the declaring object
+ /*
+ * ColumnFamilyDescriptorBuilder should have a builder style setup where setXXX/addXXX methods
+ * can be chainable together: . For example: ColumnFamilyDescriptorBuilder builder =
+ * ColumnFamilyDescriptorBuilder.newBuilder() .setFoo(foo) .setBar(bar) .setBuz(buz) This test
+ * ensures that all methods starting with "set" returns the declaring object
*/
BuilderStyleTest.assertClassesAreBuilderStyle(ColumnFamilyDescriptorBuilder.class);
@@ -162,8 +150,8 @@ public class TestColumnFamilyDescriptorBuilder {
@Test
public void testSetTimeToLive() throws HBaseException {
String ttl;
- ColumnFamilyDescriptorBuilder builder
- = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo"));
+ ColumnFamilyDescriptorBuilder builder =
+ ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo"));
ttl = "50000";
builder.setTimeToLive(ttl);
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 3410e13..89d740c 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
@@ -42,39 +42,32 @@ import org.slf4j.LoggerFactory;
/**
* Test setting values in the descriptor.
*/
-@Category({MiscTests.class, SmallTests.class})
+@Category({ MiscTests.class, SmallTests.class })
public class TestTableDescriptorBuilder {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestTableDescriptorBuilder.class);
+ HBaseClassTestRule.forClass(TestTableDescriptorBuilder.class);
private static final Logger LOG = LoggerFactory.getLogger(TestTableDescriptorBuilder.class);
@Rule
public TestName name = new TestName();
- @Test (expected=IOException.class)
+ @Test(expected = IOException.class)
public void testAddCoprocessorTwice() throws IOException {
String cpName = "a.b.c.d";
- TableDescriptor htd
- = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
- .setCoprocessor(cpName)
- .setCoprocessor(cpName)
- .build();
+ TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).setCoprocessor(cpName)
+ .setCoprocessor(cpName).build();
}
@Test
public void testPb() throws DeserializationException, IOException {
final int v = 123;
- TableDescriptor htd
- = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
- .setMaxFileSize(v)
- .setDurability(Durability.ASYNC_WAL)
- .setReadOnly(true)
- .setRegionReplication(2)
- .build();
+ TableDescriptor htd =
+ TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).setMaxFileSize(v)
+ .setDurability(Durability.ASYNC_WAL).setReadOnly(true).setRegionReplication(2).build();
- byte [] bytes = TableDescriptorBuilder.toByteArray(htd);
+ byte[] bytes = TableDescriptorBuilder.toByteArray(htd);
TableDescriptor deserializedHtd = TableDescriptorBuilder.parseFrom(bytes);
assertEquals(htd, deserializedHtd);
assertEquals(v, deserializedHtd.getMaxFileSize());
@@ -85,69 +78,66 @@ public class TestTableDescriptorBuilder {
/**
* Test cps in the table description.
- *
* @throws Exception if setting a coprocessor fails
*/
@Test
public void testGetSetRemoveCP() throws Exception {
// simple CP
String className = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
- TableDescriptor desc
- = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
- .setCoprocessor(className) // add and check that it is present
- .build();
+ TableDescriptor desc = TableDescriptorBuilder
+ .newBuilder(TableName.valueOf(name.getMethodName())).setCoprocessor(className) // add and
+ // check that
+ // it is
+ // present
+ .build();
assertTrue(desc.hasCoprocessor(className));
- desc = TableDescriptorBuilder.newBuilder(desc)
- .removeCoprocessor(className) // remove it and check that it is gone
- .build();
+ desc = TableDescriptorBuilder.newBuilder(desc).removeCoprocessor(className) // remove it and
+ // check that it is
+ // gone
+ .build();
assertFalse(desc.hasCoprocessor(className));
}
/**
* Test cps in the table description.
- *
* @throws Exception if setting a coprocessor fails
*/
@Test
public void testSetListRemoveCP() throws Exception {
- TableDescriptor desc
- = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
+ TableDescriptor desc =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
// Check that any coprocessor is present.
assertTrue(desc.getCoprocessorDescriptors().isEmpty());
// simple CP
String className1 = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
String className2 = "org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver";
- desc = TableDescriptorBuilder.newBuilder(desc)
- .setCoprocessor(className1) // Add the 1 coprocessor and check if present.
- .build();
+ // Add the 1 coprocessor and check if present.
+ desc = TableDescriptorBuilder.newBuilder(desc).setCoprocessor(className1).build();
assertTrue(desc.getCoprocessorDescriptors().size() == 1);
assertTrue(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
.anyMatch(name -> name.equals(className1)));
-
+ // Add the 2nd coprocessor and check if present.
+ // remove it and check that it is gone
desc = TableDescriptorBuilder.newBuilder(desc)
- // Add the 2nd coprocessor and check if present.
- // remove it and check that it is gone
- .setCoprocessor(className2)
- .build();
+
+ .setCoprocessor(className2).build();
assertTrue(desc.getCoprocessorDescriptors().size() == 2);
assertTrue(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
.anyMatch(name -> name.equals(className2)));
-
+ // Remove one and check
desc = TableDescriptorBuilder.newBuilder(desc)
- // Remove one and check
- .removeCoprocessor(className1)
- .build();
+
+ .removeCoprocessor(className1).build();
assertTrue(desc.getCoprocessorDescriptors().size() == 1);
assertFalse(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
.anyMatch(name -> name.equals(className1)));
assertTrue(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
.anyMatch(name -> name.equals(className2)));
-
+ // Remove the last and check
desc = TableDescriptorBuilder.newBuilder(desc)
- // Remove the last and check
- .removeCoprocessor(className2)
- .build();
+
+ .removeCoprocessor(className2).build();
assertTrue(desc.getCoprocessorDescriptors().isEmpty());
assertFalse(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
.anyMatch(name -> name.equals(className1)));
@@ -162,24 +152,20 @@ public class TestTableDescriptorBuilder {
public void testRemoveString() {
byte[] key = Bytes.toBytes("Some");
byte[] value = Bytes.toBytes("value");
- TableDescriptor desc
- = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
- .setValue(key, value)
- .build();
+ TableDescriptor desc = TableDescriptorBuilder
+ .newBuilder(TableName.valueOf(name.getMethodName())).setValue(key, value).build();
assertTrue(Bytes.equals(value, desc.getValue(key)));
- desc = TableDescriptorBuilder.newBuilder(desc)
- .removeValue(key)
- .build();
+ desc = TableDescriptorBuilder.newBuilder(desc).removeValue(key).build();
assertTrue(desc.getValue(key) == null);
}
String[] legalTableNames = { "foo", "with-dash_under.dot", "_under_start_ok",
"with-dash.with_underscore", "02-01-2012.my_table_01-02", "xyz._mytable_", "9_9_0.table_02",
"dot1.dot2.table", "new.-mytable", "with-dash.with.dot", "legal..t2", "legal..legal.t2",
- "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02"};
+ "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02" };
String[] illegalTableNames = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok",
"-dash-.start_illegal", "new.table with space", "01 .table", "ns:-illegaldash",
- "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2"};
+ "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2" };
@Test
public void testLegalTableNames() {
@@ -204,8 +190,8 @@ public class TestTableDescriptorBuilder {
public void testLegalTableNamesRegex() {
for (String tn : legalTableNames) {
TableName tName = TableName.valueOf(tn);
- assertTrue("Testing: '" + tn + "'", Pattern.matches(TableName.VALID_USER_TABLE_REGEX,
- tName.getNameAsString()));
+ assertTrue("Testing: '" + tn + "'",
+ Pattern.matches(TableName.VALID_USER_TABLE_REGEX, tName.getNameAsString()));
}
}
@@ -217,17 +203,16 @@ public class TestTableDescriptorBuilder {
}
}
- /**
+ /**
* Test default value handling for maxFileSize
*/
@Test
public void testGetMaxFileSize() {
- TableDescriptor desc = TableDescriptorBuilder
- .newBuilder(TableName.valueOf(name.getMethodName())).build();
+ TableDescriptor desc =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
assertEquals(-1, desc.getMaxFileSize());
- desc = TableDescriptorBuilder
- .newBuilder(TableName.valueOf(name.getMethodName()))
- .setMaxFileSize(1111L).build();
+ desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setMaxFileSize(1111L).build();
assertEquals(1111L, desc.getMaxFileSize());
}
@@ -236,12 +221,11 @@ public class TestTableDescriptorBuilder {
*/
@Test
public void testGetMemStoreFlushSize() {
- TableDescriptor desc = TableDescriptorBuilder
- .newBuilder(TableName.valueOf(name.getMethodName())).build();
+ TableDescriptor desc =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
assertEquals(-1, desc.getMemStoreFlushSize());
- desc = TableDescriptorBuilder
- .newBuilder(TableName.valueOf(name.getMethodName()))
- .setMemStoreFlushSize(1111L).build();
+ desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setMemStoreFlushSize(1111L).build();
assertEquals(1111L, desc.getMemStoreFlushSize());
}
@@ -254,51 +238,35 @@ public class TestTableDescriptorBuilder {
public void testModifyFamily() {
byte[] familyName = Bytes.toBytes("cf");
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
- .setBlocksize(1000)
- .setDFSReplication((short) 3)
- .build();
- TableDescriptor htd
- = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
- .setColumnFamily(hcd)
- .build();
+ .setBlocksize(1000).setDFSReplication((short) 3).build();
+ TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setColumnFamily(hcd).build();
assertEquals(1000, htd.getColumnFamily(familyName).getBlocksize());
assertEquals(3, htd.getColumnFamily(familyName).getDFSReplication());
- hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
- .setBlocksize(2000)
- .setDFSReplication((short) 1)
- .build();
- htd = TableDescriptorBuilder.newBuilder(htd)
- .modifyColumnFamily(hcd)
- .build();
+ hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName).setBlocksize(2000)
+ .setDFSReplication((short) 1).build();
+ htd = TableDescriptorBuilder.newBuilder(htd).modifyColumnFamily(hcd).build();
assertEquals(2000, htd.getColumnFamily(familyName).getBlocksize());
assertEquals(1, htd.getColumnFamily(familyName).getDFSReplication());
}
- @Test(expected=IllegalArgumentException.class)
+ @Test(expected = IllegalArgumentException.class)
public void testModifyInexistentFamily() {
byte[] familyName = Bytes.toBytes("cf");
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName);
- TableDescriptor htd = TableDescriptorBuilder
- .newBuilder(TableName.valueOf(name.getMethodName()))
- .modifyColumnFamily(familyDescriptor)
- .build();
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .modifyColumnFamily(ColumnFamilyDescriptorBuilder.of(familyName)).build();
}
- @Test(expected=IllegalArgumentException.class)
+ @Test(expected = IllegalArgumentException.class)
public void testAddDuplicateFamilies() {
byte[] familyName = Bytes.toBytes("cf");
- ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
- .setBlocksize(1000)
- .build();
+ ColumnFamilyDescriptor hcd =
+ ColumnFamilyDescriptorBuilder.newBuilder(familyName).setBlocksize(1000).build();
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
- .setColumnFamily(hcd)
- .build();
+ .setColumnFamily(hcd).build();
assertEquals(1000, htd.getColumnFamily(familyName).getBlocksize());
- hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
- .setBlocksize(2000)
- .build();
+ hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName).setBlocksize(2000).build();
// add duplicate column
TableDescriptorBuilder.newBuilder(htd).setColumnFamily(hcd).build();
}
@@ -306,21 +274,17 @@ public class TestTableDescriptorBuilder {
@Test
public void testPriority() {
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
- .setPriority(42)
- .build();
+ .setPriority(42).build();
assertEquals(42, htd.getPriority());
}
@Test
public void testStringCustomizedValues() {
byte[] familyName = Bytes.toBytes("cf");
- ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
- .setBlocksize(1000)
- .build();
+ ColumnFamilyDescriptor hcd =
+ ColumnFamilyDescriptorBuilder.newBuilder(familyName).setBlocksize(1000).build();
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
- .setColumnFamily(hcd)
- .setDurability(Durability.ASYNC_WAL)
- .build();
+ .setColumnFamily(hcd).setDurability(Durability.ASYNC_WAL).build();
assertEquals(
"'testStringCustomizedValues', " +
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 d4ed2f2..caf4e34 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
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -95,11 +96,9 @@ public class TestBatchCoprocessorEndpoint {
util.startMiniCluster(2);
Admin admin = util.getAdmin();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
- admin.createTable(tableDescriptor, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]});
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TEST_TABLE)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
+ admin.createTable(tableDescriptor, new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] });
util.waitUntilAllRegionsAssigned(TEST_TABLE);
admin.close();
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 8e2c9e6..a0b100f 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@@ -101,11 +102,9 @@ public class TestCoprocessorEndpoint {
util.startMiniCluster(2);
Admin admin = util.getAdmin();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
- admin.createTable(tableDescriptor, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]});
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TEST_TABLE)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
+ admin.createTable(tableDescriptor, new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] });
util.waitUntilAllRegionsAssigned(TEST_TABLE);
Table table = util.getConnection().getTable(TEST_TABLE);
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 24ec48f..9704404 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
@@ -25,10 +25,10 @@ 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.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
@@ -81,13 +81,9 @@ public class TestCoprocessorTableEndpoint {
public void testCoprocessorTableEndpoint() throws Throwable {
final TableName tableName = TableName.valueOf(name.getMethodName());
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY);
-
- tableDescriptor.setColumnFamily(familyDescriptor);
- tableDescriptor.setCoprocessor(ColumnAggregationEndpoint.class.getName());
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY))
+ .setCoprocessor(ColumnAggregationEndpoint.class.getName()).build();
createTable(tableDescriptor);
verifyTable(tableName);
@@ -97,16 +93,12 @@ public class TestCoprocessorTableEndpoint {
public void testDynamicCoprocessorTableEndpoint() throws Throwable {
final TableName tableName = TableName.valueOf(name.getMethodName());
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY);
-
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
createTable(tableDescriptor);
- tableDescriptor.setCoprocessor(ColumnAggregationEndpoint.class.getName());
- updateTable(tableDescriptor);
+ updateTable(TableDescriptorBuilder.newBuilder(tableDescriptor)
+ .setCoprocessor(ColumnAggregationEndpoint.class.getName()).build());
verifyTable(tableName);
}
@@ -142,10 +134,9 @@ public class TestCoprocessorTableEndpoint {
});
}
- private static final void createTable(
- final TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) throws Exception {
+ private static final void createTable(final TableDescriptor tableDescriptor) throws Exception {
Admin admin = TEST_UTIL.getAdmin();
- admin.createTable(tableDescriptor, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]});
+ admin.createTable(tableDescriptor, new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] });
TEST_UTIL.waitUntilAllRegionsAssigned(tableDescriptor.getTableName());
Table table = TEST_UTIL.getConnection().getTable(tableDescriptor.getTableName());
try {
@@ -159,8 +150,7 @@ public class TestCoprocessorTableEndpoint {
}
}
- private static void updateTable(
- final TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) throws Exception {
+ private static void updateTable(TableDescriptor tableDescriptor) throws Exception {
Admin admin = TEST_UTIL.getAdmin();
admin.disableTable(tableDescriptor.getTableName());
admin.modifyTable(tableDescriptor);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java
index feca9d0..0d511c9 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java
@@ -19,9 +19,10 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
-
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
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.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.StoreEngine;
@@ -40,14 +41,11 @@ public class IntegrationTestIngestStripeCompactions extends IntegrationTestInges
@Override
protected void initTable() throws IOException {
// Do the same as the LoadTestTool does, but with different table configuration.
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(getTablename());
- tableDescriptor.setValue(StoreEngine.STORE_ENGINE_CLASS_KEY,
- StripeStoreEngine.class.getName());
- tableDescriptor.setValue(HStore.BLOCKING_STOREFILES_KEY, "100");
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HFileTestUtil.DEFAULT_COLUMN_FAMILY);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(getTablename())
+ .setValue(StoreEngine.STORE_ENGINE_CLASS_KEY, StripeStoreEngine.class.getName())
+ .setValue(HStore.BLOCKING_STOREFILES_KEY, "100").build();
+ ColumnFamilyDescriptor familyDescriptor =
+ ColumnFamilyDescriptorBuilder.of(HFileTestUtil.DEFAULT_COLUMN_FAMILY);
HBaseTestingUtility.createPreSplitLoadTestTable(util.getConfiguration(),
tableDescriptor, familyDescriptor);
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
index 1bd91fe..2b8c2fb 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
@@ -21,13 +21,13 @@ 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.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.HFileTestUtil;
@@ -120,14 +120,12 @@ public class IntegrationTestIngestWithMOB extends IntegrationTestIngest {
TableName tableName = getTablename();
try (Connection connection = ConnectionFactory.createConnection();
Admin admin = connection.getAdmin()) {
- HTableDescriptor tableDesc = new HTableDescriptor(admin.getDescriptor(tableName));
+ TableDescriptor tableDesc = admin.getDescriptor(tableName);
LOG.info("Disabling table " + getTablename());
admin.disableTable(tableName);
ColumnFamilyDescriptor mobColumn = tableDesc.getColumnFamily(mobColumnFamily);
ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(mobColumn)
- .setMobEnabled(true)
- .setMobThreshold((long) threshold)
- .build();
+ .setMobEnabled(true).setMobThreshold((long) threshold).build();
admin.modifyColumnFamily(tableName, cfd);
LOG.info("Enabling table " + getTablename());
admin.enableTable(tableName);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
index 8deccd4..376bc56 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
@@ -24,6 +24,7 @@ import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -189,18 +190,16 @@ public class IntegrationTestLazyCfLoading {
Configuration conf = util.getConfiguration();
String encodingKey = String.format(ENCODING_KEY, this.getClass().getSimpleName());
DataBlockEncoding blockEncoding = DataBlockEncoding.valueOf(conf.get(encodingKey, "FAST_DIFF"));
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TABLE_NAME);
for (byte[] cf : dataGen.getColumnFamilies()) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cf);
- familyDescriptor.setDataBlockEncoding(blockEncoding);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ ColumnFamilyDescriptor familyDescriptor =
+ ColumnFamilyDescriptorBuilder.newBuilder(cf).setDataBlockEncoding(blockEncoding).build();
+ builder.setColumnFamily(familyDescriptor);
}
int serverCount = util.getHBaseClusterInterface().getClusterMetrics()
.getLiveServerMetrics().size();
byte[][] splits = new RegionSplitter.HexStringSplit().split(serverCount * REGIONS_PER_SERVER);
- util.getAdmin().createTable(tableDescriptor, splits);
+ util.getAdmin().createTable(builder.build(), splits);
LOG.info("Created table");
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
index 5e8cd80..d368f30 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
@@ -24,29 +24,27 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.Arrays;
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.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
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.Table;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
import org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobFileCleanerChore;
import org.apache.hadoop.hbase.mob.MobStoreEngine;
import org.apache.hadoop.hbase.mob.MobUtils;
-
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.ToolRunner;
-
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -98,8 +96,8 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase {
.toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
private static Configuration conf;
- private static TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
- private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
+ private static TableDescriptor tableDescriptor;
+ private static ColumnFamilyDescriptor familyDescriptor;
private static Admin admin;
private static Table table = null;
private static MobFileCleanerChore chore;
@@ -126,12 +124,10 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase {
private void createTestTable() throws IOException {
// Create test table
- tableDescriptor = util.createModifyableTableDescriptor("testMobCompactTable");
- familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam);
- familyDescriptor.setMobEnabled(true);
- familyDescriptor.setMobThreshold(mobLen);
- familyDescriptor.setMaxVersions(1);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(fam).setMobEnabled(true)
+ .setMobThreshold(mobLen).setMaxVersions(1).build();
+ tableDescriptor = util.createModifyableTableDescriptor("testMobCompactTable")
+ .setColumnFamily(familyDescriptor).build();
table = util.createTable(tableDescriptor, null);
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
index 3e94a19..7f12d07 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
@@ -23,7 +23,6 @@ import java.util.Locale;
import java.util.Set;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
@@ -267,48 +266,41 @@ public class StripeCompactionsPerformanceEvaluation extends AbstractHBaseTool {
System.out.println(s);
}
- private TableDescriptorBuilder.ModifyableTableDescriptor createHtd(boolean isStripe)
- throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_FAMILY);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ private TableDescriptorBuilder createHtd(boolean isStripe) throws Exception {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(COLUMN_FAMILY));
String noSplitsPolicy = DisabledRegionSplitPolicy.class.getName();
- tableDescriptor.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, noSplitsPolicy);
+ builder.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, noSplitsPolicy);
if (isStripe) {
- tableDescriptor.setValue(StoreEngine.STORE_ENGINE_CLASS_KEY,
- StripeStoreEngine.class.getName());
+ builder.setValue(StoreEngine.STORE_ENGINE_CLASS_KEY, StripeStoreEngine.class.getName());
if (initialStripeCount != null) {
- tableDescriptor.setValue(
- StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialStripeCount.toString());
- tableDescriptor.setValue(
- HStore.BLOCKING_STOREFILES_KEY, Long.toString(10 * initialStripeCount));
+ builder.setValue(StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialStripeCount.toString());
+ builder.setValue(HStore.BLOCKING_STOREFILES_KEY, Long.toString(10 * initialStripeCount));
} else {
- tableDescriptor.setValue(HStore.BLOCKING_STOREFILES_KEY, "500");
+ builder.setValue(HStore.BLOCKING_STOREFILES_KEY, "500");
}
if (splitSize != null) {
- tableDescriptor.setValue(StripeStoreConfig.SIZE_TO_SPLIT_KEY, splitSize.toString());
+ builder.setValue(StripeStoreConfig.SIZE_TO_SPLIT_KEY, splitSize.toString());
}
if (splitParts != null) {
- tableDescriptor.setValue(StripeStoreConfig.SPLIT_PARTS_KEY, splitParts.toString());
+ builder.setValue(StripeStoreConfig.SPLIT_PARTS_KEY, splitParts.toString());
}
} else {
- tableDescriptor.setValue(HStore.BLOCKING_STOREFILES_KEY, "10"); // default
+ builder.setValue(HStore.BLOCKING_STOREFILES_KEY, "10"); // default
}
- return tableDescriptor;
+ return builder;
}
- protected void createTable(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor)
+ private void createTable(TableDescriptorBuilder builder)
throws Exception {
deleteTable();
if (util.getHBaseClusterInterface() instanceof MiniHBaseCluster) {
LOG.warn("Test does not make a lot of sense for minicluster. Will set flush size low.");
- tableDescriptor.setValue(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, "1048576");
+ builder.setValue(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, "1048576");
}
byte[][] splits = new RegionSplitter.HexStringSplit().split(
util.getHBaseClusterInterface().getClusterMetrics().getLiveServerMetrics().size());
- util.getAdmin().createTable(tableDescriptor, splits);
+ util.getAdmin().createTable(builder.build(), splits);
}
public static class SeqShardedDataGenerator extends LoadTestDataGenerator {
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index ce2314e..1d30e17 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -64,6 +64,7 @@ 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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -756,24 +757,20 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
try (Connection conn = ConnectionFactory.createConnection(conf);
Admin admin = conn.getAdmin()) {
if (!admin.tableExists(tableName)) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(getTableName(getConf()));
-
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME);
- // if -DuseMob=true force all data through mob path.
- setMobProperties(conf, familyDescriptor);
- tableDescriptor.setColumnFamily(familyDescriptor);
- // Always add these families. Just skip writing to them when we do not test per CF flush.
- familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(BIG_FAMILY_NAME);
- setMobProperties(conf, familyDescriptor);
- tableDescriptor.setColumnFamily(familyDescriptor);
-
- familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TINY_FAMILY_NAME);
- setMobProperties(conf, familyDescriptor);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder
+ .newBuilder(getTableName(getConf()))
+ // if -DuseMob=true force all data through mob path.
+ .setColumnFamily(
+ setMobProperties(conf, ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_NAME)).build())
+ // Always add these families. Just skip writing to them when we do not test per CF
+ // flush.
+ .setColumnFamily(
+ setMobProperties(conf, ColumnFamilyDescriptorBuilder.newBuilder(BIG_FAMILY_NAME))
+ .build())
+ .setColumnFamily(
+ setMobProperties(conf, ColumnFamilyDescriptorBuilder.newBuilder(TINY_FAMILY_NAME))
+ .build())
+ .build();
// If we want to pre-split compute how many splits.
if (conf.getBoolean(HBaseTestingUtility.PRESPLIT_TEST_TABLE_KEY,
@@ -914,12 +911,13 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
}
}
- private static void setMobProperties(final Configuration conf,
- final ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor) {
+ private static ColumnFamilyDescriptorBuilder setMobProperties(final Configuration conf,
+ final ColumnFamilyDescriptorBuilder builder) {
if (conf.getBoolean("useMob", false)) {
- familyDescriptor.setMobEnabled(true);
- familyDescriptor.setMobThreshold(4);
+ builder.setMobEnabled(true);
+ builder.setMobThreshold(4);
}
+ return builder;
}
/**
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
index 8550f4e..513a5ae 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.log.HBaseMarkers;
@@ -147,14 +148,13 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
private void createTable(Admin admin, TableName tableName, boolean setVersion,
boolean acl) throws IOException {
if (!admin.tableExists(tableName)) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME);
+ ColumnFamilyDescriptorBuilder cfBuilder =
+ ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_NAME);
if (setVersion) {
- familyDescriptor.setMaxVersions(DEFAULT_TABLES_COUNT);
+ cfBuilder.setMaxVersions(DEFAULT_TABLES_COUNT);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(cfBuilder.build()).build();
admin.createTable(tableDescriptor);
if (acl) {
LOG.info("Granting permissions for user " + USER.getShortName());
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
index 6fc4ba7..29a7160 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
@@ -533,11 +533,9 @@ public void cleanUpCluster() throws Exception {
@Test
public void testLoadAndVerify() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_NAME));
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(TEST_NAME))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
Admin admin = getTestingUtil(getConf()).getAdmin();
admin.createTable(tableDescriptor, Bytes.toBytes(0L), Bytes.toBytes(-1L), 40);
@@ -617,10 +615,8 @@ public void cleanUpCluster() throws Exception {
// create HTableDescriptor for specified table
TableName table = getTablename();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(table);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(table)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
if (doLoad) {
try (Connection conn = ConnectionFactory.createConnection(getConf());
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
index 6c71057..e196aa7 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
@@ -365,10 +365,8 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
IntegrationTestingUtility.setUseDistributedCluster(getConf());
int numPresplits = getConf().getInt("loadmapper.numPresplits", 5);
// create HTableDescriptor for specified table
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(getTablename());
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(getTablename())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
try (Connection conn = ConnectionFactory.createConnection(getConf());
Admin admin = conn.getAdmin()) {
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
index d56e181..3d41d8c 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
@@ -57,7 +57,7 @@ implements RecordReader<ImmutableBytesWritable, Result> {
}
/**
- * @param htable the {@link org.apache.hadoop.hbase.HTableDescriptor} to scan.
+ * @param htable the table to scan.
*/
public void setHTable(Table htable) {
this.recordReaderImpl.setHTable(htable);
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
index 6328b5e..3455d6a 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
@@ -104,13 +104,13 @@ public class TableRecordReaderImpl {
byte[] getStartRow() {
return this.startRow;
}
+
/**
- * @param htable the {@link org.apache.hadoop.hbase.HTableDescriptor} to scan.
+ * @param htable the table to scan.
*/
public void setHTable(Table htable) {
Configuration conf = htable.getConfiguration();
- logScannerActivity = conf.getBoolean(
- ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false);
+ logScannerActivity = conf.getBoolean(ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false);
logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100);
this.htable = htable;
}
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
index 5cb8e85..9b71479 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
import java.util.Base64;
import java.util.HashSet;
import java.util.Set;
-
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@@ -36,17 +35,16 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
@@ -59,9 +57,10 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -596,17 +595,15 @@ public class ImportTsv extends Configured implements Tool {
private static void createTable(Admin admin, TableName tableName, String[] columns)
throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(tableName);
Set<String> cfSet = getColumnFamilies(columns);
for (String cf : cfSet) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes(cf));
- tableDescriptor.setColumnFamily(familyDescriptor);
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf));
}
LOG.warn(format("Creating table '%s' with '%s' columns and default descriptors.",
tableName, cfSet));
- admin.createTable(tableDescriptor);
+ admin.createTable(builder.build());
}
private static void deleteTable(Configuration conf, String[] args) {
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
index 8aef458..21589fd 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
@@ -118,13 +118,11 @@ public class TableRecordReaderImpl {
/**
* Sets the HBase table.
- *
- * @param htable The {@link org.apache.hadoop.hbase.HTableDescriptor} to scan.
+ * @param htable The table to scan.
*/
public void setHTable(Table htable) {
Configuration conf = htable.getConfiguration();
- logScannerActivity = conf.getBoolean(
- ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false);
+ logScannerActivity = conf.getBoolean(ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false);
logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100);
this.htable = htable;
}
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index b7f3a3e..134ffa6 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -400,30 +400,30 @@ public class PerformanceEvaluation extends Configured implements Tool {
* Create an HTableDescriptor from provided TestOptions.
*/
protected static TableDescriptor getTableDescriptor(TestOptions opts) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(opts.tableName));
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(opts.tableName));
for (int family = 0; family < opts.families; family++) {
byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDesc =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName);
- familyDesc.setDataBlockEncoding(opts.blockEncoding);
- familyDesc.setCompressionType(opts.compression);
- familyDesc.setBloomFilterType(opts.bloomType);
- familyDesc.setBlocksize(opts.blockSize);
+ ColumnFamilyDescriptorBuilder cfBuilder =
+ ColumnFamilyDescriptorBuilder.newBuilder(familyName);
+ cfBuilder.setDataBlockEncoding(opts.blockEncoding);
+ cfBuilder.setCompressionType(opts.compression);
+ cfBuilder.setBloomFilterType(opts.bloomType);
+ cfBuilder.setBlocksize(opts.blockSize);
if (opts.inMemoryCF) {
- familyDesc.setInMemory(true);
+ cfBuilder.setInMemory(true);
}
- familyDesc.setInMemoryCompaction(opts.inMemoryCompaction);
- tableDescriptor.setColumnFamily(familyDesc);
+ cfBuilder.setInMemoryCompaction(opts.inMemoryCompaction);
+ builder.setColumnFamily(cfBuilder.build());
}
if (opts.replicas != DEFAULT_OPTS.replicas) {
- tableDescriptor.setRegionReplication(opts.replicas);
+ builder.setRegionReplication(opts.replicas);
}
if (opts.splitPolicy != null && !opts.splitPolicy.equals(DEFAULT_OPTS.splitPolicy)) {
- tableDescriptor.setRegionSplitPolicyClassName(opts.splitPolicy);
+ builder.setRegionSplitPolicyClassName(opts.splitPolicy);
}
- return tableDescriptor;
+ return builder.build();
}
/**
@@ -692,7 +692,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
int randomSleep = 0;
boolean inMemoryCF = false;
int presplitRegions = 0;
- int replicas = HTableDescriptor.DEFAULT_REGION_REPLICATION;
+ int replicas = TableDescriptorBuilder.DEFAULT_REGION_REPLICATION;
String splitPolicy = null;
Compression.Algorithm compression = Compression.Algorithm.NONE;
BloomType bloomType = BloomType.ROW;
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 b5fa1e5..b7c2f21 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
@@ -42,6 +42,7 @@ 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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -150,13 +151,10 @@ public class TestTimeRangeMapRed {
@Test
public void testTimeRangeMapRed()
- throws IOException, InterruptedException, ClassNotFoundException {
- final TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
- final ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME);
- familyDescriptor.setMaxVersions(Integer.MAX_VALUE);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ throws IOException, InterruptedException, ClassNotFoundException {
+ final TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(ColumnFamilyDescriptorBuilder
+ .newBuilder(FAMILY_NAME).setMaxVersions(Integer.MAX_VALUE).build()).build();
admin.createTable(tableDescriptor);
List<Put> puts = new ArrayList<>();
for (Map.Entry<Long, Boolean> entry : TIMESTAMP.entrySet()) {
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
index bef16c5..2194f54 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
@@ -32,7 +32,6 @@ import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.UriInfo;
import javax.xml.namespace.QName;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
@@ -71,9 +70,9 @@ public class SchemaResource extends ResourceBase {
this.tableResource = tableResource;
}
- private HTableDescriptor getTableSchema() throws IOException, TableNotFoundException {
+ private TableDescriptor getTableSchema() throws IOException, TableNotFoundException {
try (Table table = servlet.getTable(tableResource.getName())) {
- return new HTableDescriptor(table.getDescriptor());
+ return table.getDescriptor();
}
}
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java
index 967f6ba..6de4b77 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java
@@ -19,21 +19,18 @@
package org.apache.hadoop.hbase.rest.model;
+import com.fasterxml.jackson.annotation.JsonAnyGetter;
+import com.fasterxml.jackson.annotation.JsonAnySetter;
import java.io.Serializable;
import java.util.LinkedHashMap;
import java.util.Map;
-
import javax.xml.bind.annotation.XmlAnyAttribute;
import javax.xml.bind.annotation.XmlAttribute;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.namespace.QName;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-
-import com.fasterxml.jackson.annotation.JsonAnyGetter;
-import com.fasterxml.jackson.annotation.JsonAnySetter;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Representation of a column family schema.
@@ -49,12 +46,12 @@ import com.fasterxml.jackson.annotation.JsonAnySetter;
@InterfaceAudience.Private
public class ColumnSchemaModel implements Serializable {
private static final long serialVersionUID = 1L;
- private static QName BLOCKCACHE = new QName(HColumnDescriptor.BLOCKCACHE);
- private static QName BLOCKSIZE = new QName(HColumnDescriptor.BLOCKSIZE);
- private static QName BLOOMFILTER = new QName(HColumnDescriptor.BLOOMFILTER);
- private static QName COMPRESSION = new QName(HColumnDescriptor.COMPRESSION);
+ private static QName BLOCKCACHE = new QName(ColumnFamilyDescriptorBuilder.BLOCKCACHE);
+ private static QName BLOCKSIZE = new QName(ColumnFamilyDescriptorBuilder.BLOCKSIZE);
+ private static QName BLOOMFILTER = new QName(ColumnFamilyDescriptorBuilder.BLOOMFILTER);
+ private static QName COMPRESSION = new QName(ColumnFamilyDescriptorBuilder.COMPRESSION);
private static QName IN_MEMORY = new QName(HConstants.IN_MEMORY);
- private static QName TTL = new QName(HColumnDescriptor.TTL);
+ private static QName TTL = new QName(ColumnFamilyDescriptorBuilder.TTL);
private static QName VERSIONS = new QName(HConstants.VERSIONS);
private String name;
@@ -138,8 +135,8 @@ public class ColumnSchemaModel implements Serializable {
*/
public boolean __getBlockcache() {
Object o = attrs.get(BLOCKCACHE);
- return o != null ?
- Boolean.parseBoolean(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKCACHE;
+ return o != null ? Boolean.parseBoolean(o.toString()) :
+ ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKCACHE;
}
/**
@@ -147,8 +144,8 @@ public class ColumnSchemaModel implements Serializable {
*/
public int __getBlocksize() {
Object o = attrs.get(BLOCKSIZE);
- return o != null ?
- Integer.parseInt(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKSIZE;
+ return o != null ? Integer.parseInt(o.toString()) :
+ ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKSIZE;
}
/**
@@ -156,7 +153,7 @@ public class ColumnSchemaModel implements Serializable {
*/
public String __getBloomfilter() {
Object o = attrs.get(BLOOMFILTER);
- return o != null ? o.toString() : HColumnDescriptor.DEFAULT_BLOOMFILTER;
+ return o != null ? o.toString() : ColumnFamilyDescriptorBuilder.DEFAULT_BLOOMFILTER.name();
}
/**
@@ -164,7 +161,7 @@ public class ColumnSchemaModel implements Serializable {
*/
public String __getCompression() {
Object o = attrs.get(COMPRESSION);
- return o != null ? o.toString() : HColumnDescriptor.DEFAULT_COMPRESSION;
+ return o != null ? o.toString() : ColumnFamilyDescriptorBuilder.DEFAULT_COMPRESSION.name();
}
/**
@@ -173,7 +170,7 @@ public class ColumnSchemaModel implements Serializable {
public boolean __getInMemory() {
Object o = attrs.get(IN_MEMORY);
return o != null ?
- Boolean.parseBoolean(o.toString()) : HColumnDescriptor.DEFAULT_IN_MEMORY;
+ Boolean.parseBoolean(o.toString()) : ColumnFamilyDescriptorBuilder.DEFAULT_IN_MEMORY;
}
/**
@@ -181,8 +178,7 @@ public class ColumnSchemaModel implements Serializable {
*/
public int __getTTL() {
Object o = attrs.get(TTL);
- return o != null ?
- Integer.parseInt(o.toString()) : HColumnDescriptor.DEFAULT_TTL;
+ return o != null ? Integer.parseInt(o.toString()) : ColumnFamilyDescriptorBuilder.DEFAULT_TTL;
}
/**
@@ -190,8 +186,8 @@ public class ColumnSchemaModel implements Serializable {
*/
public int __getVersions() {
Object o = attrs.get(VERSIONS);
- return o != null ?
- Integer.parseInt(o.toString()) : HColumnDescriptor.DEFAULT_VERSIONS;
+ return o != null ? Integer.parseInt(o.toString()) :
+ ColumnFamilyDescriptorBuilder.DEFAULT_MAX_VERSIONS;
}
/**
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java
index 3d60490..b5578b7 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.rest.model;
import com.fasterxml.jackson.annotation.JsonAnyGetter;
import com.fasterxml.jackson.annotation.JsonAnySetter;
import com.fasterxml.jackson.annotation.JsonIgnore;
-
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
@@ -29,28 +28,24 @@ import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
-
import javax.xml.bind.annotation.XmlAnyAttribute;
import javax.xml.bind.annotation.XmlAttribute;
import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.namespace.QName;
-
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.yetus.audience.InterfaceAudience;
-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.client.ColumnFamilyDescriptor;
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.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.TableSchemaMessage.TableSchema;
import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.ColumnSchemaMessage.ColumnSchema;
+import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.TableSchemaMessage.TableSchema;
/**
* A representation of HBase table descriptors.
@@ -70,13 +65,12 @@ import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.ColumnSchemaMessag
@InterfaceAudience.Private
public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
private static final long serialVersionUID = 1L;
- private static final QName IS_META = new QName(HTableDescriptor.IS_META);
- private static final QName IS_ROOT = new QName(HTableDescriptor.IS_ROOT);
- private static final QName READONLY = new QName(HTableDescriptor.READONLY);
- private static final QName TTL = new QName(HColumnDescriptor.TTL);
+ private static final QName IS_META = new QName(TableDescriptorBuilder.IS_META);
+ private static final QName IS_ROOT = new QName("IS_ROOT");
+ private static final QName READONLY = new QName(TableDescriptorBuilder.READONLY);
+ private static final QName TTL = new QName(ColumnFamilyDescriptorBuilder.TTL);
private static final QName VERSIONS = new QName(HConstants.VERSIONS);
- private static final QName COMPRESSION =
- new QName(HColumnDescriptor.COMPRESSION);
+ private static final QName COMPRESSION = new QName(ColumnFamilyDescriptorBuilder.COMPRESSION);
private String name;
private Map<QName,Object> attrs = new LinkedHashMap<>();
@@ -242,7 +236,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
*/
public boolean __getReadOnly() {
Object o = attrs.get(READONLY);
- return o != null ? Boolean.parseBoolean(o.toString()) : HTableDescriptor.DEFAULT_READONLY;
+ return o != null ? Boolean.parseBoolean(o.toString()) : TableDescriptorBuilder.DEFAULT_READONLY;
}
/**
@@ -315,7 +309,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
this.addAttribute(attr.getName(), attr.getValue());
}
if (builder.hasReadOnly()) {
- this.addAttribute(HTableDescriptor.READONLY, builder.getReadOnly());
+ this.addAttribute(TableDescriptorBuilder.READONLY, builder.getReadOnly());
}
for (ColumnSchema family : builder.getColumnsList()) {
ColumnSchemaModel familyModel = new ColumnSchemaModel();
@@ -324,14 +318,14 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
familyModel.addAttribute(attr.getName(), attr.getValue());
}
if (family.hasTtl()) {
- familyModel.addAttribute(HColumnDescriptor.TTL, family.getTtl());
+ familyModel.addAttribute(ColumnFamilyDescriptorBuilder.TTL, family.getTtl());
}
if (family.hasMaxVersions()) {
familyModel.addAttribute(HConstants.VERSIONS,
family.getMaxVersions());
}
if (family.hasCompression()) {
- familyModel.addAttribute(HColumnDescriptor.COMPRESSION,
+ familyModel.addAttribute(ColumnFamilyDescriptorBuilder.COMPRESSION,
family.getCompression());
}
this.addColumnFamily(familyModel);
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
index b895c4d..d7a5cfd 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
@@ -33,7 +33,6 @@ import java.util.Random;
import java.util.TreeMap;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -76,7 +75,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Hash;
import org.apache.hadoop.hbase.util.MurmurHash;
import org.apache.hadoop.hbase.util.Pair;
-
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
@@ -93,7 +91,6 @@ import org.apache.hadoop.mapreduce.lib.reduce.LongSumReducer;
import org.apache.hadoop.util.LineReader;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -128,7 +125,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
public static final byte[] QUALIFIER_NAME = Bytes.toBytes("data");
private TableName tableName = TABLE_NAME;
- protected TableDescriptorBuilder.ModifyableTableDescriptor TABLE_DESCRIPTOR;
+ protected TableDescriptor TABLE_DESCRIPTOR;
protected Map<String, CmdDescriptor> commands = new TreeMap<>();
protected static Cluster cluster = new Cluster();
@@ -539,15 +536,12 @@ public class PerformanceEvaluation extends Configured implements Tool {
protected TableDescriptor getDescriptor() {
if (TABLE_DESCRIPTOR == null) {
- TABLE_DESCRIPTOR = new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME);
- familyDescriptor.setDataBlockEncoding(blockEncoding);
- familyDescriptor.setCompressionType(compression);
- if (inMemoryCF) {
- familyDescriptor.setInMemory(true);
- }
- TABLE_DESCRIPTOR.setColumnFamily(familyDescriptor);
+ TABLE_DESCRIPTOR =
+ TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_NAME)
+ .setDataBlockEncoding(blockEncoding).setCompressionType(compression)
+ .setInMemory(inMemoryCF).build())
+ .build();
}
return TABLE_DESCRIPTOR;
}
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 a6add87..e1fcaf8 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
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.Durability;
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.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.Filter;
@@ -140,12 +141,9 @@ public class TestScannersWithFilters {
REST_TEST_UTIL.getServletPort()));
Admin admin = TEST_UTIL.getAdmin();
if (!admin.tableExists(TABLE)) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0]));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[1]));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[0]))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[1])).build();
admin.createTable(tableDescriptor);
Table table = TEST_UTIL.getConnection().getTable(TABLE);
// Insert first half
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 4d0359d..6f4dee5 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
@@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.rest.client;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
@@ -31,8 +31,8 @@ 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;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.RestTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -117,12 +117,12 @@ public class TestRemoteAdminRetries {
}
@Test
- @SuppressWarnings("deprecation")
public void testFailingCreateTable() throws Exception {
testTimedOutCall(new CallExecutor() {
@Override
public void run() throws Exception {
- remoteAdmin.createTable(new HTableDescriptor(TableName.valueOf("TestTable")));
+ remoteAdmin
+ .createTable(TableDescriptorBuilder.newBuilder(TableName.valueOf("TestTable")).build());
}
});
verify(client, times(RETRIES)).put(anyString(), anyString(), any());
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 2d3af51..66baf16 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
@@ -122,17 +122,11 @@ public class TestRemoteTable {
admin.deleteTable(TABLE);
}
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_1)
- .setMaxVersions(3));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_2)
- .setMaxVersions(3));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_3)
- .setMaxVersions(3));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_1).setMaxVersions(3).build())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_2).setMaxVersions(3).build())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_3).setMaxVersions(3).build())
+ .build();
admin.createTable(tableDescriptor);
try (Table table = TEST_UTIL.getConnection().getTable(TABLE)) {
Put put = new Put(ROW_1);
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index 19759b5..d95a359 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -38,7 +38,6 @@ org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
org.apache.hadoop.hbase.HBaseConfiguration;
org.apache.hadoop.hbase.HConstants;
-org.apache.hadoop.hbase.HTableDescriptor;
org.apache.hadoop.hbase.NamespaceDescriptor;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.TableName;
@@ -610,8 +609,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
<tbody>
<%for TableDescriptor desc : tables%>
<%java>
- HTableDescriptor htDesc = new HTableDescriptor(desc);
- TableName tableName = htDesc.getTableName();
+ TableName tableName = desc.getTableName();
TableState tableState = master.getTableStateManager().getTableState(tableName);
Map<RegionState.State, List<RegionInfo>> tableRegions =
master.getAssignmentManager().getRegionStates()
@@ -649,7 +647,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
<td><% failedRegionsCount %></td>
<td><% splitRegionsCount %></td>
<td><% otherRegionsCount %></td>
- <td><% htDesc.toStringCustomizedValues() %></td>
+ <td><% desc.toStringCustomizedValues() %></td>
</tr>
</%for>
<p><% tables.size() %> table(s) in set. [<a href=tablesDetailed.jsp>Details</a>]</p>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
index b67fffa..f4847b9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
@@ -23,22 +23,22 @@ import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.hadoop.conf.Configuration;
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.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
-
-import java.util.concurrent.CopyOnWriteArrayList;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* This class creates a single process HBase cluster. One thread is created for
@@ -451,23 +451,18 @@ public class LocalHBaseCluster {
/**
* Test things basically work.
- * @param args
- * @throws IOException
*/
public static void main(String[] args) throws IOException {
Configuration conf = HBaseConfiguration.create();
LocalHBaseCluster cluster = new LocalHBaseCluster(conf);
cluster.startup();
- Connection connection = ConnectionFactory.createConnection(conf);
- Admin admin = connection.getAdmin();
- try {
- HTableDescriptor htd =
- new HTableDescriptor(TableName.valueOf(cluster.getClass().getName()));
+ try (Connection connection = ConnectionFactory.createConnection(conf);
+ Admin admin = connection.getAdmin()) {
+ TableDescriptor htd =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(cluster.getClass().getName())).build();
admin.createTable(htd);
} finally {
- admin.close();
+ cluster.shutdown();
}
- connection.close();
- cluster.shutdown();
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java
index ee942db..301adc1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java
@@ -286,7 +286,6 @@ public final class Constraints {
* Write the configuration to a String
* @param conf to write
* @return String representation of that configuration
- * @throws IOException
*/
private static String serializeConfiguration(Configuration conf) throws IOException {
// write the configuration out to the data stream
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
index 5587c40..2978eed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
@@ -50,46 +50,41 @@ import org.apache.hbase.thirdparty.com.google.common.base.Objects;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
- * A block cache implementation that is memory-aware using {@link HeapSize},
- * memory-bound using an LRU eviction algorithm, and concurrent: backed by a
- * {@link ConcurrentHashMap} and with a non-blocking eviction thread giving
- * constant-time {@link #cacheBlock} and {@link #getBlock} operations.<p>
- *
+ * A block cache implementation that is memory-aware using {@link HeapSize}, memory-bound using an
+ * LRU eviction algorithm, and concurrent: backed by a {@link ConcurrentHashMap} and with a
+ * non-blocking eviction thread giving constant-time {@link #cacheBlock} and {@link #getBlock}
+ * operations.
+ * </p>
* Contains three levels of block priority to allow for scan-resistance and in-memory families
- * {@link org.apache.hadoop.hbase.HColumnDescriptor#setInMemory(boolean)} (An in-memory column
- * family is a column family that should be served from memory if possible):
- * single-access, multiple-accesses, and in-memory priority.
- * A block is added with an in-memory priority flag if
- * {@link org.apache.hadoop.hbase.HColumnDescriptor#isInMemory()}, otherwise a block becomes a
- * single access priority the first time it is read into this block cache. If a block is
- * accessed again while in cache, it is marked as a multiple access priority block. This
- * delineation of blocks is used to prevent scans from thrashing the cache adding a
- * least-frequently-used element to the eviction algorithm.<p>
- *
- * Each priority is given its own chunk of the total cache to ensure
- * fairness during eviction. Each priority will retain close to its maximum
- * size, however, if any priority is not using its entire chunk the others
- * are able to grow beyond their chunk size.<p>
- *
- * Instantiated at a minimum with the total size and average block size.
- * All sizes are in bytes. The block size is not especially important as this
- * cache is fully dynamic in its sizing of blocks. It is only used for
- * pre-allocating data structures and in initial heap estimation of the map.<p>
- *
- * The detailed constructor defines the sizes for the three priorities (they
- * should total to the <code>maximum size</code> defined). It also sets the levels that
- * trigger and control the eviction thread.<p>
- *
- * The <code>acceptable size</code> is the cache size level which triggers the eviction
- * process to start. It evicts enough blocks to get the size below the
- * minimum size specified.<p>
- *
- * Eviction happens in a separate thread and involves a single full-scan
- * of the map. It determines how many bytes must be freed to reach the minimum
- * size, and then while scanning determines the fewest least-recently-used
- * blocks necessary from each of the three priorities (would be 3 times bytes
- * to free). It then uses the priority chunk sizes to evict fairly according
- * to the relative sizes and usage.
+ * {@link org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder#setInMemory(boolean)} (An
+ * in-memory column family is a column family that should be served from memory if possible):
+ * single-access, multiple-accesses, and in-memory priority. A block is added with an in-memory
+ * priority flag if {@link org.apache.hadoop.hbase.client.ColumnFamilyDescriptor#isInMemory()},
+ * otherwise a block becomes a single access priority the first time it is read into this block
+ * cache. If a block is accessed again while in cache, it is marked as a multiple access priority
+ * block. This delineation of blocks is used to prevent scans from thrashing the cache adding a
+ * least-frequently-used element to the eviction algorithm.
+ * <p/>
+ * Each priority is given its own chunk of the total cache to ensure fairness during eviction. Each
+ * priority will retain close to its maximum size, however, if any priority is not using its entire
+ * chunk the others are able to grow beyond their chunk size.
+ * <p/>
+ * Instantiated at a minimum with the total size and average block size. All sizes are in bytes. The
+ * block size is not especially important as this cache is fully dynamic in its sizing of blocks. It
+ * is only used for pre-allocating data structures and in initial heap estimation of the map.
+ * <p/>
+ * The detailed constructor defines the sizes for the three priorities (they should total to the
+ * <code>maximum size</code> defined). It also sets the levels that trigger and control the eviction
+ * thread.
+ * <p/>
+ * The <code>acceptable size</code> is the cache size level which triggers the eviction process to
+ * start. It evicts enough blocks to get the size below the minimum size specified.
+ * <p/>
+ * Eviction happens in a separate thread and involves a single full-scan of the map. It determines
+ * how many bytes must be freed to reach the minimum size, and then while scanning determines the
+ * fewest least-recently-used blocks necessary from each of the three priorities (would be 3 times
+ * bytes to free). It then uses the priority chunk sizes to evict fairly according to the relative
+ * sizes and usage.
*/
@InterfaceAudience.Private
public class LruBlockCache implements FirstLevelBlockCache {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java
index 1d816d6..36dcd26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java
@@ -23,7 +23,6 @@ import java.util.Arrays;
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.DoNotRetryIOException;
@@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Mutation;
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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.util.Bytes;
@@ -75,22 +75,15 @@ public class QuotaUtil extends QuotaTableUtil {
public static final long DEFAULT_WRITE_CAPACITY_UNIT = 1024;
/** Table descriptor for Quota internal table */
- public static final TableDescriptorBuilder.ModifyableTableDescriptor QUOTA_TABLE_DESC =
- new TableDescriptorBuilder.ModifyableTableDescriptor(QUOTA_TABLE_NAME);
- static {
- QUOTA_TABLE_DESC.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(QUOTA_FAMILY_INFO)
- .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
- .setBloomFilterType(BloomType.ROW)
- .setMaxVersions(1)
- );
- QUOTA_TABLE_DESC.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(QUOTA_FAMILY_USAGE)
- .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
- .setBloomFilterType(BloomType.ROW)
- .setMaxVersions(1)
- );
- }
+ public static final TableDescriptor QUOTA_TABLE_DESC =
+ TableDescriptorBuilder.newBuilder(QUOTA_TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(QUOTA_FAMILY_INFO)
+ .setScope(HConstants.REPLICATION_SCOPE_LOCAL).setBloomFilterType(BloomType.ROW)
+ .setMaxVersions(1).build())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(QUOTA_FAMILY_USAGE)
+ .setScope(HConstants.REPLICATION_SCOPE_LOCAL).setBloomFilterType(BloomType.ROW)
+ .setMaxVersions(1).build())
+ .build();
/** Returns true if the support for quota is enabled */
public static boolean isQuotaEnabled(final Configuration conf) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 6570e77e..6cc690f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.TableName;
@@ -211,19 +210,14 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
// Need to create the new system table for labels here
if (!MetaTableAccessor.tableExists(ctx.getEnvironment().getConnection(), LABELS_TABLE_NAME)) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(LABELS_TABLE_NAME);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(LABELS_TABLE_FAMILY);
- familyDescriptor.setBloomFilterType(BloomType.NONE);
- // We will cache all the labels. No need of normal
- // table block cache.
- familyDescriptor.setBlockCacheEnabled(false);
- tableDescriptor.setColumnFamily(familyDescriptor);
- // Let the "labels" table having only one region always. We are not expecting too many labels in
- // the system.
- tableDescriptor.setValue(HTableDescriptor.SPLIT_POLICY,
- DisabledRegionSplitPolicy.class.getName());
+ // We will cache all the labels. No need of normal table block cache.
+ // Let the "labels" table having only one region always. We are not expecting too many labels
+ // in the system.
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(LABELS_TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(LABELS_TABLE_FAMILY)
+ .setBloomFilterType(BloomType.NONE).setBlockCacheEnabled(false).build())
+ .setValue(TableDescriptorBuilder.SPLIT_POLICY, DisabledRegionSplitPolicy.class.getName())
+ .build();
try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {
admin.createTable(tableDescriptor);
}
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp b/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
index 94590c8..6a28de1 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
@@ -23,10 +23,10 @@
import="java.util.Date"
import="java.util.List"
import="java.util.Map"
+ import="java.util.function.Function"
import="java.util.regex.Pattern"
import="java.util.stream.Stream"
import="java.util.stream.Collectors"
- import="org.apache.hadoop.hbase.HTableDescriptor"
import="org.apache.hadoop.hbase.ServerName"
import="org.apache.hadoop.hbase.TableName"
import="org.apache.hadoop.hbase.client.Admin"
@@ -438,8 +438,8 @@
try (Admin admin = master.getConnection().getAdmin()) {
tables = master.isInitialized() ? admin.listTableDescriptors(true) : null;
}
- Map<TableName, HTableDescriptor> tableDescriptors
- = tables.stream().collect(Collectors.toMap(TableDescriptor::getTableName, p -> new HTableDescriptor(p)));
+ Map<TableName, TableDescriptor> tableDescriptors = tables.stream().collect(
+ Collectors.toMap(TableDescriptor::getTableName, Function.identity()));
%>
<table class="table table-striped">
<tr>
@@ -454,8 +454,8 @@
<th>Description</th>
</tr>
<% for(TableName tableName : rsGroupTables) {
- HTableDescriptor htDesc = tableDescriptors.get(tableName);
- if(htDesc == null) {
+ TableDescriptor desc = tableDescriptors.get(tableName);
+ if(desc == null) {
%>
<tr>
<td><%= tableName.getNamespaceAsString() %></td>
@@ -500,7 +500,7 @@
<td><%= failedRegionsCount %></td>
<td><%= splitRegionsCount %></td>
<td><%= otherRegionsCount %></td>
- <td><%= htDesc.toStringCustomizedValues() %></td>
+ <td><%= desc.toStringCustomizedValues() %></td>
</tr>
<% }
} %>
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index 7683242..e46b277 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -35,7 +35,6 @@
import="org.apache.hadoop.conf.Configuration"
import="org.apache.hadoop.hbase.HConstants"
import="org.apache.hadoop.hbase.HRegionLocation"
- import="org.apache.hadoop.hbase.HTableDescriptor"
import="org.apache.hadoop.hbase.NotServingRegionException"
import="org.apache.hadoop.hbase.RegionMetrics"
import="org.apache.hadoop.hbase.RegionMetricsBuilder"
@@ -46,6 +45,7 @@
import="org.apache.hadoop.hbase.TableNotFoundException"
import="org.apache.hadoop.hbase.client.AsyncAdmin"
import="org.apache.hadoop.hbase.client.AsyncConnection"
+ import="org.apache.hadoop.hbase.client.ColumnFamilyDescriptor"
import="org.apache.hadoop.hbase.client.CompactionState"
import="org.apache.hadoop.hbase.client.RegionInfo"
import="org.apache.hadoop.hbase.client.RegionInfoBuilder"
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 0c4f7fb..ede1196 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1565,10 +1565,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
new Configuration(getConfiguration()));
}
- public Table createTable(TableName tableName, byte[][] families,
- int numVersions, byte[] startKey, byte[] endKey, int numRegions)
- throws IOException{
- HTableDescriptor desc = createTableDescriptor(tableName, families, numVersions);
+ public Table createTable(TableName tableName, byte[][] families, int numVersions, byte[] startKey,
+ byte[] endKey, int numRegions) throws IOException {
+ TableDescriptor desc = createTableDescriptor(tableName, families, numVersions);
getAdmin().createTable(desc, startKey, endKey, numRegions);
// HBaseAdmin only waits for regions to appear in hbase:meta we
@@ -1579,14 +1578,11 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
/**
* Create a table.
- * @param htd
- * @param families
* @param c Configuration to use
* @return A Table instance for the created table.
- * @throws IOException
*/
public Table createTable(TableDescriptor htd, byte[][] families, Configuration c)
- throws IOException {
+ throws IOException {
return createTable(htd, families, null, c);
}
@@ -1678,35 +1674,25 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
* @param replicaCount the replica count
* @param c Configuration to use
* @return A Table instance for the created table.
- * @throws IOException
*/
public Table createTable(TableName tableName, byte[][] families, byte[][] splitKeys,
- int replicaCount, final Configuration c) throws IOException {
- HTableDescriptor htd = new HTableDescriptor(tableName);
- htd.setRegionReplication(replicaCount);
+ int replicaCount, final Configuration c) throws IOException {
+ TableDescriptor htd =
+ TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(replicaCount).build();
return createTable(htd, families, splitKeys, c);
}
/**
* Create a table.
- * @param tableName
- * @param family
- * @param numVersions
* @return A Table instance for the created table.
- * @throws IOException
*/
- public Table createTable(TableName tableName, byte[] family, int numVersions)
- throws IOException {
- return createTable(tableName, new byte[][]{family}, numVersions);
+ public Table createTable(TableName tableName, byte[] family, int numVersions) throws IOException {
+ return createTable(tableName, new byte[][] { family }, numVersions);
}
/**
* Create a table.
- * @param tableName
- * @param families
- * @param numVersions
* @return A Table instance for the created table.
- * @throws IOException
*/
public Table createTable(TableName tableName, byte[][] families, int numVersions)
throws IOException {
@@ -1715,30 +1701,23 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
/**
* Create a table.
- * @param tableName
- * @param families
- * @param numVersions
- * @param splitKeys
* @return A Table instance for the created table.
- * @throws IOException
*/
public Table createTable(TableName tableName, byte[][] families, int numVersions,
byte[][] splitKeys) throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
for (byte[] family : families) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
- .setMaxVersions(numVersions);
+ ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family)
+ .setMaxVersions(numVersions);
if (isNewVersionBehaviorEnabled()) {
- familyDescriptor.setNewVersionBehavior(true);
+ cfBuilder.setNewVersionBehavior(true);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
+ builder.setColumnFamily(cfBuilder.build());
}
if (splitKeys != null) {
- getAdmin().createTable(tableDescriptor, splitKeys);
+ getAdmin().createTable(builder.build(), splitKeys);
} else {
- getAdmin().createTable(tableDescriptor);
+ getAdmin().createTable(builder.build());
}
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
@@ -1748,11 +1727,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
/**
* Create a table with multiple regions.
- * @param tableName
- * @param families
- * @param numVersions
* @return A Table instance for the created table.
- * @throws IOException
*/
public Table createMultiRegionTable(TableName tableName, byte[][] families, int numVersions)
throws IOException {
@@ -1761,28 +1736,20 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
/**
* Create a table.
- * @param tableName
- * @param families
- * @param numVersions
- * @param blockSize
* @return A Table instance for the created table.
- * @throws IOException
*/
public Table createTable(TableName tableName, byte[][] families,
int numVersions, int blockSize) throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
for (byte[] family : families) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
- .setMaxVersions(numVersions)
- .setBlocksize(blockSize);
+ ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family)
+ .setMaxVersions(numVersions).setBlocksize(blockSize);
if (isNewVersionBehaviorEnabled()) {
- familyDescriptor.setNewVersionBehavior(true);
+ cfBuilder.setNewVersionBehavior(true);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
+ builder.setColumnFamily(cfBuilder.build());
}
- getAdmin().createTable(tableDescriptor);
+ getAdmin().createTable(builder.build());
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(tableName);
@@ -1791,22 +1758,19 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
public Table createTable(TableName tableName, byte[][] families,
int numVersions, int blockSize, String cpName) throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
for (byte[] family : families) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
- .setMaxVersions(numVersions)
- .setBlocksize(blockSize);
+ ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family)
+ .setMaxVersions(numVersions).setBlocksize(blockSize);
if (isNewVersionBehaviorEnabled()) {
- familyDescriptor.setNewVersionBehavior(true);
+ cfBuilder.setNewVersionBehavior(true);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
+ builder.setColumnFamily(cfBuilder.build());
}
if (cpName != null) {
- tableDescriptor.setCoprocessor(cpName);
+ builder.setCoprocessor(cpName);
}
- getAdmin().createTable(tableDescriptor);
+ getAdmin().createTable(builder.build());
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(tableName);
@@ -1815,28 +1779,22 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
/**
* Create a table.
- * @param tableName
- * @param families
- * @param numVersions
* @return A Table instance for the created table.
- * @throws IOException
*/
- public Table createTable(TableName tableName, byte[][] families,
- int[] numVersions) throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+ public Table createTable(TableName tableName, byte[][] families, int[] numVersions)
+ throws IOException {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
int i = 0;
for (byte[] family : families) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
- .setMaxVersions(numVersions[i]);
+ ColumnFamilyDescriptorBuilder cfBuilder =
+ ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(numVersions[i]);
if (isNewVersionBehaviorEnabled()) {
- familyDescriptor.setNewVersionBehavior(true);
+ cfBuilder.setNewVersionBehavior(true);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
+ builder.setColumnFamily(cfBuilder.build());
i++;
}
- getAdmin().createTable(tableDescriptor);
+ getAdmin().createTable(builder.build());
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(tableName);
@@ -1845,23 +1803,17 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
/**
* Create a table.
- * @param tableName
- * @param family
- * @param splitRows
* @return A Table instance for the created table.
- * @throws IOException
*/
public Table createTable(TableName tableName, byte[] family, byte[][] splitRows)
- throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
+ throws IOException {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+ ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family);
if (isNewVersionBehaviorEnabled()) {
- familyDescriptor.setNewVersionBehavior(true);
+ cfBuilder.setNewVersionBehavior(true);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
- getAdmin().createTable(tableDescriptor, splitRows);
+ builder.setColumnFamily(cfBuilder.build());
+ getAdmin().createTable(builder.build(), splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(tableName);
@@ -1870,10 +1822,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
/**
* Create a table with multiple regions.
- * @param tableName
- * @param family
* @return A Table instance for the created table.
- * @throws IOException
*/
public Table createMultiRegionTable(TableName tableName, byte[] family) throws IOException {
return createTable(tableName, family, KEYS_FOR_HBA_CREATE_TABLE);
@@ -1896,10 +1845,10 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
* Set the number of Region replicas.
*/
public static void setReplicas(Admin admin, TableName table, int replicaCount)
- throws IOException, InterruptedException {
+ throws IOException, InterruptedException {
admin.disableTable(table);
- HTableDescriptor desc = new HTableDescriptor(admin.getDescriptor(table));
- desc.setRegionReplication(replicaCount);
+ TableDescriptor desc = TableDescriptorBuilder.newBuilder(admin.getDescriptor(table))
+ .setRegionReplication(replicaCount).build();
admin.modifyTable(desc);
admin.enableTable(table);
}
@@ -1931,7 +1880,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
// ==========================================================================
// Canned table and table descriptor creation
- // TODO replace HBaseTestCase
public final static byte [] fam1 = Bytes.toBytes("colfamily11");
public final static byte [] fam2 = Bytes.toBytes("colfamily21");
@@ -1944,52 +1892,40 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
public static final byte [] START_KEY_BYTES = {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
public static final String START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_CHARSET);
- public TableDescriptorBuilder.ModifyableTableDescriptor createModifyableTableDescriptor(
- final String name) {
+ public TableDescriptorBuilder createModifyableTableDescriptor(final String name) {
return createModifyableTableDescriptor(TableName.valueOf(name),
- HColumnDescriptor.DEFAULT_MIN_VERSIONS,
- MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
- }
-
- public HTableDescriptor createTableDescriptor(final TableName name,
- final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(name);
- for (byte[] cfName : new byte[][]{fam1, fam2, fam3}) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName)
- .setMinVersions(minVersions)
- .setMaxVersions(versions)
- .setKeepDeletedCells(keepDeleted)
- .setBlockCacheEnabled(false)
- .setTimeToLive(ttl);
+ ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, MAXVERSIONS, HConstants.FOREVER,
+ ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+ }
+
+ public TableDescriptor createTableDescriptor(final TableName name, final int minVersions,
+ final int versions, final int ttl, KeepDeletedCells keepDeleted) {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name);
+ for (byte[] cfName : new byte[][] { fam1, fam2, fam3 }) {
+ ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(cfName)
+ .setMinVersions(minVersions).setMaxVersions(versions).setKeepDeletedCells(keepDeleted)
+ .setBlockCacheEnabled(false).setTimeToLive(ttl);
if (isNewVersionBehaviorEnabled()) {
- familyDescriptor.setNewVersionBehavior(true);
+ cfBuilder.setNewVersionBehavior(true);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
- }
- return new HTableDescriptor(tableDescriptor);
- }
-
- public TableDescriptorBuilder.ModifyableTableDescriptor createModifyableTableDescriptor(
- final TableName name, final int minVersions, final int versions, final int ttl,
- KeepDeletedCells keepDeleted) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(name);
- for (byte[] cfName : new byte[][]{fam1, fam2, fam3}) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName)
- .setMinVersions(minVersions)
- .setMaxVersions(versions)
- .setKeepDeletedCells(keepDeleted)
- .setBlockCacheEnabled(false)
- .setTimeToLive(ttl);
+ builder.setColumnFamily(cfBuilder.build());
+ }
+ return builder.build();
+ }
+
+ public TableDescriptorBuilder createModifyableTableDescriptor(final TableName name,
+ final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name);
+ for (byte[] cfName : new byte[][] { fam1, fam2, fam3 }) {
+ ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(cfName)
+ .setMinVersions(minVersions).setMaxVersions(versions).setKeepDeletedCells(keepDeleted)
+ .setBlockCacheEnabled(false).setTimeToLive(ttl);
if (isNewVersionBehaviorEnabled()) {
- familyDescriptor.setNewVersionBehavior(true);
+ cfBuilder.setNewVersionBehavior(true);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
+ builder.setColumnFamily(cfBuilder.build());
}
- return tableDescriptor;
+ return builder;
}
/**
@@ -1997,31 +1933,27 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
* @param name Name to give table.
* @return Column descriptor.
*/
- public HTableDescriptor createTableDescriptor(final TableName name) {
- return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS,
- MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
+ public TableDescriptor createTableDescriptor(final TableName name) {
+ return createTableDescriptor(name, ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS,
+ MAXVERSIONS, HConstants.FOREVER, ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
}
- public HTableDescriptor createTableDescriptor(final TableName tableName,
- byte[] family) {
- return createTableDescriptor(tableName, new byte[][] {family}, 1);
+ public TableDescriptor createTableDescriptor(final TableName tableName, byte[] family) {
+ return createTableDescriptor(tableName, new byte[][] { family }, 1);
}
- public HTableDescriptor createTableDescriptor(final TableName tableName,
- byte[][] families, int maxVersions) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
+ public TableDescriptor createTableDescriptor(final TableName tableName, byte[][] families,
+ int maxVersions) {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
for (byte[] family : families) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
- .setMaxVersions(maxVersions);
+ ColumnFamilyDescriptorBuilder cfBuilder =
+ ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(maxVersions);
if (isNewVersionBehaviorEnabled()) {
- familyDescriptor.setNewVersionBehavior(true);
+ cfBuilder.setNewVersionBehavior(true);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
+ builder.setColumnFamily(cfBuilder.build());
}
- return new HTableDescriptor(tableDescriptor);
+ return builder.build();
}
/**
@@ -2078,28 +2010,26 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
public HRegion createLocalHRegionWithInMemoryFlags(TableName tableName, byte[] startKey,
byte[] stopKey, boolean isReadOnly, Durability durability, WAL wal, boolean[] compactedMemStore,
byte[]... families) throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setReadOnly(isReadOnly);
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+ builder.setReadOnly(isReadOnly);
int i = 0;
for (byte[] family : families) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
+ ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family);
if (compactedMemStore != null && i < compactedMemStore.length) {
- familyDescriptor.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
+ cfBuilder.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
} else {
- familyDescriptor.setInMemoryCompaction(MemoryCompactionPolicy.NONE);
+ cfBuilder.setInMemoryCompaction(MemoryCompactionPolicy.NONE);
}
i++;
// Set default to be three versions.
- familyDescriptor.setMaxVersions(Integer.MAX_VALUE);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ cfBuilder.setMaxVersions(Integer.MAX_VALUE);
+ builder.setColumnFamily(cfBuilder.build());
}
- tableDescriptor.setDurability(durability);
- RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
- .setStartKey(startKey).setEndKey(stopKey).build();
- return createLocalHRegion(info, tableDescriptor, wal);
+ builder.setDurability(durability);
+ RegionInfo info =
+ RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).setEndKey(stopKey).build();
+ return createLocalHRegion(info, builder.build(), wal);
}
//
@@ -2776,7 +2706,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
} catch (SecurityException e) {
throw new RuntimeException(e);
} catch (NoSuchFieldException e) {
- // TODO Auto-generated catch block
throw new RuntimeException(e);
} catch (IllegalArgumentException e) {
throw new RuntimeException(e);
@@ -3892,20 +3821,17 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
* logs a warning and continues.
* @return the number of regions the table was split into
*/
- public static int createPreSplitLoadTestTable(Configuration conf,
- TableName tableName, byte[] columnFamily, Algorithm compression,
- DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication,
- Durability durability)
- throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setDurability(durability);
- tableDescriptor.setRegionReplication(regionReplication);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamily);
- familyDescriptor.setDataBlockEncoding(dataBlockEncoding);
- familyDescriptor.setCompressionType(compression);
- return createPreSplitLoadTestTable(conf, tableDescriptor, familyDescriptor,
+ public static int createPreSplitLoadTestTable(Configuration conf, TableName tableName,
+ byte[] columnFamily, Algorithm compression, DataBlockEncoding dataBlockEncoding,
+ int numRegionsPerServer, int regionReplication, Durability durability) throws IOException {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+ builder.setDurability(durability);
+ builder.setRegionReplication(regionReplication);
+ ColumnFamilyDescriptorBuilder cfBuilder =
+ ColumnFamilyDescriptorBuilder.newBuilder(columnFamily);
+ cfBuilder.setDataBlockEncoding(dataBlockEncoding);
+ cfBuilder.setCompressionType(compression);
+ return createPreSplitLoadTestTable(conf, builder.build(), cfBuilder.build(),
numRegionsPerServer);
}
@@ -3914,24 +3840,21 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
* logs a warning and continues.
* @return the number of regions the table was split into
*/
- public static int createPreSplitLoadTestTable(Configuration conf,
- TableName tableName, byte[][] columnFamilies, Algorithm compression,
- DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication,
- Durability durability)
- throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setDurability(durability);
- tableDescriptor.setRegionReplication(regionReplication);
+ public static int createPreSplitLoadTestTable(Configuration conf, TableName tableName,
+ byte[][] columnFamilies, Algorithm compression, DataBlockEncoding dataBlockEncoding,
+ int numRegionsPerServer, int regionReplication, Durability durability) throws IOException {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+ builder.setDurability(durability);
+ builder.setRegionReplication(regionReplication);
ColumnFamilyDescriptor[] hcds = new ColumnFamilyDescriptor[columnFamilies.length];
for (int i = 0; i < columnFamilies.length; i++) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamilies[i]);
- familyDescriptor.setDataBlockEncoding(dataBlockEncoding);
- familyDescriptor.setCompressionType(compression);
- hcds[i] = familyDescriptor;
+ ColumnFamilyDescriptorBuilder cfBuilder =
+ ColumnFamilyDescriptorBuilder.newBuilder(columnFamilies[i]);
+ cfBuilder.setDataBlockEncoding(dataBlockEncoding);
+ cfBuilder.setCompressionType(compression);
+ hcds[i] = cfBuilder.build();
}
- return createPreSplitLoadTestTable(conf, tableDescriptor, hcds, numRegionsPerServer);
+ return createPreSplitLoadTestTable(conf, builder.build(), hcds, numRegionsPerServer);
}
/**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestColumnFamilyDescriptorDefaultVersions.java
similarity index 72%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/TestColumnFamilyDescriptorDefaultVersions.java
index 0416b1a..eab1916 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestColumnFamilyDescriptorDefaultVersions.java
@@ -42,15 +42,15 @@ 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
+ * Verify that the ColumnFamilyDescriptor version is set correctly by default, hbase-site.xml, and
+ * user input
*/
-@Category({MiscTests.class, MediumTests.class})
-public class TestHColumnDescriptorDefaultVersions {
+@Category({ MiscTests.class, MediumTests.class })
+public class TestColumnFamilyDescriptorDefaultVersions {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestHColumnDescriptorDefaultVersions.class);
+ HBaseClassTestRule.forClass(TestColumnFamilyDescriptorDefaultVersions.class);
@Rule
public TestName name = new TestName();
@@ -60,7 +60,6 @@ public class TestHColumnDescriptorDefaultVersions {
/**
* Start up a mini cluster and put a small table of empty regions into it.
- * @throws Exception
*/
@BeforeClass
public static void beforeAllTests() throws Exception {
@@ -82,11 +81,8 @@ public class TestHColumnDescriptorDefaultVersions {
public void testCreateTableWithDefault() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
// Create a table with one family
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -105,12 +101,10 @@ public class TestHColumnDescriptorDefaultVersions {
Admin admin = TEST_UTIL.getAdmin();
// Create a table with one family
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
- .setMaxVersions(TEST_UTIL.getConfiguration().getInt("hbase.column.max.version", 1));
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY)
+ .setMaxVersions(TEST_UTIL.getConfiguration().getInt("hbase.column.max.version", 1)).build())
+ .build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -129,12 +123,9 @@ public class TestHColumnDescriptorDefaultVersions {
Admin admin = TEST_UTIL.getAdmin();
// Create a table with one family
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
- .setMaxVersions(5);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(5).build())
+ .build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -148,41 +139,39 @@ public class TestHColumnDescriptorDefaultVersions {
@Test
public void testHColumnDescriptorCachedMaxVersions() throws Exception {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
- familyDescriptor.setMaxVersions(5);
+ ColumnFamilyDescriptor familyDescriptor =
+ ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(5).build();
// Verify the max version
assertEquals(5, familyDescriptor.getMaxVersions());
// modify the max version
- familyDescriptor.setValue(Bytes.toBytes(HConstants.VERSIONS), Bytes.toBytes("8"));
+ familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(familyDescriptor)
+ .setValue(Bytes.toBytes(HConstants.VERSIONS), Bytes.toBytes("8")).build();
// Verify the max version
assertEquals(8, familyDescriptor.getMaxVersions());
}
private void verifyHColumnDescriptor(int expected, final TableName tableName,
- final byte[]... families) throws IOException {
+ final byte[]... families) throws IOException {
Admin admin = TEST_UTIL.getAdmin();
// Verify descriptor from master
TableDescriptor htd = admin.getDescriptor(tableName);
ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies();
- verifyHColumnDescriptor(expected, hcds, tableName, families);
+ verifyColumnFamilyDescriptor(expected, hcds, tableName, families);
// Verify descriptor from HDFS
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName);
TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
hcds = td.getColumnFamilies();
- verifyHColumnDescriptor(expected, hcds, tableName, families);
+ verifyColumnFamilyDescriptor(expected, hcds, tableName, families);
}
- private void verifyHColumnDescriptor(int expected, final ColumnFamilyDescriptor[] hcds,
- final TableName tableName,
- final byte[]... families) {
+ private void verifyColumnFamilyDescriptor(int expected, final ColumnFamilyDescriptor[] hcds,
+ final TableName tableName, final byte[]... families) {
for (ColumnFamilyDescriptor hcd : hcds) {
assertEquals(expected, hcd.getMaxVersions());
}
}
-
}
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 5a36b81..edf553b 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
@@ -35,6 +35,7 @@ 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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -98,15 +99,11 @@ public class TestMultiVersions {
*/
@Test
public void testTimestamps() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- TimestampTestBase.FAMILY_NAME);
-
- familyDescriptor.setMaxVersions(3);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(TimestampTestBase.FAMILY_NAME)
+ .setMaxVersions(3).build())
+ .build();
this.admin.createTable(tableDescriptor);
Table table = UTIL.getConnection().getTable(tableDescriptor.getTableName());
// TODO: Remove these deprecated classes or pull them in here if this is
@@ -144,14 +141,11 @@ public class TestMultiVersions {
final byte [] value2 = Bytes.toBytes("value2");
final long timestamp1 = 100L;
final long timestamp2 = 200L;
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(contents);
-
- familyDescriptor.setMaxVersions(3);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(contents)
+ .setMaxVersions(3).build())
+ .build();
this.admin.createTable(tableDescriptor);
Put put = new Put(row, timestamp1);
put.addColumn(contents, contents, value1);
@@ -205,21 +199,14 @@ public class TestMultiVersions {
* crazyness.
*
* <p>Tests five cases of scans and timestamps.
- * @throws Exception
*/
@Test
public void testScanMultipleVersions() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
- final byte [][] rows = new byte[][] {
- Bytes.toBytes("row_0200"),
- Bytes.toBytes("row_0800")
- };
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
+
+ final byte[][] rows = new byte[][] { Bytes.toBytes("row_0200"), Bytes.toBytes("row_0800") };
final byte [][] splitRows = new byte[][] {Bytes.toBytes("row_0500")};
final long [] timestamp = new long[] {100L, 1000L};
this.admin.createTable(tableDescriptor, splitRows);
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 170182e..08ef065 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
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.FlakeyTests;
@@ -77,7 +78,7 @@ public class TestRegionRebalancing {
private static final Logger LOG = LoggerFactory.getLogger(TestRegionRebalancing.class);
private final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private RegionLocator regionLocator;
- private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
+ private TableDescriptor tableDescriptor;
private String balancerName;
public TestRegionRebalancing(String balancerName) {
@@ -96,10 +97,8 @@ public class TestRegionRebalancing {
// set minCostNeedBalance to 0, make sure balancer run
UTIL.startMiniCluster(1);
- this.tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf("test"));
- this.tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME));
+ this.tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf("test"))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_NAME)).build();
}
/**
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 d7d0542..69cf139 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
@@ -141,9 +141,9 @@ public class TestSerialization {
@Test
public void testTableDescriptor() throws Exception {
final String name = "testTableDescriptor";
- HTableDescriptor htd = createTableDescriptor(name);
- byte[] mb = htd.toByteArray();
- HTableDescriptor deserializedHtd = HTableDescriptor.parseFrom(mb);
+ TableDescriptor htd = createTableDescriptor(name);
+ byte[] mb = TableDescriptorBuilder.toByteArray(htd);
+ TableDescriptor deserializedHtd = TableDescriptorBuilder.parseFrom(mb);
assertEquals(htd.getTableName(), deserializedHtd.getTableName());
}
@@ -296,7 +296,7 @@ public class TestSerialization {
* @param name Name to give table.
* @return Column descriptor.
*/
- protected HTableDescriptor createTableDescriptor(final String name) {
+ protected TableDescriptor createTableDescriptor(final String name) {
return createTableDescriptor(name, MAXVERSIONS);
}
@@ -306,18 +306,15 @@ public class TestSerialization {
* @param versions How many versions to allow per column.
* @return Column descriptor.
*/
- protected HTableDescriptor createTableDescriptor(final String name, final int versions) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name));
- tableDescriptor
- .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1)
- .setMaxVersions(versions).setBlockCacheEnabled(false));
- tableDescriptor
- .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2)
- .setMaxVersions(versions).setBlockCacheEnabled(false));
- tableDescriptor
- .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3)
- .setMaxVersions(versions).setBlockCacheEnabled(false));
- return new HTableDescriptor(tableDescriptor);
+ protected TableDescriptor createTableDescriptor(final String name, final int versions) {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf(name));
+ builder
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam1).setMaxVersions(versions)
+ .setBlockCacheEnabled(false).build())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam2).setMaxVersions(versions)
+ .setBlockCacheEnabled(false).build())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam3).setMaxVersions(versions)
+ .setBlockCacheEnabled(false).build());
+ return builder.build();
}
}
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 09c9ab2..5bb1cc2 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -36,7 +37,6 @@ import java.util.stream.Collectors;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
@@ -85,8 +85,8 @@ public class TestAdmin2 extends TestAdminBase {
public void testCreateBadTables() throws IOException {
String msg = null;
try {
- ADMIN.createTable(new HTableDescriptor(TableName.META_TABLE_NAME));
- } catch(TableExistsException e) {
+ ADMIN.createTable(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build());
+ } catch (TableExistsException e) {
msg = e.toString();
}
assertTrue("Unexcepted exception message " + msg, msg != null &&
@@ -94,13 +94,9 @@ public class TestAdmin2 extends TestAdminBase {
msg.contains(TableName.META_TABLE_NAME.getNameAsString()));
// Now try and do concurrent creation with a bunch of threads.
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
int count = 10;
Thread [] threads = new Thread [count];
final AtomicInteger successes = new AtomicInteger(0);
@@ -141,22 +137,15 @@ public class TestAdmin2 extends TestAdminBase {
/**
* Test for hadoop-1581 'HBASE: Unopenable tablename bug'.
- * @throws Exception
*/
@Test
public void testTableNameClash() throws Exception {
final String name = this.name.getMethodName();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor1 =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name + "SOMEUPPERCASE"));
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor2 =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name));
- tableDescriptor1.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
- tableDescriptor2.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor1 =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name + "SOMEUPPERCASE"))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
+ TableDescriptor tableDescriptor2 = TableDescriptorBuilder.newBuilder(TableName.valueOf(name))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
ADMIN.createTable(tableDescriptor1);
ADMIN.createTable(tableDescriptor2);
// Before fix, below would fail throwing a NoServerForRegionException.
@@ -181,11 +170,8 @@ public class TestAdmin2 extends TestAdminBase {
byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
Admin hbaseadmin = TEST_UTIL.getAdmin();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf(name))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
hbaseadmin.createTable(tableDescriptor, startKey, endKey, expectedRegions);
} finally {
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, oldTimeout);
@@ -214,26 +200,18 @@ public class TestAdmin2 extends TestAdminBase {
*/
@Test
public void testTableNames() throws IOException {
- byte[][] illegalNames = new byte[][] {
- Bytes.toBytes("-bad"),
- Bytes.toBytes(".bad")
- };
+ byte[][] illegalNames = new byte[][] { Bytes.toBytes("-bad"), Bytes.toBytes(".bad") };
for (byte[] illegalName : illegalNames) {
- try {
- new HTableDescriptor(TableName.valueOf(illegalName));
- throw new IOException("Did not detect '" +
- Bytes.toString(illegalName) + "' as an illegal user table name");
- } catch (IllegalArgumentException e) {
- // expected
- }
+ assertThrows(
+ "Did not detect '" + Bytes.toString(illegalName) + "' as an illegal user table name",
+ IllegalArgumentException.class, () -> TableName.valueOf(illegalName));
}
byte[] legalName = Bytes.toBytes("g-oo.d");
try {
- new HTableDescriptor(TableName.valueOf(legalName));
+ TableName.valueOf(legalName);
} catch (IllegalArgumentException e) {
- throw new IOException("Legal user table name: '" +
- Bytes.toString(legalName) + "' caused IllegalArgumentException: " +
- e.getMessage());
+ fail("Legal user table name: '" + Bytes.toString(legalName) +
+ "' caused IllegalArgumentException: " + e.getMessage());
}
}
@@ -404,11 +382,8 @@ public class TestAdmin2 extends TestAdminBase {
byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
ADMIN.createTable(tableDescriptor, startKey, endKey, expectedRegions);
List<RegionInfo> RegionInfos = ADMIN.getRegions(tableName);
@@ -505,11 +480,8 @@ public class TestAdmin2 extends TestAdminBase {
TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close();
// Create the test table and open it
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
ADMIN.createTable(tableDescriptor);
Table table = TEST_UTIL.getConnection().getTable(tableName);
@@ -541,14 +513,9 @@ public class TestAdmin2 extends TestAdminBase {
}
// Before the fix for HBASE-6146, the below table creation was failing as the hbase:meta table
// actually getting disabled by the disableTable() call.
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(Bytes.toBytes(name.getMethodName())));
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- Bytes.toBytes("cf1"));
-
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(Bytes.toBytes(name.getMethodName())))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf1"))).build();
TEST_UTIL.getAdmin().createTable(tableDescriptor);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
index 6ba960b..4a71baf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
@@ -19,9 +19,9 @@ package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -147,9 +146,8 @@ public class TestAsyncTableAdminApi3 extends TestAsyncAdminBase {
}
TableDescriptor desc = builder.build();
admin.createTable(desc).join();
- ModifyableTableDescriptor modifyableDesc = ((ModifyableTableDescriptor) desc);
TableDescriptor confirmedHtd = admin.getDescriptor(tableName).get();
- assertEquals(0, modifyableDesc.compareTo((ModifyableTableDescriptor) confirmedHtd));
+ assertEquals(0, TableDescriptor.COMPARATOR.compare(desc, confirmedHtd));
}
@Test
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 dbf0ce8..b8c994c 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
@@ -36,8 +36,6 @@ 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;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -575,14 +573,15 @@ public class TestBlockEvictionFromClient {
Table table = null;
try {
final TableName tableName = TableName.valueOf(name.getMethodName());
- HTableDescriptor desc = TEST_UTIL.createTableDescriptor(tableName);
+ TableDescriptor desc = TEST_UTIL.createTableDescriptor(tableName);
// This test expects rpc refcount of cached data blocks to be 0 after split. After split,
// two daughter regions are opened and a compaction is scheduled to get rid of reference
// of the parent region hfiles. Compaction will increase refcount of cached data blocks by 1.
// It is flakey since compaction can kick in anytime. To solve this issue, table is created
// with compaction disabled.
- desc.setCompactionEnabled(false);
- table = TEST_UTIL.createTable(desc, FAMILIES_1, null, BloomType.ROW, 1024, null);
+ table = TEST_UTIL.createTable(
+ TableDescriptorBuilder.newBuilder(desc).setCompactionEnabled(false).build(), FAMILIES_1,
+ null, BloomType.ROW, 1024, null);
// get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegion().getEncodedName();
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 92bc15e..166fade 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
@@ -76,23 +76,18 @@ public class TestEnableTable {
* We were only clearing rows that had a hregioninfo column in hbase:meta. Mangled rows that
* were missing the hregioninfo because of error were being left behind messing up any
* subsequent table made with the same name. HBASE-12980
- * @throws IOException
- * @throws InterruptedException
*/
@Test
public void testDeleteForSureClearsAllTableRowsFromMeta()
- throws IOException, InterruptedException {
+ throws IOException, InterruptedException {
final TableName tableName = TableName.valueOf(name.getMethodName());
final Admin admin = TEST_UTIL.getAdmin();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILYNAME);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILYNAME)).build();
try {
createTable(TEST_UTIL, tableDescriptor, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
} catch (Exception e) {
- e.printStackTrace();
+ LOG.error("", e);
fail("Got an exception while creating " + tableName);
}
// Now I have a nice table, mangle it by removing the HConstants.REGIONINFO_QUALIFIER_STR
@@ -115,7 +110,7 @@ public class TestEnableTable {
try {
deleteTable(TEST_UTIL, tableName);
} catch (Exception e) {
- e.printStackTrace();
+ LOG.error("", e);
fail("Got an exception while deleting " + tableName);
}
int rowCount = 0;
@@ -163,13 +158,12 @@ public class TestEnableTable {
}
}
- public static void createTable(HBaseTestingUtility testUtil,
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, byte [][] splitKeys)
- throws Exception {
+ public static void createTable(HBaseTestingUtility testUtil, TableDescriptor tableDescriptor,
+ byte[][] splitKeys) throws Exception {
// NOTE: We need a latch because admin is not sync,
// so the postOp coprocessor method may be called after the admin operation returned.
- MasterSyncObserver observer = testUtil.getHBaseCluster().getMaster()
- .getMasterCoprocessorHost().findCoprocessor(MasterSyncObserver.class);
+ MasterSyncObserver observer = testUtil.getHBaseCluster().getMaster().getMasterCoprocessorHost()
+ .findCoprocessor(MasterSyncObserver.class);
observer.tableCreationLatch = new CountDownLatch(1);
Admin admin = testUtil.getAdmin();
if (splitKeys != null) {
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 70d832f..bd56e5f 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
@@ -113,18 +113,18 @@ public class TestFromClientSide extends FromClientSideBase {
*/
@Test
public void testDuplicateAppend() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor mtd = TEST_UTIL
+ TableDescriptorBuilder builder = TEST_UTIL
.createModifyableTableDescriptor(name.getTableName(),
ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
Map<String, String> kvs = new HashMap<>();
kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
- mtd.setCoprocessor(CoprocessorDescriptorBuilder
+ builder.setCoprocessor(CoprocessorDescriptorBuilder
.newBuilder(SleepAtFirstRpcCall.class.getName())
.setPriority(1)
.setProperties(kvs)
.build());
- TEST_UTIL.createTable(mtd, new byte[][] { ROW }).close();
+ TEST_UTIL.createTable(builder.build(), new byte[][] { ROW }).close();
Configuration c = new Configuration(TEST_UTIL.getConfiguration());
c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
@@ -156,18 +156,18 @@ public class TestFromClientSide extends FromClientSideBase {
*/
@Test
public void testDuplicateBatchAppend() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor mtd = TEST_UTIL
+ TableDescriptorBuilder builder = TEST_UTIL
.createModifyableTableDescriptor(name.getTableName(),
ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
Map<String, String> kvs = new HashMap<>();
kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
- mtd.setCoprocessor(CoprocessorDescriptorBuilder
+ builder.setCoprocessor(CoprocessorDescriptorBuilder
.newBuilder(SleepAtFirstRpcCall.class.getName())
.setPriority(1)
.setProperties(kvs)
.build());
- TEST_UTIL.createTable(mtd, new byte[][] { ROW }).close();
+ TEST_UTIL.createTable(builder.build(), new byte[][] { ROW }).close();
Configuration c = new Configuration(TEST_UTIL.getConfiguration());
c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
@@ -200,7 +200,8 @@ public class TestFromClientSide extends FromClientSideBase {
/**
* Basic client side validation of HBASE-4536
*/
- @Test public void testKeepDeletedCells() throws Exception {
+ @Test
+ public void testKeepDeletedCells() throws Exception {
final TableName tableName = name.getTableName();
final byte[] FAMILY = Bytes.toBytes("family");
final byte[] C0 = Bytes.toBytes("c0");
@@ -208,13 +209,11 @@ public class TestFromClientSide extends FromClientSideBase {
final byte[] T1 = Bytes.toBytes("T1");
final byte[] T2 = Bytes.toBytes("T2");
final byte[] T3 = Bytes.toBytes("T3");
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
- .setKeepDeletedCells(KeepDeletedCells.TRUE).setMaxVersions(3);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY)
+ .setKeepDeletedCells(KeepDeletedCells.TRUE).setMaxVersions(3).build())
+ .build();
TEST_UTIL.getAdmin().createTable(tableDescriptor);
try (Table h = TEST_UTIL.getConnection().getTable(tableName)) {
long ts = System.currentTimeMillis();
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 f600ab7..5e127d6 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
@@ -43,10 +43,8 @@ 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;
import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@@ -308,8 +306,8 @@ public class TestFromClientSide3 {
// change the compaction.min config option for this table to 5
LOG.info("hbase.hstore.compaction.min should now be 5");
- HTableDescriptor htd = new HTableDescriptor(table.getDescriptor());
- htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
+ TableDescriptor htd = TableDescriptorBuilder.newBuilder(table.getDescriptor())
+ .setValue("hbase.hstore.compaction.min", String.valueOf(5)).build();
admin.modifyTable(htd);
LOG.info("alter status finished");
@@ -327,9 +325,10 @@ public class TestFromClientSide3 {
// change an individual CF's config option to 2 & online schema update
LOG.info("hbase.hstore.compaction.min should now be 2");
- HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
- hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2));
- htd.modifyFamily(hcd);
+ htd = TableDescriptorBuilder.newBuilder(htd)
+ .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(htd.getColumnFamily(FAMILY))
+ .setValue("hbase.hstore.compaction.min", String.valueOf(2)).build())
+ .build();
admin.modifyTable(htd);
LOG.info("alter status finished");
@@ -356,9 +355,10 @@ public class TestFromClientSide3 {
// Finally, ensure that we can remove a custom config value after we made it
LOG.info("Removing CF config value");
LOG.info("hbase.hstore.compaction.min should now be 5");
- hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
- hcd.setValue("hbase.hstore.compaction.min", null);
- htd.modifyFamily(hcd);
+ htd = TableDescriptorBuilder.newBuilder(htd)
+ .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(htd.getColumnFamily(FAMILY))
+ .setValue("hbase.hstore.compaction.min", null).build())
+ .build();
admin.modifyTable(htd);
LOG.info("alter status finished");
assertNull(table.getDescriptor().getColumnFamily(FAMILY)
@@ -735,14 +735,10 @@ public class TestFromClientSide3 {
});
}
- private void testPreBatchMutate(TableName tableName, Runnable rn)throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
- tableDescriptor.setCoprocessor(WaitingForScanObserver.class.getName());
- tableDescriptor.setColumnFamily(familyDescriptor);
+ private void testPreBatchMutate(TableName tableName, Runnable rn) throws Exception {
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY))
+ .setCoprocessor(WaitingForScanObserver.class.getName()).build();
TEST_UTIL.getAdmin().createTable(tableDescriptor);
// Don't use waitTableAvailable(), because the scanner will mess up the co-processor
@@ -774,14 +770,10 @@ public class TestFromClientSide3 {
@Test
public void testLockLeakWithDelta() throws Exception, Throwable {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
- tableDescriptor.setCoprocessor(WaitingForMultiMutationsObserver.class.getName());
- tableDescriptor.setValue("hbase.rowlock.wait.duration", String.valueOf(5000));
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY))
+ .setCoprocessor(WaitingForMultiMutationsObserver.class.getName())
+ .setValue("hbase.rowlock.wait.duration", String.valueOf(5000)).build();
TEST_UTIL.getAdmin().createTable(tableDescriptor);
TEST_UTIL.waitTableAvailable(tableName, WAITTABLE_MILLIS);
@@ -833,15 +825,11 @@ public class TestFromClientSide3 {
@Test
public void testMultiRowMutations() throws Exception, Throwable {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
- tableDescriptor.setCoprocessor(MultiRowMutationEndpoint.class.getName());
- tableDescriptor.setCoprocessor(WaitingForMultiMutationsObserver.class.getName());
- tableDescriptor.setValue("hbase.rowlock.wait.duration", String.valueOf(5000));
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY))
+ .setCoprocessor(MultiRowMutationEndpoint.class.getName())
+ .setCoprocessor(WaitingForMultiMutationsObserver.class.getName())
+ .setValue("hbase.rowlock.wait.duration", String.valueOf(5000)).build();
TEST_UTIL.getAdmin().createTable(tableDescriptor);
TEST_UTIL.waitTableAvailable(tableName, WAITTABLE_MILLIS);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide4.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide4.java
index 78aa56d..4c5985a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide4.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide4.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
+
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -37,9 +38,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
@@ -1309,29 +1308,31 @@ public class TestFromClientSide4 extends FromClientSideBase {
// Test user metadata
Admin admin = TEST_UTIL.getAdmin();
// make a modifiable descriptor
- HTableDescriptor desc = new HTableDescriptor(a.getDescriptor());
+ TableDescriptor desc = a.getDescriptor();
// offline the table
admin.disableTable(tableAname);
// add a user attribute to HTD
- desc.setValue(attrName, attrValue);
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(desc).setValue(attrName, attrValue);
// add a user attribute to HCD
- for (HColumnDescriptor c : desc.getFamilies()) {
- c.setValue(attrName, attrValue);
+ for (ColumnFamilyDescriptor c : desc.getColumnFamilies()) {
+ builder.modifyColumnFamily(
+ ColumnFamilyDescriptorBuilder.newBuilder(c).setValue(attrName, attrValue).build());
}
// update metadata for all regions of this table
- admin.modifyTable(desc);
+ admin.modifyTable(builder.build());
// enable the table
admin.enableTable(tableAname);
// Test that attribute changes were applied
- desc = new HTableDescriptor(a.getDescriptor());
+ desc = a.getDescriptor();
assertEquals("wrong table descriptor returned", desc.getTableName(), tableAname);
// check HTD attribute
value = desc.getValue(attrName);
assertNotNull("missing HTD attribute value", value);
assertFalse("HTD attribute value is incorrect", Bytes.compareTo(value, attrValue) != 0);
// check HCD attribute
- for (HColumnDescriptor c : desc.getFamilies()) {
+ for (ColumnFamilyDescriptor c : desc.getColumnFamilies()) {
value = c.getValue(attrName);
assertNotNull("missing HCD attribute value", value);
assertFalse("HCD attribute value is incorrect", Bytes.compareTo(value, attrValue) != 0);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java
index 6fb50bd..3213c0f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java
@@ -2240,13 +2240,9 @@ public class TestFromClientSide5 extends FromClientSideBase {
@Test
public void testCellSizeLimit() throws IOException {
final TableName tableName = name.getTableName();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
- .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024));
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
try (Admin admin = TEST_UTIL.getAdmin()) {
admin.createTable(tableDescriptor);
}
@@ -2279,12 +2275,9 @@ public class TestFromClientSide5 extends FromClientSideBase {
@Test
public void testCellSizeNoLimit() throws IOException {
final TableName tableName = name.getTableName();
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
- .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(0));
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(0))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
try (Admin admin = TEST_UTIL.getAdmin()) {
admin.createTable(tableDescriptor);
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 021a1d2..1a9e5b5 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
@@ -26,7 +26,6 @@ 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;
@@ -84,107 +83,104 @@ public class TestIllegalTableDescriptor {
@Test
public void testIllegalTableDescriptor() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
+ ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY);
// create table with 0 families
- checkTableIsIllegal(tableDescriptor);
- tableDescriptor.setColumnFamily(familyDescriptor);
- checkTableIsLegal(tableDescriptor);
-
- tableDescriptor.setMaxFileSize(1024); // 1K
- checkTableIsIllegal(tableDescriptor);
- tableDescriptor.setMaxFileSize(0);
- checkTableIsIllegal(tableDescriptor);
- tableDescriptor.setMaxFileSize(1024 * 1024 * 1024); // 1G
- checkTableIsLegal(tableDescriptor);
-
- tableDescriptor.setMemStoreFlushSize(1024);
- checkTableIsIllegal(tableDescriptor);
- tableDescriptor.setMemStoreFlushSize(0);
- checkTableIsIllegal(tableDescriptor);
- tableDescriptor.setMemStoreFlushSize(128 * 1024 * 1024); // 128M
- checkTableIsLegal(tableDescriptor);
-
- tableDescriptor.setRegionSplitPolicyClassName("nonexisting.foo.class");
- checkTableIsIllegal(tableDescriptor);
- tableDescriptor.setRegionSplitPolicyClassName(null);
- checkTableIsLegal(tableDescriptor);
-
- tableDescriptor.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, "nonexisting.foo.class");
- checkTableIsIllegal(tableDescriptor);
- tableDescriptor.removeValue(Bytes.toBytes(HConstants.HBASE_REGION_SPLIT_POLICY_KEY));
- checkTableIsLegal(tableDescriptor);
-
- familyDescriptor.setBlocksize(0);
- checkTableIsIllegal(tableDescriptor);
- familyDescriptor.setBlocksize(1024 * 1024 * 128); // 128M
- checkTableIsIllegal(tableDescriptor);
- familyDescriptor.setBlocksize(1024);
- checkTableIsLegal(tableDescriptor);
-
- familyDescriptor.setTimeToLive(0);
- checkTableIsIllegal(tableDescriptor);
- familyDescriptor.setTimeToLive(-1);
- checkTableIsIllegal(tableDescriptor);
- familyDescriptor.setTimeToLive(1);
- checkTableIsLegal(tableDescriptor);
-
- familyDescriptor.setMinVersions(-1);
- checkTableIsIllegal(tableDescriptor);
- familyDescriptor.setMinVersions(3);
+ checkTableIsIllegal(builder.build());
+ checkTableIsLegal(builder.setColumnFamily(cfBuilder.build()).build());
+
+ builder.setMaxFileSize(1024); // 1K
+ checkTableIsIllegal(builder.build());
+ builder.setMaxFileSize(0);
+ checkTableIsIllegal(builder.build());
+ builder.setMaxFileSize(1024 * 1024 * 1024); // 1G
+ checkTableIsLegal(builder.build());
+
+ builder.setMemStoreFlushSize(1024);
+ checkTableIsIllegal(builder.build());
+ builder.setMemStoreFlushSize(0);
+ checkTableIsIllegal(builder.build());
+ builder.setMemStoreFlushSize(128 * 1024 * 1024); // 128M
+ checkTableIsLegal(builder.build());
+
+ builder.setRegionSplitPolicyClassName("nonexisting.foo.class");
+ checkTableIsIllegal(builder.build());
+ builder.setRegionSplitPolicyClassName(null);
+ checkTableIsLegal(builder.build());
+
+ builder.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, "nonexisting.foo.class");
+ checkTableIsIllegal(builder.build());
+ builder.removeValue(Bytes.toBytes(HConstants.HBASE_REGION_SPLIT_POLICY_KEY));
+ checkTableIsLegal(builder.build());
+
+ cfBuilder.setBlocksize(0);
+ checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+ cfBuilder.setBlocksize(1024 * 1024 * 128); // 128M
+ checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+ cfBuilder.setBlocksize(1024);
+ checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+
+ cfBuilder.setTimeToLive(0);
+ checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+ cfBuilder.setTimeToLive(-1);
+ checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+ cfBuilder.setTimeToLive(1);
+ checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+
+ cfBuilder.setMinVersions(-1);
+ checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+ cfBuilder.setMinVersions(3);
try {
- familyDescriptor.setMaxVersions(2);
+ cfBuilder.setMaxVersions(2);
fail();
} catch (IllegalArgumentException ex) {
// expected
- familyDescriptor.setMaxVersions(10);
+ cfBuilder.setMaxVersions(10);
}
- checkTableIsLegal(tableDescriptor);
+ checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
// HBASE-13776 Setting illegal versions for HColumnDescriptor
// does not throw IllegalArgumentException
// finally, minVersions must be less than or equal to maxVersions
- familyDescriptor.setMaxVersions(4);
- familyDescriptor.setMinVersions(5);
- checkTableIsIllegal(tableDescriptor);
- familyDescriptor.setMinVersions(3);
+ cfBuilder.setMaxVersions(4);
+ cfBuilder.setMinVersions(5);
+ checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+ cfBuilder.setMinVersions(3);
- familyDescriptor.setScope(-1);
- checkTableIsIllegal(tableDescriptor);
- familyDescriptor.setScope(0);
- checkTableIsLegal(tableDescriptor);
+ cfBuilder.setScope(-1);
+ checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+ cfBuilder.setScope(0);
+ checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
- familyDescriptor.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "INVALID");
- checkTableIsIllegal(tableDescriptor);
- familyDescriptor.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "NONE");
- checkTableIsLegal(tableDescriptor);
+ cfBuilder.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "INVALID");
+ checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+ cfBuilder.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "NONE");
+ checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
try {
- familyDescriptor.setDFSReplication((short) -1);
+ cfBuilder.setDFSReplication((short) -1);
fail("Illegal value for setDFSReplication did not throw");
} catch (IllegalArgumentException e) {
// pass
}
// set an illegal DFS replication value by hand
- familyDescriptor.setValue(ColumnFamilyDescriptorBuilder.DFS_REPLICATION, "-1");
- checkTableIsIllegal(tableDescriptor);
+ cfBuilder.setValue(ColumnFamilyDescriptorBuilder.DFS_REPLICATION, "-1");
+ checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
try {
- familyDescriptor.setDFSReplication((short) -1);
+ cfBuilder.setDFSReplication((short) -1);
fail("Should throw exception if an illegal value is explicitly being set");
} catch (IllegalArgumentException e) {
// pass
}
// check the conf settings to disable sanity checks
- tableDescriptor.setMemStoreFlushSize(0);
+ builder.setMemStoreFlushSize(0);
// Check that logs warn on invalid table but allow it.
- tableDescriptor.setValue(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString());
- checkTableIsLegal(tableDescriptor);
+ builder.setValue(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString());
+ checkTableIsLegal(builder.build());
verify(LOGGER).warn(contains("MEMSTORE_FLUSHSIZE for table "
+ "descriptor or \"hbase.hregion.memstore.flush.size\" (0) is too small, which might "
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 58c88b5..237eada 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
@@ -101,16 +101,16 @@ public class TestIncrementsFromClientSide {
*/
@Test
public void testDuplicateIncrement() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor mtd =
+ TableDescriptorBuilder builder =
TEST_UTIL.createModifyableTableDescriptor(name.getMethodName());
Map<String, String> kvs = new HashMap<>();
kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
- mtd.setCoprocessor(CoprocessorDescriptorBuilder
+ builder.setCoprocessor(CoprocessorDescriptorBuilder
.newBuilder(SleepAtFirstRpcCall.class.getName())
.setPriority(1)
.setProperties(kvs)
.build());
- TEST_UTIL.createTable(mtd, new byte[][] { ROW }).close();
+ TEST_UTIL.createTable(builder.build(), new byte[][] { ROW }).close();
Configuration c = new Configuration(TEST_UTIL.getConfiguration());
c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
@@ -141,16 +141,16 @@ public class TestIncrementsFromClientSide {
*/
@Test
public void testDuplicateBatchIncrement() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor mtd =
+ TableDescriptorBuilder builder =
TEST_UTIL.createModifyableTableDescriptor(name.getMethodName());
Map<String, String> kvs = new HashMap<>();
kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
- mtd.setCoprocessor(CoprocessorDescriptorBuilder
+ builder.setCoprocessor(CoprocessorDescriptorBuilder
.newBuilder(SleepAtFirstRpcCall.class.getName())
.setPriority(1)
.setProperties(kvs)
.build());
- TEST_UTIL.createTable(mtd, new byte[][] { ROW }).close();
+ TEST_UTIL.createTable(builder.build(), new byte[][] { ROW }).close();
Configuration c = new Configuration(TEST_UTIL.getConfiguration());
c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
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 3567e50..7cada26 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
@@ -55,18 +55,15 @@ public class TestIntraRowPagination {
byte [][] FAMILIES = HTestConst.makeNAscii(HTestConst.DEFAULT_CF_BYTES, 3);
byte [][] QUALIFIERS = HTestConst.makeNAscii(HTestConst.DEFAULT_QUALIFIER_BYTES, 10);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(HTestConst.DEFAULT_TABLE_BYTES));
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(HTestConst.DEFAULT_TABLE_BYTES));
RegionInfo info = RegionInfoBuilder.newBuilder(HTestConst.DEFAULT_TABLE).build();
for (byte[] family : FAMILIES) {
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
}
HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
- TEST_UTIL.getConfiguration(), tableDescriptor);
+ TEST_UTIL.getConfiguration(), builder.build());
try {
Put put;
Scan scan;
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 5019fb3..3215ea8 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
@@ -34,9 +34,7 @@ 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;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
@@ -278,12 +276,14 @@ public class TestReplicaWithCluster {
@Test
public void testCreateDeleteTable() throws IOException {
// Create table then get the single region for our new table.
- HTableDescriptor hdt = HTU.createTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
- HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
- HColumnDescriptor.DEFAULT_KEEP_DELETED);
- hdt.setRegionReplication(NB_SERVERS);
- hdt.addCoprocessor(SlowMeCopro.class.getName());
- Table table = HTU.createTable(hdt, new byte[][]{f}, null);
+ TableDescriptorBuilder builder =
+ HTU.createModifyableTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
+ ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+ ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+ builder.setRegionReplication(NB_SERVERS);
+ builder.setCoprocessor(SlowMeCopro.class.getName());
+ TableDescriptor hdt = builder.build();
+ Table table = HTU.createTable(hdt, new byte[][] { f }, null);
Put p = new Put(row);
p.addColumn(f, row, row);
@@ -372,17 +372,14 @@ public class TestReplicaWithCluster {
@SuppressWarnings("deprecation")
@Test
public void testReplicaAndReplication() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
+ TableDescriptorBuilder builder =
HTU.createModifyableTableDescriptor("testReplicaAndReplication");
- tableDescriptor.setRegionReplication(NB_SERVERS);
+ builder.setRegionReplication(NB_SERVERS);
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(row)
+ .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build());
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(row);
-
- familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
- tableDescriptor.setColumnFamily(familyDescriptor);
-
- tableDescriptor.setCoprocessor(SlowMeCopro.class.getName());
+ builder.setCoprocessor(SlowMeCopro.class.getName());
+ TableDescriptor tableDescriptor = builder.build();
HTU.getAdmin().createTable(tableDescriptor, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
Configuration conf2 = HBaseConfiguration.create(HTU.getConfiguration());
@@ -461,12 +458,13 @@ public class TestReplicaWithCluster {
public void testBulkLoad() throws IOException {
// Create table then get the single region for our new table.
LOG.debug("Creating test table");
- HTableDescriptor hdt = HTU.createTableDescriptor(TableName.valueOf("testBulkLoad"),
- HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
- HColumnDescriptor.DEFAULT_KEEP_DELETED);
- hdt.setRegionReplication(NB_SERVERS);
- hdt.addCoprocessor(SlowMeCopro.class.getName());
- Table table = HTU.createTable(hdt, new byte[][]{f}, null);
+ TableDescriptorBuilder builder = HTU.createModifyableTableDescriptor(
+ TableName.valueOf("testBulkLoad"), ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3,
+ HConstants.FOREVER, ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+ builder.setRegionReplication(NB_SERVERS);
+ builder.setCoprocessor(SlowMeCopro.class.getName());
+ TableDescriptor hdt = builder.build();
+ Table table = HTU.createTable(hdt, new byte[][] { f }, null);
// create hfiles to load.
LOG.debug("Creating test data");
@@ -475,7 +473,7 @@ public class TestReplicaWithCluster {
final byte[] qual = Bytes.toBytes("qual");
final byte[] val = Bytes.toBytes("val");
Map<byte[], List<Path>> family2Files = new TreeMap<>(Bytes.BYTES_COMPARATOR);
- for (HColumnDescriptor col : hdt.getColumnFamilies()) {
+ for (ColumnFamilyDescriptor col : hdt.getColumnFamilies()) {
Path hfile = new Path(dir, col.getNameAsString());
TestHRegionServerBulkLoad.createHFile(HTU.getTestFileSystem(), hfile, col.getName(), qual,
val, numRows);
@@ -519,11 +517,13 @@ public class TestReplicaWithCluster {
@Test
public void testReplicaGetWithPrimaryDown() throws IOException {
// Create table then get the single region for our new table.
- HTableDescriptor hdt = HTU.createTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
- HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
- HColumnDescriptor.DEFAULT_KEEP_DELETED);
- hdt.setRegionReplication(NB_SERVERS);
- hdt.addCoprocessor(RegionServerStoppedCopro.class.getName());
+ TableDescriptorBuilder builder =
+ HTU.createModifyableTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
+ ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+ ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+ builder.setRegionReplication(NB_SERVERS);
+ builder.setCoprocessor(RegionServerStoppedCopro.class.getName());
+ TableDescriptor hdt = builder.build();
try {
Table table = HTU.createTable(hdt, new byte[][] { f }, null);
@@ -555,12 +555,13 @@ public class TestReplicaWithCluster {
@Test
public void testReplicaScanWithPrimaryDown() throws IOException {
// Create table then get the single region for our new table.
- HTableDescriptor hdt = HTU.createTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
- HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
- HColumnDescriptor.DEFAULT_KEEP_DELETED);
- hdt.setRegionReplication(NB_SERVERS);
- hdt.addCoprocessor(RegionServerStoppedCopro.class.getName());
-
+ TableDescriptorBuilder builder =
+ HTU.createModifyableTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
+ ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+ ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+ builder.setRegionReplication(NB_SERVERS);
+ builder.setCoprocessor(RegionServerStoppedCopro.class.getName());
+ TableDescriptor hdt = builder.build();
try {
Table table = HTU.createTable(hdt, new byte[][] { f }, null);
@@ -601,16 +602,16 @@ public class TestReplicaWithCluster {
@Test
public void testReplicaGetWithAsyncRpcClientImpl() throws IOException {
HTU.getConfiguration().setBoolean("hbase.ipc.client.specificThreadForWriting", true);
- HTU.getConfiguration().set(
- "hbase.rpc.client.impl", "org.apache.hadoop.hbase.ipc.AsyncRpcClient");
+ HTU.getConfiguration().set("hbase.rpc.client.impl",
+ "org.apache.hadoop.hbase.ipc.AsyncRpcClient");
// Create table then get the single region for our new table.
- HTableDescriptor hdt = HTU.createTableDescriptor(
- TableName.valueOf("testReplicaGetWithAsyncRpcClientImpl"),
- HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
- HColumnDescriptor.DEFAULT_KEEP_DELETED);
- hdt.setRegionReplication(NB_SERVERS);
- hdt.addCoprocessor(SlowMeCopro.class.getName());
-
+ TableDescriptorBuilder builder =
+ HTU.createModifyableTableDescriptor(TableName.valueOf("testReplicaGetWithAsyncRpcClientImpl"),
+ ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+ ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+ builder.setRegionReplication(NB_SERVERS);
+ builder.setCoprocessor(SlowMeCopro.class.getName());
+ TableDescriptor hdt = builder.build();
try {
Table table = HTU.createTable(hdt, new byte[][] { f }, null);
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 d7fc8e0..a64f296 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
@@ -31,10 +31,8 @@ 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.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.StartMiniClusterOption;
import org.apache.hadoop.hbase.TableName;
@@ -195,11 +193,12 @@ public class TestReplicasClient {
HTU.startMiniCluster(option);
// Create table then get the single region for our new table.
- HTableDescriptor hdt = HTU.createTableDescriptor(
+ TableDescriptorBuilder builder = HTU.createModifyableTableDescriptor(
TableName.valueOf(TestReplicasClient.class.getSimpleName()),
- HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
- HColumnDescriptor.DEFAULT_KEEP_DELETED);
- hdt.addCoprocessor(SlowMeCopro.class.getName());
+ ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+ ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+ builder.setCoprocessor(SlowMeCopro.class.getName());
+ TableDescriptor hdt = builder.build();
HTU.createTable(hdt, new byte[][]{f}, null);
TABLE_NAME = hdt.getTableName();
try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) {
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 1dce029..00b8a64 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
@@ -768,8 +768,7 @@ public class TestScannersFromClientSide {
table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, ts, value));
assertArrayEquals(value, table.get(new Get(ROW)).getValue(FAMILY, QUALIFIER));
TEST_UTIL.getAdmin().modifyColumnFamily(tableName,
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
- .setTimeToLive(5));
+ ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setTimeToLive(5).build());
try (ResultScanner scanner = table.getScanner(FAMILY)) {
assertNull(scanner.next());
}
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 0dce2f3..9e6f702 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
@@ -28,8 +28,6 @@ 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;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -185,8 +183,8 @@ public class TestServerBusyException {
@Test()
public void testServerBusyException() throws Exception {
- HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(TableName.valueOf(name.getMethodName()));
- hdt.addCoprocessor(SleepCoprocessor.class.getName());
+ TableDescriptor hdt = TEST_UTIL.createModifyableTableDescriptor(name.getMethodName())
+ .setCoprocessor(SleepCoprocessor.class.getName()).build();
Configuration c = new Configuration(TEST_UTIL.getConfiguration());
TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c);
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 ecb6973..85a912a 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
@@ -23,7 +23,6 @@ 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.TableName;
@@ -60,7 +59,6 @@ public class TestSizeFailures {
//((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
//((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
//((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
- Configuration conf = TEST_UTIL.getConfiguration();
TEST_UTIL.startMiniCluster(SLAVES);
// Write a bunch of data
@@ -70,12 +68,8 @@ public class TestSizeFailures {
qualifiers.add(Bytes.toBytes(Integer.toString(i)));
}
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLENAME);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLENAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
byte[][] splits = new byte[9][2];
for (int i = 1; i < 10; i++) {
int split = 48 + i;
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 9b35e70..cfa0b6d 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
@@ -25,7 +25,6 @@ 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.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
@@ -65,7 +64,6 @@ public class TestSnapshotCloneIndependence {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
protected static final int NUM_RS = 2;
- private static final String STRING_TABLE_NAME = "test";
private static final String TEST_FAM_STR = "fam";
protected static final byte[] TEST_FAM = Bytes.toBytes(TEST_FAM_STR);
private static final int CLEANER_INTERVAL = 100;
@@ -321,8 +319,7 @@ public class TestSnapshotCloneIndependence {
private void runTestSnapshotMetadataChangesIndependent() throws Exception {
// Add a new column family to the original table
byte[] TEST_FAM_2 = Bytes.toBytes("fam2");
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAM_2);
+ ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.of(TEST_FAM_2);
admin.disableTable(originalTableName);
admin.addColumnFamily(originalTableName, familyDescriptor);
@@ -334,19 +331,18 @@ public class TestSnapshotCloneIndependence {
// get a description of the cloned table
// get a list of its families
// assert that the family is there
- HTableDescriptor originalTableDescriptor = new HTableDescriptor(originalTable.getDescriptor());
- HTableDescriptor clonedTableDescriptor =
- new HTableDescriptor(admin.getDescriptor(cloneTableName));
+ TableDescriptor originalTableDescriptor = originalTable.getDescriptor();
+ TableDescriptor clonedTableDescriptor = admin.getDescriptor(cloneTableName);
Assert.assertTrue("The original family was not found. There is something wrong. ",
- originalTableDescriptor.hasFamily(TEST_FAM));
+ originalTableDescriptor.hasColumnFamily(TEST_FAM));
Assert.assertTrue("The original family was not found in the clone. There is something wrong. ",
- clonedTableDescriptor.hasFamily(TEST_FAM));
+ clonedTableDescriptor.hasColumnFamily(TEST_FAM));
Assert.assertTrue("The new family was not found. ",
- originalTableDescriptor.hasFamily(TEST_FAM_2));
+ originalTableDescriptor.hasColumnFamily(TEST_FAM_2));
Assert.assertTrue("The new family was not found. ",
- !clonedTableDescriptor.hasFamily(TEST_FAM_2));
+ !clonedTableDescriptor.hasColumnFamily(TEST_FAM_2));
}
/**
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 9c57920..63bf72f 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
@@ -31,7 +31,6 @@ 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.TableNotFoundException;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -119,9 +118,9 @@ public class TestSnapshotFromClient {
}
protected void createTable() throws Exception {
- HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
- htd.setRegionReplication(getNumReplicas());
- UTIL.createTable(htd, new byte[][]{TEST_FAM}, null);
+ TableDescriptor htd =
+ TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(getNumReplicas()).build();
+ UTIL.createTable(htd, new byte[][] { TEST_FAM }, null);
}
protected int getNumReplicas() {
@@ -321,7 +320,7 @@ public class TestSnapshotFromClient {
try {
admin = UTIL.getAdmin();
- HTableDescriptor htd = new HTableDescriptor(tableName);
+ TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName).build();
UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
String table1Snapshot1 = "Table1Snapshot1";
@@ -406,7 +405,7 @@ public class TestSnapshotFromClient {
try {
admin = UTIL.getAdmin();
- HTableDescriptor htd = new HTableDescriptor(tableName);
+ TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName).build();
UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
String table1Snapshot1 = "Table1Snapshot1";
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 77462f8..4c5afb1 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
@@ -29,7 +29,6 @@ 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.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -91,7 +90,7 @@ public class TestSnapshotMetadata {
private Admin admin;
private String originalTableDescription;
- private HTableDescriptor originalTableDescriptor;
+ private TableDescriptor originalTableDescriptor;
TableName originalTableName;
private static FileSystem fs;
@@ -155,35 +154,26 @@ public class TestSnapshotMetadata {
originalTableName = TableName.valueOf(sourceTableNameAsString);
// enable replication on a column family
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor maxVersionsColumn =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(MAX_VERSIONS_FAM);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor bloomFilterColumn =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(BLOOMFILTER_FAM);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor dataBlockColumn =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COMPRESSED_FAM);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor blockSizeColumn =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(BLOCKSIZE_FAM);
-
- maxVersionsColumn.setMaxVersions(MAX_VERSIONS);
- bloomFilterColumn.setBloomFilterType(BLOOM_TYPE);
- dataBlockColumn.setDataBlockEncoding(DATA_BLOCK_ENCODING_TYPE);
- blockSizeColumn.setBlocksize(BLOCK_SIZE);
-
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(sourceTableNameAsString));
- tableDescriptor.setColumnFamily(maxVersionsColumn);
- tableDescriptor.setColumnFamily(bloomFilterColumn);
- tableDescriptor.setColumnFamily(dataBlockColumn);
- tableDescriptor.setColumnFamily(blockSizeColumn);
- tableDescriptor.setValue(TEST_CUSTOM_VALUE, TEST_CUSTOM_VALUE);
- tableDescriptor.setValue(TEST_CONF_CUSTOM_VALUE, TEST_CONF_CUSTOM_VALUE);
- assertTrue(tableDescriptor.getConfiguration().size() > 0);
+ ColumnFamilyDescriptor maxVersionsColumn = ColumnFamilyDescriptorBuilder
+ .newBuilder(MAX_VERSIONS_FAM).setMaxVersions(MAX_VERSIONS).build();
+ ColumnFamilyDescriptor bloomFilterColumn = ColumnFamilyDescriptorBuilder
+ .newBuilder(BLOOMFILTER_FAM).setBloomFilterType(BLOOM_TYPE).build();
+ ColumnFamilyDescriptor dataBlockColumn = ColumnFamilyDescriptorBuilder
+ .newBuilder(COMPRESSED_FAM).setDataBlockEncoding(DATA_BLOCK_ENCODING_TYPE).build();
+ ColumnFamilyDescriptor blockSizeColumn =
+ ColumnFamilyDescriptorBuilder.newBuilder(BLOCKSIZE_FAM).setBlocksize(BLOCK_SIZE).build();
+
+ TableDescriptor tableDescriptor = TableDescriptorBuilder
+ .newBuilder(TableName.valueOf(sourceTableNameAsString)).setColumnFamily(maxVersionsColumn)
+ .setColumnFamily(bloomFilterColumn).setColumnFamily(dataBlockColumn)
+ .setColumnFamily(blockSizeColumn).setValue(TEST_CUSTOM_VALUE, TEST_CUSTOM_VALUE)
+ .setValue(TEST_CONF_CUSTOM_VALUE, TEST_CONF_CUSTOM_VALUE).build();
+ assertTrue(tableDescriptor.getValues().size() > 0);
admin.createTable(tableDescriptor);
Table original = UTIL.getConnection().getTable(originalTableName);
originalTableName = TableName.valueOf(sourceTableNameAsString);
- originalTableDescriptor = new HTableDescriptor(admin.getDescriptor(originalTableName));
+ originalTableDescriptor = admin.getDescriptor(originalTableName);
originalTableDescription = originalTableDescriptor.toStringCustomizedValues();
original.close();
@@ -212,7 +202,7 @@ public class TestSnapshotMetadata {
admin.cloneSnapshot(snapshotName, clonedTableName);
Table clonedTable = UTIL.getConnection().getTable(clonedTableName);
- HTableDescriptor cloneHtd = new HTableDescriptor(admin.getDescriptor(clonedTableName));
+ TableDescriptor cloneHtd = admin.getDescriptor(clonedTableName);
assertEquals(
originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString),
cloneHtd.toStringCustomizedValues());
@@ -220,12 +210,9 @@ public class TestSnapshotMetadata {
// Verify the custom fields
assertEquals(originalTableDescriptor.getValues().size(),
cloneHtd.getValues().size());
- assertEquals(originalTableDescriptor.getConfiguration().size(),
- cloneHtd.getConfiguration().size());
assertEquals(TEST_CUSTOM_VALUE, cloneHtd.getValue(TEST_CUSTOM_VALUE));
- assertEquals(TEST_CONF_CUSTOM_VALUE, cloneHtd.getConfigurationValue(TEST_CONF_CUSTOM_VALUE));
+ assertEquals(TEST_CONF_CUSTOM_VALUE, cloneHtd.getValue(TEST_CONF_CUSTOM_VALUE));
assertEquals(originalTableDescriptor.getValues(), cloneHtd.getValues());
- assertEquals(originalTableDescriptor.getConfiguration(), cloneHtd.getConfiguration());
admin.enableTable(originalTableName);
clonedTable.close();
@@ -304,8 +291,7 @@ public class TestSnapshotMetadata {
final byte[] newFamilyName = Bytes.toBytes(newFamilyNameAsString);
admin.disableTable(originalTableName);
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(newFamilyName);
+ ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.of(newFamilyName);
admin.addColumnFamily(originalTableName, familyDescriptor);
assertTrue("New column family was not added.",
admin.getDescriptor(originalTableName).toString().contains(newFamilyNameAsString));
@@ -321,9 +307,8 @@ public class TestSnapshotMetadata {
// verify that the descrption is reverted
try (Table original = UTIL.getConnection().getTable(originalTableName)) {
- assertEquals(originalTableDescriptor,
- new HTableDescriptor(admin.getDescriptor(originalTableName)));
- assertEquals(originalTableDescriptor, new HTableDescriptor(original.getDescriptor()));
+ assertEquals(originalTableDescriptor, admin.getDescriptor(originalTableName));
+ assertEquals(originalTableDescriptor, original.getDescriptor());
}
}
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
index e61b1d2..aa2ecd4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
@@ -37,7 +37,6 @@ 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.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
@@ -128,8 +127,8 @@ public class TestSnapshotTemporaryDirectory {
@Before
public void setup() throws Exception {
- HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
- htd.setRegionReplication(getNumReplicas());
+ TableDescriptor htd =
+ TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(getNumReplicas()).build();
UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
}
@@ -426,7 +425,7 @@ public class TestSnapshotTemporaryDirectory {
Admin admin = UTIL.getAdmin();
TableName tableName2 = TableName.valueOf("testListTableSnapshots");
try {
- HTableDescriptor htd = new HTableDescriptor(tableName2);
+ TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName2).build();
UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
String table1Snapshot1 = "Table1Snapshot1";
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 94b3948..84d4046 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
@@ -29,10 +29,11 @@ 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.HTableDescriptor;
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.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -70,10 +71,11 @@ public class TestCoprocessorConfiguration {
}
private static final TableName TABLENAME = TableName.valueOf("TestCoprocessorConfiguration");
private static final RegionInfo REGIONINFO = RegionInfoBuilder.newBuilder(TABLENAME).build();
- private static final HTableDescriptor TABLEDESC = new HTableDescriptor(TABLENAME);
+ private static final TableDescriptor TABLEDESC;
static {
try {
- TABLEDESC.addCoprocessor(TableCoprocessor.class.getName());
+ TABLEDESC = TableDescriptorBuilder.newBuilder(TABLENAME)
+ .setCoprocessor(TableCoprocessor.class.getName()).build();
} catch (IOException e) {
throw new RuntimeException(e);
}
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 03e031a..a21b63a 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
@@ -409,14 +409,11 @@ public class TestCoprocessorInterface {
return r;
}
- HRegion initHRegion (TableName tableName, String callingMethod,
- Configuration conf, Class<?> [] implClasses, byte [][] families)
- throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+ HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
+ Class<?>[] implClasses, byte[][] families) throws IOException {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
for (byte[] family : families) {
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
}
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
RegionInfo info = RegionInfoBuilder.newBuilder(tableName)
@@ -425,7 +422,7 @@ public class TestCoprocessorInterface {
.setSplit(false)
.build();
Path path = new Path(DIR + callingMethod);
- HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, tableDescriptor);
+ HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, builder.build());
// this following piece is a hack.
RegionCoprocessorHost host =
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 0ca7e0c..d7ac38a 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
@@ -398,12 +398,11 @@ public class TestCoprocessorMetrics {
public void testRegionObserverSingleRegion() throws IOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
- Admin admin = connection.getAdmin()) {
- admin.createTable(
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
- .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
- // add the coprocessor for the region
- .setCoprocessor(CustomRegionObserver.class.getName()));
+ Admin admin = connection.getAdmin()) {
+ admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+ // add the coprocessor for the region
+ .setCoprocessor(CustomRegionObserver.class.getName()).build());
try (Table table = connection.getTable(tableName)) {
table.get(new Get(foo));
table.get(new Get(foo)); // 2 gets
@@ -418,12 +417,10 @@ public class TestCoprocessorMetrics {
final TableName tableName = TableName.valueOf(name.getMethodName());
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
Admin admin = connection.getAdmin()) {
- admin.createTable(
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
- .setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
- // add the coprocessor for the region
- .setCoprocessor(CustomRegionObserver.class.getName()), new byte[][]{foo});
+ admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+ // add the coprocessor for the region
+ .setCoprocessor(CustomRegionObserver.class.getName()).build(), new byte[][] { foo });
// create with 2 regions
try (Table table = connection.getTable(tableName);
RegionLocator locator = connection.getRegionLocator(tableName)) {
@@ -444,18 +441,14 @@ public class TestCoprocessorMetrics {
final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2");
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
Admin admin = connection.getAdmin()) {
- admin.createTable(
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName1)
- .setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
- // add the coprocessor for the region
- .setCoprocessor(CustomRegionObserver.class.getName()));
- admin.createTable(
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName2)
- .setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
- // add the coprocessor for the region
- .setCoprocessor(CustomRegionObserver.class.getName()));
+ admin.createTable(TableDescriptorBuilder.newBuilder(tableName1)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+ // add the coprocessor for the region
+ .setCoprocessor(CustomRegionObserver.class.getName()).build());
+ admin.createTable(TableDescriptorBuilder.newBuilder(tableName2)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+ // add the coprocessor for the region
+ .setCoprocessor(CustomRegionObserver.class.getName()).build());
try (Table table1 = connection.getTable(tableName1);
Table table2 = connection.getTable(tableName2)) {
table1.get(new Get(bar));
@@ -470,13 +463,11 @@ public class TestCoprocessorMetrics {
final TableName tableName = TableName.valueOf(name.getMethodName());
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
Admin admin = connection.getAdmin()) {
- admin.createTable(
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
- .setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
- // add the coprocessor for the region. We add two different coprocessors
- .setCoprocessor(CustomRegionObserver.class.getName())
- .setCoprocessor(CustomRegionObserver2.class.getName()));
+ admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+ // add the coprocessor for the region. We add two different coprocessors
+ .setCoprocessor(CustomRegionObserver.class.getName())
+ .setCoprocessor(CustomRegionObserver2.class.getName()).build());
try (Table table = connection.getTable(tableName)) {
table.get(new Get(foo));
table.get(new Get(foo)); // 2 gets
@@ -493,12 +484,10 @@ public class TestCoprocessorMetrics {
final TableName tableName = TableName.valueOf(name.getMethodName());
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
Admin admin = connection.getAdmin()) {
- admin.createTable(
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
- .setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
- // add the coprocessor for the region
- .setCoprocessor(CustomRegionObserver.class.getName()), new byte[][]{foo});
+ admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+ // add the coprocessor for the region
+ .setCoprocessor(CustomRegionObserver.class.getName()).build(), new byte[][] { foo });
// create with 2 regions
try (Table table = connection.getTable(tableName)) {
table.get(new Get(foo));
@@ -538,12 +527,10 @@ public class TestCoprocessorMetrics {
final TableName tableName = TableName.valueOf(name.getMethodName());
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
Admin admin = connection.getAdmin()) {
- admin.createTable(
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
- .setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
- // add the coprocessor for the region
- .setCoprocessor(CustomRegionEndpoint.class.getName()));
+ admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+ // add the coprocessor for the region
+ .setCoprocessor(CustomRegionEndpoint.class.getName()).build());
try (Table table = connection.getTable(tableName)) {
List<Mutation> mutations = Lists.newArrayList(new Put(foo), new Put(bar));
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 a0b0403..2a4cf61 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
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -89,11 +88,9 @@ public class TestMasterCoprocessorExceptionWithAbort {
public void run() {
// create a table : master coprocessor will throw an exception and not
// catch it.
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE));
- ColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(TEST_TABLE))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
try {
Admin admin = UTIL.getAdmin();
admin.createTable(tableDescriptor);
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 c4636fb..5379bb3 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
@@ -196,11 +196,9 @@ public class TestMasterCoprocessorExceptionWithRemove {
BuggyMasterObserver.class.getName();
assertTrue(HMaster.getLoadedCoprocessors().contains(coprocessorName));
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor1 =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE1));
-
- tableDescriptor1.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY1));
+ TableDescriptor tableDescriptor1 =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(TEST_TABLE1))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY1)).build();
boolean threwDNRE = false;
try {
@@ -229,10 +227,9 @@ public class TestMasterCoprocessorExceptionWithRemove {
// Verify that BuggyMasterObserver has been removed due to its misbehavior
// by creating another table: should not have a problem this time.
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor2 =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE2));
- tableDescriptor2.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY2));
+ TableDescriptor tableDescriptor2 =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(TEST_TABLE2))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY2)).build();
Admin admin = UTIL.getAdmin();
try {
admin.createTable(tableDescriptor2);
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 eef3229..7120758 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
@@ -33,7 +33,6 @@ 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.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
@@ -1324,11 +1323,8 @@ public class TestMasterObserver {
assertFalse("No table created yet", cp.wasCreateTableCalled());
// create a table
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
try(Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
Admin admin = connection.getAdmin()) {
tableCreationLatch = new CountDownLatch(1);
@@ -1371,7 +1367,8 @@ public class TestMasterObserver {
assertTrue(admin.isTableDisabled(tableName));
// modify table
- tableDescriptor.setMaxFileSize(512 * 1024 * 1024);
+ tableDescriptor = TableDescriptorBuilder.newBuilder(tableDescriptor)
+ .setMaxFileSize(512 * 1024 * 1024).build();
modifyTableSync(admin, tableName, tableDescriptor);
assertTrue("Test table should have been modified",
cp.wasModifyTableCalled());
@@ -1412,7 +1409,8 @@ public class TestMasterObserver {
cp.wasDisableTableActionCalled());
// modify table
- tableDescriptor.setMaxFileSize(512 * 1024 * 1024);
+ tableDescriptor = TableDescriptorBuilder.newBuilder(tableDescriptor)
+ .setMaxFileSize(512 * 1024 * 1024).build();
modifyTableSync(admin, tableName, tableDescriptor);
assertTrue("Test table should have been modified",
cp.wasModifyTableCalled());
@@ -1455,11 +1453,8 @@ public class TestMasterObserver {
cp.resetStates();
// create a table
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
Admin admin = UTIL.getAdmin();
tableCreationLatch = new CountDownLatch(1);
@@ -1535,12 +1530,11 @@ public class TestMasterObserver {
}
private void modifyTableSync(Admin admin, TableName tableName, TableDescriptor tableDescriptor)
- throws IOException {
+ throws IOException {
admin.modifyTable(tableDescriptor);
- //wait until modify table finishes
- for (int t = 0; t < 100; t++) { //10 sec timeout
- HTableDescriptor td = new HTableDescriptor(
- admin.getDescriptor(tableDescriptor.getTableName()));
+ // wait until modify table finishes
+ for (int t = 0; t < 100; t++) { // 10 sec timeout
+ TableDescriptor td = admin.getDescriptor(tableDescriptor.getTableName());
if (td.equals(tableDescriptor)) {
break;
}
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 2caf376..5ee837a 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
@@ -38,6 +38,7 @@ 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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -160,23 +161,18 @@ public class TestOpenTableInCoprocessor {
runCoprocessorConnectionToRemoteTable(CustomThreadPoolCoprocessor.class, completedWithPool);
}
- private void runCoprocessorConnectionToRemoteTable(Class clazz, boolean[] completeCheck)
+ private void runCoprocessorConnectionToRemoteTable(Class<?> clazz, boolean[] completeCheck)
throws Throwable {
// Check if given class implements RegionObserver.
- assert(RegionObserver.class.isAssignableFrom(clazz));
- TableDescriptorBuilder.ModifyableTableDescriptor primaryDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(primaryTable);
-
- primaryDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ assert (RegionObserver.class.isAssignableFrom(clazz));
// add our coprocessor
- primaryDescriptor.setCoprocessor(clazz.getName());
+ TableDescriptor primaryDescriptor = TableDescriptorBuilder.newBuilder(primaryTable)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCoprocessor(clazz.getName())
+ .build();
- TableDescriptorBuilder.ModifyableTableDescriptor otherDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(otherTable);
- otherDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ TableDescriptor otherDescriptor = TableDescriptorBuilder.newBuilder(otherTable)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build();
Admin admin = UTIL.getAdmin();
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 9e2bd7d..15954f8 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
@@ -39,6 +39,7 @@ 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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -98,17 +99,14 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
}
private void createTable(String coprocessor) throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
- .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(dummy))
- .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(test))
- .setCoprocessor(coprocessor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(dummy))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(test)).setCoprocessor(coprocessor).build();
util.getAdmin().createTable(tableDescriptor);
}
/**
* Test various multiput operations.
- * @throws Exception
*/
@Test
public void testMulti() throws Exception {
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 d252d5e..b679c32 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
@@ -577,12 +577,9 @@ public class TestRegionObserverInterface {
admin.deleteTable(compactTable);
}
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(compactTable);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(A));
- tableDescriptor.setCoprocessor(EvenOnlyCompactor.class.getName());
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(compactTable)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(A))
+ .setCoprocessor(EvenOnlyCompactor.class.getName()).build();
admin.createTable(tableDescriptor);
Table table = util.getConnection().getTable(compactTable);
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 713a3e5..ace8c2a 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
@@ -185,12 +185,13 @@ public class TestRegionObserverScannerOpenHook {
HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
byte[]... families) throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName));
for (byte[] family : families) {
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ builder.setColumnFamily(
+ ColumnFamilyDescriptorBuilder.of(family));
}
+ TableDescriptor tableDescriptor = builder.build();
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
Path path = new Path(DIR + callingMethod);
@@ -307,24 +308,16 @@ public class TestRegionObserverScannerOpenHook {
UTIL.startMiniCluster();
byte[] ROW = Bytes.toBytes("testRow");
byte[] A = Bytes.toBytes("A");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(A));
- tableDescriptor.setCoprocessor(
- CoprocessorDescriptorBuilder.newBuilder(EmptyRegionObsever.class.getName())
- .setJarPath(null)
- .setPriority(Coprocessor.PRIORITY_USER)
- .setProperties(Collections.emptyMap())
- .build());
- tableDescriptor.setCoprocessor(
- CoprocessorDescriptorBuilder.newBuilder(NoDataFromCompaction.class.getName())
- .setJarPath(null)
- .setPriority(Coprocessor.PRIORITY_HIGHEST)
- .setProperties(Collections.emptyMap())
- .build());
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(A))
+ .setCoprocessor(CoprocessorDescriptorBuilder
+ .newBuilder(EmptyRegionObsever.class.getName()).setJarPath(null)
+ .setPriority(Coprocessor.PRIORITY_USER).setProperties(Collections.emptyMap()).build())
+ .setCoprocessor(CoprocessorDescriptorBuilder
+ .newBuilder(NoDataFromCompaction.class.getName()).setJarPath(null)
+ .setPriority(Coprocessor.PRIORITY_HIGHEST).setProperties(Collections.emptyMap()).build())
+ .build();
Admin admin = UTIL.getAdmin();
admin.createTable(tableDescriptor);
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 69a1454..eff278a 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
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Durability;
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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.ChunkCreator;
import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -119,16 +120,14 @@ public class TestRegionObserverStacking extends TestCase {
}
}
- HRegion initHRegion (byte [] tableName, String callingMethod,
- Configuration conf, byte [] ... families) throws IOException {
-
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
+ HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
+ byte[]... families) throws IOException {
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName));
for (byte[] family : families) {
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
}
+ TableDescriptor tableDescriptor = builder.build();
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
Path path = new Path(DIR + callingMethod);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java
index 50dd3e3..88ecb19 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java
@@ -26,7 +26,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.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.FilterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -57,12 +57,8 @@ public class FilterTestingCluster {
protected static void createTable(TableName tableName, String columnFamilyName) {
assertNotNull("HBaseAdmin is not initialized successfully.", admin);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- Bytes.toBytes(columnFamilyName)));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes(columnFamilyName))).build();
try {
admin.createTable(tableDescriptor);
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 b249005..a13aed2 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
@@ -37,6 +37,7 @@ 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.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -84,18 +85,13 @@ public class TestDependentColumnFilter {
public void setUp() throws Exception {
testVals = makeTestVals();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(this.getClass().getSimpleName()));
-
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd0 =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0])
- .setMaxVersions(3);
- tableDescriptor.setColumnFamily(hcd0);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd1 =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[1])
- .setMaxVersions(3);
- tableDescriptor.setColumnFamily(hcd1);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(this.getClass().getSimpleName()))
+ .setColumnFamily(
+ ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[0]).setMaxVersions(3).build())
+ .setColumnFamily(
+ ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[1]).setMaxVersions(3).build())
+ .build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
TEST_UTIL.getConfiguration(), tableDescriptor);
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 eb88529..816524c 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
@@ -41,6 +41,7 @@ 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.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.filter.FilterList.Operator;
import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -150,23 +151,15 @@ public class TestFilter {
@Before
public void setUp() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf("TestFilter"));
-
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor family0 =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0])
- .setVersions(100, 100);
- tableDescriptor.setColumnFamily(family0);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[1]));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES_1[0]));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES_1[1]));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(NEW_FAMILIES[0]));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(NEW_FAMILIES[1]));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder
+ .newBuilder(TableName.valueOf("TestFilter"))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[0]).setMinVersions(100)
+ .setMaxVersions(100).build())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[1]))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES_1[0]))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES_1[1]))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(NEW_FAMILIES[0]))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(NEW_FAMILIES[1])).build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
TEST_UTIL.getConfiguration(), tableDescriptor);
@@ -1519,12 +1512,9 @@ public class TestFilter {
public void testFilterListWithPrefixFilter() throws IOException {
byte[] family = Bytes.toBytes("f1");
byte[] qualifier = Bytes.toBytes("q1");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
TEST_UTIL.getConfiguration(), tableDescriptor);
@@ -2293,12 +2283,9 @@ public class TestFilter {
@Ignore("TODO: intentionally disabled?")
public void testNestedFilterListWithSCVF() throws IOException {
byte[] columnStatus = Bytes.toBytes("S");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0]));
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[0])).build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
TEST_UTIL.getConfiguration(), tableDescriptor);
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 7be3bad..ac06289 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
@@ -35,6 +35,7 @@ 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.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -83,18 +84,16 @@ public class TestFilterFromRegionSide {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TABLE_NAME);
for (byte[] family : FAMILIES) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
}
+ TableDescriptor tableDescriptor = builder.build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
REGION = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
TEST_UTIL.getConfiguration(), tableDescriptor);
- for(Put put:createPuts(ROWS, FAMILIES, QUALIFIERS, VALUE)){
+ for (Put put : createPuts(ROWS, FAMILIES, QUALIFIERS, VALUE)) {
REGION.put(put);
}
}
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 a00b1b1..bd9e83c 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
@@ -17,7 +17,10 @@
*/
package org.apache.hadoop.hbase.filter;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
@@ -42,6 +45,7 @@ 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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.FilterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -149,13 +153,8 @@ public class TestFilterWrapper {
private static void createTable() {
assertNotNull("HBaseAdmin is not initialized successfully.", admin);
if (admin != null) {
-
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(name);
-
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("f1"));
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(name)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("f1"))).build();
try {
admin.createTable(tableDescriptor);
@@ -163,7 +162,6 @@ public class TestFilterWrapper {
} catch (IOException e) {
assertNull("Exception found while creating table", e);
}
-
}
}
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 b752d12..1a4296c 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
@@ -30,6 +30,7 @@ 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.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -68,14 +69,11 @@ public class TestInvocationRecordFilter {
@Before
public void setUp() throws Exception {
- TableDescriptorBuilder.ModifyableTableDescriptor htd =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TABLE_NAME_BYTES));
-
- htd.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME_BYTES));
+ TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE_NAME_BYTES))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_NAME_BYTES)).build();
RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
- TEST_UTIL.getConfiguration(), htd);
+ TEST_UTIL.getConfiguration(), htd);
Put put = new Put(ROW_BYTES);
for (int i = 0; i < 10; i += 2) {
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 29f1fdd..0e7218f 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
@@ -33,6 +33,7 @@ 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.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -63,12 +64,9 @@ public class TestSeekBeforeWithReverseScan {
@Before
public void setUp() throws Exception {
TableName tableName = TableName.valueOf(getClass().getSimpleName());
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName)
- .setDataBlockEncoding(DataBlockEncoding.FAST_DIFF));
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+ .newBuilder(cfName).setDataBlockEncoding(DataBlockEncoding.FAST_DIFF).build()).build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
Path path = testUtil.getDataTestDir(getClass().getSimpleName());
region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(),
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 f2b7801..e9f6192 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
@@ -36,6 +36,7 @@ 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.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -102,14 +103,11 @@ public class TestScannerSelectionUsingKeyRange {
public void testScannerSelection() throws IOException {
Configuration conf = TEST_UTIL.getConfiguration();
conf.setInt("hbase.hstore.compactionThreshold", 10000);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_BYTES)
- .setBlockCacheEnabled(true)
- .setBloomFilterType(bloomType);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_BYTES)
+ .setBlockCacheEnabled(true).setBloomFilterType(bloomType).build())
+ .build();
- tableDescriptor.setColumnFamily(familyDescriptor);
RegionInfo info = RegionInfoBuilder.newBuilder(TABLE).build();
HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf,
tableDescriptor);
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 e653234..53bebec 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
@@ -48,6 +48,7 @@ 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.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -241,11 +242,8 @@ public class TestMaster {
int msgInterval = conf.getInt("hbase.regionserver.msginterval", 100);
// insert some data into META
TableName tableName = TableName.valueOf("testFlushSeqId");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("cf")));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))).build();
Table table = TEST_UTIL.createTable(tableDescriptor, null);
// flush META region
TEST_UTIL.flush(TableName.META_TABLE_NAME);
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 1241e26..ff88be1 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
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
@@ -125,12 +126,9 @@ public class TestMasterMetricsWrapper {
TableName table = TableName.valueOf("testRegionNumber");
try {
RegionInfo hri;
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(table);
-
byte[] FAMILY = Bytes.toBytes("FAMILY");
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(table)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
TEST_UTIL.getAdmin().createTable(tableDescriptor, Bytes.toBytes("A"),
Bytes.toBytes("Z"), 5);
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 e9e4a39..3983040 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
@@ -551,12 +551,8 @@ public class TestRegionPlacement {
splitKeys[i - 1] = new byte[] { splitKey, splitKey, splitKey };
}
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, splitKeys);
try (RegionLocator r = CONNECTION.getRegionLocator(tableName)) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlansWithThrottle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlansWithThrottle.java
index 7108340..5084064 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlansWithThrottle.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlansWithThrottle.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -65,9 +66,8 @@ public class TestRegionPlansWithThrottle {
public void testExecuteRegionPlansWithThrottling() throws Exception {
final TableName tableName = TableName.valueOf("testExecuteRegionPlansWithThrottling");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName).setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("cf")));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))).build();
UTIL.getAdmin().createTable(tableDescriptor);
Table table = UTIL.getConnection().getTable(tableName);
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 6a4fc8c..40345e8 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
@@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORE
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
+
import java.util.List;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
@@ -32,6 +33,7 @@ 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.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.master.HMaster;
@@ -44,6 +46,7 @@ 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;
/*
@@ -71,7 +74,6 @@ public class TestFavoredNodeTableImport {
@Test
public void testTableCreation() throws Exception {
-
conf.set(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, StochasticLoadBalancer.class.getName());
LOG.info("Starting up cluster");
@@ -83,12 +85,9 @@ public class TestFavoredNodeTableImport {
admin.balancerSwitch(false, true);
String tableName = "testFNImport";
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, Bytes.toBytes("a"), Bytes.toBytes("z"), REGION_NUM);
UTIL.waitTableAvailable(tableDescriptor.getTableName());
admin.balancerSwitch(true, true);
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 8a1ee8a..d3e1e83 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
@@ -117,12 +118,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
public void testBasicBalance() throws Exception {
TableName tableName = TableName.valueOf("testBasicBalance");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY);
@@ -154,12 +151,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
public void testRoundRobinAssignment() throws Exception {
TableName tableName = TableName.valueOf("testRoundRobinAssignment");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY);
@@ -180,14 +173,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
@Test
public void testBasicRegionPlacementAndReplicaLoad() throws Exception {
-
String tableName = "testBasicRegionPlacement";
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName());
@@ -226,12 +215,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
public void testRandomAssignmentWithNoFavNodes() throws Exception {
final String tableName = "testRandomAssignmentWithNoFavNodes";
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor);
TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName());
@@ -261,12 +247,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
public void testBalancerWithoutFavoredNodes() throws Exception {
TableName tableName = TableName.valueOf("testBalancerWithoutFavoredNodes");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
@@ -301,14 +283,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
@Ignore @Test
public void testMisplacedRegions() throws Exception {
-
TableName tableName = TableName.valueOf("testMisplacedRegions");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
@@ -352,14 +329,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
@Test
public void test2FavoredNodesDead() throws Exception {
-
TableName tableName = TableName.valueOf("testAllFavoredNodesDead");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
@@ -394,14 +366,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
@Ignore @Test
public void testAllFavoredNodesDead() throws Exception {
-
TableName tableName = TableName.valueOf("testAllFavoredNodesDead");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
@@ -460,14 +427,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
@Ignore @Test
public void testAllFavoredNodesDeadMasterRestarted() throws Exception {
-
TableName tableName = TableName.valueOf("testAllFavoredNodesDeadMasterRestarted");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- HConstants.CATALOG_FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
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 c7ed3b2..670d7a3 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
@@ -28,13 +28,13 @@ 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;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -101,7 +101,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
@Test
public void deleteColumnFamilyWithMultipleRegions() throws Exception {
Admin admin = TEST_UTIL.getAdmin();
- HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
+ TableDescriptor beforehtd = admin.getDescriptor(TABLENAME);
FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
@@ -110,7 +110,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
// 2 - Check if all three families exist in descriptor
assertEquals(3, beforehtd.getColumnFamilyCount());
- HColumnDescriptor[] families = beforehtd.getColumnFamilies();
+ ColumnFamilyDescriptor[] families = beforehtd.getColumnFamilies();
for (int i = 0; i < families.length; i++) {
assertTrue(families[i].getNameAsString().equals("cf" + (i + 1)));
}
@@ -148,9 +148,9 @@ public class TestDeleteColumnFamilyProcedureFromClient {
admin.deleteColumnFamily(TABLENAME, Bytes.toBytes("cf2"));
// 5 - Check if only 2 column families exist in the descriptor
- HTableDescriptor afterhtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
+ TableDescriptor afterhtd = admin.getDescriptor(TABLENAME);
assertEquals(2, afterhtd.getColumnFamilyCount());
- HColumnDescriptor[] newFamilies = afterhtd.getColumnFamilies();
+ ColumnFamilyDescriptor[] newFamilies = afterhtd.getColumnFamilies();
assertTrue(newFamilies[0].getNameAsString().equals("cf1"));
assertTrue(newFamilies[1].getNameAsString().equals("cf3"));
@@ -179,7 +179,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
@Test
public void deleteColumnFamilyTwice() throws Exception {
Admin admin = TEST_UTIL.getAdmin();
- HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
+ TableDescriptor beforehtd = admin.getDescriptor(TABLENAME);
String cfToDelete = "cf1";
FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
@@ -188,7 +188,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
assertTrue(admin.isTableAvailable(TABLENAME));
// 2 - Check if all the target column family exist in descriptor
- HColumnDescriptor[] families = beforehtd.getColumnFamilies();
+ ColumnFamilyDescriptor[] families = beforehtd.getColumnFamilies();
Boolean foundCF = false;
for (int i = 0; i < families.length; i++) {
if (families[i].getNameAsString().equals(cfToDelete)) {
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 fefba23..4461f2f 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
@@ -20,11 +20,11 @@ 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 java.io.IOException;
import org.apache.hadoop.hbase.ConcurrentTableModificationException;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@@ -71,31 +71,31 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
UTIL.getAdmin().disableTable(tableName);
// Modify the table descriptor
- HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
+ TableDescriptor htd = UTIL.getAdmin().getDescriptor(tableName);
// Test 1: Modify 1 property
long newMaxFileSize = htd.getMaxFileSize() * 2;
- htd.setMaxFileSize(newMaxFileSize);
- htd.setRegionReplication(3);
+ htd = TableDescriptorBuilder.newBuilder(htd).setMaxFileSize(newMaxFileSize)
+ .setRegionReplication(3).build();
- long procId1 = ProcedureTestingUtility.submitAndWait(
- procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
+ long procId1 = ProcedureTestingUtility.submitAndWait(procExec,
+ new ModifyTableProcedure(procExec.getEnvironment(), htd));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
- HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
+ TableDescriptor currentHtd = UTIL.getAdmin().getDescriptor(tableName);
assertEquals(newMaxFileSize, currentHtd.getMaxFileSize());
// Test 2: Modify multiple properties
boolean newReadOnlyOption = htd.isReadOnly() ? false : true;
long newMemStoreFlushSize = htd.getMemStoreFlushSize() * 2;
- htd.setReadOnly(newReadOnlyOption);
- htd.setMemStoreFlushSize(newMemStoreFlushSize);
+ htd = TableDescriptorBuilder.newBuilder(htd).setReadOnly(newReadOnlyOption)
+ .setMemStoreFlushSize(newMemStoreFlushSize).build();
long procId2 = ProcedureTestingUtility.submitAndWait(
procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
- currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
+ currentHtd = UTIL.getAdmin().getDescriptor(tableName);
assertEquals(newReadOnlyOption, currentHtd.isReadOnly());
assertEquals(newMemStoreFlushSize, currentHtd.getMemStoreFlushSize());
}
@@ -106,8 +106,8 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "cf1");
- HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
- assertEquals(1, currentHtd.getFamiliesKeys().size());
+ TableDescriptor currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+ assertEquals(1, currentHtd.getColumnFamilyNames().size());
// Test 1: Modify the table descriptor online
String cf2 = "cf2";
@@ -122,9 +122,9 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
procExec.getEnvironment(), tableDescriptorBuilder.build()));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
- currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
- assertEquals(2, currentHtd.getFamiliesKeys().size());
- assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
+ currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+ assertEquals(2, currentHtd.getColumnFamilyNames().size());
+ assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf2)));
// Test 2: Modify the table descriptor offline
UTIL.getAdmin().disableTable(tableName);
@@ -141,9 +141,9 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
new ModifyTableProcedure(procExec.getEnvironment(), tableDescriptorBuilder.build()));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
- currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
- assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf3)));
- assertEquals(3, currentHtd.getFamiliesKeys().size());
+ currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+ assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf3)));
+ assertEquals(3, currentHtd.getColumnFamilyNames().size());
}
@Test
@@ -155,44 +155,43 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, cf2, cf3);
- HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
- assertEquals(3, currentHtd.getFamiliesKeys().size());
+ TableDescriptor currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+ assertEquals(3, currentHtd.getColumnFamilyNames().size());
// Test 1: Modify the table descriptor
- HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
- htd.removeFamily(Bytes.toBytes(cf2));
+ TableDescriptor htd = UTIL.getAdmin().getDescriptor(tableName);
+ htd = TableDescriptorBuilder.newBuilder(htd).removeColumnFamily(Bytes.toBytes(cf2)).build();
long procId = ProcedureTestingUtility.submitAndWait(
procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
- currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
- assertEquals(2, currentHtd.getFamiliesKeys().size());
- assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf2)));
+ currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+ assertEquals(2, currentHtd.getColumnFamilyNames().size());
+ assertFalse(currentHtd.hasColumnFamily(Bytes.toBytes(cf2)));
// Test 2: Modify the table descriptor offline
UTIL.getAdmin().disableTable(tableName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
- HTableDescriptor htd2 =
- new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
- htd2.removeFamily(Bytes.toBytes(cf3));
+ TableDescriptor htd2 = UTIL.getAdmin().getDescriptor(tableName);
// Disable Sanity check
- htd2.setConfiguration(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString());
+ htd2 = TableDescriptorBuilder.newBuilder(htd2).removeColumnFamily(Bytes.toBytes(cf3))
+ .setValue(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString())
+ .build();
long procId2 =
ProcedureTestingUtility.submitAndWait(procExec,
new ModifyTableProcedure(procExec.getEnvironment(), htd2));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
- currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
- assertEquals(1, currentHtd.getFamiliesKeys().size());
- assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf3)));
+ currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+ assertEquals(1, currentHtd.getColumnFamilyNames().size());
+ assertFalse(currentHtd.hasColumnFamily(Bytes.toBytes(cf3)));
- //Removing the last family will fail
- HTableDescriptor htd3 =
- new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
- htd3.removeFamily(Bytes.toBytes(cf1));
+ // Removing the last family will fail
+ TableDescriptor htd3 = UTIL.getAdmin().getDescriptor(tableName);
+ htd3 = TableDescriptorBuilder.newBuilder(htd3).removeColumnFamily(Bytes.toBytes(cf1)).build();
long procId3 =
ProcedureTestingUtility.submitAndWait(procExec,
new ModifyTableProcedure(procExec.getEnvironment(), htd3));
@@ -201,8 +200,8 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
Throwable cause = ProcedureTestingUtility.getExceptionCause(result);
assertTrue("expected DoNotRetryIOException, got " + cause,
cause instanceof DoNotRetryIOException);
- assertEquals(1, currentHtd.getFamiliesKeys().size());
- assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf1)));
+ assertEquals(1, currentHtd.getColumnFamilyNames().size());
+ assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf1)));
}
@Test
@@ -277,11 +276,11 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
// Validate descriptor
- HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
+ TableDescriptor currentHtd = UTIL.getAdmin().getDescriptor(tableName);
assertEquals(newCompactionEnableOption, currentHtd.isCompactionEnabled());
- assertEquals(2, currentHtd.getFamiliesKeys().size());
- assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
- assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf3)));
+ assertEquals(2, currentHtd.getColumnFamilyNames().size());
+ assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf2)));
+ assertFalse(currentHtd.hasColumnFamily(Bytes.toBytes(cf3)));
// cf2 should be added cf3 should be removed
MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
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 64ecb42..afe6b96 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
@@ -21,14 +21,15 @@ 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 java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.HBaseClassTestRule;
-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.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -49,6 +50,7 @@ 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;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
@@ -118,10 +120,10 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
snapshot = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotList.get(0));
// modify the table
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor columnFamilyDescriptor3 =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(CF3);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor columnFamilyDescriptor4 =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(CF4);
+ ColumnFamilyDescriptor columnFamilyDescriptor3 =
+ ColumnFamilyDescriptorBuilder.of(CF3);
+ ColumnFamilyDescriptor columnFamilyDescriptor4 =
+ ColumnFamilyDescriptorBuilder.of(CF4);
admin.addColumnFamily(snapshotTableName, columnFamilyDescriptor3);
admin.addColumnFamily(snapshotTableName, columnFamilyDescriptor4);
admin.deleteColumnFamily(snapshotTableName, CF2);
@@ -130,27 +132,26 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF3, CF3);
SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF4, CF4);
SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF1addition, CF1);
- HTableDescriptor currentHTD = new HTableDescriptor(admin.getDescriptor(snapshotTableName));
- assertTrue(currentHTD.hasFamily(CF1));
- assertFalse(currentHTD.hasFamily(CF2));
- assertTrue(currentHTD.hasFamily(CF3));
- assertTrue(currentHTD.hasFamily(CF4));
- assertNotEquals(currentHTD.getFamiliesKeys().size(), snapshotHTD.getColumnFamilies().length);
- SnapshotTestingUtils.verifyRowCount(
- UTIL, snapshotTableName, rowCountCF1 + rowCountCF3 + rowCountCF4 + rowCountCF1addition);
+ TableDescriptor currentHTD = admin.getDescriptor(snapshotTableName);
+ assertTrue(currentHTD.hasColumnFamily(CF1));
+ assertFalse(currentHTD.hasColumnFamily(CF2));
+ assertTrue(currentHTD.hasColumnFamily(CF3));
+ assertTrue(currentHTD.hasColumnFamily(CF4));
+ assertNotEquals(currentHTD.getColumnFamilyNames().size(),
+ snapshotHTD.getColumnFamilies().length);
+ SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName,
+ rowCountCF1 + rowCountCF3 + rowCountCF4 + rowCountCF1addition);
admin.disableTable(snapshotTableName);
}
- private static TableDescriptor createHTableDescriptor(
- final TableName tableName, final byte[] ... family) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+ private static TableDescriptor createHTableDescriptor(final TableName tableName,
+ final byte[]... family) {
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
for (int i = 0; i < family.length; ++i) {
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family[i]));
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family[i]));
}
- return tableDescriptor;
+ return builder.build();
}
@Test
@@ -222,13 +223,13 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
try {
UTIL.getAdmin().enableTable(snapshotTableName);
- HTableDescriptor currentHTD =
- new HTableDescriptor(UTIL.getAdmin().getDescriptor(snapshotTableName));
- assertTrue(currentHTD.hasFamily(CF1));
- assertTrue(currentHTD.hasFamily(CF2));
- assertFalse(currentHTD.hasFamily(CF3));
- assertFalse(currentHTD.hasFamily(CF4));
- assertEquals(currentHTD.getFamiliesKeys().size(), snapshotHTD.getColumnFamilies().length);
+ TableDescriptor currentHTD = UTIL.getAdmin().getDescriptor(snapshotTableName);
+ assertTrue(currentHTD.hasColumnFamily(CF1));
+ assertTrue(currentHTD.hasColumnFamily(CF2));
+ assertFalse(currentHTD.hasColumnFamily(CF3));
+ assertFalse(currentHTD.hasColumnFamily(CF4));
+ assertEquals(currentHTD.getColumnFamilyNames().size(),
+ snapshotHTD.getColumnFamilies().length);
SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName, rowCountCF1 + rowCountCF2);
} finally {
UTIL.getAdmin().disableTable(snapshotTableName);
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 cdd1aa3..9196921 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
@@ -25,11 +25,10 @@ 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;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@@ -91,11 +90,8 @@ public class TestTableDescriptorModificationFromClient {
public void testModifyTable() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
// Create a table with one family
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0)).build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -103,13 +99,9 @@ public class TestTableDescriptorModificationFromClient {
verifyTableDescriptor(TABLE_NAME, FAMILY_0);
// Modify the table adding another family and verify the descriptor
- TableDescriptorBuilder.ModifyableTableDescriptor modifiedtableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
- modifiedtableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
- modifiedtableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+ TableDescriptor modifiedtableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_1)).build();
admin.modifyTable(modifiedtableDescriptor);
verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
} finally {
@@ -121,11 +113,8 @@ public class TestTableDescriptorModificationFromClient {
public void testAddColumn() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
// Create a table with two families
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0)).build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -133,8 +122,7 @@ public class TestTableDescriptorModificationFromClient {
verifyTableDescriptor(TABLE_NAME, FAMILY_0);
// Modify the table removing one family and verify the descriptor
- admin.addColumnFamily(TABLE_NAME,
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+ admin.addColumnFamily(TABLE_NAME, ColumnFamilyDescriptorBuilder.of(FAMILY_1));
verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
} finally {
admin.deleteTable(TABLE_NAME);
@@ -145,11 +133,8 @@ public class TestTableDescriptorModificationFromClient {
public void testAddSameColumnFamilyTwice() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
// Create a table with one families
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0)).build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -157,14 +142,12 @@ public class TestTableDescriptorModificationFromClient {
verifyTableDescriptor(TABLE_NAME, FAMILY_0);
// Modify the table removing one family and verify the descriptor
- admin.addColumnFamily(TABLE_NAME,
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+ admin.addColumnFamily(TABLE_NAME, ColumnFamilyDescriptorBuilder.of(FAMILY_1));
verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
try {
// Add same column family again - expect failure
- admin.addColumnFamily(TABLE_NAME,
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+ admin.addColumnFamily(TABLE_NAME, ColumnFamilyDescriptorBuilder.of(FAMILY_1));
Assert.fail("Delete a non-exist column family should fail");
} catch (InvalidFamilyOperationException e) {
// Expected.
@@ -179,14 +162,11 @@ public class TestTableDescriptorModificationFromClient {
public void testModifyColumnFamily() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor cfDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0);
+ ColumnFamilyDescriptor cfDescriptor = ColumnFamilyDescriptorBuilder.of(FAMILY_0);
int blockSize = cfDescriptor.getBlocksize();
// Create a table with one families
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
- tableDescriptor.setColumnFamily(cfDescriptor);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(cfDescriptor).build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -194,13 +174,14 @@ public class TestTableDescriptorModificationFromClient {
verifyTableDescriptor(TABLE_NAME, FAMILY_0);
int newBlockSize = 2 * blockSize;
- cfDescriptor.setBlocksize(newBlockSize);
+ cfDescriptor =
+ ColumnFamilyDescriptorBuilder.newBuilder(cfDescriptor).setBlocksize(newBlockSize).build();
// Modify colymn family
admin.modifyColumnFamily(TABLE_NAME, cfDescriptor);
- HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TABLE_NAME));
- HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
+ TableDescriptor htd = admin.getDescriptor(TABLE_NAME);
+ ColumnFamilyDescriptor hcfd = htd.getColumnFamily(FAMILY_0);
assertTrue(hcfd.getBlocksize() == newBlockSize);
} finally {
admin.deleteTable(TABLE_NAME);
@@ -211,15 +192,11 @@ public class TestTableDescriptorModificationFromClient {
public void testModifyNonExistingColumnFamily() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor cfDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1);
+ ColumnFamilyDescriptor cfDescriptor = ColumnFamilyDescriptorBuilder.of(FAMILY_1);
int blockSize = cfDescriptor.getBlocksize();
// Create a table with one families
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0)).build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -227,7 +204,8 @@ public class TestTableDescriptorModificationFromClient {
verifyTableDescriptor(TABLE_NAME, FAMILY_0);
int newBlockSize = 2 * blockSize;
- cfDescriptor.setBlocksize(newBlockSize);
+ cfDescriptor =
+ ColumnFamilyDescriptorBuilder.newBuilder(cfDescriptor).setBlocksize(newBlockSize).build();
// Modify a column family that is not in the table.
try {
@@ -246,13 +224,9 @@ public class TestTableDescriptorModificationFromClient {
public void testDeleteColumn() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
// Create a table with two families
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_1)).build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -271,13 +245,9 @@ public class TestTableDescriptorModificationFromClient {
public void testDeleteSameColumnFamilyTwice() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
// Create a table with two families
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_1)).build();
admin.createTable(tableDescriptor);
admin.disableTable(TABLE_NAME);
try {
@@ -305,7 +275,7 @@ public class TestTableDescriptorModificationFromClient {
Admin admin = TEST_UTIL.getAdmin();
// Verify descriptor from master
- HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(tableName));
+ TableDescriptor htd = admin.getDescriptor(tableName);
verifyTableDescriptor(htd, tableName, families);
// Verify descriptor from HDFS
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java
index cf6a422..9b63b9d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java
@@ -22,7 +22,6 @@ 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.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -30,6 +29,7 @@ import org.apache.hadoop.fs.Path;
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.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -37,6 +37,7 @@ 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.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
import org.apache.hadoop.hbase.util.Bytes;
@@ -56,7 +57,6 @@ import org.slf4j.LoggerFactory;
This class is used by MobStressTool only. This is not a unit test
*/
-@SuppressWarnings("deprecation")
public class MobStressToolRunner {
private static final Logger LOG = LoggerFactory.getLogger(MobStressToolRunner.class);
@@ -71,8 +71,8 @@ public class MobStressToolRunner {
.toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
private Configuration conf;
- private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
- private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
+ private TableDescriptor tableDescriptor;
+ private ColumnFamilyDescriptor familyDescriptor;
private Admin admin;
private long count = 500000;
private double failureProb = 0.1;
@@ -82,7 +82,6 @@ public class MobStressToolRunner {
private static volatile boolean run = true;
public MobStressToolRunner() {
-
}
public void init(Configuration conf, long numRows) throws IOException {
@@ -90,15 +89,13 @@ public class MobStressToolRunner {
this.count = numRows;
initConf();
printConf();
- tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf("testMobCompactTable"));
Connection conn = ConnectionFactory.createConnection(this.conf);
this.admin = conn.getAdmin();
- this.familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam);
- this.familyDescriptor.setMobEnabled(true);
- this.familyDescriptor.setMobThreshold(mobLen);
- this.familyDescriptor.setMaxVersions(1);
- this.tableDescriptor.setColumnFamily(familyDescriptor);
+ this.familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(fam).setMobEnabled(true)
+ .setMobThreshold(mobLen).setMaxVersions(1).build();
+ this.tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf("testMobCompactTable"))
+ .setColumnFamily(familyDescriptor).build();
if (admin.tableExists(tableDescriptor.getTableName())) {
admin.disableTable(tableDescriptor.getTableName());
admin.deleteTable(tableDescriptor.getTableName());
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 1fa0a68..e0d7d2c 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
@@ -29,6 +29,7 @@ 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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -46,86 +47,74 @@ 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");
- private final static byte [] family = Bytes.toBytes("family");
- private final static byte [] qf1 = Bytes.toBytes("qf1");
- private final static byte [] qf2 = Bytes.toBytes("qf2");
- private final static byte [] value1 = Bytes.toBytes("value1");
- private final static byte [] value2 = Bytes.toBytes("value2");
-
- @Rule
- public TestName name = new TestName();
-
- @BeforeClass
- public static void setUpBeforeClass() throws Exception {
- TEST_UTIL.startMiniCluster(1);
- }
+ 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");
+ private final static byte[] family = Bytes.toBytes("family");
+ private final static byte[] qf1 = Bytes.toBytes("qf1");
+ private final static byte[] qf2 = Bytes.toBytes("qf2");
+ private final static byte[] value1 = Bytes.toBytes("value1");
+ private final static byte[] value2 = Bytes.toBytes("value2");
+
+ @Rule
+ public TestName name = new TestName();
+
+ @BeforeClass
+ public static void setUpBeforeClass() throws Exception {
+ TEST_UTIL.startMiniCluster(1);
+ }
- @AfterClass
- public static void tearDownAfterClass() throws Exception {
- TEST_UTIL.shutdownMiniCluster();
- }
+ @AfterClass
+ public static void tearDownAfterClass() throws Exception {
+ TEST_UTIL.shutdownMiniCluster();
+ }
@Test
public void testFlushNonMobFile() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
- familyDescriptor.setMaxVersions(4);
- tableDescriptor.setColumnFamily(familyDescriptor);
-
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(4).build())
+ .build();
testFlushFile(tableDescriptor);
}
@Test
public void testFlushMobFile() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
- hcd.setMobEnabled(true);
- hcd.setMobThreshold(3L);
- hcd.setMaxVersions(4);
- tableDescriptor.setColumnFamily(hcd);
-
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true)
+ .setMobThreshold(3L).setMaxVersions(4).build())
+ .build();
testFlushFile(tableDescriptor);
}
- private void testFlushFile(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor)
- throws Exception {
+ private void testFlushFile(TableDescriptor tableDescriptor) throws Exception {
Table table = null;
try {
table = TEST_UTIL.createTable(tableDescriptor, null);
- //put data
+ // put data
Put put0 = new Put(row1);
put0.addColumn(family, qf1, 1, value1);
table.put(put0);
- //put more data
+ // put more data
Put put1 = new Put(row2);
put1.addColumn(family, qf2, 1, value2);
table.put(put1);
- //flush
+ // flush
TEST_UTIL.flush(tableDescriptor.getTableName());
- //Scan
+ // Scan
Scan scan = new Scan();
scan.addColumn(family, qf1);
scan.readVersions(4);
ResultScanner scanner = table.getScanner(scan);
- //Compare
+ // Compare
int size = 0;
for (Result result : scanner) {
size++;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
index 09f7f58..cacc90e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
@@ -18,14 +18,11 @@
*/
package org.apache.hadoop.hbase.mob;
import java.io.IOException;
-
import org.apache.hadoop.hbase.HBaseClassTestRule;
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;
/**
* Mob file compaction chore in a generational non-batch mode test.
@@ -42,11 +39,9 @@ import org.slf4j.LoggerFactory;
* 11 Verifies that number of MOB files in a mob directory is 20.
* 12 Runs scanner and checks all 3 * 1000 rows.
*/
-@SuppressWarnings("deprecation")
@Category(LargeTests.class)
public class TestMobCompactionOptMode extends TestMobCompactionWithDefaults {
- private static final Logger LOG =
- LoggerFactory.getLogger(TestMobCompactionOptMode.class);
+
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMobCompactionOptMode.class);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
index 117b9ee..46ef3b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
@@ -46,7 +46,6 @@ import org.slf4j.LoggerFactory;
* 11 Verifies that number of MOB files in a mob directory is 20.
* 12 Runs scanner and checks all 3 * 1000 rows.
*/
-@SuppressWarnings("deprecation")
@Category(LargeTests.class)
public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionWithDefaults {
private static final Logger LOG =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
index 5151789..bc70425 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
@@ -46,7 +46,6 @@ import org.slf4j.LoggerFactory;
* 11 Verifies that number of MOB files in a mob directory is 20.
* 12 Runs scanner and checks all 3 * 1000 rows.
*/
-@SuppressWarnings("deprecation")
@Category(LargeTests.class)
public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionWithDefaults {
private static final Logger LOG =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java
index 22fb31f..149de20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java
@@ -26,7 +26,6 @@ import java.util.Arrays;
import java.util.List;
import java.util.Random;
import java.util.stream.Collectors;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -43,7 +42,6 @@ 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.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.RegionSplitter;
@@ -75,7 +73,6 @@ import org.slf4j.LoggerFactory;
* 11 Verifies that number of MOB files in a mob directory is 20.
* 12 Runs scanner and checks all 3 * 1000 rows.
*/
-@SuppressWarnings("deprecation")
@Category(LargeTests.class)
public class TestMobCompactionWithDefaults {
private static final Logger LOG =
@@ -97,8 +94,8 @@ public class TestMobCompactionWithDefaults {
@Rule
public TestName test = new TestName();
- protected TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
- private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
+ protected TableDescriptor tableDescriptor;
+ private ColumnFamilyDescriptor familyDescriptor;
protected Admin admin;
protected TableName table = null;
protected int numRegions = 20;
@@ -130,14 +127,12 @@ public class TestMobCompactionWithDefaults {
@Before
public void setUp() throws Exception {
- tableDescriptor = HTU.createModifyableTableDescriptor(test.getMethodName());
admin = HTU.getAdmin();
cleanerChore = new MobFileCleanerChore();
- familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam);
- familyDescriptor.setMobEnabled(true);
- familyDescriptor.setMobThreshold(mobLen);
- familyDescriptor.setMaxVersions(1);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(fam).setMobEnabled(true)
+ .setMobThreshold(mobLen).setMaxVersions(1).build();
+ tableDescriptor = HTU.createModifyableTableDescriptor(test.getMethodName())
+ .setColumnFamily(familyDescriptor).build();
RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit();
byte[][] splitKeys = splitAlgo.split(numRegions);
table = HTU.createTable(tableDescriptor, splitKeys).getName();
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 2b1595d..fefa898 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
@@ -22,11 +22,13 @@ 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.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.ConnectionFactory;
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.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -52,9 +54,8 @@ public class TestMobDataBlockEncoding {
protected final byte[] qf3 = Bytes.toBytes("qualifier3");
private static Table table;
private static Admin admin;
- private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor
- columnFamilyDescriptor;
- private static TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
+ private static ColumnFamilyDescriptor columnFamilyDescriptor;
+ private static TableDescriptor tableDescriptor;
private static Random random = new Random();
private static long defaultThreshold = 10;
@@ -70,15 +71,10 @@ public class TestMobDataBlockEncoding {
public void setUp(long threshold, String TN, DataBlockEncoding encoding)
throws Exception {
- tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TN));
- columnFamilyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
- columnFamilyDescriptor.setMobEnabled(true);
- columnFamilyDescriptor.setMobThreshold(threshold);
- columnFamilyDescriptor.setMaxVersions(4);
- columnFamilyDescriptor.setDataBlockEncoding(encoding);
- tableDescriptor.setColumnFamily(columnFamilyDescriptor);
+ columnFamilyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true)
+ .setMobThreshold(threshold).setMaxVersions(4).setDataBlockEncoding(encoding).build();
+ tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf(TN))
+ .setColumnFamily(columnFamilyDescriptor).build();
admin = TEST_UTIL.getAdmin();
admin.createTable(tableDescriptor);
table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java
index 274d045..5465122 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java
@@ -22,7 +22,6 @@ 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.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -31,13 +30,14 @@ 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.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.CompactionState;
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.Table;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -59,7 +59,6 @@ import org.slf4j.LoggerFactory;
* 6. Runs Mob cleaner chore
* 7 Verifies that number of MOB files in a mob directory is 1.
*/
-@SuppressWarnings("deprecation")
@Category(MediumTests.class)
public class TestMobFileCleanerChore {
private static final Logger LOG = LoggerFactory.getLogger(TestMobFileCleanerChore.class);
@@ -77,8 +76,8 @@ public class TestMobFileCleanerChore {
.toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
private Configuration conf;
- private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
- private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
+ private TableDescriptor tableDescriptor;
+ private ColumnFamilyDescriptor familyDescriptor;
private Admin admin;
private Table table = null;
private MobFileCleanerChore chore;
@@ -90,7 +89,6 @@ public class TestMobFileCleanerChore {
@Before
public void setUp() throws Exception {
HTU = new HBaseTestingUtility();
- tableDescriptor = HTU.createModifyableTableDescriptor("testMobCompactTable");
conf = HTU.getConfiguration();
initConf();
@@ -98,11 +96,10 @@ public class TestMobFileCleanerChore {
HTU.startMiniCluster();
admin = HTU.getAdmin();
chore = new MobFileCleanerChore();
- familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam);
- familyDescriptor.setMobEnabled(true);
- familyDescriptor.setMobThreshold(mobLen);
- familyDescriptor.setMaxVersions(1);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(fam).setMobEnabled(true)
+ .setMobThreshold(mobLen).setMaxVersions(1).build();
+ tableDescriptor = HTU.createModifyableTableDescriptor("testMobCompactTable")
+ .setColumnFamily(familyDescriptor).build();
table = HTU.createTable(tableDescriptor, null);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
index de1dba9..1865c0b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
@@ -96,8 +96,8 @@ public class TestMobStoreCompaction {
private Configuration conf = null;
private HRegion region = null;
- private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = null;
- private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = null;
+ private TableDescriptor tableDescriptor = null;
+ private ColumnFamilyDescriptor familyDescriptor = null;
private long mobCellThreshold = 1000;
private FileSystem fs;
@@ -112,13 +112,10 @@ public class TestMobStoreCompaction {
HBaseTestingUtility UTIL = new HBaseTestingUtility(conf);
compactionThreshold = conf.getInt("hbase.hstore.compactionThreshold", 3);
- tableDescriptor = UTIL.createModifyableTableDescriptor(name.getMethodName());
- familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_FAMILY);
- familyDescriptor.setMobEnabled(true);
- familyDescriptor.setMobThreshold(mobThreshold);
- familyDescriptor.setMaxVersions(1);
- tableDescriptor.modifyColumnFamily(familyDescriptor);
+ familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_FAMILY).setMobEnabled(true)
+ .setMobThreshold(mobThreshold).setMaxVersions(1).build();
+ tableDescriptor = UTIL.createModifyableTableDescriptor(name.getMethodName())
+ .modifyColumnFamily(familyDescriptor).build();
RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
region = HBaseTestingUtility.createRegionAndWAL(regionInfo,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
index eec4e99..32c62ca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
@@ -30,6 +30,7 @@ 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.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.ConnectionConfiguration;
import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -40,6 +41,7 @@ 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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
import org.apache.hadoop.hbase.io.hfile.TestHFile;
@@ -73,8 +75,8 @@ public class TestMobStoreScanner {
protected final byte[] qf3 = Bytes.toBytes("qualifier3");
private static Table table;
private static Admin admin;
- private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
- private static TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
+ private static ColumnFamilyDescriptor familyDescriptor;
+ private static TableDescriptor tableDescriptor;
private static Random random = new Random();
private static long defaultThreshold = 10;
private FileSystem fs;
@@ -99,16 +101,13 @@ public class TestMobStoreScanner {
public void setUp(long threshold, TableName tn) throws Exception {
conf = TEST_UTIL.getConfiguration();
fs = FileSystem.get(conf);
- tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor(tn);
- familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
- familyDescriptor.setMobEnabled(true);
- familyDescriptor.setMobThreshold(threshold);
- familyDescriptor.setMaxVersions(4);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true)
+ .setMobThreshold(threshold).setMaxVersions(4).build();
+ tableDescriptor =
+ TableDescriptorBuilder.newBuilder(tn).setColumnFamily(familyDescriptor).build();
admin = TEST_UTIL.getAdmin();
admin.createTable(tableDescriptor);
- table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
- .getTable(tn);
+ table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()).getTable(tn);
}
/**
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 bc491c8..7d67387 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
@@ -344,10 +344,8 @@ public class TestNamespaceAuditor {
ADMIN.createNamespace(nspDesc);
final TableName tableTwo = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table2");
byte[] columnFamily = Bytes.toBytes("info");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableTwo);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamily));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableTwo)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(columnFamily)).build();
ADMIN.createTable(tableDescriptor, Bytes.toBytes("0"), Bytes.toBytes("9"), initialRegions);
Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
try (Table table = connection.getTable(tableTwo)) {
@@ -442,11 +440,8 @@ public class TestNamespaceAuditor {
ADMIN.createNamespace(nspDesc);
final TableName tableOne = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table1");
byte[] columnFamily = Bytes.toBytes("info");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableOne);
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamily));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableOne)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(columnFamily)).build();
MasterSyncObserver.throwExceptionInPreCreateTableAction = true;
try {
try {
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 a5a1d8f..db96342 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
@@ -296,19 +296,18 @@ public class TestAtomicOperation {
initHRegion(tableName, callingMethod, null, families);
}
- private void initHRegion (byte [] tableName, String callingMethod, int [] maxVersions,
- byte[] ... families)
- throws IOException {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
- int i=0;
- for(byte [] family : families) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
- familyDescriptor.setMaxVersions(maxVersions != null ? maxVersions[i++] : 1);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ private void initHRegion(byte[] tableName, String callingMethod, int[] maxVersions,
+ byte[]... families) throws IOException {
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName));
+
+ int i = 0;
+ for (byte[] family : families) {
+ ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(family)
+ .setMaxVersions(maxVersions != null ? maxVersions[i++] : 1).build();
+ builder.setColumnFamily(familyDescriptor);
}
+ TableDescriptor tableDescriptor = builder.build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
}
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 c3af661..69b1446 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
@@ -75,34 +75,25 @@ public class TestBlocksScanned {
@Test
public void testBlocksScanned() throws Exception {
- byte [] tableName = Bytes.toBytes("TestBlocksScanned");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
- .setMaxVersions(10)
- .setBlockCacheEnabled(true)
- .setBlocksize(BLOCK_SIZE)
- .setCompressionType(Compression.Algorithm.NONE)
- );
+ byte[] tableName = Bytes.toBytes("TestBlocksScanned");
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(10)
+ .setBlockCacheEnabled(true).setBlocksize(BLOCK_SIZE)
+ .setCompressionType(Compression.Algorithm.NONE).build())
+ .build();
_testBlocksScanned(tableDescriptor);
}
@Test
public void testBlocksScannedWithEncoding() throws Exception {
- byte [] tableName = Bytes.toBytes("TestBlocksScannedWithEncoding");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
- .setMaxVersions(10)
- .setBlockCacheEnabled(true)
- .setDataBlockEncoding(DataBlockEncoding.FAST_DIFF)
- .setBlocksize(BLOCK_SIZE)
- .setCompressionType(Compression.Algorithm.NONE)
- );
+ byte[] tableName = Bytes.toBytes("TestBlocksScannedWithEncoding");
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(10)
+ .setBlockCacheEnabled(true).setDataBlockEncoding(DataBlockEncoding.FAST_DIFF)
+ .setBlocksize(BLOCK_SIZE).setCompressionType(Compression.Algorithm.NONE).build())
+ .build();
_testBlocksScanned(tableDescriptor);
}
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 fd36b41..f1f26ba 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
@@ -256,21 +256,18 @@ public class TestBulkLoad {
return new Pair<>(new byte[]{0x00, 0x01, 0x02}, getNotExistFilePath());
}
-
private HRegion testRegionWithFamiliesAndSpecifiedTableName(TableName tableName,
byte[]... families) throws IOException {
RegionInfo hRegionInfo = RegionInfoBuilder.newBuilder(tableName).build();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+ TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
for (byte[] family : families) {
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
}
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
// TODO We need a way to do this without creating files
return HRegion.createHRegion(hRegionInfo, new Path(testFolder.newFolder().toURI()), conf,
- tableDescriptor, log);
+ builder.build(), log);
}
private HRegion testRegionWithFamilies(byte[]... families) throws IOException {
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 fc82991..cf39dc0 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
@@ -40,6 +40,7 @@ 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.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -71,14 +72,10 @@ public class TestColumnSeeking {
byte[] familyBytes = Bytes.toBytes("Family");
TableName table = TableName.valueOf(name.getMethodName());
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyBytes)
- .setMaxVersions(1000);
- familyDescriptor.setMaxVersions(3);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(table);
-
- tableDescriptor.setColumnFamily(familyDescriptor);
+ ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(familyBytes)
+ .setMaxVersions(1000).setMaxVersions(3).build();
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(table).setColumnFamily(familyDescriptor).build();
RegionInfo info = RegionInfoBuilder.newBuilder(table).build();
// Set this so that the archiver writes to the temp dir as well.
HRegion region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
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 cc2921b..f191e48 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
@@ -25,11 +25,11 @@ 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.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -112,11 +112,9 @@ public class TestCompactSplitThread {
Configuration conf = TEST_UTIL.getConfiguration();
Connection conn = ConnectionFactory.createConnection(conf);
try {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
- tableDescriptor.setCompactionEnabled(false);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCompactionEnabled(false)
+ .build();
TEST_UTIL.getAdmin().createTable(tableDescriptor);
TEST_UTIL.waitTableAvailable(tableName);
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
@@ -162,8 +160,8 @@ public class TestCompactSplitThread {
@Test
public void testFlushWithTableCompactionDisabled() throws Exception {
- HTableDescriptor htd = new HTableDescriptor(tableName);
- htd.setCompactionEnabled(false);
+ TableDescriptor htd =
+ TableDescriptorBuilder.newBuilder(tableName).setCompactionEnabled(false).build();
TEST_UTIL.createTable(htd, new byte[][] { family }, null);
// load the table
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 aeb5045..e4b720e 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
@@ -40,10 +40,12 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
@@ -108,12 +110,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000);
HBaseTestingUtility hbaseUtility = new HBaseTestingUtility(conf);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf("foobar"));
-
- tableDescriptor.setColumnFamily(familyDescriptor);
+ ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.of(FAMILY);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf("foobar"))
+ .setColumnFamily(familyDescriptor).build();
RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf("foobar")).build();
WAL wal = HBaseTestingUtility.createWal(conf, hbaseUtility.getDataTestDir(), info);
this.region = HRegion.createHRegion(info, hbaseUtility.getDataTestDir(), conf,
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 7c8494a..2174d51 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
@@ -52,11 +52,13 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTestConst;
import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Delete;
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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
@@ -99,7 +101,7 @@ public class TestCompaction {
protected Configuration conf = UTIL.getConfiguration();
private HRegion r = null;
- private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = null;
+ private TableDescriptor tableDescriptor = null;
private static final byte [] COLUMN_FAMILY = fam1;
private final byte [] STARTROW = Bytes.toBytes(START_KEY);
private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY;
@@ -129,17 +131,16 @@ public class TestCompaction {
@Before
public void setUp() throws Exception {
- this.tableDescriptor = UTIL.createModifyableTableDescriptor(name.getMethodName());
+ TableDescriptorBuilder builder = UTIL.createModifyableTableDescriptor(name.getMethodName());
if (name.getMethodName().equals("testCompactionSeqId")) {
UTIL.getConfiguration().set("hbase.hstore.compaction.kv.max", "10");
- UTIL.getConfiguration().set(
- DefaultStoreEngine.DEFAULT_COMPACTOR_CLASS_KEY,
- DummyCompactor.class.getName());
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
- familyDescriptor.setMaxVersions(65536);
- this.tableDescriptor.setColumnFamily(familyDescriptor);
+ UTIL.getConfiguration().set(DefaultStoreEngine.DEFAULT_COMPACTOR_CLASS_KEY,
+ DummyCompactor.class.getName());
+ ColumnFamilyDescriptor familyDescriptor =
+ ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(65536).build();
+ builder.setColumnFamily(familyDescriptor);
}
+ this.tableDescriptor = builder.build();
this.r = UTIL.createLocalHRegion(tableDescriptor, null, null);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
index 0173008..33dc8d5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -28,9 +28,11 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
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.regionserver.compactions.CompactionConfiguration;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
@@ -91,16 +93,15 @@ public class TestCompactionPolicy {
Path basedir = new Path(DIR);
String logName = "logs";
Path logdir = new Path(DIR, logName);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("family"));
+ ColumnFamilyDescriptor familyDescriptor =
+ ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("family"));
FileSystem fs = FileSystem.get(conf);
fs.delete(logdir, true);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(Bytes.toBytes("table")));
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(Bytes.toBytes("table")))
+ .setColumnFamily(familyDescriptor).build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
hlog = new FSHLog(fs, basedir, logName, conf);
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 f894787..7804d70 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
@@ -87,18 +87,13 @@ public class TestDeleteMobTable {
return mobVal;
}
- private TableDescriptorBuilder.ModifyableTableDescriptor createTableDescriptor(
- TableName tableName, boolean hasMob) {
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
+ private TableDescriptor createTableDescriptor(TableName tableName, boolean hasMob) {
+ ColumnFamilyDescriptorBuilder builder = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY);
if (hasMob) {
- familyDescriptor.setMobEnabled(true);
- familyDescriptor.setMobThreshold(0);
+ builder.setMobEnabled(true);
+ builder.setMobThreshold(0);
}
- tableDescriptor.setColumnFamily(familyDescriptor);
- return tableDescriptor;
+ return TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(builder.build()).build();
}
private Table createTableWithOneFile(TableDescriptor tableDescriptor) throws IOException {
@@ -173,12 +168,10 @@ public class TestDeleteMobTable {
@Test
public void testMobFamilyDelete() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- createTableDescriptor(tableName, true);
+ TableDescriptor tableDescriptor = createTableDescriptor(tableName, true);
ColumnFamilyDescriptor familyDescriptor = tableDescriptor.getColumnFamily(FAMILY);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
- Bytes.toBytes("family2")));
+ tableDescriptor = TableDescriptorBuilder.newBuilder(tableDescriptor)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("family2"))).build();
Table table = createTableWithOneFile(tableDescriptor);
try {
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 4cafac5..732d0f5 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
@@ -41,6 +41,7 @@ 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.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -201,13 +202,9 @@ public class TestFSErrorsExposed {
byte[] fam = Bytes.toBytes("fam");
Admin admin = util.getAdmin();
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam)
- .setMaxVersions(1)
- .setBlockCacheEnabled(false)
- );
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+ .newBuilder(fam).setMaxVersions(1).setBlockCacheEnabled(false).build()).build();
admin.createTable(tableDescriptor);
// Make a new Configuration so it makes a new connection that has the
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 fdda388..f44137d 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
@@ -3382,13 +3382,11 @@ public class TestHRegion {
byte[] value2 = Bytes.toBytes("value2");
final int maxVersions = 3;
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1);
- familyDescriptor.setMaxVersions(maxVersions);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf("testFilterAndColumnTracker"));
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf("testFilterAndColumnTracker"))
+ .setColumnFamily(
+ ColumnFamilyDescriptorBuilder.newBuilder(fam1).setMaxVersions(maxVersions).build())
+ .build();
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + ".log");
@@ -4695,14 +4693,10 @@ public class TestHRegion {
byte[] qf1 = Bytes.toBytes("col");
byte[] val1 = Bytes.toBytes("value1");
// Create Table
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1)
- .setMaxVersions(Integer.MAX_VALUE)
- .setBloomFilterType(BloomType.ROWCOL);
-
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam1)
+ .setMaxVersions(Integer.MAX_VALUE).setBloomFilterType(BloomType.ROWCOL).build())
+ .build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
int num_unique_rows = 10;
@@ -4752,13 +4746,10 @@ public class TestHRegion {
byte[] FAMILY = Bytes.toBytes("family");
// Create table
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
- .setMaxVersions(Integer.MAX_VALUE)
- .setBloomFilterType(BloomType.ROWCOL);
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TABLE));
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY)
+ .setMaxVersions(Integer.MAX_VALUE).setBloomFilterType(BloomType.ROWCOL).build())
+ .build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
// For row:0, col:0: insert versions 1 through 5.
@@ -4797,14 +4788,10 @@ public class TestHRegion {
byte[] familyName = Bytes.toBytes("familyName");
// Create Table
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
- .setMaxVersions(Integer.MAX_VALUE)
- .setBloomFilterType(BloomType.ROWCOL);
-
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(familyName)
+ .setMaxVersions(Integer.MAX_VALUE).setBloomFilterType(BloomType.ROWCOL).build())
+ .build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
// Insert some data
@@ -5345,14 +5332,13 @@ public class TestHRegion {
Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
};
byte[] cq = Bytes.toBytes("cq");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(
TableName.valueOf(name.getMethodName()));
for (byte[] family : families) {
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
}
-
+ TableDescriptor tableDescriptor = builder.build();
long time = System.currentTimeMillis();
RegionInfo primaryHri = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
.setRegionId(time).setReplicaId(0).build();
@@ -5396,14 +5382,12 @@ public class TestHRegion {
Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
};
byte[] cq = Bytes.toBytes("cq");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
for (byte[] family : families) {
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
}
-
+ TableDescriptor tableDescriptor = builder.build();
long time = System.currentTimeMillis();
RegionInfo primaryHri = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
.setRegionId(time).setReplicaId(0).build();
@@ -5456,14 +5440,12 @@ public class TestHRegion {
Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
};
byte[] cq = Bytes.toBytes("cq");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
for (byte[] family : families) {
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+ builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
}
-
+ TableDescriptor tableDescriptor = builder.build();
long time = System.currentTimeMillis();
RegionInfo primaryHri = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
.setRegionId(time).setReplicaId(0).build();
@@ -6492,10 +6474,8 @@ public class TestHRegion {
@Test
public void testFlushedFileWithNoTags() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam1)).build();
RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
Path path = TEST_UTIL.getDataTestDir(getClass().getSimpleName());
region = HBaseTestingUtility.createRegionAndWAL(info, path,
@@ -6665,13 +6645,11 @@ public class TestHRegion {
final byte[] q3 = Bytes.toBytes("q3");
final byte[] q4 = Bytes.toBytes("q4");
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1);
- familyDescriptor.setTimeToLive(10); // 10 seconds
- tableDescriptor.setColumnFamily(familyDescriptor);
+ // 10 seconds
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam1).setTimeToLive(10).build())
+ .build();
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS);
@@ -7173,11 +7151,9 @@ public class TestHRegion {
final ServerName serverName = ServerName.valueOf(name.getMethodName(), 100, 42);
final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(
- TableName.valueOf(name.getMethodName()));
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1));
+ TableDescriptor tableDescriptor =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam1)).build();
RegionInfo hri = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
region = HRegion.openHRegion(hri, tableDescriptor, rss.getWAL(hri),
TEST_UTIL.getConfiguration(), rss, null);
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 7685815c..85848a0 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
@@ -37,6 +37,7 @@ 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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -81,10 +82,8 @@ public class TestHRegionOnCluster {
HMaster master = cluster.getMaster();
// Create table
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
- tableDescriptor.setColumnFamily(
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY));
+ TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
hbaseAdmin = master.getConnection().getAdmin();
hbaseAdmin.createTable(tableDescriptor);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java
index 8420358..13e3110 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Mutation;
@@ -29,6 +28,8 @@ import org.apache.hadoop.hbase.client.Put;
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.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
import org.apache.hadoop.hbase.testclassification.FilterTests;
@@ -73,7 +74,8 @@ public class TestIsDeleteFailure {
@Test
public void testIsDeleteFailure() throws Exception {
- final HTableDescriptor table = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
+ final TableDescriptor table =
+ TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
final byte[] family = Bytes.toBytes("0");
final byte[] c1 = Bytes.toBytes("C01");
final byte[] c2 = Bytes.toBytes("C02");
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 d11e046..ca3ae63 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
@@ -21,7 +21,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
-
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -29,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.StartMiniClusterOption;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
@@ -52,6 +52,7 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
import org.apache.hbase.thirdparty.org.apache.commons.cli.GnuParser;
@@ -111,14 +112,14 @@ public class TestJoinedScanners {
byte[][] families = {cf_essential, cf_joined};
final TableName tableName = TableName.valueOf(name.getMethodName());
- TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
- new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(tableName);
for (byte[] family : families) {
- ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
- familyDescriptor.setDataBlockEncoding(blockEncoding);
- tableDescriptor.setColumnFamily(familyDescriptor);
+ ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(family)
+ .setDataBlockEncoding(blockEncoding).build();
+ builder.setColumnFamily(familyDescriptor);
}
+ TableDescriptor tableDescriptor = builder.build();
TEST_UTIL.getAdmin().createTable(tableDescriptor);
Table ht = TEST_UTIL.getConnection().getTable(tableName);
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 b2c9ef2..ca9e0f6 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
@@ -23,6 +23,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 java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -31,7 +32,6 @@ 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.KeepDeletedCells;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.TableName;
@@ -40,6 +40,7 @@ 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.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -104,7 +105,7 @@ public class TestKeepDeletes {
@Test
public void testBasicScenario() throws Exception {
// keep 3 versions, rows do not expire
- HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+ TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
HConstants.FOREVER, KeepDeletedCells.TRUE);
HRegion region = hbu.createLocalHRegion(htd, null, null);
@@ -201,7 +202,7 @@ public class TestKeepDeletes {
@Test
public void testRawScanWithoutKeepingDeletes() throws Exception {
// KEEP_DELETED_CELLS is NOT enabled
- HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+ TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
HConstants.FOREVER, KeepDeletedCells.FALSE);
HRegion region = hbu.createLocalHRegion(htd, null, null);
@@ -246,7 +247,7 @@ public class TestKeepDeletes {
@Test
public void testWithoutKeepingDeletes() throws Exception {
// KEEP_DELETED_CELLS is NOT enabled
- HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+ TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
HConstants.FOREVER, KeepDeletedCells.FALSE);
HRegion region = hbu.createLocalHRegion(htd, null, null);
@@ -300,7 +301,7 @@ public class TestKeepDeletes {
*/
@Test
public void testRawScanWithColumns() throws Exception {
- HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+ TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
HConstants.FOREVER, KeepDeletedCells.TRUE);
Region region = hbu.createLocalHRegion(htd, null, null);
@@ -324,7 +325,7 @@ public class TestKeepDeletes {
*/
@Test
public void testRawScan() throws Exception {
- HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+ TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
HConstants.FOREVER, KeepDeletedCells.TRUE);
Region region = hbu.createLocalHRegion(htd, null, null);
@@ -414,7 +415,7 @@ public class TestKeepDeletes {
*/
... 3784 lines suppressed ...