HBASE-23662 : Replace HColumnDescriptor(String cf) with ColumnFamilyDescriptor
Signed-off-by: Peter Somogyi <psomogyi@apache.org> Signed-off-by: Jan Hentschel <janh@apache.org>
This commit is contained in:
parent
4ed466966c
commit
2e0edacf72
|
@ -101,21 +101,6 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
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(Bytes.toBytes(familyName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct a column descriptor specifying only the family name
|
||||
* The other attributes are defaulted.
|
||||
|
|
|
@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
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.compress.Compression.Algorithm;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
|
@ -31,7 +30,6 @@ 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.Assert;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
|
@ -112,20 +110,6 @@ public class TestHColumnDescriptor {
|
|||
new HColumnDescriptor(Bytes.toBytes(""));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that we add and remove strings from configuration properly.
|
||||
*/
|
||||
@Test
|
||||
public void testAddGetRemoveConfiguration() throws Exception {
|
||||
HColumnDescriptor desc = new HColumnDescriptor("foo");
|
||||
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 testMobValuesInHColumnDescriptorShouldReadable() {
|
||||
boolean isMob = true;
|
||||
|
@ -161,37 +145,4 @@ public class TestHColumnDescriptor {
|
|||
BuilderStyleTest.assertClassesAreBuilderStyle(HColumnDescriptor.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetTimeToLive() throws HBaseException {
|
||||
String ttl;
|
||||
HColumnDescriptor desc = new HColumnDescriptor("foo");
|
||||
|
||||
ttl = "50000";
|
||||
desc.setTimeToLive(ttl);
|
||||
Assert.assertEquals(50000, desc.getTimeToLive());
|
||||
|
||||
ttl = "50000 seconds";
|
||||
desc.setTimeToLive(ttl);
|
||||
Assert.assertEquals(50000, desc.getTimeToLive());
|
||||
|
||||
ttl = "";
|
||||
desc.setTimeToLive(ttl);
|
||||
Assert.assertEquals(0, desc.getTimeToLive());
|
||||
|
||||
ttl = "FOREVER";
|
||||
desc.setTimeToLive(ttl);
|
||||
Assert.assertEquals(HConstants.FOREVER, desc.getTimeToLive());
|
||||
|
||||
ttl = "1 HOUR 10 minutes 1 second";
|
||||
desc.setTimeToLive(ttl);
|
||||
Assert.assertEquals(4201, desc.getTimeToLive());
|
||||
|
||||
ttl = "500 Days 23 HOURS";
|
||||
desc.setTimeToLive(ttl);
|
||||
Assert.assertEquals(43282800, desc.getTimeToLive());
|
||||
|
||||
ttl = "43282800 SECONDS (500 Days 23 hours)";
|
||||
desc.setTimeToLive(ttl);
|
||||
Assert.assertEquals(43282800, desc.getTimeToLive());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
|
@ -37,19 +38,23 @@ import org.apache.hadoop.hbase.Coprocessor;
|
|||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.RegionMetrics;
|
||||
import org.apache.hadoop.hbase.ServerMetrics;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
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.CoprocessorDescriptor;
|
||||
import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder;
|
||||
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.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.TestServerCustomProtocol;
|
||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ClassLoaderTestHelper;
|
||||
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
|
@ -167,14 +172,15 @@ public class TestClassLoading {
|
|||
LOG.info("Copied jar file to HDFS: " + jarFileOnHDFS2);
|
||||
|
||||
// create a table that references the coprocessors
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor("test"));
|
||||
// without configuration values
|
||||
htd.setValue("COPROCESSOR$1", jarFileOnHDFS1.toString() + "|" + cpName1 +
|
||||
"|" + Coprocessor.PRIORITY_USER);
|
||||
// with configuration values
|
||||
htd.setValue("COPROCESSOR$2", jarFileOnHDFS2.toString() + "|" + cpName2 +
|
||||
"|" + Coprocessor.PRIORITY_USER + "|k1=v1,k2=v2,k3=v3");
|
||||
TableDescriptorBuilder tdb = TableDescriptorBuilder.newBuilder(tableName);
|
||||
tdb.setColumnFamily(ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("test")).build());
|
||||
// without configuration values
|
||||
tdb.setValue("COPROCESSOR$1", jarFileOnHDFS1 + "|" + cpName1
|
||||
+ "|" + Coprocessor.PRIORITY_USER);
|
||||
// with configuration values
|
||||
tdb.setValue("COPROCESSOR$2", jarFileOnHDFS2 + "|" + cpName2
|
||||
+ "|" + Coprocessor.PRIORITY_USER + "|k1=v1,k2=v2,k3=v3");
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
if (admin.tableExists(tableName)) {
|
||||
if (admin.isTableEnabled(tableName)) {
|
||||
|
@ -185,8 +191,9 @@ public class TestClassLoading {
|
|||
CoprocessorClassLoader.clearCache();
|
||||
byte[] startKey = {10, 63};
|
||||
byte[] endKey = {12, 43};
|
||||
admin.createTable(htd, startKey, endKey, 4);
|
||||
waitForTable(htd.getTableName());
|
||||
TableDescriptor tableDescriptor = tdb.build();
|
||||
admin.createTable(tableDescriptor, startKey, endKey, 4);
|
||||
waitForTable(tableDescriptor.getTableName());
|
||||
|
||||
// verify that the coprocessors were loaded
|
||||
boolean foundTableRegion=false;
|
||||
|
@ -253,13 +260,15 @@ public class TestClassLoading {
|
|||
File jarFile = buildCoprocessorJar(cpName3);
|
||||
|
||||
// create a table that references the jar
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(cpName3));
|
||||
htd.addFamily(new HColumnDescriptor("test"));
|
||||
htd.setValue("COPROCESSOR$1", getLocalPath(jarFile) + "|" + cpName3 + "|" +
|
||||
TableDescriptorBuilder tdb = TableDescriptorBuilder.newBuilder(TableName.valueOf(cpName3));
|
||||
tdb.setColumnFamily(ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("test")).build());
|
||||
tdb.setValue("COPROCESSOR$1", getLocalPath(jarFile) + "|" + cpName3 + "|" +
|
||||
Coprocessor.PRIORITY_USER);
|
||||
TableDescriptor tableDescriptor = tdb.build();
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
admin.createTable(htd);
|
||||
waitForTable(htd.getTableName());
|
||||
admin.createTable(tableDescriptor);
|
||||
waitForTable(tableDescriptor.getTableName());
|
||||
|
||||
// verify that the coprocessor was loaded
|
||||
boolean found = false;
|
||||
|
@ -278,13 +287,15 @@ public class TestClassLoading {
|
|||
File jarFile = buildCoprocessorJar(cpName4);
|
||||
|
||||
// create a table that references the jar
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(cpName4));
|
||||
htd.addFamily(new HColumnDescriptor("test"));
|
||||
htd.setValue("COPROCESSOR$1", getLocalPath(jarFile) + "|" + cpName4 + "|" +
|
||||
TableDescriptorBuilder tdb = TableDescriptorBuilder.newBuilder(TableName.valueOf(cpName4));
|
||||
tdb.setColumnFamily(ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("test")).build());
|
||||
tdb.setValue("COPROCESSOR$1", getLocalPath(jarFile) + "|" + cpName4 + "|" +
|
||||
Coprocessor.PRIORITY_USER);
|
||||
TableDescriptor tableDescriptor = tdb.build();
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
admin.createTable(htd);
|
||||
waitForTable(htd.getTableName());
|
||||
admin.createTable(tableDescriptor);
|
||||
waitForTable(tableDescriptor.getTableName());
|
||||
|
||||
// verify that the coprocessor was loaded correctly
|
||||
boolean found = false;
|
||||
|
@ -325,23 +336,35 @@ public class TestClassLoading {
|
|||
" | org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver | | k=v ";
|
||||
|
||||
// create a table that references the jar
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor("test"));
|
||||
TableDescriptorBuilder tdb = TableDescriptorBuilder.newBuilder(tableName);
|
||||
tdb.setColumnFamily(ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("test")).build());
|
||||
|
||||
// add 3 coprocessors by setting htd attributes directly.
|
||||
htd.setValue(cpKey1, cpValue1);
|
||||
htd.setValue(cpKey2, cpValue2);
|
||||
htd.setValue(cpKey3, cpValue3);
|
||||
tdb.setValue(cpKey1, cpValue1);
|
||||
tdb.setValue(cpKey2, cpValue2);
|
||||
tdb.setValue(cpKey3, cpValue3);
|
||||
|
||||
// add 2 coprocessor by using new htd.setCoprocessor() api
|
||||
htd.addCoprocessor(cpName5, new Path(getLocalPath(jarFile5)),
|
||||
Coprocessor.PRIORITY_USER, null);
|
||||
CoprocessorDescriptor coprocessorDescriptor = CoprocessorDescriptorBuilder
|
||||
.newBuilder(cpName5)
|
||||
.setJarPath(new Path(getLocalPath(jarFile5)).toString())
|
||||
.setPriority(Coprocessor.PRIORITY_USER)
|
||||
.setProperties(Collections.emptyMap())
|
||||
.build();
|
||||
tdb.setCoprocessor(coprocessorDescriptor);
|
||||
Map<String, String> kvs = new HashMap<>();
|
||||
kvs.put("k1", "v1");
|
||||
kvs.put("k2", "v2");
|
||||
kvs.put("k3", "v3");
|
||||
htd.addCoprocessor(cpName6, new Path(getLocalPath(jarFile6)),
|
||||
Coprocessor.PRIORITY_USER, kvs);
|
||||
|
||||
coprocessorDescriptor = CoprocessorDescriptorBuilder
|
||||
.newBuilder(cpName6)
|
||||
.setJarPath(new Path(getLocalPath(jarFile6)).toString())
|
||||
.setPriority(Coprocessor.PRIORITY_USER)
|
||||
.setProperties(kvs)
|
||||
.build();
|
||||
tdb.setCoprocessor(coprocessorDescriptor);
|
||||
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
if (admin.tableExists(tableName)) {
|
||||
|
@ -350,8 +373,10 @@ public class TestClassLoading {
|
|||
}
|
||||
admin.deleteTable(tableName);
|
||||
}
|
||||
admin.createTable(htd);
|
||||
waitForTable(htd.getTableName());
|
||||
|
||||
TableDescriptor tableDescriptor = tdb.build();
|
||||
admin.createTable(tableDescriptor);
|
||||
waitForTable(tableDescriptor.getTableName());
|
||||
|
||||
// verify that the coprocessor was loaded
|
||||
boolean found_2 = false, found_1 = false, found_3 = false,
|
||||
|
@ -426,14 +451,15 @@ public class TestClassLoading {
|
|||
LOG.info("Copied jar file to HDFS: " + jarFileOnHDFS);
|
||||
|
||||
// create a table that references the coprocessors
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor("test"));
|
||||
TableDescriptorBuilder tdb = TableDescriptorBuilder.newBuilder(tableName);
|
||||
tdb.setColumnFamily(ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("test")).build());
|
||||
// without configuration values
|
||||
htd.setValue("COPROCESSOR$1", jarFileOnHDFS.toString() + "|" + cpName1 +
|
||||
"|" + Coprocessor.PRIORITY_USER);
|
||||
tdb.setValue("COPROCESSOR$1", jarFileOnHDFS + "|" + cpName1
|
||||
+ "|" + Coprocessor.PRIORITY_USER);
|
||||
// with configuration values
|
||||
htd.setValue("COPROCESSOR$2", jarFileOnHDFS.toString() + "|" + cpName2 +
|
||||
"|" + Coprocessor.PRIORITY_USER + "|k1=v1,k2=v2,k3=v3");
|
||||
tdb.setValue("COPROCESSOR$2", jarFileOnHDFS + "|" + cpName2
|
||||
+ "|" + Coprocessor.PRIORITY_USER + "|k1=v1,k2=v2,k3=v3");
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
if (admin.tableExists(tableName)) {
|
||||
if (admin.isTableEnabled(tableName)) {
|
||||
|
@ -441,8 +467,10 @@ public class TestClassLoading {
|
|||
}
|
||||
admin.deleteTable(tableName);
|
||||
}
|
||||
admin.createTable(htd);
|
||||
waitForTable(htd.getTableName());
|
||||
|
||||
TableDescriptor tableDescriptor = tdb.build();
|
||||
admin.createTable(tableDescriptor);
|
||||
waitForTable(tableDescriptor.getTableName());
|
||||
|
||||
// verify that the coprocessors were loaded
|
||||
boolean found1 = false, found2 = false, found2_k1 = false,
|
||||
|
|
|
@ -22,7 +22,12 @@ import java.io.IOException;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
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;
|
||||
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.RegionSplitter;
|
||||
import org.apache.hadoop.hbase.util.RegionSplitter.SplitAlgorithm;
|
||||
import org.junit.After;
|
||||
|
@ -109,15 +114,21 @@ public class IntegrationTestManyRegions {
|
|||
|
||||
@Test
|
||||
public void testCreateTableWithRegions() throws Exception {
|
||||
HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
|
||||
desc.addFamily(new HColumnDescriptor("cf"));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("cf"))
|
||||
.build();
|
||||
TableDescriptor tableDescriptor =
|
||||
TableDescriptorBuilder.newBuilder(TABLE_NAME)
|
||||
.setColumnFamily(columnFamilyDescriptor)
|
||||
.build();
|
||||
|
||||
SplitAlgorithm algo = new RegionSplitter.HexStringSplit();
|
||||
byte[][] splits = algo.split(REGION_COUNT);
|
||||
|
||||
LOG.info(String.format("Creating table %s with %d splits.", TABLE_NAME, REGION_COUNT));
|
||||
long startTime = System.currentTimeMillis();
|
||||
try {
|
||||
admin.createTable(desc, splits);
|
||||
admin.createTable(tableDescriptor, splits);
|
||||
LOG.info(String.format("Pre-split table created successfully in %dms.",
|
||||
(System.currentTimeMillis() - startTime)));
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -54,10 +54,8 @@ import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
|||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.HadoopShims;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.PerformanceEvaluation;
|
||||
|
@ -67,6 +65,8 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
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;
|
||||
|
@ -76,6 +76,7 @@ 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.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
|
||||
|
@ -856,15 +857,21 @@ public class TestHFileOutputFormat2 {
|
|||
|
||||
private void setupMockColumnFamiliesForCompression(Table table,
|
||||
Map<String, Compression.Algorithm> familyToCompression) throws IOException {
|
||||
HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAMES[0]);
|
||||
|
||||
TableDescriptorBuilder mockTableDescriptor =
|
||||
TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);
|
||||
for (Entry<String, Compression.Algorithm> entry : familyToCompression.entrySet()) {
|
||||
mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey())
|
||||
.setMaxVersions(1)
|
||||
.setCompressionType(entry.getValue())
|
||||
.setBlockCacheEnabled(false)
|
||||
.setTimeToLive(0));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(entry.getKey()))
|
||||
.setMaxVersions(1)
|
||||
.setCompressionType(entry.getValue())
|
||||
.setBlockCacheEnabled(false)
|
||||
.setTimeToLive(0)
|
||||
.build();
|
||||
|
||||
mockTableDescriptor.setColumnFamily(columnFamilyDescriptor);
|
||||
}
|
||||
Mockito.doReturn(mockTableDescriptor).when(table).getDescriptor();
|
||||
Mockito.doReturn(mockTableDescriptor.build()).when(table).getDescriptor();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -929,13 +936,16 @@ public class TestHFileOutputFormat2 {
|
|||
|
||||
private void setupMockColumnFamiliesForBloomType(Table table,
|
||||
Map<String, BloomType> familyToDataBlockEncoding) throws IOException {
|
||||
HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAMES[0]);
|
||||
TableDescriptorBuilder mockTableDescriptor =
|
||||
TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);
|
||||
for (Entry<String, BloomType> entry : familyToDataBlockEncoding.entrySet()) {
|
||||
mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey())
|
||||
.setMaxVersions(1)
|
||||
.setBloomFilterType(entry.getValue())
|
||||
.setBlockCacheEnabled(false)
|
||||
.setTimeToLive(0));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(entry.getKey()))
|
||||
.setMaxVersions(1)
|
||||
.setBloomFilterType(entry.getValue())
|
||||
.setBlockCacheEnabled(false)
|
||||
.setTimeToLive(0).build();
|
||||
mockTableDescriptor.setColumnFamily(columnFamilyDescriptor);
|
||||
}
|
||||
Mockito.doReturn(mockTableDescriptor).when(table).getDescriptor();
|
||||
}
|
||||
|
@ -1001,13 +1011,16 @@ public class TestHFileOutputFormat2 {
|
|||
|
||||
private void setupMockColumnFamiliesForBlockSize(Table table,
|
||||
Map<String, Integer> familyToDataBlockEncoding) throws IOException {
|
||||
HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAMES[0]);
|
||||
TableDescriptorBuilder mockTableDescriptor =
|
||||
TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);
|
||||
for (Entry<String, Integer> entry : familyToDataBlockEncoding.entrySet()) {
|
||||
mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey())
|
||||
.setMaxVersions(1)
|
||||
.setBlocksize(entry.getValue())
|
||||
.setBlockCacheEnabled(false)
|
||||
.setTimeToLive(0));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(entry.getKey()))
|
||||
.setMaxVersions(1)
|
||||
.setBlocksize(entry.getValue())
|
||||
.setBlockCacheEnabled(false)
|
||||
.setTimeToLive(0).build();
|
||||
mockTableDescriptor.setColumnFamily(columnFamilyDescriptor);
|
||||
}
|
||||
Mockito.doReturn(mockTableDescriptor).when(table).getDescriptor();
|
||||
}
|
||||
|
@ -1077,13 +1090,16 @@ public class TestHFileOutputFormat2 {
|
|||
|
||||
private void setupMockColumnFamiliesForDataBlockEncoding(Table table,
|
||||
Map<String, DataBlockEncoding> familyToDataBlockEncoding) throws IOException {
|
||||
HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAMES[0]);
|
||||
TableDescriptorBuilder mockTableDescriptor =
|
||||
TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);
|
||||
for (Entry<String, DataBlockEncoding> entry : familyToDataBlockEncoding.entrySet()) {
|
||||
mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey())
|
||||
.setMaxVersions(1)
|
||||
.setDataBlockEncoding(entry.getValue())
|
||||
.setBlockCacheEnabled(false)
|
||||
.setTimeToLive(0));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(entry.getKey()))
|
||||
.setMaxVersions(1)
|
||||
.setDataBlockEncoding(entry.getValue())
|
||||
.setBlockCacheEnabled(false)
|
||||
.setTimeToLive(0).build();
|
||||
mockTableDescriptor.setColumnFamily(columnFamilyDescriptor);
|
||||
}
|
||||
Mockito.doReturn(mockTableDescriptor).when(table).getDescriptor();
|
||||
}
|
||||
|
@ -1142,10 +1158,12 @@ public class TestHFileOutputFormat2 {
|
|||
// Setup table descriptor
|
||||
Table table = Mockito.mock(Table.class);
|
||||
RegionLocator regionLocator = Mockito.mock(RegionLocator.class);
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLE_NAMES[0]);
|
||||
Mockito.doReturn(htd).when(table).getDescriptor();
|
||||
for (HColumnDescriptor hcd: HBaseTestingUtility.generateColumnDescriptors()) {
|
||||
htd.addFamily(hcd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);
|
||||
|
||||
Mockito.doReturn(tableDescriptorBuilder.build()).when(table).getDescriptor();
|
||||
for (ColumnFamilyDescriptor hcd : HBaseTestingUtility.generateColumnDescriptors()) {
|
||||
tableDescriptorBuilder.setColumnFamily(hcd);
|
||||
}
|
||||
|
||||
// set up the table to return some mock keys
|
||||
|
@ -1170,7 +1188,8 @@ public class TestHFileOutputFormat2 {
|
|||
writer = hof.getRecordWriter(context);
|
||||
|
||||
// write out random rows
|
||||
writeRandomKeyValues(writer, context, htd.getFamiliesKeys(), ROWSPERSPLIT);
|
||||
writeRandomKeyValues(writer, context,
|
||||
tableDescriptorBuilder.build().getColumnFamilyNames(), ROWSPERSPLIT);
|
||||
writer.close(context);
|
||||
|
||||
// Make sure that a directory was created for every CF
|
||||
|
@ -1180,10 +1199,11 @@ public class TestHFileOutputFormat2 {
|
|||
hof.getOutputCommitter(context).commitTask(context);
|
||||
hof.getOutputCommitter(context).commitJob(context);
|
||||
FileStatus[] families = FSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));
|
||||
assertEquals(htd.getFamilies().size(), families.length);
|
||||
assertEquals(tableDescriptorBuilder.build().getColumnFamilies().length, families.length);
|
||||
for (FileStatus f : families) {
|
||||
String familyStr = f.getPath().getName();
|
||||
HColumnDescriptor hcd = htd.getFamily(Bytes.toBytes(familyStr));
|
||||
ColumnFamilyDescriptor hcd = tableDescriptorBuilder.build()
|
||||
.getColumnFamily(Bytes.toBytes(familyStr));
|
||||
// verify that the compression on this file matches the configured
|
||||
// compression
|
||||
Path dataFilePath = fs.listStatus(f.getPath())[0].getPath();
|
||||
|
|
|
@ -32,16 +32,20 @@ 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.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||
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.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.rest.model.ColumnSchemaModel;
|
||||
import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -101,30 +105,36 @@ public class SchemaResource extends ResourceBase {
|
|||
.build();
|
||||
}
|
||||
try {
|
||||
HTableDescriptor htd = new HTableDescriptor(name);
|
||||
for (Map.Entry<QName,Object> e: model.getAny().entrySet()) {
|
||||
htd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(name);
|
||||
for (Map.Entry<QName, Object> e : model.getAny().entrySet()) {
|
||||
tableDescriptorBuilder.setValue(e.getKey().getLocalPart(), e.getValue().toString());
|
||||
}
|
||||
for (ColumnSchemaModel family: model.getColumns()) {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family.getName());
|
||||
for (Map.Entry<QName,Object> e: family.getAny().entrySet()) {
|
||||
hcd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
|
||||
for (ColumnSchemaModel family : model.getColumns()) {
|
||||
ColumnFamilyDescriptorBuilder columnFamilyDescriptorBuilder =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family.getName()));
|
||||
for (Map.Entry<QName, Object> e : family.getAny().entrySet()) {
|
||||
columnFamilyDescriptorBuilder.setValue(e.getKey().getLocalPart(),
|
||||
e.getValue().toString());
|
||||
}
|
||||
htd.addFamily(hcd);
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptorBuilder.build());
|
||||
}
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
if (admin.tableExists(name)) {
|
||||
admin.disableTable(name);
|
||||
admin.modifyTable(htd);
|
||||
admin.modifyTable(tableDescriptor);
|
||||
admin.enableTable(name);
|
||||
servlet.getMetrics().incrementSucessfulPutRequests(1);
|
||||
} else try {
|
||||
admin.createTable(htd);
|
||||
servlet.getMetrics().incrementSucessfulPutRequests(1);
|
||||
} catch (TableExistsException e) {
|
||||
// race, someone else created a table with the same name
|
||||
return Response.status(Response.Status.NOT_MODIFIED)
|
||||
.type(MIMETYPE_TEXT).entity("Not modified" + CRLF)
|
||||
.build();
|
||||
} else {
|
||||
try {
|
||||
admin.createTable(tableDescriptor);
|
||||
servlet.getMetrics().incrementSucessfulPutRequests(1);
|
||||
} catch (TableExistsException e) {
|
||||
// race, someone else created a table with the same name
|
||||
return Response.status(Response.Status.NOT_MODIFIED)
|
||||
.type(MIMETYPE_TEXT).entity("Not modified" + CRLF)
|
||||
.build();
|
||||
}
|
||||
}
|
||||
return Response.created(uriInfo.getAbsolutePath()).build();
|
||||
} catch (Exception e) {
|
||||
|
@ -142,18 +152,23 @@ public class SchemaResource extends ResourceBase {
|
|||
.build();
|
||||
}
|
||||
try {
|
||||
HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(name));
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(admin.getDescriptor(name));
|
||||
admin.disableTable(name);
|
||||
try {
|
||||
for (ColumnSchemaModel family: model.getColumns()) {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family.getName());
|
||||
for (Map.Entry<QName,Object> e: family.getAny().entrySet()) {
|
||||
hcd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
|
||||
for (ColumnSchemaModel family : model.getColumns()) {
|
||||
ColumnFamilyDescriptorBuilder columnFamilyDescriptorBuilder =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family.getName()));
|
||||
for (Map.Entry<QName, Object> e : family.getAny().entrySet()) {
|
||||
columnFamilyDescriptorBuilder.setValue(e.getKey().getLocalPart(),
|
||||
e.getValue().toString());
|
||||
}
|
||||
if (htd.hasFamily(hcd.getName())) {
|
||||
admin.modifyColumnFamily(name, hcd);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor = columnFamilyDescriptorBuilder.build();
|
||||
if (tableDescriptor.hasColumnFamily(columnFamilyDescriptor.getName())) {
|
||||
admin.modifyColumnFamily(name, columnFamilyDescriptor);
|
||||
} else {
|
||||
admin.addColumnFamily(name, hcd);
|
||||
admin.addColumnFamily(name, columnFamilyDescriptor);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -33,6 +33,9 @@ import javax.xml.bind.annotation.XmlElement;
|
|||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
import javax.xml.namespace.QName;
|
||||
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -340,19 +343,21 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
|
|||
* @return a table descriptor
|
||||
*/
|
||||
@JsonIgnore
|
||||
public HTableDescriptor getTableDescriptor() {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(getName()));
|
||||
public TableDescriptor getTableDescriptor() {
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(getName()));
|
||||
for (Map.Entry<QName, Object> e: getAny().entrySet()) {
|
||||
htd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
|
||||
tableDescriptorBuilder.setValue(e.getKey().getLocalPart(), e.getValue().toString());
|
||||
}
|
||||
for (ColumnSchemaModel column: getColumns()) {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(column.getName());
|
||||
ColumnFamilyDescriptorBuilder cfdb = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(column.getName()));
|
||||
for (Map.Entry<QName, Object> e: column.getAny().entrySet()) {
|
||||
hcd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
|
||||
cfdb.setValue(e.getKey().getLocalPart(), e.getValue().toString());
|
||||
}
|
||||
htd.addFamily(hcd);
|
||||
tableDescriptorBuilder.setColumnFamily(cfdb.build());
|
||||
}
|
||||
return htd;
|
||||
return tableDescriptorBuilder.build();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -35,10 +35,11 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.rest.client.Client;
|
||||
import org.apache.hadoop.hbase.rest.client.Cluster;
|
||||
import org.apache.hadoop.hbase.rest.client.Response;
|
||||
|
@ -113,10 +114,15 @@ public class RowResourceBase {
|
|||
if (admin.tableExists(TABLE_NAME)) {
|
||||
TEST_UTIL.deleteTable(TABLE_NAME);
|
||||
}
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
|
||||
htd.addFamily(new HColumnDescriptor(CFA));
|
||||
htd.addFamily(new HColumnDescriptor(CFB));
|
||||
admin.createTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(CFA)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(CFB)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
|
@ -27,13 +27,14 @@ import java.util.zip.GZIPInputStream;
|
|||
import java.util.zip.GZIPOutputStream;
|
||||
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.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.Get;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.rest.client.Client;
|
||||
import org.apache.hadoop.hbase.rest.client.Cluster;
|
||||
import org.apache.hadoop.hbase.rest.client.Response;
|
||||
|
@ -77,9 +78,12 @@ public class TestGzipFilter {
|
|||
if (admin.tableExists(TABLE)) {
|
||||
return;
|
||||
}
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLE);
|
||||
htd.addFamily(new HColumnDescriptor(CFA));
|
||||
admin.createTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TABLE);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFA)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
|
|
@ -32,10 +32,11 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.rest.client.Client;
|
||||
import org.apache.hadoop.hbase.rest.client.Cluster;
|
||||
import org.apache.hadoop.hbase.rest.client.Response;
|
||||
|
@ -117,10 +118,15 @@ public class TestMultiRowResource {
|
|||
if (admin.tableExists(TABLE)) {
|
||||
return;
|
||||
}
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLE);
|
||||
htd.addFamily(new HColumnDescriptor(CFA));
|
||||
htd.addFamily(new HColumnDescriptor(CFB));
|
||||
admin.createTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TABLE);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFA)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFB)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
|
|
@ -38,11 +38,12 @@ import javax.xml.bind.JAXBException;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.rest.client.Client;
|
||||
import org.apache.hadoop.hbase.rest.client.Cluster;
|
||||
import org.apache.hadoop.hbase.rest.client.Response;
|
||||
|
@ -186,15 +187,17 @@ public class TestNamespacesInstanceResource {
|
|||
admin.createNamespace(nsd);
|
||||
|
||||
// Create two tables via admin.
|
||||
HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
|
||||
TableName tn1 = TableName.valueOf(nsName + ":table1");
|
||||
HTableDescriptor table = new HTableDescriptor(tn1);
|
||||
table.addFamily(colDesc);
|
||||
admin.createTable(table);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tn1);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf1")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
TableName tn2 = TableName.valueOf(nsName + ":table2");
|
||||
table = new HTableDescriptor(tn2);
|
||||
table.addFamily(colDesc);
|
||||
admin.createTable(table);
|
||||
tableDescriptorBuilder = TableDescriptorBuilder.newBuilder(tn2);
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
|
||||
Map<String, String> nsProperties = new HashMap<>();
|
||||
nsProperties.put("key1", "value1");
|
||||
|
|
|
@ -37,15 +37,16 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
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;
|
||||
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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.rest.client.Client;
|
||||
import org.apache.hadoop.hbase.rest.client.Cluster;
|
||||
import org.apache.hadoop.hbase.rest.client.Response;
|
||||
|
@ -186,17 +187,28 @@ public class TestScannerResource {
|
|||
if (admin.tableExists(TABLE)) {
|
||||
return;
|
||||
}
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLE);
|
||||
htd.addFamily(new HColumnDescriptor(CFA));
|
||||
htd.addFamily(new HColumnDescriptor(CFB));
|
||||
admin.createTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TABLE);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFA)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFB)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
expectedRows1 = insertData(TEST_UTIL.getConfiguration(), TABLE, COLUMN_1, 1.0);
|
||||
expectedRows2 = insertData(TEST_UTIL.getConfiguration(), TABLE, COLUMN_2, 0.5);
|
||||
|
||||
htd = new HTableDescriptor(TABLE_TO_BE_DISABLED);
|
||||
htd.addFamily(new HColumnDescriptor(CFA));
|
||||
htd.addFamily(new HColumnDescriptor(CFB));
|
||||
admin.createTable(htd);
|
||||
tableDescriptorBuilder=TableDescriptorBuilder.newBuilder(TABLE_TO_BE_DISABLED);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFA)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFB)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
|
|
@ -35,15 +35,16 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
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;
|
||||
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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
|
||||
import org.apache.hadoop.hbase.rest.client.Client;
|
||||
import org.apache.hadoop.hbase.rest.client.Cluster;
|
||||
|
@ -153,10 +154,15 @@ public class TestScannersWithLabels {
|
|||
if (admin.tableExists(TABLE)) {
|
||||
return;
|
||||
}
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLE);
|
||||
htd.addFamily(new HColumnDescriptor(CFA));
|
||||
htd.addFamily(new HColumnDescriptor(CFB));
|
||||
admin.createTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TABLE);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFA)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFB)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
insertData(TABLE, COLUMN_1, 1.0);
|
||||
insertData(TABLE, COLUMN_2, 0.5);
|
||||
}
|
||||
|
|
|
@ -50,10 +50,11 @@ 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.HTableDescriptor;
|
||||
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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.ParseFilter;
|
||||
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
||||
|
@ -106,10 +107,15 @@ public class TestTableScan {
|
|||
REST_TEST_UTIL.getServletPort()));
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
if (!admin.tableExists(TABLE)) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLE);
|
||||
htd.addFamily(new HColumnDescriptor(CFA));
|
||||
htd.addFamily(new HColumnDescriptor(CFB));
|
||||
admin.createTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TABLE);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFA)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CFB)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
expectedRows1 = TestScannerResource.insertData(conf, TABLE, COLUMN_1, 1.0);
|
||||
expectedRows2 = TestScannerResource.insertData(conf, TABLE, COLUMN_2, 0.5);
|
||||
expectedRows3 = TestScannerResource.insertData(conf, TABLE, COLUMN_EMPTY, 1.0);
|
||||
|
|
|
@ -157,7 +157,7 @@ public class TestRemoteTable {
|
|||
public void testGetTableDescriptor() throws IOException {
|
||||
try (Table table = TEST_UTIL.getConnection().getTable(TABLE)) {
|
||||
TableDescriptor local = table.getDescriptor();
|
||||
assertEquals(remoteTable.getDescriptor(), new HTableDescriptor(local));
|
||||
assertEquals(remoteTable.getDescriptor(), local);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -57,10 +57,8 @@ import org.apache.hadoop.hbase.ClusterMetrics.Option;
|
|||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
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.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.ScheduledChore;
|
||||
|
@ -70,6 +68,7 @@ import org.apache.hadoop.hbase.TableNotEnabledException;
|
|||
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.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
|
@ -80,6 +79,7 @@ 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.filter.FirstKeyOnlyFilter;
|
||||
import org.apache.hadoop.hbase.tool.CanaryTool.RegionTask.TaskType;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -1542,12 +1542,11 @@ public class CanaryTool implements Tool, Canary {
|
|||
"(current lower limit of regions per server is {} and you can change it with config {}).",
|
||||
numberOfServers, numberOfRegions, regionsLowerLimit,
|
||||
HConstants.HBASE_CANARY_WRITE_PERSERVER_REGIONS_LOWERLIMIT_KEY);
|
||||
HTableDescriptor desc = new HTableDescriptor(writeTableName);
|
||||
HColumnDescriptor family = new HColumnDescriptor(CANARY_TABLE_FAMILY_NAME);
|
||||
family.setMaxVersions(1);
|
||||
family.setTimeToLive(writeDataTTL);
|
||||
|
||||
desc.addFamily(family);
|
||||
ColumnFamilyDescriptor family = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(CANARY_TABLE_FAMILY_NAME)).setMaxVersions(1)
|
||||
.setTimeToLive(writeDataTTL).build();
|
||||
TableDescriptor desc = TableDescriptorBuilder.newBuilder(writeTableName)
|
||||
.setColumnFamily(family).build();
|
||||
byte[][] splits = new RegionSplitter.HexStringSplit().split(numberOfRegions);
|
||||
admin.createTable(desc, splits);
|
||||
}
|
||||
|
|
|
@ -4297,7 +4297,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
|
|||
* encoding, bloom codecs available.
|
||||
* @return the list of column descriptors
|
||||
*/
|
||||
public static List<HColumnDescriptor> generateColumnDescriptors() {
|
||||
public static List<ColumnFamilyDescriptor> generateColumnDescriptors() {
|
||||
return generateColumnDescriptors("");
|
||||
}
|
||||
|
||||
|
@ -4307,23 +4307,24 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
|
|||
* @param prefix family names prefix
|
||||
* @return the list of column descriptors
|
||||
*/
|
||||
public static List<HColumnDescriptor> generateColumnDescriptors(final String prefix) {
|
||||
List<HColumnDescriptor> htds = new ArrayList<>();
|
||||
public static List<ColumnFamilyDescriptor> generateColumnDescriptors(final String prefix) {
|
||||
List<ColumnFamilyDescriptor> columnFamilyDescriptors = new ArrayList<>();
|
||||
long familyId = 0;
|
||||
for (Compression.Algorithm compressionType: getSupportedCompressionAlgorithms()) {
|
||||
for (DataBlockEncoding encodingType: DataBlockEncoding.values()) {
|
||||
for (BloomType bloomType: BloomType.values()) {
|
||||
String name = String.format("%s-cf-!@#&-%d!@#", prefix, familyId);
|
||||
HColumnDescriptor htd = new HColumnDescriptor(name);
|
||||
htd.setCompressionType(compressionType);
|
||||
htd.setDataBlockEncoding(encodingType);
|
||||
htd.setBloomFilterType(bloomType);
|
||||
htds.add(htd);
|
||||
ColumnFamilyDescriptorBuilder columnFamilyDescriptorBuilder =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(name));
|
||||
columnFamilyDescriptorBuilder.setCompressionType(compressionType);
|
||||
columnFamilyDescriptorBuilder.setDataBlockEncoding(encodingType);
|
||||
columnFamilyDescriptorBuilder.setBloomFilterType(bloomType);
|
||||
columnFamilyDescriptors.add(columnFamilyDescriptorBuilder.build());
|
||||
familyId++;
|
||||
}
|
||||
}
|
||||
}
|
||||
return htds;
|
||||
return columnFamilyDescriptors;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -30,10 +30,13 @@ import java.util.regex.Pattern;
|
|||
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.Get;
|
||||
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.HMaster;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
|
@ -211,23 +214,26 @@ public class TestNamespace {
|
|||
String nsName = prefix + "_" + name.getMethodName();
|
||||
LOG.info(name.getMethodName());
|
||||
|
||||
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(nsName + ":" + name.getMethodName()));
|
||||
HColumnDescriptor colDesc = new HColumnDescriptor("my_cf");
|
||||
desc.addFamily(colDesc);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(nsName + ":" + name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("my_cf")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
try {
|
||||
admin.createTable(desc);
|
||||
admin.createTable(tableDescriptor);
|
||||
fail("Expected no namespace exists exception");
|
||||
} catch (NamespaceNotFoundException ex) {
|
||||
}
|
||||
//create table and in new namespace
|
||||
admin.createNamespace(NamespaceDescriptor.create(nsName).build());
|
||||
admin.createTable(desc);
|
||||
TEST_UTIL.waitTableAvailable(desc.getTableName().getName(), 10000);
|
||||
admin.createTable(tableDescriptor);
|
||||
TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName().getName(), 10000);
|
||||
FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
|
||||
assertTrue(fs.exists(
|
||||
new Path(master.getMasterFileSystem().getRootDir(),
|
||||
new Path(HConstants.BASE_NAMESPACE_DIR,
|
||||
new Path(nsName, desc.getTableName().getQualifierAsString())))));
|
||||
new Path(nsName, tableDescriptor.getTableName().getQualifierAsString())))));
|
||||
assertEquals(1, admin.listTableDescriptors().size());
|
||||
|
||||
//verify non-empty namespace can't be removed
|
||||
|
@ -239,42 +245,48 @@ public class TestNamespace {
|
|||
}
|
||||
|
||||
//sanity check try to write and read from table
|
||||
Table table = TEST_UTIL.getConnection().getTable(desc.getTableName());
|
||||
Table table = TEST_UTIL.getConnection().getTable(tableDescriptor.getTableName());
|
||||
Put p = new Put(Bytes.toBytes("row1"));
|
||||
p.addColumn(Bytes.toBytes("my_cf"), Bytes.toBytes("my_col"), Bytes.toBytes("value1"));
|
||||
table.put(p);
|
||||
//flush and read from disk to make sure directory changes are working
|
||||
admin.flush(desc.getTableName());
|
||||
admin.flush(tableDescriptor.getTableName());
|
||||
Get g = new Get(Bytes.toBytes("row1"));
|
||||
assertTrue(table.exists(g));
|
||||
|
||||
//normal case of removing namespace
|
||||
TEST_UTIL.deleteTable(desc.getTableName());
|
||||
TEST_UTIL.deleteTable(tableDescriptor.getTableName());
|
||||
admin.deleteNamespace(nsName);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void createTableInDefaultNamespace() throws Exception {
|
||||
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
|
||||
desc.addFamily(colDesc);
|
||||
admin.createTable(desc);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf1")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
admin.createTable(tableDescriptor);
|
||||
assertTrue(admin.listTableDescriptors().size() == 1);
|
||||
admin.disableTable(desc.getTableName());
|
||||
admin.deleteTable(desc.getTableName());
|
||||
admin.disableTable(tableDescriptor.getTableName());
|
||||
admin.deleteTable(tableDescriptor.getTableName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void createTableInSystemNamespace() throws Exception {
|
||||
final TableName tableName = TableName.valueOf("hbase:" + name.getMethodName());
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
|
||||
desc.addFamily(colDesc);
|
||||
admin.createTable(desc);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf1")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
admin.createTable(tableDescriptor);
|
||||
assertEquals(0, admin.listTableDescriptors().size());
|
||||
assertTrue(admin.tableExists(tableName));
|
||||
admin.disableTable(desc.getTableName());
|
||||
admin.deleteTable(desc.getTableName());
|
||||
admin.disableTable(tableDescriptor.getTableName());
|
||||
admin.deleteTable(tableDescriptor.getTableName());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -295,10 +307,12 @@ public class TestNamespace {
|
|||
runWithExpectedException(new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
HTableDescriptor htd =
|
||||
new HTableDescriptor(TableName.valueOf("non_existing_namespace", name.getMethodName()));
|
||||
htd.addFamily(new HColumnDescriptor("family1"));
|
||||
admin.createTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder = TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf("non_existing_namespace", name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("family1")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
return null;
|
||||
}
|
||||
}, NamespaceNotFoundException.class);
|
||||
|
@ -341,9 +355,12 @@ public class TestNamespace {
|
|||
}, NamespaceNotFoundException.class);
|
||||
|
||||
// get table descriptors for existing namespace
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(prefix + "ns1", name.getMethodName()));
|
||||
htd.addFamily(new HColumnDescriptor("family1"));
|
||||
admin.createTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(prefix + "ns1", name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("family1")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
List<TableDescriptor> htds =
|
||||
admin.listTableDescriptorsByNamespace(Bytes.toBytes(prefix + "ns1"));
|
||||
assertNotNull("Should have not returned null", htds);
|
||||
|
|
|
@ -31,8 +31,13 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
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.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
||||
|
@ -173,12 +178,16 @@ public class TestSerialization {
|
|||
}
|
||||
|
||||
private HRegionInfo createRandomRegion(final String name) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
String [] families = new String [] {"info", "anchor"};
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name));
|
||||
String[] families = new String[]{"info", "anchor"};
|
||||
for (int i = 0; i < families.length; i++) {
|
||||
htd.addFamily(new HColumnDescriptor(families[i]));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(families[i])).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
}
|
||||
return new HRegionInfo(htd.getTableName(), HConstants.EMPTY_START_ROW,
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
return new HRegionInfo(tableDescriptor.getTableName(), HConstants.EMPTY_START_ROW,
|
||||
HConstants.EMPTY_END_ROW);
|
||||
}
|
||||
|
||||
|
|
|
@ -348,11 +348,13 @@ public class TestAdmin2 extends TestAdminBase {
|
|||
private Admin createTable(TableName tableName) throws IOException {
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("value");
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("value")).build();
|
||||
|
||||
htd.addFamily(hcd);
|
||||
admin.createTable(htd);
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
return admin;
|
||||
}
|
||||
|
||||
|
@ -361,11 +363,13 @@ public class TestAdmin2 extends TestAdminBase {
|
|||
}
|
||||
|
||||
private void createTableWithDefaultConf(TableName TABLENAME) throws IOException {
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLENAME);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("value");
|
||||
htd.addFamily(hcd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TABLENAME);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("value")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
ADMIN.createTable(htd);
|
||||
ADMIN.createTable(tableDescriptorBuilder.build());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -24,15 +24,14 @@ import static org.junit.Assert.fail;
|
|||
|
||||
import java.util.concurrent.Callable;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceExistException;
|
||||
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
|
@ -119,10 +118,13 @@ public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase {
|
|||
runWithExpectedException(new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("non_existing_namespace",
|
||||
"table1"));
|
||||
htd.addFamily(new HColumnDescriptor("family1"));
|
||||
admin.createTable(htd).join();
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf("non_existing_namespace",
|
||||
"table1"));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("family1")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build()).join();
|
||||
return null;
|
||||
}
|
||||
}, NamespaceNotFoundException.class);
|
||||
|
|
|
@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.HRegionLocation;
|
|||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.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.Get;
|
||||
|
@ -48,6 +50,7 @@ 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.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.metrics.Counter;
|
||||
import org.apache.hadoop.hbase.metrics.Metric;
|
||||
|
@ -309,10 +312,12 @@ public class TestCoprocessorMetrics {
|
|||
Timer createTableTimer = (Timer)metric.get();
|
||||
long prevCount = createTableTimer.getHistogram().getCount();
|
||||
LOG.info("Creating table");
|
||||
admin.createTable(
|
||||
new HTableDescriptor(TableName.valueOf(name.getMethodName()))
|
||||
.addFamily(new HColumnDescriptor("foo")));
|
||||
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
assertEquals(1, createTableTimer.getHistogram().getCount() - prevCount);
|
||||
}
|
||||
}
|
||||
|
@ -353,9 +358,12 @@ public class TestCoprocessorMetrics {
|
|||
|
||||
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
|
||||
Admin admin = connection.getAdmin()) {
|
||||
admin.createTable(
|
||||
new HTableDescriptor(TableName.valueOf(name.getMethodName()))
|
||||
.addFamily(new HColumnDescriptor("foo")));
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
|
||||
Counter rollWalRequests = (Counter)metric.get();
|
||||
long prevCount = rollWalRequests.getCount();
|
||||
|
|
|
@ -29,15 +29,17 @@ import java.util.Set;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||
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.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
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;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
|
@ -65,11 +67,18 @@ public class TestColumnPrefixFilter {
|
|||
@Test
|
||||
public void testColumnPrefixFilter() throws IOException {
|
||||
String family = "Family";
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
htd.addFamily((new HColumnDescriptor(family)).setMaxVersions(3));
|
||||
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family))
|
||||
.setMaxVersions(3)
|
||||
.build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false);
|
||||
HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
|
||||
TEST_UTIL.getConfiguration(), htd);
|
||||
TEST_UTIL.getConfiguration(), tableDescriptor);
|
||||
try {
|
||||
List<String> rows = generateRandomWords(100, "row");
|
||||
List<String> columns = generateRandomWords(10000, "column");
|
||||
|
@ -127,11 +136,18 @@ public class TestColumnPrefixFilter {
|
|||
@Test
|
||||
public void testColumnPrefixFilterWithFilterList() throws IOException {
|
||||
String family = "Family";
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
htd.addFamily((new HColumnDescriptor(family)).setMaxVersions(3));
|
||||
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family))
|
||||
.setMaxVersions(3)
|
||||
.build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false);
|
||||
HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
|
||||
TEST_UTIL.getConfiguration(), htd);
|
||||
TEST_UTIL.getConfiguration(), tableDescriptor);
|
||||
try {
|
||||
List<String> rows = generateRandomWords(100, "row");
|
||||
List<String> columns = generateRandomWords(10000, "column");
|
||||
|
|
|
@ -29,15 +29,17 @@ import java.util.Set;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||
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.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
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;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
|
@ -65,14 +67,19 @@ public class TestMultipleColumnPrefixFilter {
|
|||
@Test
|
||||
public void testMultipleColumnPrefixFilter() throws IOException {
|
||||
String family = "Family";
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMaxVersions(3);
|
||||
htd.addFamily(hcd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family))
|
||||
.setMaxVersions(3)
|
||||
.build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
// HRegionInfo info = new HRegionInfo(htd, null, null, false);
|
||||
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
||||
HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false);
|
||||
HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.
|
||||
getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
|
||||
getDataTestDir(), TEST_UTIL.getConfiguration(), tableDescriptor);
|
||||
|
||||
List<String> rows = generateRandomWords(100, "row");
|
||||
List<String> columns = generateRandomWords(10000, "column");
|
||||
|
@ -129,16 +136,23 @@ public class TestMultipleColumnPrefixFilter {
|
|||
public void testMultipleColumnPrefixFilterWithManyFamilies() throws IOException {
|
||||
String family1 = "Family1";
|
||||
String family2 = "Family2";
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
HColumnDescriptor hcd1 = new HColumnDescriptor(family1);
|
||||
hcd1.setMaxVersions(3);
|
||||
htd.addFamily(hcd1);
|
||||
HColumnDescriptor hcd2 = new HColumnDescriptor(family2);
|
||||
hcd2.setMaxVersions(3);
|
||||
htd.addFamily(hcd2);
|
||||
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family1))
|
||||
.setMaxVersions(3)
|
||||
.build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family2))
|
||||
.setMaxVersions(3)
|
||||
.build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false);
|
||||
HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.
|
||||
getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
|
||||
getDataTestDir(), TEST_UTIL.getConfiguration(), tableDescriptor);
|
||||
|
||||
List<String> rows = generateRandomWords(100, "row");
|
||||
List<String> columns = generateRandomWords(10000, "column");
|
||||
|
@ -199,11 +213,15 @@ public class TestMultipleColumnPrefixFilter {
|
|||
@Test
|
||||
public void testMultipleColumnPrefixFilterWithColumnPrefixFilter() throws IOException {
|
||||
String family = "Family";
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
htd.addFamily(new HColumnDescriptor(family));
|
||||
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false);
|
||||
HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.
|
||||
getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
|
||||
getDataTestDir(), TEST_UTIL.getConfiguration(), tableDescriptor);
|
||||
|
||||
List<String> rows = generateRandomWords(100, "row");
|
||||
List<String> columns = generateRandomWords(10000, "column");
|
||||
|
|
|
@ -30,11 +30,10 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
|
@ -42,6 +41,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.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
|
@ -77,7 +77,7 @@ public class TestChangingEncoding {
|
|||
|
||||
private static final int TIMEOUT_MS = 600000;
|
||||
|
||||
private HColumnDescriptor hcd;
|
||||
private ColumnFamilyDescriptorBuilder columnFamilyDescriptorBuilder;
|
||||
|
||||
private TableName tableName;
|
||||
private static final List<DataBlockEncoding> ENCODINGS_TO_ITERATE =
|
||||
|
@ -94,11 +94,13 @@ public class TestChangingEncoding {
|
|||
|
||||
private void prepareTest(String testId) throws IOException {
|
||||
tableName = TableName.valueOf("test_table_" + testId);
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
hcd = new HColumnDescriptor(CF);
|
||||
htd.addFamily(hcd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
columnFamilyDescriptorBuilder =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CF));
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptorBuilder.build());
|
||||
try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
|
||||
admin.createTable(htd);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
}
|
||||
numBatchesWritten = 0;
|
||||
}
|
||||
|
@ -185,12 +187,12 @@ public class TestChangingEncoding {
|
|||
boolean onlineChange) throws Exception {
|
||||
LOG.debug("Setting CF encoding to " + encoding + " (ordinal="
|
||||
+ encoding.ordinal() + "), onlineChange=" + onlineChange);
|
||||
hcd.setDataBlockEncoding(encoding);
|
||||
columnFamilyDescriptorBuilder.setDataBlockEncoding(encoding);
|
||||
try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
|
||||
if (!onlineChange) {
|
||||
admin.disableTable(tableName);
|
||||
}
|
||||
admin.modifyColumnFamily(tableName, hcd);
|
||||
admin.modifyColumnFamily(tableName, columnFamilyDescriptorBuilder.build());
|
||||
if (!onlineChange) {
|
||||
admin.enableTable(tableName);
|
||||
}
|
||||
|
|
|
@ -32,8 +32,9 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.io.ByteBuffAllocator;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
|
||||
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
|
||||
|
@ -41,6 +42,7 @@ import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
|||
import org.apache.hadoop.hbase.nio.ByteBuff;
|
||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.junit.Before;
|
||||
import org.junit.ClassRule;
|
||||
|
@ -237,10 +239,13 @@ public class TestCacheConfig {
|
|||
conf.setBoolean(CacheConfig.CACHE_DATA_ON_READ_KEY, true);
|
||||
conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false);
|
||||
|
||||
HColumnDescriptor family = new HColumnDescriptor("testDisableCacheDataBlock");
|
||||
family.setBlockCacheEnabled(false);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("testDisableCacheDataBlock"))
|
||||
.setBlockCacheEnabled(false)
|
||||
.build();
|
||||
|
||||
cacheConfig = new CacheConfig(conf, family, null, ByteBuffAllocator.HEAP);
|
||||
cacheConfig = new CacheConfig(conf, columnFamilyDescriptor, null, ByteBuffAllocator.HEAP);
|
||||
assertFalse(cacheConfig.shouldCacheBlockOnRead(BlockCategory.DATA));
|
||||
assertFalse(cacheConfig.shouldCacheCompressed(BlockCategory.DATA));
|
||||
assertFalse(cacheConfig.shouldCacheDataCompressed());
|
||||
|
|
|
@ -38,12 +38,13 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.MetaMockingUtil;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
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.Result;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
|
@ -172,9 +173,14 @@ public class TestCatalogJanitor {
|
|||
* family that is MockMasterServices.DEFAULT_COLUMN_FAMILY_NAME)
|
||||
*/
|
||||
private TableDescriptor createTableDescriptorForCurrentMethod() {
|
||||
return TableDescriptorBuilder.newBuilder(TableName.valueOf(this.name.getMethodName())).
|
||||
setColumnFamily(new HColumnDescriptor(MockMasterServices.DEFAULT_COLUMN_FAMILY_NAME)).
|
||||
build();
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(MockMasterServices.DEFAULT_COLUMN_FAMILY_NAME))
|
||||
.build();
|
||||
return TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(this.name.getMethodName()))
|
||||
.setColumnFamily(columnFamilyDescriptor)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -42,10 +42,13 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||
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.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -186,11 +189,13 @@ public class TestMaster {
|
|||
@Test
|
||||
public void testMoveThrowsUnknownRegionException() throws IOException {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("value");
|
||||
htd.addFamily(hcd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("value")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
admin.createTable(htd);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
try {
|
||||
RegionInfo hri = RegionInfoBuilder.newBuilder(tableName)
|
||||
.setStartKey(Bytes.toBytes("A"))
|
||||
|
@ -209,11 +214,13 @@ public class TestMaster {
|
|||
public void testMoveThrowsPleaseHoldException() throws IOException {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("value");
|
||||
htd.addFamily(hcd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("value")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
admin.createTable(htd);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
try {
|
||||
List<RegionInfo> tableRegions = admin.getRegions(tableName);
|
||||
|
||||
|
|
|
@ -26,7 +26,6 @@ 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.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -114,11 +113,15 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
|||
|
||||
// Test 1: Modify the table descriptor online
|
||||
String cf2 = "cf2";
|
||||
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
|
||||
htd.addFamily(new HColumnDescriptor(cf2));
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(UTIL.getAdmin().getDescriptor(tableName));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(cf2)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
long procId = ProcedureTestingUtility.submitAndWait(
|
||||
procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
|
||||
procExec, new ModifyTableProcedure(
|
||||
procExec.getEnvironment(), tableDescriptorBuilder.build()));
|
||||
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
|
||||
|
||||
currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
|
||||
|
@ -129,13 +132,15 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
|||
UTIL.getAdmin().disableTable(tableName);
|
||||
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
|
||||
String cf3 = "cf3";
|
||||
HTableDescriptor htd2 =
|
||||
new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
|
||||
htd2.addFamily(new HColumnDescriptor(cf3));
|
||||
tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(UTIL.getAdmin().getDescriptor(tableName));
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(cf3)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
long procId2 =
|
||||
ProcedureTestingUtility.submitAndWait(procExec,
|
||||
new ModifyTableProcedure(procExec.getEnvironment(), htd2));
|
||||
ProcedureTestingUtility.submitAndWait(procExec,
|
||||
new ModifyTableProcedure(procExec.getEnvironment(), tableDescriptorBuilder.build()));
|
||||
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
|
||||
|
||||
currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
|
||||
|
@ -257,15 +262,18 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
|||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
||||
|
||||
// Modify multiple properties of the table.
|
||||
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
|
||||
boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
|
||||
htd.setCompactionEnabled(newCompactionEnableOption);
|
||||
htd.addFamily(new HColumnDescriptor(cf2));
|
||||
htd.removeFamily(Bytes.toBytes(cf3));
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(UTIL.getAdmin().getDescriptor(tableName));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(cf2)).build();
|
||||
boolean newCompactionEnableOption = !tableDescriptorBuilder.build().isCompactionEnabled();
|
||||
tableDescriptorBuilder.setCompactionEnabled(newCompactionEnableOption);
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
tableDescriptorBuilder.removeColumnFamily(Bytes.toBytes(cf3));
|
||||
|
||||
// Start the Modify procedure && kill the executor
|
||||
long procId = procExec.submitProcedure(
|
||||
new ModifyTableProcedure(procExec.getEnvironment(), htd));
|
||||
new ModifyTableProcedure(procExec.getEnvironment(), tableDescriptorBuilder.build()));
|
||||
|
||||
// Restart the executor and execute the step twice
|
||||
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
|
||||
|
@ -414,18 +422,19 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
|||
|
||||
class ConcurrentAddColumnFamily extends Thread {
|
||||
TableName tableName = null;
|
||||
HColumnDescriptor hcd = null;
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor;
|
||||
boolean exception;
|
||||
|
||||
public ConcurrentAddColumnFamily(TableName tableName, HColumnDescriptor hcd) {
|
||||
public ConcurrentAddColumnFamily(TableName tableName,
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor) {
|
||||
this.tableName = tableName;
|
||||
this.hcd = hcd;
|
||||
this.columnFamilyDescriptor = columnFamilyDescriptor;
|
||||
this.exception = false;
|
||||
}
|
||||
|
||||
public void run() {
|
||||
try {
|
||||
UTIL.getAdmin().addColumnFamily(tableName, hcd);
|
||||
UTIL.getAdmin().addColumnFamily(tableName, columnFamilyDescriptor);
|
||||
} catch (Exception e) {
|
||||
if (e.getClass().equals(ConcurrentTableModificationException.class)) {
|
||||
this.exception = true;
|
||||
|
@ -433,10 +442,14 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
|||
}
|
||||
}
|
||||
}
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(column_Family2)).build();
|
||||
ConcurrentAddColumnFamily t1 =
|
||||
new ConcurrentAddColumnFamily(tableName, new HColumnDescriptor(column_Family2));
|
||||
new ConcurrentAddColumnFamily(tableName, columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(column_Family3)).build();
|
||||
ConcurrentAddColumnFamily t2 =
|
||||
new ConcurrentAddColumnFamily(tableName, new HColumnDescriptor(column_Family3));
|
||||
new ConcurrentAddColumnFamily(tableName, columnFamilyDescriptor);
|
||||
|
||||
t1.start();
|
||||
t2.start();
|
||||
|
@ -451,11 +464,18 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
|||
@Test
|
||||
public void testConcurrentDeleteColumnFamily() throws IOException, InterruptedException {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor(column_Family1));
|
||||
htd.addFamily(new HColumnDescriptor(column_Family2));
|
||||
htd.addFamily(new HColumnDescriptor(column_Family3));
|
||||
UTIL.getAdmin().createTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(column_Family1)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(column_Family2)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(column_Family3)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
UTIL.getAdmin().createTable(tableDescriptorBuilder.build());
|
||||
|
||||
class ConcurrentCreateDeleteTable extends Thread {
|
||||
TableName tableName = null;
|
||||
|
|
|
@ -24,13 +24,14 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
@ -84,28 +85,34 @@ public class TestExpiredMobFileCleaner {
|
|||
}
|
||||
|
||||
private void init() throws Exception {
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(3L);
|
||||
hcd.setMaxVersions(4);
|
||||
desc.addFamily(hcd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family))
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(3L)
|
||||
.setMaxVersions(4)
|
||||
.build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
admin = TEST_UTIL.getAdmin();
|
||||
admin.createTable(desc);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
|
||||
.getBufferedMutator(tableName);
|
||||
}
|
||||
|
||||
private void modifyColumnExpiryDays(int expireDays) throws Exception {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(3L);
|
||||
ColumnFamilyDescriptorBuilder columnFamilyDescriptorBuilder =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family))
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(3L);
|
||||
// change ttl as expire days to make some row expired
|
||||
int timeToLive = expireDays * secondsOfDay();
|
||||
hcd.setTimeToLive(timeToLive);
|
||||
columnFamilyDescriptorBuilder.setTimeToLive(timeToLive);
|
||||
|
||||
admin.modifyColumnFamily(tableName, hcd);
|
||||
admin.modifyColumnFamily(tableName, columnFamilyDescriptorBuilder.build());
|
||||
}
|
||||
|
||||
private void putKVAndFlush(BufferedMutator table, byte[] row, byte[] value, long ts)
|
||||
|
|
|
@ -28,13 +28,15 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
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.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.regionserver.HMobStore;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -46,8 +48,6 @@ import org.junit.Before;
|
|||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestMobFileCache {
|
||||
|
@ -56,7 +56,6 @@ public class TestMobFileCache {
|
|||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestMobFileCache.class);
|
||||
|
||||
static final Logger LOG = LoggerFactory.getLogger(TestMobFileCache.class);
|
||||
private HBaseTestingUtility UTIL;
|
||||
private HRegion region;
|
||||
private Configuration conf;
|
||||
|
@ -88,23 +87,31 @@ public class TestMobFileCache {
|
|||
UTIL = HBaseTestingUtility.createLocalHTU();
|
||||
conf = UTIL.getConfiguration();
|
||||
conf.set(MobConstants.MOB_FILE_CACHE_SIZE_KEY, TEST_CACHE_SIZE);
|
||||
HTableDescriptor htd = UTIL.createTableDescriptor("testMobFileCache");
|
||||
HColumnDescriptor hcd1 = new HColumnDescriptor(FAMILY1);
|
||||
hcd1.setMobEnabled(true);
|
||||
hcd1.setMobThreshold(0);
|
||||
HColumnDescriptor hcd2 = new HColumnDescriptor(FAMILY2);
|
||||
hcd2.setMobEnabled(true);
|
||||
hcd2.setMobThreshold(0);
|
||||
HColumnDescriptor hcd3 = new HColumnDescriptor(FAMILY3);
|
||||
hcd3.setMobEnabled(true);
|
||||
hcd3.setMobThreshold(0);
|
||||
htd.addFamily(hcd1);
|
||||
htd.addFamily(hcd2);
|
||||
htd.addFamily(hcd3);
|
||||
RegionInfo regionInfo = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(UTIL.createTableDescriptor("testMobFileCache"));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(FAMILY1))
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(0)
|
||||
.build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(FAMILY2))
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(0)
|
||||
.build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(FAMILY3))
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(0)
|
||||
.build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
|
||||
mobFileCache = new MobFileCache(conf);
|
||||
region = HBaseTestingUtility
|
||||
.createRegionAndWAL(regionInfo, UTIL.getDataTestDir(), conf, htd, mobFileCache);
|
||||
.createRegionAndWAL(regionInfo, UTIL.getDataTestDir(), conf, tableDescriptor, mobFileCache);
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -124,30 +131,34 @@ public class TestMobFileCache {
|
|||
* Create the mob store file
|
||||
*/
|
||||
private Path createMobStoreFile(Configuration conf, String family) throws IOException {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMaxVersions(4);
|
||||
hcd.setMobEnabled(true);
|
||||
return createMobStoreFile(hcd);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family))
|
||||
.setMaxVersions(4)
|
||||
.setMobEnabled(true).build();
|
||||
return createMobStoreFile(columnFamilyDescriptor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the mob store file
|
||||
*/
|
||||
private Path createMobStoreFile(HColumnDescriptor hcd)
|
||||
private Path createMobStoreFile(ColumnFamilyDescriptor columnFamilyDescriptor)
|
||||
throws IOException {
|
||||
// Setting up a Store
|
||||
TableName tn = TableName.valueOf(TABLE);
|
||||
HTableDescriptor htd = new HTableDescriptor(tn);
|
||||
htd.addFamily(hcd);
|
||||
HMobStore mobStore = (HMobStore) region.getStore(hcd.getName());
|
||||
KeyValue key1 = new KeyValue(ROW, hcd.getName(), QF1, 1, VALUE);
|
||||
KeyValue key2 = new KeyValue(ROW, hcd.getName(), QF2, 1, VALUE);
|
||||
KeyValue key3 = new KeyValue(ROW2, hcd.getName(), QF3, 1, VALUE2);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tn);
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
HMobStore mobStore = (HMobStore) region.getStore(columnFamilyDescriptor.getName());
|
||||
KeyValue key1 = new KeyValue(ROW, columnFamilyDescriptor.getName(), QF1, 1, VALUE);
|
||||
KeyValue key2 = new KeyValue(ROW, columnFamilyDescriptor.getName(), QF2, 1, VALUE);
|
||||
KeyValue key3 = new KeyValue(ROW2, columnFamilyDescriptor.getName(), QF3, 1, VALUE2);
|
||||
KeyValue[] keys = new KeyValue[] { key1, key2, key3 };
|
||||
int maxKeyCount = keys.length;
|
||||
HRegionInfo regionInfo = new HRegionInfo(tn);
|
||||
StoreFileWriter mobWriter = mobStore.createWriterInTmp(currentDate,
|
||||
maxKeyCount, hcd.getCompactionCompression(), regionInfo.getStartKey(), false);
|
||||
maxKeyCount, columnFamilyDescriptor.getCompactionCompressionType(),
|
||||
regionInfo.getStartKey(), false);
|
||||
Path mobFilePath = mobWriter.getPath();
|
||||
String fileName = mobFilePath.getName();
|
||||
mobWriter.append(key1);
|
||||
|
|
|
@ -53,6 +53,8 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.CompactType;
|
||||
import org.apache.hadoop.hbase.client.CompactionState;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
|
@ -66,6 +68,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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
|
@ -119,9 +122,9 @@ public class TestMobCompactor {
|
|||
private BufferedMutator bufMut;
|
||||
private Table table;
|
||||
private static Admin admin;
|
||||
private HTableDescriptor desc;
|
||||
private HColumnDescriptor hcd1;
|
||||
private HColumnDescriptor hcd2;
|
||||
private TableDescriptorBuilder desc;
|
||||
private ColumnFamilyDescriptorBuilder cfdb1;
|
||||
private ColumnFamilyDescriptorBuilder cfdb2;
|
||||
private static FileSystem fs;
|
||||
private static final String family1 = "family1";
|
||||
private static final String family2 = "family2";
|
||||
|
@ -232,16 +235,16 @@ public class TestMobCompactor {
|
|||
|
||||
public void setUp(String tableNameAsString) throws IOException {
|
||||
tableName = TableName.valueOf(tableNameAsString);
|
||||
hcd1 = new HColumnDescriptor(family1);
|
||||
hcd1.setMobEnabled(true);
|
||||
hcd1.setMobThreshold(5);
|
||||
hcd2 = new HColumnDescriptor(family2);
|
||||
hcd2.setMobEnabled(true);
|
||||
hcd2.setMobThreshold(5);
|
||||
desc = new HTableDescriptor(tableName);
|
||||
desc.addFamily(hcd1);
|
||||
desc.addFamily(hcd2);
|
||||
admin.createTable(desc, getSplitKeys());
|
||||
cfdb1 = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family1));
|
||||
cfdb1.setMobEnabled(true);
|
||||
cfdb1.setMobThreshold(5);
|
||||
cfdb2 = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family2));
|
||||
cfdb2.setMobEnabled(true);
|
||||
cfdb2.setMobThreshold(5);
|
||||
desc = TableDescriptorBuilder.newBuilder(tableName);
|
||||
desc.setColumnFamily(cfdb1.build());
|
||||
desc.setColumnFamily(cfdb2.build());
|
||||
admin.createTable(desc.build(), getSplitKeys());
|
||||
table = conn.getTable(tableName);
|
||||
bufMut = conn.getBufferedMutator(tableName);
|
||||
}
|
||||
|
@ -250,13 +253,13 @@ public class TestMobCompactor {
|
|||
private void setUpForPolicyTest(String tableNameAsString, MobCompactPartitionPolicy type)
|
||||
throws IOException {
|
||||
tableName = TableName.valueOf(tableNameAsString);
|
||||
hcd1 = new HColumnDescriptor(family1);
|
||||
hcd1.setMobEnabled(true);
|
||||
hcd1.setMobThreshold(10);
|
||||
hcd1.setMobCompactPartitionPolicy(type);
|
||||
desc = new HTableDescriptor(tableName);
|
||||
desc.addFamily(hcd1);
|
||||
admin.createTable(desc);
|
||||
cfdb1 = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family1));
|
||||
cfdb1.setMobEnabled(true);
|
||||
cfdb1.setMobThreshold(10);
|
||||
cfdb1.setMobCompactPartitionPolicy(type);
|
||||
desc = TableDescriptorBuilder.newBuilder(tableName);
|
||||
desc.setColumnFamily(cfdb1.build());
|
||||
admin.createTable(desc.build());
|
||||
table = conn.getTable(tableName);
|
||||
bufMut = conn.getBufferedMutator(tableName);
|
||||
}
|
||||
|
@ -265,9 +268,9 @@ public class TestMobCompactor {
|
|||
private void alterForPolicyTest(final MobCompactPartitionPolicy type)
|
||||
throws Exception {
|
||||
|
||||
hcd1.setMobCompactPartitionPolicy(type);
|
||||
desc.modifyFamily(hcd1);
|
||||
admin.modifyTable(desc);
|
||||
cfdb1.setMobCompactPartitionPolicy(type);
|
||||
desc.modifyColumnFamily(cfdb1.build());
|
||||
admin.modifyTable(desc.build());
|
||||
LOG.info("alter status finished");
|
||||
}
|
||||
|
||||
|
@ -312,7 +315,8 @@ public class TestMobCompactor {
|
|||
countFiles(tableName, false, family2));
|
||||
|
||||
// do the mob file compaction
|
||||
MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, hcd1, pool);
|
||||
MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName,
|
||||
cfdb1.build(), pool);
|
||||
compactor.compact();
|
||||
|
||||
assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum),
|
||||
|
@ -452,7 +456,8 @@ public class TestMobCompactor {
|
|||
countFiles(tableName, false, family2));
|
||||
|
||||
// do the mob compaction
|
||||
MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, hcd1, pool);
|
||||
MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName,
|
||||
cfdb1.build(), pool);
|
||||
compactor.compact();
|
||||
|
||||
assertEquals("After first compaction: mob rows count", regionNum
|
||||
|
@ -524,18 +529,23 @@ public class TestMobCompactor {
|
|||
byte[] encryptionKey = EncryptionUtil.wrapKey(conf,
|
||||
conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()), cfKey);
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
HColumnDescriptor hcd1 = new HColumnDescriptor(family1);
|
||||
hcd1.setMobEnabled(true);
|
||||
hcd1.setMobThreshold(0);
|
||||
hcd1.setEncryptionType(algorithm);
|
||||
hcd1.setEncryptionKey(encryptionKey);
|
||||
HColumnDescriptor hcd2 = new HColumnDescriptor(family2);
|
||||
hcd2.setMobEnabled(true);
|
||||
hcd2.setMobThreshold(0);
|
||||
desc.addFamily(hcd1);
|
||||
desc.addFamily(hcd2);
|
||||
admin.createTable(desc, getSplitKeys());
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor cfd1 =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family1))
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(0)
|
||||
.setEncryptionType(algorithm)
|
||||
.setEncryptionKey(encryptionKey).build();
|
||||
ColumnFamilyDescriptor cfd2 =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family2))
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(0).build();
|
||||
tableDescriptorBuilder.setColumnFamily(cfd1);
|
||||
tableDescriptorBuilder.setColumnFamily(cfd2);
|
||||
admin.createTable(tableDescriptorBuilder.build(), getSplitKeys());
|
||||
Table table = conn.getTable(tableName);
|
||||
BufferedMutator bufMut = conn.getBufferedMutator(tableName);
|
||||
int count = 4;
|
||||
|
@ -566,7 +576,7 @@ public class TestMobCompactor {
|
|||
countFiles(tableName, false, family2));
|
||||
|
||||
// do the major mob compaction, it will force all files to compaction
|
||||
admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB);
|
||||
admin.majorCompact(tableName, cfd1.getName(), CompactType.MOB);
|
||||
|
||||
waitUntilMobCompactionFinished(tableName);
|
||||
assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum),
|
||||
|
@ -606,25 +616,27 @@ public class TestMobCompactor {
|
|||
// read the latest cell of key0.
|
||||
Get get = new Get(key0);
|
||||
Result result = table.get(get);
|
||||
Cell cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1));
|
||||
ColumnFamilyDescriptor cfd1 = cfdb1.build();
|
||||
Cell cell = result.getColumnLatestCell(cfd1.getName(), Bytes.toBytes(qf1));
|
||||
assertEquals("Before compaction: mob value of k0", newValue0,
|
||||
Bytes.toString(CellUtil.cloneValue(cell)));
|
||||
admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB);
|
||||
admin.majorCompact(tableName, cfd1.getName(), CompactType.MOB);
|
||||
waitUntilMobCompactionFinished(tableName);
|
||||
// read the latest cell of key0, the cell seqId in bulk loaded file is not reset in the
|
||||
// scanner. The cell that has "new" value is still visible.
|
||||
result = table.get(get);
|
||||
cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1));
|
||||
cell = result.getColumnLatestCell(cfd1.getName(), Bytes.toBytes(qf1));
|
||||
assertEquals("After compaction: mob value of k0", newValue0,
|
||||
Bytes.toString(CellUtil.cloneValue(cell)));
|
||||
// read the ref cell, not read further to the mob cell.
|
||||
get = new Get(key1);
|
||||
get.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(true));
|
||||
result = table.get(get);
|
||||
cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1));
|
||||
cell = result.getColumnLatestCell(cfd1.getName(), Bytes.toBytes(qf1));
|
||||
// the ref name is the new file
|
||||
Path mobFamilyPath =
|
||||
MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, hcd1.getNameAsString());
|
||||
MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName,
|
||||
cfdb1.getNameAsString());
|
||||
List<Path> paths = new ArrayList<>();
|
||||
if (fs.exists(mobFamilyPath)) {
|
||||
FileStatus[] files = fs.listStatus(mobFamilyPath);
|
||||
|
@ -1166,9 +1178,9 @@ public class TestMobCompactor {
|
|||
}
|
||||
|
||||
if (majorCompact) {
|
||||
admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB);
|
||||
admin.majorCompact(tableName, cfdb1.build().getName(), CompactType.MOB);
|
||||
} else {
|
||||
admin.compact(tableName, hcd1.getName(), CompactType.MOB);
|
||||
admin.compact(tableName, cfdb1.build().getName(), CompactType.MOB);
|
||||
}
|
||||
|
||||
waitUntilMobCompactionFinished(tableName);
|
||||
|
|
|
@ -46,12 +46,12 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
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.MobCompactPartitionPolicy;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
|
@ -99,7 +99,8 @@ public class TestPartitionedMobCompactor {
|
|||
private final static String qf = "qf";
|
||||
private final long DAY_IN_MS = 1000 * 60 * 60 * 24;
|
||||
private static byte[] KEYS = Bytes.toBytes("012");
|
||||
private HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
private ColumnFamilyDescriptorBuilder cfdb =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family));
|
||||
private Configuration conf = TEST_UTIL.getConfiguration();
|
||||
private CacheConfig cacheConf = new CacheConfig(conf);
|
||||
private FileSystem fs;
|
||||
|
@ -340,12 +341,12 @@ public class TestPartitionedMobCompactor {
|
|||
}
|
||||
|
||||
// Set the policy
|
||||
this.hcd.setMobCompactPartitionPolicy(policy);
|
||||
this.cfdb.setMobCompactPartitionPolicy(policy);
|
||||
// set the mob compaction mergeable threshold
|
||||
conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize);
|
||||
testSelectFiles(tableName, type, isForceAllFiles, expectedStartKeys);
|
||||
// go back to the default daily policy
|
||||
this.hcd.setMobCompactPartitionPolicy(MobCompactPartitionPolicy.DAILY);
|
||||
this.cfdb.setMobCompactPartitionPolicy(MobCompactPartitionPolicy.DAILY);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -380,7 +381,8 @@ public class TestPartitionedMobCompactor {
|
|||
listFiles();
|
||||
|
||||
TableName tName = TableName.valueOf(tableName);
|
||||
MobCompactor compactor = new PartitionedMobCompactor(conf, faultyFs, tName, hcd, pool);
|
||||
MobCompactor compactor = new PartitionedMobCompactor(conf, faultyFs, tName,
|
||||
cfdb.build(), pool);
|
||||
faultyFs.setThrowException(true);
|
||||
try {
|
||||
compactor.compact(allFiles, true);
|
||||
|
@ -486,7 +488,7 @@ public class TestPartitionedMobCompactor {
|
|||
listFiles();
|
||||
|
||||
PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs,
|
||||
TableName.valueOf(tableName), hcd, pool) {
|
||||
TableName.valueOf(tableName), cfdb.build(), pool) {
|
||||
@Override
|
||||
public List<Path> compact(List<FileStatus> files, boolean isForceAllFiles)
|
||||
throws IOException {
|
||||
|
@ -619,7 +621,7 @@ public class TestPartitionedMobCompactor {
|
|||
listFiles();
|
||||
|
||||
MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs,
|
||||
TableName.valueOf(tableName), hcd, pool, 1, cacheConf, 1);
|
||||
TableName.valueOf(tableName), cfdb.build(), pool, 1, cacheConf, 1);
|
||||
|
||||
compactor.compact(allFiles, true);
|
||||
}
|
||||
|
@ -639,7 +641,7 @@ public class TestPartitionedMobCompactor {
|
|||
listFiles();
|
||||
|
||||
MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs,
|
||||
TableName.valueOf(tableName), hcd, pool, 3, cacheConf, 3);
|
||||
TableName.valueOf(tableName), cfdb.build(), pool, 3, cacheConf, 3);
|
||||
compactor.compact(allFiles, true);
|
||||
}
|
||||
|
||||
|
@ -670,7 +672,7 @@ public class TestPartitionedMobCompactor {
|
|||
private void testSelectFiles(String tableName, final CompactionType type,
|
||||
final boolean isForceAllFiles, final List<String> expected) throws IOException {
|
||||
PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs,
|
||||
TableName.valueOf(tableName), hcd, pool) {
|
||||
TableName.valueOf(tableName), cfdb.build(), pool) {
|
||||
@Override
|
||||
public List<Path> compact(List<FileStatus> files, boolean isForceAllFiles)
|
||||
throws IOException {
|
||||
|
@ -740,7 +742,7 @@ public class TestPartitionedMobCompactor {
|
|||
private void testCompactDelFiles(String tableName, final int expectedFileCount,
|
||||
final int expectedCellCount, boolean isForceAllFiles) throws IOException {
|
||||
PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs,
|
||||
TableName.valueOf(tableName), hcd, pool) {
|
||||
TableName.valueOf(tableName), cfdb.build(), pool) {
|
||||
@Override
|
||||
protected List<Path> performCompaction(PartitionedMobCompactionRequest request)
|
||||
throws IOException {
|
||||
|
@ -886,8 +888,9 @@ public class TestPartitionedMobCompactor {
|
|||
List<KeyValueScanner> scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs,
|
||||
false, true, false, false, HConstants.LATEST_TIMESTAMP));
|
||||
long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
|
||||
long ttl = HStore.determineTTLFromFamily(hcd);
|
||||
ScanInfo scanInfo = new ScanInfo(conf, hcd, ttl, timeToPurgeDeletes, CellComparatorImpl.COMPARATOR);
|
||||
long ttl = HStore.determineTTLFromFamily(cfdb.build());
|
||||
ScanInfo scanInfo = new ScanInfo(conf, cfdb.build(), ttl, timeToPurgeDeletes,
|
||||
CellComparatorImpl.COMPARATOR);
|
||||
StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_RETAIN_DELETES, scanners);
|
||||
List<Cell> results = new ArrayList<>();
|
||||
boolean hasMore = true;
|
||||
|
|
|
@ -48,6 +48,8 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Waiter;
|
||||
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.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
|
@ -56,6 +58,7 @@ 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.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
|
@ -148,41 +151,42 @@ public class TestNamespaceAuditor {
|
|||
ADMIN.createNamespace(nspDesc);
|
||||
assertNotNull("Namespace descriptor found null.", ADMIN.getNamespaceDescriptor(nsp));
|
||||
assertEquals(3, ADMIN.listNamespaceDescriptors().length);
|
||||
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("fam1")).build();
|
||||
|
||||
HTableDescriptor tableDescOne =
|
||||
new HTableDescriptor(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1"));
|
||||
tableDescOne.addFamily(fam1);
|
||||
HTableDescriptor tableDescTwo =
|
||||
new HTableDescriptor(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table2"));
|
||||
tableDescTwo.addFamily(fam1);
|
||||
HTableDescriptor tableDescThree =
|
||||
new HTableDescriptor(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table3"));
|
||||
tableDescThree.addFamily(fam1);
|
||||
ADMIN.createTable(tableDescOne);
|
||||
TableDescriptorBuilder tableDescOne = TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1"));
|
||||
tableDescOne.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptorBuilder tableDescTwo = TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table2"));
|
||||
tableDescTwo.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptorBuilder tableDescThree = TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table3"));
|
||||
tableDescThree.setColumnFamily(columnFamilyDescriptor);
|
||||
ADMIN.createTable(tableDescOne.build());
|
||||
boolean constraintViolated = false;
|
||||
try {
|
||||
ADMIN.createTable(tableDescTwo, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 5);
|
||||
ADMIN.createTable(tableDescTwo.build(), Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 5);
|
||||
} catch (Exception exp) {
|
||||
assertTrue(exp instanceof IOException);
|
||||
constraintViolated = true;
|
||||
} finally {
|
||||
assertTrue("Constraint not violated for table " + tableDescTwo.getTableName(),
|
||||
assertTrue("Constraint not violated for table " + tableDescTwo.build().getTableName(),
|
||||
constraintViolated);
|
||||
}
|
||||
ADMIN.createTable(tableDescTwo, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
ADMIN.createTable(tableDescTwo.build(), Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
NamespaceTableAndRegionInfo nspState = getQuotaManager().getState(nsp);
|
||||
assertNotNull(nspState);
|
||||
assertTrue(nspState.getTables().size() == 2);
|
||||
assertTrue(nspState.getRegionCount() == 5);
|
||||
constraintViolated = false;
|
||||
try {
|
||||
ADMIN.createTable(tableDescThree);
|
||||
ADMIN.createTable(tableDescThree.build());
|
||||
} catch (Exception exp) {
|
||||
assertTrue(exp instanceof IOException);
|
||||
constraintViolated = true;
|
||||
} finally {
|
||||
assertTrue("Constraint not violated for table " + tableDescThree.getTableName(),
|
||||
assertTrue("Constraint not violated for table " + tableDescThree.build().getTableName(),
|
||||
constraintViolated);
|
||||
}
|
||||
}
|
||||
|
@ -257,28 +261,29 @@ public class TestNamespaceAuditor {
|
|||
assertNotNull("Namespace descriptor found null.", ADMIN.getNamespaceDescriptor(namespace));
|
||||
NamespaceTableAndRegionInfo stateInfo = getNamespaceState(nspDesc.getName());
|
||||
assertNotNull("Namespace state found null for " + namespace, stateInfo);
|
||||
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
|
||||
HTableDescriptor tableDescOne =
|
||||
new HTableDescriptor(TableName.valueOf(namespace + TableName.NAMESPACE_DELIM + "table1"));
|
||||
tableDescOne.addFamily(fam1);
|
||||
HTableDescriptor tableDescTwo =
|
||||
new HTableDescriptor(TableName.valueOf(namespace + TableName.NAMESPACE_DELIM + "table2"));
|
||||
tableDescTwo.addFamily(fam1);
|
||||
ADMIN.createTable(tableDescOne);
|
||||
ADMIN.createTable(tableDescTwo, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 5);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("fam1")).build();
|
||||
TableDescriptorBuilder tableDescOne = TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(namespace + TableName.NAMESPACE_DELIM + "table1"));
|
||||
tableDescOne.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptorBuilder tableDescTwo = TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(namespace + TableName.NAMESPACE_DELIM + "table2"));
|
||||
tableDescTwo.setColumnFamily(columnFamilyDescriptor);
|
||||
ADMIN.createTable(tableDescOne.build());
|
||||
ADMIN.createTable(tableDescTwo.build(), Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 5);
|
||||
stateInfo = getNamespaceState(nspDesc.getName());
|
||||
assertNotNull("Namespace state found to be null.", stateInfo);
|
||||
assertEquals(2, stateInfo.getTables().size());
|
||||
assertEquals(5, stateInfo.getRegionCountOfTable(tableDescTwo.getTableName()));
|
||||
assertEquals(5, stateInfo.getRegionCountOfTable(tableDescTwo.build().getTableName()));
|
||||
assertEquals(6, stateInfo.getRegionCount());
|
||||
ADMIN.disableTable(tableDescOne.getTableName());
|
||||
deleteTable(tableDescOne.getTableName());
|
||||
ADMIN.disableTable(tableDescOne.build().getTableName());
|
||||
deleteTable(tableDescOne.build().getTableName());
|
||||
stateInfo = getNamespaceState(nspDesc.getName());
|
||||
assertNotNull("Namespace state found to be null.", stateInfo);
|
||||
assertEquals(5, stateInfo.getRegionCount());
|
||||
assertEquals(1, stateInfo.getTables().size());
|
||||
ADMIN.disableTable(tableDescTwo.getTableName());
|
||||
deleteTable(tableDescTwo.getTableName());
|
||||
ADMIN.disableTable(tableDescTwo.build().getTableName());
|
||||
deleteTable(tableDescTwo.build().getTableName());
|
||||
ADMIN.deleteNamespace(namespace);
|
||||
stateInfo = getNamespaceState(namespace);
|
||||
assertNull("Namespace state not found to be null.", stateInfo);
|
||||
|
@ -520,16 +525,21 @@ public class TestNamespaceAuditor {
|
|||
TableName tableOne = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table1");
|
||||
TableName tableTwo = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table2");
|
||||
TableName tableThree = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table3");
|
||||
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
|
||||
HTableDescriptor tableDescOne = new HTableDescriptor(tableOne);
|
||||
tableDescOne.addFamily(fam1);
|
||||
HTableDescriptor tableDescTwo = new HTableDescriptor(tableTwo);
|
||||
tableDescTwo.addFamily(fam1);
|
||||
HTableDescriptor tableDescThree = new HTableDescriptor(tableThree);
|
||||
tableDescThree.addFamily(fam1);
|
||||
ADMIN.createTable(tableDescOne, Bytes.toBytes("1"), Bytes.toBytes("1000"), 3);
|
||||
ADMIN.createTable(tableDescTwo, Bytes.toBytes("1"), Bytes.toBytes("1000"), 3);
|
||||
ADMIN.createTable(tableDescThree, Bytes.toBytes("1"), Bytes.toBytes("1000"), 4);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("fam1")).build();
|
||||
TableDescriptorBuilder tableDescOne = TableDescriptorBuilder
|
||||
.newBuilder(tableOne);
|
||||
tableDescOne.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptorBuilder tableDescTwo = TableDescriptorBuilder
|
||||
.newBuilder(tableTwo);
|
||||
tableDescTwo.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptorBuilder tableDescThree = TableDescriptorBuilder
|
||||
.newBuilder(tableThree);
|
||||
tableDescThree.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
ADMIN.createTable(tableDescOne.build(), Bytes.toBytes("1"), Bytes.toBytes("1000"), 3);
|
||||
ADMIN.createTable(tableDescTwo.build(), Bytes.toBytes("1"), Bytes.toBytes("1000"), 3);
|
||||
ADMIN.createTable(tableDescThree.build(), Bytes.toBytes("1"), Bytes.toBytes("1000"), 4);
|
||||
ADMIN.disableTable(tableThree);
|
||||
deleteTable(tableThree);
|
||||
// wait for chore to complete
|
||||
|
@ -621,15 +631,16 @@ public class TestNamespaceAuditor {
|
|||
ADMIN.createNamespace(nspDesc);
|
||||
assertNotNull("Namespace descriptor found null.", ADMIN.getNamespaceDescriptor(nsp));
|
||||
assertEquals(3, ADMIN.listNamespaceDescriptors().length);
|
||||
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
|
||||
HTableDescriptor tableDescOne =
|
||||
new HTableDescriptor(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1"));
|
||||
tableDescOne.addFamily(fam1);
|
||||
HTableDescriptor tableDescTwo =
|
||||
new HTableDescriptor(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table2"));
|
||||
tableDescTwo.addFamily(fam1);
|
||||
ADMIN.createTable(tableDescOne);
|
||||
ADMIN.createTable(tableDescTwo, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("fam1")).build();
|
||||
TableDescriptorBuilder tableDescOne = TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1"));
|
||||
tableDescOne.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptorBuilder tableDescTwo = TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table2"));
|
||||
tableDescTwo.setColumnFamily(columnFamilyDescriptor);
|
||||
ADMIN.createTable(tableDescOne.build());
|
||||
ADMIN.createTable(tableDescTwo.build(), Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
}
|
||||
|
||||
@Test(expected = QuotaExceededException.class)
|
||||
|
@ -642,10 +653,12 @@ public class TestNamespaceAuditor {
|
|||
assertNotNull("Namespace descriptor found null.", ADMIN.getNamespaceDescriptor(nsp));
|
||||
TableName tableName = TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1");
|
||||
TableName cloneTableName = TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table2");
|
||||
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
|
||||
HTableDescriptor tableDescOne = new HTableDescriptor(tableName);
|
||||
tableDescOne.addFamily(fam1);
|
||||
ADMIN.createTable(tableDescOne);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("fam1")).build();
|
||||
TableDescriptorBuilder tableDescOne = TableDescriptorBuilder
|
||||
.newBuilder(tableName);
|
||||
tableDescOne.setColumnFamily(columnFamilyDescriptor);
|
||||
ADMIN.createTable(tableDescOne.build());
|
||||
String snapshot = "snapshot_testTableQuotaExceedWithCloneSnapshot";
|
||||
ADMIN.snapshot(snapshot, tableName);
|
||||
ADMIN.cloneSnapshot(snapshot, cloneTableName);
|
||||
|
@ -663,11 +676,13 @@ public class TestNamespaceAuditor {
|
|||
TableName tableName = TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1");
|
||||
TableName cloneTableName = TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table2");
|
||||
|
||||
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
|
||||
HTableDescriptor tableDescOne = new HTableDescriptor(tableName);
|
||||
tableDescOne.addFamily(fam1);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("fam1")).build();
|
||||
TableDescriptorBuilder tableDescOne = TableDescriptorBuilder
|
||||
.newBuilder(tableName);
|
||||
tableDescOne.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
ADMIN.createTable(tableDescOne, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
ADMIN.createTable(tableDescOne.build(), Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
String snapshot = "snapshot_testCloneSnapshot";
|
||||
ADMIN.snapshot(snapshot, tableName);
|
||||
ADMIN.cloneSnapshot(snapshot, cloneTableName);
|
||||
|
@ -699,10 +714,12 @@ public class TestNamespaceAuditor {
|
|||
ADMIN.createNamespace(nspDesc);
|
||||
assertNotNull("Namespace descriptor found null.", ADMIN.getNamespaceDescriptor(nsp));
|
||||
TableName tableName1 = TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1");
|
||||
HTableDescriptor tableDescOne = new HTableDescriptor(tableName1);
|
||||
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
|
||||
tableDescOne.addFamily(fam1);
|
||||
ADMIN.createTable(tableDescOne, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("fam1")).build();
|
||||
TableDescriptorBuilder tableDescOne = TableDescriptorBuilder
|
||||
.newBuilder(tableName1);
|
||||
tableDescOne.setColumnFamily(columnFamilyDescriptor);
|
||||
ADMIN.createTable(tableDescOne.build(), Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
|
||||
NamespaceTableAndRegionInfo nstate = getNamespaceState(nsp);
|
||||
assertEquals("Intial region count should be 4.", 4, nstate.getRegionCount());
|
||||
|
@ -736,11 +753,13 @@ public class TestNamespaceAuditor {
|
|||
NamespaceDescriptor ndesc = ADMIN.getNamespaceDescriptor(nsp);
|
||||
assertNotNull("Namespace descriptor found null.", ndesc);
|
||||
TableName tableName1 = TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1");
|
||||
HTableDescriptor tableDescOne = new HTableDescriptor(tableName1);
|
||||
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
|
||||
tableDescOne.addFamily(fam1);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("fam1")).build();
|
||||
TableDescriptorBuilder tableDescOne = TableDescriptorBuilder
|
||||
.newBuilder(tableName1);
|
||||
tableDescOne.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
ADMIN.createTable(tableDescOne, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
ADMIN.createTable(tableDescOne.build(), Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
|
||||
|
||||
NamespaceTableAndRegionInfo nstate = getNamespaceState(nsp);
|
||||
assertEquals("Intial region count should be 4.", 4, nstate.getRegionCount());
|
||||
|
@ -751,7 +770,7 @@ public class TestNamespaceAuditor {
|
|||
// recreate table with 1 region and set max regions to 3 for namespace
|
||||
ADMIN.disableTable(tableName1);
|
||||
ADMIN.deleteTable(tableName1);
|
||||
ADMIN.createTable(tableDescOne);
|
||||
ADMIN.createTable(tableDescOne.build());
|
||||
ndesc.setConfiguration(TableNamespaceManager.KEY_MAX_REGIONS, "3");
|
||||
ADMIN.modifyNamespace(ndesc);
|
||||
|
||||
|
|
|
@ -27,15 +27,17 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -349,9 +351,12 @@ public class TestMasterQuotasObserver {
|
|||
|
||||
private void createTable(Admin admin, TableName tn) throws Exception {
|
||||
// Create a table
|
||||
HTableDescriptor tableDesc = new HTableDescriptor(tn);
|
||||
tableDesc.addFamily(new HColumnDescriptor("F1"));
|
||||
admin.createTable(tableDesc);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tn);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("F1")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
}
|
||||
|
||||
private void dropTable(Admin admin, TableName tn) throws Exception {
|
||||
|
|
|
@ -28,15 +28,16 @@ import java.util.Random;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
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.Connection;
|
||||
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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -138,9 +139,14 @@ public class TestRegionSizeUse {
|
|||
}
|
||||
|
||||
// Create the table
|
||||
HTableDescriptor tableDesc = new HTableDescriptor(tn);
|
||||
tableDesc.addFamily(new HColumnDescriptor(F1));
|
||||
admin.createTable(tableDesc, Bytes.toBytes("1"), Bytes.toBytes("9"), NUM_SPLITS);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tn);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(F1)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
admin.createTable(tableDescriptorBuilder.build(), Bytes.toBytes("1"),
|
||||
Bytes.toBytes("9"), NUM_SPLITS);
|
||||
|
||||
final Table table = conn.getTable(tn);
|
||||
try {
|
||||
|
|
|
@ -50,6 +50,8 @@ import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
|
|||
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
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.Get;
|
||||
|
@ -62,6 +64,7 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
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.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockCache;
|
||||
|
@ -642,9 +645,12 @@ public class TestAtomicOperation {
|
|||
public void testPutAndCheckAndPutInParallel() throws Exception {
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setClass(HConstants.REGION_IMPL, MockHRegion.class, HeapSize.class);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()))
|
||||
.addFamily(new HColumnDescriptor(family));
|
||||
this.region = TEST_UTIL.createLocalHRegion(htd, null, null);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
this.region = TEST_UTIL.createLocalHRegion(tableDescriptorBuilder.build(), null, null);
|
||||
Put[] puts = new Put[1];
|
||||
Put put = new Put(Bytes.toBytes("r1"));
|
||||
put.addColumn(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10"));
|
||||
|
|
|
@ -36,9 +36,12 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||
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.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -182,13 +185,16 @@ public class TestColumnSeeking {
|
|||
byte[] familyBytes = Bytes.toBytes("Family");
|
||||
TableName table = TableName.valueOf(name.getMethodName());
|
||||
|
||||
HTableDescriptor htd = new HTableDescriptor(table);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMaxVersions(3);
|
||||
htd.addFamily(hcd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(table);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family))
|
||||
.setMaxVersions(3).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
HRegionInfo info = new HRegionInfo(table, null, null, false);
|
||||
HRegion region = TEST_UTIL.createLocalHRegion(info, htd);
|
||||
HRegion region = TEST_UTIL.createLocalHRegion(info, tableDescriptorBuilder.build());
|
||||
|
||||
List<String> rows = generateRandomWords(10, "row");
|
||||
List<String> allColumns = generateRandomWords(100, "column");
|
||||
|
|
|
@ -31,14 +31,16 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
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.CompactionState;
|
||||
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.crypto.Encryption;
|
||||
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
||||
import org.apache.hadoop.hbase.io.crypto.aes.AES;
|
||||
|
@ -56,8 +58,6 @@ 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;
|
||||
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestEncryptionKeyRotation {
|
||||
|
@ -66,7 +66,6 @@ public class TestEncryptionKeyRotation {
|
|||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestEncryptionKeyRotation.class);
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestEncryptionKeyRotation.class);
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private static final Configuration conf = TEST_UTIL.getConfiguration();
|
||||
private static final Key initialCFKey;
|
||||
|
@ -105,19 +104,23 @@ public class TestEncryptionKeyRotation {
|
|||
@Test
|
||||
public void testCFKeyRotation() throws Exception {
|
||||
// Create the table schema
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("default", name.getMethodName()));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("cf");
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf("default", name.getMethodName()));
|
||||
ColumnFamilyDescriptorBuilder columnFamilyDescriptorBuilder =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf"));
|
||||
String algorithm =
|
||||
conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
|
||||
hcd.setEncryptionType(algorithm);
|
||||
hcd.setEncryptionKey(EncryptionUtil.wrapKey(conf, "hbase", initialCFKey));
|
||||
htd.addFamily(hcd);
|
||||
conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
|
||||
columnFamilyDescriptorBuilder.setEncryptionType(algorithm);
|
||||
columnFamilyDescriptorBuilder.setEncryptionKey(EncryptionUtil.wrapKey(conf, "hbase",
|
||||
initialCFKey));
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptorBuilder.build());
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
|
||||
// Create the table and some on disk files
|
||||
createTableAndFlush(htd);
|
||||
createTableAndFlush(tableDescriptor);
|
||||
|
||||
// Verify we have store file(s) with the initial key
|
||||
final List<Path> initialPaths = findStorefilePaths(htd.getTableName());
|
||||
final List<Path> initialPaths = findStorefilePaths(tableDescriptor.getTableName());
|
||||
assertTrue(initialPaths.size() > 0);
|
||||
for (Path path: initialPaths) {
|
||||
assertTrue("Store file " + path + " has incorrect key",
|
||||
|
@ -125,30 +128,30 @@ public class TestEncryptionKeyRotation {
|
|||
}
|
||||
|
||||
// Update the schema with a new encryption key
|
||||
hcd = htd.getFamily(Bytes.toBytes("cf"));
|
||||
hcd.setEncryptionKey(EncryptionUtil.wrapKey(conf,
|
||||
columnFamilyDescriptorBuilder.setEncryptionKey(EncryptionUtil.wrapKey(conf,
|
||||
conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()),
|
||||
secondCFKey));
|
||||
TEST_UTIL.getAdmin().modifyColumnFamily(htd.getTableName(), hcd);
|
||||
TEST_UTIL.getAdmin().modifyColumnFamily(tableDescriptor.getTableName(),
|
||||
columnFamilyDescriptorBuilder.build());
|
||||
Thread.sleep(5000); // Need a predicate for online schema change
|
||||
|
||||
// And major compact
|
||||
TEST_UTIL.getAdmin().majorCompact(htd.getTableName());
|
||||
TEST_UTIL.getAdmin().majorCompact(tableDescriptor.getTableName());
|
||||
// waiting for the major compaction to complete
|
||||
TEST_UTIL.waitFor(30000, new Waiter.Predicate<IOException>() {
|
||||
@Override
|
||||
public boolean evaluate() throws IOException {
|
||||
return TEST_UTIL.getAdmin().getCompactionState(htd.getTableName()) ==
|
||||
CompactionState.NONE;
|
||||
return TEST_UTIL.getAdmin().getCompactionState(tableDescriptor
|
||||
.getTableName()) == CompactionState.NONE;
|
||||
}
|
||||
});
|
||||
List<Path> pathsAfterCompaction = findStorefilePaths(htd.getTableName());
|
||||
List<Path> pathsAfterCompaction = findStorefilePaths(tableDescriptor.getTableName());
|
||||
assertTrue(pathsAfterCompaction.size() > 0);
|
||||
for (Path path: pathsAfterCompaction) {
|
||||
assertTrue("Store file " + path + " has incorrect key",
|
||||
Bytes.equals(secondCFKey.getEncoded(), extractHFileKey(path)));
|
||||
}
|
||||
List<Path> compactedPaths = findCompactedStorefilePaths(htd.getTableName());
|
||||
List<Path> compactedPaths = findCompactedStorefilePaths(tableDescriptor.getTableName());
|
||||
assertTrue(compactedPaths.size() > 0);
|
||||
for (Path path: compactedPaths) {
|
||||
assertTrue("Store file " + path + " retains initial key",
|
||||
|
@ -159,19 +162,23 @@ public class TestEncryptionKeyRotation {
|
|||
@Test
|
||||
public void testMasterKeyRotation() throws Exception {
|
||||
// Create the table schema
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("default", name.getMethodName()));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("cf");
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf("default", name.getMethodName()));
|
||||
ColumnFamilyDescriptorBuilder columnFamilyDescriptorBuilder =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf"));
|
||||
String algorithm =
|
||||
conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
|
||||
hcd.setEncryptionType(algorithm);
|
||||
hcd.setEncryptionKey(EncryptionUtil.wrapKey(conf, "hbase", initialCFKey));
|
||||
htd.addFamily(hcd);
|
||||
columnFamilyDescriptorBuilder.setEncryptionType(algorithm);
|
||||
columnFamilyDescriptorBuilder.setEncryptionKey(
|
||||
EncryptionUtil.wrapKey(conf, "hbase", initialCFKey));
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptorBuilder.build());
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
|
||||
// Create the table and some on disk files
|
||||
createTableAndFlush(htd);
|
||||
createTableAndFlush(tableDescriptor);
|
||||
|
||||
// Verify we have store file(s) with the initial key
|
||||
List<Path> storeFilePaths = findStorefilePaths(htd.getTableName());
|
||||
List<Path> storeFilePaths = findStorefilePaths(tableDescriptor.getTableName());
|
||||
assertTrue(storeFilePaths.size() > 0);
|
||||
for (Path path: storeFilePaths) {
|
||||
assertTrue("Store file " + path + " has incorrect key",
|
||||
|
@ -188,9 +195,9 @@ public class TestEncryptionKeyRotation {
|
|||
// Start the cluster back up
|
||||
TEST_UTIL.startMiniHBaseCluster();
|
||||
// Verify the table can still be loaded
|
||||
TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
|
||||
TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName(), 5000);
|
||||
// Double check that the store file keys can be unwrapped
|
||||
storeFilePaths = findStorefilePaths(htd.getTableName());
|
||||
storeFilePaths = findStorefilePaths(tableDescriptor.getTableName());
|
||||
assertTrue(storeFilePaths.size() > 0);
|
||||
for (Path path: storeFilePaths) {
|
||||
assertTrue("Store file " + path + " has incorrect key",
|
||||
|
@ -228,20 +235,20 @@ public class TestEncryptionKeyRotation {
|
|||
return paths;
|
||||
}
|
||||
|
||||
private void createTableAndFlush(HTableDescriptor htd) throws Exception {
|
||||
HColumnDescriptor hcd = htd.getFamilies().iterator().next();
|
||||
private void createTableAndFlush(TableDescriptor tableDescriptor) throws Exception {
|
||||
ColumnFamilyDescriptor cfd = tableDescriptor.getColumnFamilies()[0];
|
||||
// Create the test table
|
||||
TEST_UTIL.getAdmin().createTable(htd);
|
||||
TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
|
||||
TEST_UTIL.getAdmin().createTable(tableDescriptor);
|
||||
TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName(), 5000);
|
||||
// Create a store file
|
||||
Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
|
||||
Table table = TEST_UTIL.getConnection().getTable(tableDescriptor.getTableName());
|
||||
try {
|
||||
table.put(new Put(Bytes.toBytes("testrow"))
|
||||
.addColumn(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value")));
|
||||
.addColumn(cfd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value")));
|
||||
} finally {
|
||||
table.close();
|
||||
}
|
||||
TEST_UTIL.getAdmin().flush(htd.getTableName());
|
||||
TEST_UTIL.getAdmin().flush(tableDescriptor.getTableName());
|
||||
}
|
||||
|
||||
private static byte[] extractHFileKey(Path path) throws Exception {
|
||||
|
|
|
@ -27,12 +27,12 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
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.Put;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.io.crypto.Encryption;
|
||||
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
@ -55,12 +55,12 @@ public class TestEncryptionRandomKeying {
|
|||
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private static Configuration conf = TEST_UTIL.getConfiguration();
|
||||
private static HTableDescriptor htd;
|
||||
private static TableDescriptorBuilder tdb;
|
||||
|
||||
private static List<Path> findStorefilePaths(TableName tableName) throws Exception {
|
||||
List<Path> paths = new ArrayList<>();
|
||||
for (Region region:
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions(htd.getTableName())) {
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions(tdb.build().getTableName())) {
|
||||
for (HStore store : ((HRegion) region).getStores()) {
|
||||
for (HStoreFile storefile : store.getStorefiles()) {
|
||||
paths.add(storefile.getPath());
|
||||
|
@ -94,29 +94,31 @@ public class TestEncryptionRandomKeying {
|
|||
|
||||
// Create the table schema
|
||||
// Specify an encryption algorithm without a key
|
||||
htd = new HTableDescriptor(TableName.valueOf("default", "TestEncryptionRandomKeying"));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("cf");
|
||||
String algorithm =
|
||||
conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
|
||||
hcd.setEncryptionType(algorithm);
|
||||
htd.addFamily(hcd);
|
||||
tdb = TableDescriptorBuilder.newBuilder(TableName.valueOf("default",
|
||||
"TestEncryptionRandomKeying"));
|
||||
ColumnFamilyDescriptorBuilder columnFamilyDescriptorBuilder =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf"));
|
||||
String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
|
||||
columnFamilyDescriptorBuilder.setEncryptionType(algorithm);
|
||||
tdb.setColumnFamily(columnFamilyDescriptorBuilder.build());
|
||||
|
||||
// Start the minicluster
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
|
||||
// Create the test table
|
||||
TEST_UTIL.getAdmin().createTable(htd);
|
||||
TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
|
||||
TEST_UTIL.getAdmin().createTable(tdb.build());
|
||||
TEST_UTIL.waitTableAvailable(tdb.build().getTableName(), 5000);
|
||||
|
||||
// Create a store file
|
||||
Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
|
||||
Table table = TEST_UTIL.getConnection().getTable(tdb.build().getTableName());
|
||||
try {
|
||||
table.put(new Put(Bytes.toBytes("testrow"))
|
||||
.addColumn(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value")));
|
||||
.addColumn(columnFamilyDescriptorBuilder.build().getName(),
|
||||
Bytes.toBytes("q"), Bytes.toBytes("value")));
|
||||
} finally {
|
||||
table.close();
|
||||
}
|
||||
TEST_UTIL.getAdmin().flush(htd.getTableName());
|
||||
TEST_UTIL.getAdmin().flush(tdb.build().getTableName());
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -127,7 +129,7 @@ public class TestEncryptionRandomKeying {
|
|||
@Test
|
||||
public void testRandomKeying() throws Exception {
|
||||
// Verify we have store file(s) with a random key
|
||||
final List<Path> initialPaths = findStorefilePaths(htd.getTableName());
|
||||
final List<Path> initialPaths = findStorefilePaths(tdb.build().getTableName());
|
||||
assertTrue(initialPaths.size() > 0);
|
||||
for (Path path: initialPaths) {
|
||||
assertNotNull("Store file " + path + " is not encrypted", extractHFileKey(path));
|
||||
|
|
|
@ -4187,12 +4187,16 @@ public class TestHRegion {
|
|||
try {
|
||||
FileSystem fs = Mockito.mock(FileSystem.class);
|
||||
Mockito.when(fs.exists((Path) Mockito.anyObject())).thenThrow(new IOException());
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor("cf"));
|
||||
info = new HRegionInfo(htd.getTableName(), HConstants.EMPTY_BYTE_ARRAY,
|
||||
HConstants.EMPTY_BYTE_ARRAY, false);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
info = new HRegionInfo(tableDescriptorBuilder.build().getTableName(),
|
||||
HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, false);
|
||||
Path path = new Path(dir + "testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization");
|
||||
region = HRegion.newHRegion(path, null, fs, CONF, info, htd, null);
|
||||
region = HRegion.newHRegion(path, null, fs, CONF, info,
|
||||
tableDescriptorBuilder.build(), null);
|
||||
// region initialization throws IOException and set task state to ABORTED.
|
||||
region.initialize();
|
||||
fail("Region initialization should fail due to IOException");
|
||||
|
@ -4217,13 +4221,18 @@ public class TestHRegion {
|
|||
public void testRegionInfoFileCreation() throws IOException {
|
||||
Path rootDir = new Path(dir + "testRegionInfoFileCreation");
|
||||
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
htd.addFamily(new HColumnDescriptor("cf"));
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
|
||||
|
||||
HRegionInfo hri = new HRegionInfo(htd.getTableName());
|
||||
HRegionInfo hri = new HRegionInfo(tableDescriptor.getTableName());
|
||||
|
||||
// Create a region and skip the initialization (like CreateTableHandler)
|
||||
region = HBaseTestingUtility.createRegionAndWAL(hri, rootDir, CONF, htd, false);
|
||||
region = HBaseTestingUtility.createRegionAndWAL(hri, rootDir, CONF,
|
||||
tableDescriptor, false);
|
||||
Path regionDir = region.getRegionFileSystem().getRegionDir();
|
||||
FileSystem fs = region.getRegionFileSystem().getFileSystem();
|
||||
HBaseTestingUtility.closeRegionAndWAL(region);
|
||||
|
@ -4235,7 +4244,7 @@ public class TestHRegion {
|
|||
fs.exists(regionInfoFile));
|
||||
|
||||
// Try to open the region
|
||||
region = HRegion.openHRegion(rootDir, hri, htd, null, CONF);
|
||||
region = HRegion.openHRegion(rootDir, hri, tableDescriptor, null, CONF);
|
||||
assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
|
||||
HBaseTestingUtility.closeRegionAndWAL(region);
|
||||
|
||||
|
@ -4248,7 +4257,7 @@ public class TestHRegion {
|
|||
assertFalse(HRegionFileSystem.REGION_INFO_FILE + " should be removed from the region dir",
|
||||
fs.exists(regionInfoFile));
|
||||
|
||||
region = HRegion.openHRegion(rootDir, hri, htd, null, CONF);
|
||||
region = HRegion.openHRegion(rootDir, hri, tableDescriptor, null, CONF);
|
||||
// region = TEST_UTIL.openHRegion(hri, htd);
|
||||
assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
|
||||
HBaseTestingUtility.closeRegionAndWAL(region);
|
||||
|
|
|
@ -36,9 +36,9 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
|
@ -110,19 +110,21 @@ public class TestHRegionFileSystem {
|
|||
|
||||
// alter table cf schema to change storage policies
|
||||
// and make sure it could override settings in conf
|
||||
HColumnDescriptor hcdA = new HColumnDescriptor(Bytes.toString(FAMILIES[0]));
|
||||
ColumnFamilyDescriptorBuilder cfdA =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[0]);
|
||||
// alter through setting HStore#BLOCK_STORAGE_POLICY_KEY in HColumnDescriptor
|
||||
hcdA.setValue(HStore.BLOCK_STORAGE_POLICY_KEY, "ONE_SSD");
|
||||
admin.modifyColumnFamily(TABLE_NAME, hcdA);
|
||||
cfdA.setValue(HStore.BLOCK_STORAGE_POLICY_KEY, "ONE_SSD");
|
||||
admin.modifyColumnFamily(TABLE_NAME, cfdA.build());
|
||||
while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().
|
||||
getRegionStates().hasRegionsInTransition()) {
|
||||
Thread.sleep(200);
|
||||
LOG.debug("Waiting on table to finish schema altering");
|
||||
}
|
||||
// alter through HColumnDescriptor#setStoragePolicy
|
||||
HColumnDescriptor hcdB = new HColumnDescriptor(Bytes.toString(FAMILIES[1]));
|
||||
hcdB.setStoragePolicy("ALL_SSD");
|
||||
admin.modifyColumnFamily(TABLE_NAME, hcdB);
|
||||
ColumnFamilyDescriptorBuilder cfdB =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[1]);
|
||||
cfdB.setStoragePolicy("ALL_SSD");
|
||||
admin.modifyColumnFamily(TABLE_NAME, cfdB.build());
|
||||
while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
|
||||
.hasRegionsInTransition()) {
|
||||
Thread.sleep(200);
|
||||
|
|
|
@ -38,9 +38,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.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
|
||||
|
@ -48,12 +46,15 @@ import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
|
|||
import org.apache.hadoop.hbase.StartMiniClusterOption;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
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.Connection;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
|
@ -311,14 +312,18 @@ public class TestHRegionServerBulkLoad {
|
|||
public void setupTable(TableName table, int cfs) throws IOException {
|
||||
try {
|
||||
LOG.info("Creating table " + table);
|
||||
HTableDescriptor htd = new HTableDescriptor(table);
|
||||
htd.addCoprocessor(MyObserver.class.getName());
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(table);
|
||||
|
||||
tableDescriptorBuilder.setCoprocessor(MyObserver.class.getName());
|
||||
MyObserver.sleepDuration = this.sleepDuration;
|
||||
for (int i = 0; i < 10; i++) {
|
||||
htd.addFamily(new HColumnDescriptor(family(i)));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family(i))).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
}
|
||||
|
||||
UTIL.getAdmin().createTable(htd);
|
||||
UTIL.getAdmin().createTable(tableDescriptorBuilder.build());
|
||||
} catch (TableExistsException tee) {
|
||||
LOG.info("Table " + table + " already exists");
|
||||
}
|
||||
|
|
|
@ -30,12 +30,13 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellComparator;
|
||||
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.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
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.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -75,14 +76,18 @@ public class TestMemStoreSegmentsIterator {
|
|||
public void setup() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
|
||||
htd.addFamily(hcd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(FAMILY)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
|
||||
HRegionInfo info = new HRegionInfo(TableName.valueOf(TABLE), null, null, false);
|
||||
Path rootPath = hbaseUtility.getDataTestDir(ROOT_SUB_PATH);
|
||||
this.wal = hbaseUtility.createWal(conf, rootPath, info);
|
||||
this.region = HRegion.createHRegion(info, rootPath, conf, htd, this.wal, true);
|
||||
this.store = new HStore(this.region, hcd, conf, false);
|
||||
this.region = HRegion.createHRegion(info, rootPath, conf,
|
||||
tableDescriptorBuilder.build(), this.wal, true);
|
||||
this.store = new HStore(this.region, columnFamilyDescriptor, conf, false);
|
||||
this.comparator = CellComparator.getInstance();
|
||||
this.compactionKVMax = HConstants.COMPACTION_KV_MAX_DEFAULT;
|
||||
}
|
||||
|
|
|
@ -36,9 +36,10 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||
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.Scan;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
|
@ -109,11 +110,13 @@ public class TestScanWithBloomError {
|
|||
|
||||
@Test
|
||||
public void testThreeStoreFiles() throws IOException {
|
||||
region = TEST_UTIL.createTestRegion(TABLE_NAME,
|
||||
new HColumnDescriptor(FAMILY)
|
||||
.setCompressionType(Compression.Algorithm.GZ)
|
||||
.setBloomFilterType(bloomType)
|
||||
.setMaxVersions(TestMultiColumnScanner.MAX_VERSIONS));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(FAMILY))
|
||||
.setCompressionType(Compression.Algorithm.GZ)
|
||||
.setBloomFilterType(bloomType)
|
||||
.setMaxVersions(TestMultiColumnScanner.MAX_VERSIONS).build();
|
||||
region = TEST_UTIL.createTestRegion(TABLE_NAME, columnFamilyDescriptor);
|
||||
createStoreFile(new int[] {1, 2, 6});
|
||||
createStoreFile(new int[] {1, 2, 3, 7});
|
||||
createStoreFile(new int[] {1, 9});
|
||||
|
|
|
@ -29,17 +29,18 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.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.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -73,11 +74,14 @@ public class TestScannerWithBulkload {
|
|||
}
|
||||
|
||||
private static void createTable(Admin admin, TableName tableName) throws IOException {
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("col");
|
||||
hcd.setMaxVersions(3);
|
||||
desc.addFamily(hcd);
|
||||
admin.createTable(desc);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("col"))
|
||||
.setMaxVersions(3).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.createTable(tableDescriptorBuilder.build());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -35,10 +35,11 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
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.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -150,13 +151,14 @@ public class TestSeekOptimizations {
|
|||
public void testMultipleTimestampRanges() throws IOException {
|
||||
// enable seek counting
|
||||
StoreFileScanner.instrument();
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(FAMILY))
|
||||
.setCompressionType(comprAlgo)
|
||||
.setBloomFilterType(bloomType)
|
||||
.setMaxVersions(3)
|
||||
.build();
|
||||
|
||||
region = TEST_UTIL.createTestRegion("testMultipleTimestampRanges",
|
||||
new HColumnDescriptor(FAMILY)
|
||||
.setCompressionType(comprAlgo)
|
||||
.setBloomFilterType(bloomType)
|
||||
.setMaxVersions(3)
|
||||
);
|
||||
region = TEST_UTIL.createTestRegion("testMultipleTimestampRanges", columnFamilyDescriptor);
|
||||
|
||||
// Delete the given timestamp and everything before.
|
||||
final long latestDelTS = USE_MANY_STORE_FILES ? 1397 : -1;
|
||||
|
|
|
@ -23,7 +23,8 @@ import java.io.IOException;
|
|||
import org.apache.commons.lang3.RandomStringUtils;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.filter.TimestampsFilter;
|
||||
|
@ -94,12 +95,14 @@ public class TestTimestampFilterSeekHint {
|
|||
|
||||
@Before
|
||||
public void prepareRegion() throws IOException {
|
||||
region =
|
||||
TEST_UTIL.createTestRegion("TestTimestampFilterSeekHint" + regionCount++,
|
||||
new HColumnDescriptor(FAMILY)
|
||||
.setBlocksize(1024)
|
||||
.setMaxVersions(MAX_VERSIONS)
|
||||
);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(FAMILY))
|
||||
.setBlocksize(1024)
|
||||
.setMaxVersions(MAX_VERSIONS)
|
||||
.build();
|
||||
region = TEST_UTIL
|
||||
.createTestRegion("TestTimestampFilterSeekHint" + regionCount++, columnFamilyDescriptor);
|
||||
|
||||
for (long i = 0; i <MAX_VERSIONS - 2; i++) {
|
||||
Put p = new Put(RK_BYTES, i);
|
||||
|
|
|
@ -61,6 +61,8 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
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.Delete;
|
||||
|
@ -75,6 +77,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
|
@ -410,13 +413,18 @@ public class TestAccessController extends SecureTestUtil {
|
|||
AccessTestAction modifyTable = new AccessTestAction() {
|
||||
@Override
|
||||
public Object run() throws Exception {
|
||||
HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
|
||||
htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
|
||||
htd.addFamily(new HColumnDescriptor("fam_" + User.getCurrent().getShortName()));
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TEST_TABLE);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(TEST_FAMILY).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("fam_" + User.getCurrent().getShortName())).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
ACCESS_CONTROLLER.preModifyTable(ObserverContextImpl.createAndPrepare(CP_ENV),
|
||||
TEST_TABLE,
|
||||
null, // not needed by AccessController
|
||||
htd);
|
||||
tableDescriptorBuilder.build());
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -45,6 +45,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
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.Get;
|
||||
|
@ -55,6 +57,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
|
|||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
|
||||
|
@ -662,8 +665,9 @@ public abstract class TestVisibilityLabels {
|
|||
} catch (Exception e) {
|
||||
}
|
||||
try {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("testFamily");
|
||||
admin.addColumnFamily(LABELS_TABLE_NAME, hcd);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("testFamily")).build();
|
||||
admin.addColumnFamily(LABELS_TABLE_NAME, columnFamilyDescriptor);
|
||||
fail("Lables table should not get altered by user.");
|
||||
} catch (Exception e) {
|
||||
}
|
||||
|
@ -680,10 +684,15 @@ public abstract class TestVisibilityLabels {
|
|||
} catch (Exception e) {
|
||||
}
|
||||
try {
|
||||
HTableDescriptor htd = new HTableDescriptor(LABELS_TABLE_NAME);
|
||||
htd.addFamily(new HColumnDescriptor("f1"));
|
||||
htd.addFamily(new HColumnDescriptor("f2"));
|
||||
admin.modifyTable(htd);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(LABELS_TABLE_NAME);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f1")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("f2")).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
admin.modifyTable(tableDescriptorBuilder.build());
|
||||
fail("Lables table should not get altered by user.");
|
||||
} catch (Exception e) {
|
||||
}
|
||||
|
|
|
@ -39,13 +39,14 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.ClusterMetrics;
|
||||
import org.apache.hadoop.hbase.ClusterMetrics.Option;
|
||||
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.ServerName;
|
||||
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;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -55,6 +56,7 @@ import org.apache.hadoop.hbase.client.RegionLocator;
|
|||
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.coprocessor.MasterCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
|
@ -235,11 +237,14 @@ public class BaseTestHBaseFsck {
|
|||
* @throws Exception
|
||||
*/
|
||||
void setupTableWithRegionReplica(TableName tablename, int replicaCount) throws Exception {
|
||||
HTableDescriptor desc = new HTableDescriptor(tablename);
|
||||
desc.setRegionReplication(replicaCount);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
|
||||
desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
|
||||
createTable(TEST_UTIL, desc, SPLITS);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tablename);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAM).build();
|
||||
tableDescriptorBuilder.setRegionReplication(replicaCount);
|
||||
// If a table has no CF's it doesn't get checked
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
createTable(TEST_UTIL, tableDescriptorBuilder.build(), SPLITS);
|
||||
|
||||
tbl = connection.getTable(tablename, tableExecutorService);
|
||||
List<Put> puts = new ArrayList<>(ROWKEYS.length);
|
||||
|
@ -258,12 +263,16 @@ public class BaseTestHBaseFsck {
|
|||
* @throws Exception
|
||||
*/
|
||||
void setupMobTable(TableName tablename) throws Exception {
|
||||
HTableDescriptor desc = new HTableDescriptor(tablename);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(0);
|
||||
desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
|
||||
createTable(TEST_UTIL, desc, SPLITS);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tablename);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(FAM)
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(0).build();
|
||||
// If a table has no CF's it doesn't get checked
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
createTable(TEST_UTIL, tableDescriptorBuilder.build(), SPLITS);
|
||||
|
||||
tbl = connection.getTable(tablename, tableExecutorService);
|
||||
List<Put> puts = new ArrayList<>(ROWKEYS.length);
|
||||
|
@ -600,21 +609,21 @@ public class BaseTestHBaseFsck {
|
|||
}
|
||||
}
|
||||
|
||||
public static void createTable(HBaseTestingUtility testUtil, HTableDescriptor htd,
|
||||
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.
|
||||
MasterSyncCoprocessor coproc = testUtil.getHBaseCluster().getMaster()
|
||||
.getMasterCoprocessorHost().findCoprocessor(MasterSyncCoprocessor.class);
|
||||
coproc.tableCreationLatch = new CountDownLatch(1);
|
||||
if (splitKeys != null) {
|
||||
admin.createTable(htd, splitKeys);
|
||||
admin.createTable(tableDescriptor, splitKeys);
|
||||
} else {
|
||||
admin.createTable(htd);
|
||||
admin.createTable(tableDescriptor);
|
||||
}
|
||||
coproc.tableCreationLatch.await();
|
||||
coproc.tableCreationLatch = null;
|
||||
testUtil.waitUntilAllRegionsAssigned(htd.getTableName());
|
||||
testUtil.waitUntilAllRegionsAssigned(tableDescriptor.getTableName());
|
||||
}
|
||||
|
||||
public static void deleteTable(HBaseTestingUtility testUtil, TableName tableName)
|
||||
|
|
|
@ -30,12 +30,14 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
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.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.crypto.Encryption;
|
||||
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
||||
import org.apache.hadoop.hbase.io.crypto.aes.AES;
|
||||
|
@ -67,7 +69,7 @@ public class TestHBaseFsckEncryption {
|
|||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
private Configuration conf;
|
||||
private HTableDescriptor htd;
|
||||
private TableDescriptor tableDescriptor;
|
||||
private Key cfKey;
|
||||
|
||||
@Before
|
||||
|
@ -89,15 +91,19 @@ public class TestHBaseFsckEncryption {
|
|||
TEST_UTIL.startMiniCluster(3);
|
||||
|
||||
// Create the table
|
||||
htd = new HTableDescriptor(TableName.valueOf("default", "TestHBaseFsckEncryption"));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("cf");
|
||||
hcd.setEncryptionType(algorithm);
|
||||
hcd.setEncryptionKey(EncryptionUtil.wrapKey(conf,
|
||||
conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()),
|
||||
cfKey));
|
||||
htd.addFamily(hcd);
|
||||
TEST_UTIL.getAdmin().createTable(htd);
|
||||
TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(TableName.valueOf("default", "TestHBaseFsckEncryption"));
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes("cf"))
|
||||
.setEncryptionType(algorithm)
|
||||
.setEncryptionKey(EncryptionUtil.wrapKey(conf,
|
||||
conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()),
|
||||
cfKey)).build();
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
tableDescriptor = tableDescriptorBuilder.build();
|
||||
TEST_UTIL.getAdmin().createTable(tableDescriptor);
|
||||
TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName(), 5000);
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -108,7 +114,7 @@ public class TestHBaseFsckEncryption {
|
|||
@Test
|
||||
public void testFsckWithEncryption() throws Exception {
|
||||
// Populate the table with some data
|
||||
Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
|
||||
Table table = TEST_UTIL.getConnection().getTable(tableDescriptor.getTableName());
|
||||
try {
|
||||
byte[] values = { 'A', 'B', 'C', 'D' };
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
|
@ -123,10 +129,10 @@ public class TestHBaseFsckEncryption {
|
|||
table.close();
|
||||
}
|
||||
// Flush it
|
||||
TEST_UTIL.getAdmin().flush(htd.getTableName());
|
||||
TEST_UTIL.getAdmin().flush(tableDescriptor.getTableName());
|
||||
|
||||
// Verify we have encrypted store files on disk
|
||||
final List<Path> paths = findStorefilePaths(htd.getTableName());
|
||||
final List<Path> paths = findStorefilePaths(tableDescriptor.getTableName());
|
||||
assertTrue(paths.size() > 0);
|
||||
for (Path path: paths) {
|
||||
assertTrue("Store file " + path + " has incorrect key",
|
||||
|
@ -134,7 +140,7 @@ public class TestHBaseFsckEncryption {
|
|||
}
|
||||
|
||||
// Insure HBck doesn't consider them corrupt
|
||||
HBaseFsck res = HbckTestingUtil.doHFileQuarantine(conf, htd.getTableName());
|
||||
HBaseFsck res = HbckTestingUtil.doHFileQuarantine(conf, tableDescriptor.getTableName());
|
||||
assertEquals(0, res.getRetCode());
|
||||
HFileCorruptionChecker hfcc = res.getHFilecorruptionChecker();
|
||||
assertEquals(0, hfcc.getCorrupted().size());
|
||||
|
@ -146,7 +152,7 @@ public class TestHBaseFsckEncryption {
|
|||
private List<Path> findStorefilePaths(TableName tableName) throws Exception {
|
||||
List<Path> paths = new ArrayList<>();
|
||||
for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName)
|
||||
.getRegions(htd.getTableName())) {
|
||||
.getRegions(tableDescriptor.getTableName())) {
|
||||
for (HStore store : ((HRegion) region).getStores()) {
|
||||
for (HStoreFile storefile : store.getStorefiles()) {
|
||||
paths.add(storefile.getPath());
|
||||
|
|
|
@ -27,14 +27,14 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
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;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -47,6 +47,7 @@ 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.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
|
@ -130,10 +131,13 @@ public class OfflineMetaRebuildTestCore {
|
|||
* @throws KeeperException
|
||||
*/
|
||||
private Table setupTable(TableName tablename) throws Exception {
|
||||
HTableDescriptor desc = new HTableDescriptor(tablename);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
|
||||
desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
|
||||
TEST_UTIL.getAdmin().createTable(desc, splits);
|
||||
TableDescriptorBuilder tableDescriptorBuilder =
|
||||
TableDescriptorBuilder.newBuilder(tablename);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAM).build();
|
||||
// If a table has no CF's it doesn't get checked
|
||||
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
|
||||
TEST_UTIL.getAdmin().createTable(tableDescriptorBuilder.build(), splits);
|
||||
return this.connection.getTable(tablename);
|
||||
}
|
||||
|
||||
|
|
|
@ -36,13 +36,15 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.CompatibilityFactory;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
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.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.filter.ParseFilter;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
|
||||
|
@ -726,11 +728,17 @@ public class TestThriftServer {
|
|||
String col = "c";
|
||||
// create a table which will throw exceptions for requests
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
HTableDescriptor tableDesc = new HTableDescriptor(tableName);
|
||||
tableDesc.addCoprocessor(ErrorThrowingGetObserver.class.getName());
|
||||
tableDesc.addFamily(new HColumnDescriptor(family));
|
||||
|
||||
Table table = UTIL.createTable(tableDesc, null);
|
||||
ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(family))
|
||||
.build();
|
||||
TableDescriptor tableDescriptor =
|
||||
TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setCoprocessor(ErrorThrowingGetObserver.class.getName())
|
||||
.setColumnFamily(columnFamilyDescriptor)
|
||||
.build();
|
||||
|
||||
Table table = UTIL.createTable(tableDescriptor, null);
|
||||
long now = System.currentTimeMillis();
|
||||
table.put(new Put(Bytes.toBytes(rowkey))
|
||||
.addColumn(Bytes.toBytes(family), Bytes.toBytes(col), now, Bytes.toBytes("val1")));
|
||||
|
|
Loading…
Reference in New Issue