HBASE-18503 Change ***Util and Master to use TableDescriptor and ColumnFamilyDescriptor
This commit is contained in:
parent
ae3b51a7a7
commit
b03348630c
|
@ -44,7 +44,6 @@ import org.apache.hadoop.fs.RemoteIterator;
|
|||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -56,6 +55,7 @@ import org.apache.hadoop.hbase.backup.impl.BackupManifest;
|
|||
import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -139,7 +139,7 @@ public final class BackupUtils {
|
|||
LOG.warn("Table " + table + " does not exists, skipping it.");
|
||||
continue;
|
||||
}
|
||||
HTableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table);
|
||||
TableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table);
|
||||
|
||||
// write a copy of descriptor to the target directory
|
||||
Path target = new Path(backupInfo.getTableBackupDir(table));
|
||||
|
|
|
@ -33,16 +33,17 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.backup.BackupRestoreFactory;
|
||||
import org.apache.hadoop.hbase.backup.HBackupFileSystem;
|
||||
import org.apache.hadoop.hbase.backup.RestoreJob;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
|
||||
|
@ -122,10 +123,10 @@ public class RestoreTool {
|
|||
}
|
||||
|
||||
|
||||
void modifyTableSync(Connection conn, HTableDescriptor desc) throws IOException {
|
||||
void modifyTableSync(Connection conn, TableDescriptor desc) throws IOException {
|
||||
|
||||
try (Admin admin = conn.getAdmin();) {
|
||||
admin.modifyTable(desc.getTableName(), desc);
|
||||
admin.modifyTable(desc);
|
||||
int attempt = 0;
|
||||
int maxAttempts = 600;
|
||||
while (!admin.isTableAvailable(desc.getTableName())) {
|
||||
|
@ -172,29 +173,30 @@ public class RestoreTool {
|
|||
// adjust table schema
|
||||
for (int i = 0; i < tableNames.length; i++) {
|
||||
TableName tableName = tableNames[i];
|
||||
HTableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, incrBackupId);
|
||||
TableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, incrBackupId);
|
||||
LOG.debug("Found descriptor " + tableDescriptor + " through " + incrBackupId);
|
||||
|
||||
TableName newTableName = newTableNames[i];
|
||||
HTableDescriptor newTableDescriptor = new HTableDescriptor(admin.getTableDescriptor(newTableName));
|
||||
List<HColumnDescriptor> families = Arrays.asList(tableDescriptor.getColumnFamilies());
|
||||
List<HColumnDescriptor> existingFamilies =
|
||||
TableDescriptor newTableDescriptor = admin.listTableDescriptor(newTableName);
|
||||
List<ColumnFamilyDescriptor> families = Arrays.asList(tableDescriptor.getColumnFamilies());
|
||||
List<ColumnFamilyDescriptor> existingFamilies =
|
||||
Arrays.asList(newTableDescriptor.getColumnFamilies());
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(newTableDescriptor);
|
||||
boolean schemaChangeNeeded = false;
|
||||
for (HColumnDescriptor family : families) {
|
||||
for (ColumnFamilyDescriptor family : families) {
|
||||
if (!existingFamilies.contains(family)) {
|
||||
newTableDescriptor.addFamily(family);
|
||||
builder.addColumnFamily(family);
|
||||
schemaChangeNeeded = true;
|
||||
}
|
||||
}
|
||||
for (HColumnDescriptor family : existingFamilies) {
|
||||
for (ColumnFamilyDescriptor family : existingFamilies) {
|
||||
if (!families.contains(family)) {
|
||||
newTableDescriptor.removeFamily(family.getName());
|
||||
builder.removeColumnFamily(family.getName());
|
||||
schemaChangeNeeded = true;
|
||||
}
|
||||
}
|
||||
if (schemaChangeNeeded) {
|
||||
modifyTableSync(conn, newTableDescriptor);
|
||||
modifyTableSync(conn, builder.build());
|
||||
LOG.info("Changed " + newTableDescriptor.getTableName() + " to: " + newTableDescriptor);
|
||||
}
|
||||
}
|
||||
|
@ -253,24 +255,24 @@ public class RestoreTool {
|
|||
/**
|
||||
* Get table descriptor
|
||||
* @param tableName is the table backed up
|
||||
* @return {@link HTableDescriptor} saved in backup image of the table
|
||||
* @return {@link TableDescriptor} saved in backup image of the table
|
||||
*/
|
||||
HTableDescriptor getTableDesc(TableName tableName) throws FileNotFoundException, IOException {
|
||||
TableDescriptor getTableDesc(TableName tableName) throws FileNotFoundException, IOException {
|
||||
Path tableInfoPath = this.getTableInfoPath(tableName);
|
||||
SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath);
|
||||
SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc);
|
||||
HTableDescriptor tableDescriptor = manifest.getTableDescriptor();
|
||||
TableDescriptor tableDescriptor = manifest.getTableDescriptor();
|
||||
if (!tableDescriptor.getTableName().equals(tableName)) {
|
||||
LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: "
|
||||
+ tableInfoPath.toString());
|
||||
LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString());
|
||||
LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getTableName().getNameAsString());
|
||||
throw new FileNotFoundException("couldn't find Table Desc for table: " + tableName
|
||||
+ " under tableInfoPath: " + tableInfoPath.toString());
|
||||
}
|
||||
return tableDescriptor;
|
||||
}
|
||||
|
||||
private HTableDescriptor getTableDescriptor(FileSystem fileSys, TableName tableName,
|
||||
private TableDescriptor getTableDescriptor(FileSystem fileSys, TableName tableName,
|
||||
String lastIncrBackupId) throws IOException {
|
||||
if (lastIncrBackupId != null) {
|
||||
String target =
|
||||
|
@ -289,7 +291,7 @@ public class RestoreTool {
|
|||
FileSystem fileSys = tableBackupPath.getFileSystem(this.conf);
|
||||
|
||||
// get table descriptor first
|
||||
HTableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, lastIncrBackupId);
|
||||
TableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, lastIncrBackupId);
|
||||
if (tableDescriptor != null) {
|
||||
LOG.debug("Retrieved descriptor: " + tableDescriptor + " thru " + lastIncrBackupId);
|
||||
}
|
||||
|
@ -325,7 +327,7 @@ public class RestoreTool {
|
|||
LOG.debug("find table descriptor but no archive dir for table " + tableName
|
||||
+ ", will only create table");
|
||||
}
|
||||
tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
|
||||
tableDescriptor = TableDescriptorBuilder.copy(newTableName, tableDescriptor);
|
||||
checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, null, tableDescriptor,
|
||||
truncateIfExists);
|
||||
return;
|
||||
|
@ -336,9 +338,9 @@ public class RestoreTool {
|
|||
}
|
||||
|
||||
if (tableDescriptor == null) {
|
||||
tableDescriptor = new HTableDescriptor(newTableName);
|
||||
tableDescriptor = TableDescriptorBuilder.newBuilder(newTableName).build();
|
||||
} else {
|
||||
tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
|
||||
tableDescriptor = TableDescriptorBuilder.copy(newTableName, tableDescriptor);
|
||||
}
|
||||
|
||||
// record all region dirs:
|
||||
|
@ -470,7 +472,7 @@ public class RestoreTool {
|
|||
* @throws IOException exception
|
||||
*/
|
||||
private void checkAndCreateTable(Connection conn, Path tableBackupPath, TableName tableName,
|
||||
TableName targetTableName, ArrayList<Path> regionDirList, HTableDescriptor htd,
|
||||
TableName targetTableName, ArrayList<Path> regionDirList, TableDescriptor htd,
|
||||
boolean truncateIfExists) throws IOException {
|
||||
try (Admin admin = conn.getAdmin();) {
|
||||
boolean createNew = false;
|
||||
|
|
|
@ -639,13 +639,10 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
if (obj instanceof HColumnDescriptor) {
|
||||
return delegatee.equals(((HColumnDescriptor) obj).delegatee);
|
||||
}
|
||||
if (!(obj instanceof HColumnDescriptor)) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((HColumnDescriptor)obj) == 0;
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -658,7 +655,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
|
||||
@Override
|
||||
public int compareTo(HColumnDescriptor other) {
|
||||
return delegatee.compareTo(other.delegatee);
|
||||
return COMPARATOR.compare(this, other);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -495,13 +495,10 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
if (obj instanceof HTableDescriptor) {
|
||||
return delegatee.equals(((HTableDescriptor) obj).delegatee);
|
||||
}
|
||||
if (!(obj instanceof HTableDescriptor)) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((HTableDescriptor)obj) == 0;
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -523,7 +520,7 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
*/
|
||||
@Override
|
||||
public int compareTo(final HTableDescriptor other) {
|
||||
return delegatee.compareTo(other.delegatee);
|
||||
return TableDescriptor.COMPARATOR.compare(this, other);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -1160,13 +1160,10 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
if (obj instanceof ModifyableColumnFamilyDescriptor) {
|
||||
return ColumnFamilyDescriptor.COMPARATOR.compare(this, (ModifyableColumnFamilyDescriptor) obj) == 0;
|
||||
}
|
||||
if (!(obj instanceof ModifyableColumnFamilyDescriptor)) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((ModifyableColumnFamilyDescriptor) obj) == 0;
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1188,7 +1185,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
* @see #parseFrom(byte[])
|
||||
*/
|
||||
private byte[] toByteArray() {
|
||||
return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToColumnFamilySchema(this)
|
||||
return ProtobufUtil.prependPBMagic(ProtobufUtil.toColumnFamilySchema(this)
|
||||
.toByteArray());
|
||||
}
|
||||
|
||||
|
@ -1213,7 +1210,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
} catch (IOException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
return ProtobufUtil.convertToColumnDesc(cfs);
|
||||
return ProtobufUtil.toColumnFamilyDescriptor(cfs);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -378,7 +378,7 @@ public class HBaseAdmin implements Admin {
|
|||
.setNamespaceName(Bytes.toString(name)).build())
|
||||
.getTableSchemaList()
|
||||
.stream()
|
||||
.map(ProtobufUtil::convertToTableDesc)
|
||||
.map(ProtobufUtil::toTableDescriptor)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
});
|
||||
|
@ -459,8 +459,8 @@ public class HBaseAdmin implements Admin {
|
|||
protected HTableDescriptor[] rpcCall() throws Exception {
|
||||
GetTableDescriptorsRequest req =
|
||||
RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
|
||||
return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(getRpcController(),
|
||||
req));
|
||||
return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(),
|
||||
req)).stream().map(ImmutableHTableDescriptor::new).toArray(HTableDescriptor[]::new);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -525,7 +525,7 @@ public class HBaseAdmin implements Admin {
|
|||
RequestConverter.buildGetTableDescriptorsRequest(tableName);
|
||||
GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
|
||||
if (!htds.getTableSchemaList().isEmpty()) {
|
||||
return ProtobufUtil.convertToTableDesc(htds.getTableSchemaList().get(0));
|
||||
return ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0));
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -554,7 +554,7 @@ public class HBaseAdmin implements Admin {
|
|||
RequestConverter.buildGetTableDescriptorsRequest(tableName);
|
||||
GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
|
||||
if (!htds.getTableSchemaList().isEmpty()) {
|
||||
return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
|
||||
return new ImmutableHTableDescriptor(ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)));
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -2300,7 +2300,7 @@ public class HBaseAdmin implements Admin {
|
|||
.build()).getTableSchemaList();
|
||||
HTableDescriptor[] res = new HTableDescriptor[list.size()];
|
||||
for(int i=0; i < list.size(); i++) {
|
||||
res[i] = new ImmutableHTableDescriptor(ProtobufUtil.convertToHTableDesc(list.get(i)));
|
||||
res[i] = new ImmutableHTableDescriptor(ProtobufUtil.toTableDescriptor(list.get(i)));
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
@ -2419,33 +2419,14 @@ public class HBaseAdmin implements Admin {
|
|||
protected HTableDescriptor[] rpcCall() throws Exception {
|
||||
GetTableDescriptorsRequest req =
|
||||
RequestConverter.buildGetTableDescriptorsRequest(tableNames);
|
||||
return ProtobufUtil.
|
||||
getHTableDescriptorArray(master.getTableDescriptors(getRpcController(), req));
|
||||
return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(), req))
|
||||
.stream()
|
||||
.map(ImmutableHTableDescriptor::new)
|
||||
.toArray(HTableDescriptor[]::new);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Get tableDescriptor
|
||||
* @param tableName one table name
|
||||
* @return HTD the HTableDescriptor or null if the table not exists
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
private HTableDescriptor getTableDescriptorByTableName(TableName tableName)
|
||||
throws IOException {
|
||||
List<TableName> tableNames = new ArrayList<>(1);
|
||||
tableNames.add(tableName);
|
||||
|
||||
HTableDescriptor[] htdl = getTableDescriptorsByTableName(tableNames);
|
||||
|
||||
if (htdl == null || htdl.length == 0) {
|
||||
return null;
|
||||
}
|
||||
else {
|
||||
return htdl[0];
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HTableDescriptor[] getTableDescriptors(List<String> names)
|
||||
throws IOException {
|
||||
|
@ -3709,7 +3690,7 @@ public class HBaseAdmin implements Admin {
|
|||
* @return the table descriptor
|
||||
*/
|
||||
protected TableDescriptor getTableDescriptor() throws IOException {
|
||||
return getAdmin().getTableDescriptorByTableName(getTableName());
|
||||
return getAdmin().listTableDescriptor(getTableName());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -453,7 +453,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
return;
|
||||
}
|
||||
if (!tableSchemas.isEmpty()) {
|
||||
future.complete(ProtobufUtil.convertToTableDesc(tableSchemas.get(0)));
|
||||
future.complete(ProtobufUtil.toTableDescriptor(tableSchemas.get(0)));
|
||||
} else {
|
||||
future.completeExceptionally(new TableNotFoundException(tableName.getNameAsString()));
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -252,10 +253,14 @@ public class TableDescriptorBuilder {
|
|||
return new TableDescriptorBuilder(name);
|
||||
}
|
||||
|
||||
public static TableDescriptor copy(TableDescriptor desc) throws DeserializationException {
|
||||
public static TableDescriptor copy(TableDescriptor desc) {
|
||||
return new ModifyableTableDescriptor(desc);
|
||||
}
|
||||
|
||||
public static TableDescriptor copy(TableName name, TableDescriptor desc) {
|
||||
return new ModifyableTableDescriptor(name, desc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Copy all configuration, values, families, and name from the input.
|
||||
* @param desc The desciptor to copy
|
||||
|
@ -1012,13 +1017,10 @@ public class TableDescriptorBuilder {
|
|||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
if (obj instanceof ModifyableTableDescriptor) {
|
||||
return TableDescriptor.COMPARATOR.compare(this, (ModifyableTableDescriptor) obj) == 0;
|
||||
}
|
||||
if (!(obj instanceof ModifyableTableDescriptor)) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((ModifyableTableDescriptor) obj) == 0;
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1395,7 +1397,7 @@ public class TableDescriptorBuilder {
|
|||
* @return the bytes in pb format
|
||||
*/
|
||||
private byte[] toByteArray() {
|
||||
return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToTableSchema(this).toByteArray());
|
||||
return ProtobufUtil.prependPBMagic(ProtobufUtil.toTableSchema(this).toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1415,7 +1417,7 @@ public class TableDescriptorBuilder {
|
|||
HBaseProtos.TableSchema.Builder builder = HBaseProtos.TableSchema.newBuilder();
|
||||
try {
|
||||
ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
|
||||
return ProtobufUtil.convertToTableDesc(builder.build());
|
||||
return ProtobufUtil.toTableDescriptor(builder.build());
|
||||
} catch (IOException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
|
|
|
@ -17,15 +17,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.shaded.protobuf;
|
||||
|
||||
import java.awt.image.BandCombineOp;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.Method;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
|
@ -50,10 +47,8 @@ import org.apache.hadoop.hbase.ClusterStatus;
|
|||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
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.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.ProcedureInfo;
|
||||
|
@ -74,7 +69,6 @@ import org.apache.hadoop.hbase.client.Cursor;
|
|||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
|
||||
|
@ -102,7 +96,6 @@ import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
|
|||
import org.apache.hadoop.hbase.quotas.ThrottleType;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.security.visibility.Authorizations;
|
||||
import org.apache.hadoop.hbase.security.visibility.CellVisibility;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
|
||||
|
@ -182,7 +175,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
|
|||
import org.apache.hadoop.hbase.util.Addressing;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.DynamicClassLoader;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
||||
import org.apache.hadoop.hbase.util.Methods;
|
||||
|
@ -424,24 +416,6 @@ public final class ProtobufUtil {
|
|||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get HTableDescriptor[] from GetTableDescriptorsResponse protobuf
|
||||
*
|
||||
* @param proto the GetTableDescriptorsResponse
|
||||
* @return a immutable HTableDescriptor array
|
||||
* @deprecated Use {@link #toTableDescriptorList} after removing the HTableDescriptor
|
||||
*/
|
||||
@Deprecated
|
||||
public static HTableDescriptor[] getHTableDescriptorArray(GetTableDescriptorsResponse proto) {
|
||||
if (proto == null) return null;
|
||||
|
||||
HTableDescriptor[] ret = new HTableDescriptor[proto.getTableSchemaCount()];
|
||||
for (int i = 0; i < proto.getTableSchemaCount(); ++i) {
|
||||
ret[i] = new ImmutableHTableDescriptor(convertToHTableDesc(proto.getTableSchema(i)));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of TableDescriptor from GetTableDescriptorsResponse protobuf
|
||||
*
|
||||
|
@ -450,7 +424,7 @@ public final class ProtobufUtil {
|
|||
*/
|
||||
public static List<TableDescriptor> toTableDescriptorList(GetTableDescriptorsResponse proto) {
|
||||
if (proto == null) return new ArrayList<>();
|
||||
return proto.getTableSchemaList().stream().map(ProtobufUtil::convertToTableDesc)
|
||||
return proto.getTableSchemaList().stream().map(ProtobufUtil::toTableDescriptor)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
|
@ -2841,11 +2815,11 @@ public final class ProtobufUtil {
|
|||
}
|
||||
|
||||
/**
|
||||
* Converts an HColumnDescriptor to ColumnFamilySchema
|
||||
* @param hcd the HColummnDescriptor
|
||||
* Converts an ColumnFamilyDescriptor to ColumnFamilySchema
|
||||
* @param hcd the ColumnFamilySchema
|
||||
* @return Convert this instance to a the pb column family type
|
||||
*/
|
||||
public static ColumnFamilySchema convertToColumnFamilySchema(ColumnFamilyDescriptor hcd) {
|
||||
public static ColumnFamilySchema toColumnFamilySchema(ColumnFamilyDescriptor hcd) {
|
||||
ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
|
||||
builder.setName(UnsafeByteOperations.unsafeWrap(hcd.getName()));
|
||||
for (Map.Entry<Bytes, Bytes> e : hcd.getValues().entrySet()) {
|
||||
|
@ -2864,31 +2838,11 @@ public final class ProtobufUtil {
|
|||
}
|
||||
|
||||
/**
|
||||
* Converts a ColumnFamilySchema to HColumnDescriptor
|
||||
* Converts a ColumnFamilySchema to ColumnFamilyDescriptor
|
||||
* @param cfs the ColumnFamilySchema
|
||||
* @return An {@link HColumnDescriptor} made from the passed in <code>cfs</code>
|
||||
* @return An {@link ColumnFamilyDescriptor} made from the passed in <code>cfs</code>
|
||||
*/
|
||||
@Deprecated
|
||||
public static HColumnDescriptor convertToHColumnDesc(final ColumnFamilySchema cfs) {
|
||||
// Use the empty constructor so we preserve the initial values set on construction for things
|
||||
// like maxVersion. Otherwise, we pick up wrong values on deserialization which makes for
|
||||
// unrelated-looking test failures that are hard to trace back to here.
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(cfs.getName().toByteArray());
|
||||
for (BytesBytesPair a: cfs.getAttributesList()) {
|
||||
hcd.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray());
|
||||
}
|
||||
for (NameStringPair a: cfs.getConfigurationList()) {
|
||||
hcd.setConfiguration(a.getName(), a.getValue());
|
||||
}
|
||||
return hcd;
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a ColumnFamilySchema to HColumnDescriptor
|
||||
* @param cfs the ColumnFamilySchema
|
||||
* @return An {@link HColumnDescriptor} made from the passed in <code>cfs</code>
|
||||
*/
|
||||
public static ColumnFamilyDescriptor convertToColumnDesc(final ColumnFamilySchema cfs) {
|
||||
public static ColumnFamilyDescriptor toColumnFamilyDescriptor(final ColumnFamilySchema cfs) {
|
||||
// Use the empty constructor so we preserve the initial values set on construction for things
|
||||
// like maxVersion. Otherwise, we pick up wrong values on deserialization which makes for
|
||||
// unrelated-looking test failures that are hard to trace back to here.
|
||||
|
@ -2900,11 +2854,11 @@ public final class ProtobufUtil {
|
|||
}
|
||||
|
||||
/**
|
||||
* Converts an HTableDescriptor to TableSchema
|
||||
* @param htd the HTableDescriptor
|
||||
* @return Convert the current {@link HTableDescriptor} into a pb TableSchema instance.
|
||||
* Converts an TableDescriptor to TableSchema
|
||||
* @param htd the TableDescriptor
|
||||
* @return Convert the current {@link TableDescriptor} into a pb TableSchema instance.
|
||||
*/
|
||||
public static TableSchema convertToTableSchema(TableDescriptor htd) {
|
||||
public static TableSchema toTableSchema(TableDescriptor htd) {
|
||||
TableSchema.Builder builder = TableSchema.newBuilder();
|
||||
builder.setTableName(toProtoTableName(htd.getTableName()));
|
||||
for (Map.Entry<Bytes, Bytes> e : htd.getValues().entrySet()) {
|
||||
|
@ -2914,7 +2868,7 @@ public final class ProtobufUtil {
|
|||
builder.addAttributes(aBuilder.build());
|
||||
}
|
||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||
builder.addColumnFamilies(convertToColumnFamilySchema(hcd));
|
||||
builder.addColumnFamilies(toColumnFamilySchema(hcd));
|
||||
}
|
||||
for (Map.Entry<String, String> e : htd.getConfiguration().entrySet()) {
|
||||
NameStringPair.Builder aBuilder = NameStringPair.newBuilder();
|
||||
|
@ -2925,44 +2879,17 @@ public final class ProtobufUtil {
|
|||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a TableSchema to HTableDescriptor
|
||||
* @param ts A pb TableSchema instance.
|
||||
* @return An {@link HTableDescriptor} made from the passed in pb <code>ts</code>.
|
||||
* @deprecated Use {@link #convertToTableDesc} after removing the HTableDescriptor
|
||||
*/
|
||||
@Deprecated
|
||||
public static HTableDescriptor convertToHTableDesc(final TableSchema ts) {
|
||||
List<ColumnFamilySchema> list = ts.getColumnFamiliesList();
|
||||
HColumnDescriptor [] hcds = new HColumnDescriptor[list.size()];
|
||||
int index = 0;
|
||||
for (ColumnFamilySchema cfs: list) {
|
||||
hcds[index++] = ProtobufUtil.convertToHColumnDesc(cfs);
|
||||
}
|
||||
HTableDescriptor htd = new HTableDescriptor(ProtobufUtil.toTableName(ts.getTableName()));
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
htd.addFamily(hcd);
|
||||
}
|
||||
for (BytesBytesPair a: ts.getAttributesList()) {
|
||||
htd.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray());
|
||||
}
|
||||
for (NameStringPair a: ts.getConfigurationList()) {
|
||||
htd.setConfiguration(a.getName(), a.getValue());
|
||||
}
|
||||
return htd;
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a TableSchema to TableDescriptor
|
||||
* @param ts A pb TableSchema instance.
|
||||
* @return An {@link TableDescriptor} made from the passed in pb <code>ts</code>.
|
||||
*/
|
||||
public static TableDescriptor convertToTableDesc(final TableSchema ts) {
|
||||
public static TableDescriptor toTableDescriptor(final TableSchema ts) {
|
||||
TableDescriptorBuilder builder
|
||||
= TableDescriptorBuilder.newBuilder(ProtobufUtil.toTableName(ts.getTableName()));
|
||||
ts.getColumnFamiliesList()
|
||||
.stream()
|
||||
.map(ProtobufUtil::convertToColumnDesc)
|
||||
.map(ProtobufUtil::toColumnFamilyDescriptor)
|
||||
.forEach(builder::addColumnFamily);
|
||||
ts.getAttributesList()
|
||||
.forEach(a -> builder.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray()));
|
||||
|
|
|
@ -1080,7 +1080,7 @@ public final class RequestConverter {
|
|||
final long nonce) {
|
||||
AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
|
||||
builder.setColumnFamilies(ProtobufUtil.convertToColumnFamilySchema(column));
|
||||
builder.setColumnFamilies(ProtobufUtil.toColumnFamilySchema(column));
|
||||
builder.setNonceGroup(nonceGroup);
|
||||
builder.setNonce(nonce);
|
||||
return builder.build();
|
||||
|
@ -1120,7 +1120,7 @@ public final class RequestConverter {
|
|||
final long nonce) {
|
||||
ModifyColumnRequest.Builder builder = ModifyColumnRequest.newBuilder();
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
|
||||
builder.setColumnFamilies(ProtobufUtil.convertToColumnFamilySchema(column));
|
||||
builder.setColumnFamilies(ProtobufUtil.toColumnFamilySchema(column));
|
||||
builder.setNonceGroup(nonceGroup);
|
||||
builder.setNonce(nonce);
|
||||
return builder.build();
|
||||
|
@ -1306,28 +1306,28 @@ public final class RequestConverter {
|
|||
/**
|
||||
* Creates a protocol buffer CreateTableRequest
|
||||
*
|
||||
* @param hTableDesc
|
||||
* @param tableDescriptor
|
||||
* @param splitKeys
|
||||
* @return a CreateTableRequest
|
||||
*/
|
||||
public static CreateTableRequest buildCreateTableRequest(
|
||||
final TableDescriptor hTableDesc,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final byte [][] splitKeys,
|
||||
final long nonceGroup,
|
||||
final long nonce) {
|
||||
return buildCreateTableRequest(hTableDesc, Optional.ofNullable(splitKeys), nonceGroup, nonce);
|
||||
return buildCreateTableRequest(tableDescriptor, Optional.ofNullable(splitKeys), nonceGroup, nonce);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a protocol buffer CreateTableRequest
|
||||
* @param hTableDesc
|
||||
* @param tableDescriptor
|
||||
* @param splitKeys
|
||||
* @return a CreateTableRequest
|
||||
*/
|
||||
public static CreateTableRequest buildCreateTableRequest(TableDescriptor hTableDesc,
|
||||
public static CreateTableRequest buildCreateTableRequest(TableDescriptor tableDescriptor,
|
||||
Optional<byte[][]> splitKeys, long nonceGroup, long nonce) {
|
||||
CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
|
||||
builder.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDesc));
|
||||
builder.setTableSchema(ProtobufUtil.toTableSchema(tableDescriptor));
|
||||
splitKeys.ifPresent(keys -> Arrays.stream(keys).forEach(
|
||||
key -> builder.addSplitKeys(UnsafeByteOperations.unsafeWrap(key))));
|
||||
builder.setNonceGroup(nonceGroup);
|
||||
|
@ -1349,7 +1349,7 @@ public final class RequestConverter {
|
|||
final long nonce) {
|
||||
ModifyTableRequest.Builder builder = ModifyTableRequest.newBuilder();
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
|
||||
builder.setTableSchema(ProtobufUtil.convertToTableSchema(tableDesc));
|
||||
builder.setTableSchema(ProtobufUtil.toTableSchema(tableDesc));
|
||||
builder.setNonceGroup(nonceGroup);
|
||||
builder.setNonce(nonce);
|
||||
return builder.build();
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
||||
/**
|
||||
* Get, remove and modify table descriptors.
|
||||
|
@ -33,7 +34,7 @@ public interface TableDescriptors {
|
|||
* @return TableDescriptor for tablename
|
||||
* @throws IOException
|
||||
*/
|
||||
HTableDescriptor get(final TableName tableName)
|
||||
TableDescriptor get(final TableName tableName)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -41,16 +42,7 @@ public interface TableDescriptors {
|
|||
* @return Map of all descriptors.
|
||||
* @throws IOException
|
||||
*/
|
||||
Map<String, HTableDescriptor> getByNamespace(String name)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Get Map of all HTableDescriptors. Populates the descriptor cache as a
|
||||
* side effect.
|
||||
* @return Map of all descriptors.
|
||||
* @throws IOException
|
||||
*/
|
||||
Map<String, HTableDescriptor> getAll()
|
||||
Map<String, TableDescriptor> getByNamespace(String name)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -59,7 +51,16 @@ public interface TableDescriptors {
|
|||
* @return Map of all descriptors.
|
||||
* @throws IOException
|
||||
*/
|
||||
Map<String, HTableDescriptor> getAllDescriptors()
|
||||
Map<String, TableDescriptor> getAll()
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Get Map of all TableDescriptors. Populates the descriptor cache as a
|
||||
* side effect.
|
||||
* @return Map of all descriptors.
|
||||
* @throws IOException
|
||||
*/
|
||||
Map<String, TableDescriptor> getAllDescriptors()
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -67,7 +68,7 @@ public interface TableDescriptors {
|
|||
* @param htd Descriptor to set into TableDescriptors
|
||||
* @throws IOException
|
||||
*/
|
||||
void add(final HTableDescriptor htd)
|
||||
void add(final TableDescriptor htd)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -75,7 +76,7 @@ public interface TableDescriptors {
|
|||
* @return Instance of table descriptor or null if none found.
|
||||
* @throws IOException
|
||||
*/
|
||||
HTableDescriptor remove(final TableName tablename)
|
||||
TableDescriptor remove(final TableName tablename)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -50,7 +49,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
|
|||
List<Cell> values;
|
||||
|
||||
public ClientSideRegionScanner(Configuration conf, FileSystem fs,
|
||||
Path rootDir, HTableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics)
|
||||
Path rootDir, TableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics)
|
||||
throws IOException {
|
||||
// region is immutable, set isolation level
|
||||
scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
@ -75,7 +74,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
|
|||
private Path restoreDir;
|
||||
private Scan scan;
|
||||
private ArrayList<HRegionInfo> regions;
|
||||
private HTableDescriptor htd;
|
||||
private TableDescriptor htd;
|
||||
|
||||
private ClientSideRegionScanner currentRegionScanner = null;
|
||||
private int currentRegion = -1;
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -28,7 +29,6 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution.HostAndWeight;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.ClientSideRegionScanner;
|
||||
|
@ -81,7 +81,7 @@ public class TableSnapshotInputFormatImpl {
|
|||
*/
|
||||
public static class InputSplit implements Writable {
|
||||
|
||||
private HTableDescriptor htd;
|
||||
private TableDescriptor htd;
|
||||
private HRegionInfo regionInfo;
|
||||
private String[] locations;
|
||||
private String scan;
|
||||
|
@ -90,7 +90,7 @@ public class TableSnapshotInputFormatImpl {
|
|||
// constructor for mapreduce framework / Writable
|
||||
public InputSplit() {}
|
||||
|
||||
public InputSplit(HTableDescriptor htd, HRegionInfo regionInfo, List<String> locations,
|
||||
public InputSplit(TableDescriptor htd, HRegionInfo regionInfo, List<String> locations,
|
||||
Scan scan, Path restoreDir) {
|
||||
this.htd = htd;
|
||||
this.regionInfo = regionInfo;
|
||||
|
@ -108,7 +108,7 @@ public class TableSnapshotInputFormatImpl {
|
|||
this.restoreDir = restoreDir.toString();
|
||||
}
|
||||
|
||||
public HTableDescriptor getHtd() {
|
||||
public TableDescriptor getHtd() {
|
||||
return htd;
|
||||
}
|
||||
|
||||
|
@ -129,7 +129,7 @@ public class TableSnapshotInputFormatImpl {
|
|||
return locations;
|
||||
}
|
||||
|
||||
public HTableDescriptor getTableDescriptor() {
|
||||
public TableDescriptor getTableDescriptor() {
|
||||
return htd;
|
||||
}
|
||||
|
||||
|
@ -142,7 +142,7 @@ public class TableSnapshotInputFormatImpl {
|
|||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
TableSnapshotRegionSplit.Builder builder = TableSnapshotRegionSplit.newBuilder()
|
||||
.setTable(ProtobufUtil.convertToTableSchema(htd))
|
||||
.setTable(ProtobufUtil.toTableSchema(htd))
|
||||
.setRegion(HRegionInfo.convert(regionInfo));
|
||||
|
||||
for (String location : locations) {
|
||||
|
@ -169,7 +169,7 @@ public class TableSnapshotInputFormatImpl {
|
|||
byte[] buf = new byte[len];
|
||||
in.readFully(buf);
|
||||
TableSnapshotRegionSplit split = TableSnapshotRegionSplit.PARSER.parseFrom(buf);
|
||||
this.htd = ProtobufUtil.convertToHTableDesc(split.getTable());
|
||||
this.htd = ProtobufUtil.toTableDescriptor(split.getTable());
|
||||
this.regionInfo = HRegionInfo.convert(split.getRegion());
|
||||
List<String> locationsList = split.getLocationsList();
|
||||
this.locations = locationsList.toArray(new String[locationsList.size()]);
|
||||
|
@ -196,7 +196,7 @@ public class TableSnapshotInputFormatImpl {
|
|||
public void initialize(InputSplit split, Configuration conf) throws IOException {
|
||||
this.scan = TableMapReduceUtil.convertStringToScan(split.getScan());
|
||||
this.split = split;
|
||||
HTableDescriptor htd = split.htd;
|
||||
TableDescriptor htd = split.htd;
|
||||
HRegionInfo hri = this.split.getRegionInfo();
|
||||
FileSystem fs = FSUtils.getCurrentFileSystem(conf);
|
||||
|
||||
|
@ -311,7 +311,7 @@ public class TableSnapshotInputFormatImpl {
|
|||
public static List<InputSplit> getSplits(Scan scan, SnapshotManifest manifest,
|
||||
List<HRegionInfo> regionManifests, Path restoreDir, Configuration conf) throws IOException {
|
||||
// load table descriptor
|
||||
HTableDescriptor htd = manifest.getTableDescriptor();
|
||||
TableDescriptor htd = manifest.getTableDescriptor();
|
||||
|
||||
Path tableDir = FSUtils.getTableDir(restoreDir, htd.getTableName());
|
||||
|
||||
|
|
|
@ -31,21 +31,20 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ScheduledChore;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.assignment.GCMergedRegionsProcedure;
|
||||
import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -206,7 +205,7 @@ public class CatalogJanitor extends ScheduledChore {
|
|||
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
|
||||
Path rootdir = this.services.getMasterFileSystem().getRootDir();
|
||||
Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
|
||||
HTableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
|
||||
TableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
|
||||
HRegionFileSystem regionFs = null;
|
||||
try {
|
||||
regionFs = HRegionFileSystem.openRegionFromFileSystem(
|
||||
|
@ -414,12 +413,12 @@ public class CatalogJanitor extends ScheduledChore {
|
|||
}
|
||||
|
||||
boolean references = false;
|
||||
HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
|
||||
TableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
|
||||
try {
|
||||
regionFs = HRegionFileSystem.openRegionFromFileSystem(
|
||||
this.services.getConfiguration(), fs, tabledir, daughter, true);
|
||||
|
||||
for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
|
||||
for (ColumnFamilyDescriptor family: parentDescriptor.getColumnFamilies()) {
|
||||
if ((references = regionFs.hasReferences(family.getNameAsString()))) {
|
||||
break;
|
||||
}
|
||||
|
@ -432,7 +431,7 @@ public class CatalogJanitor extends ScheduledChore {
|
|||
return new Pair<>(Boolean.TRUE, Boolean.valueOf(references));
|
||||
}
|
||||
|
||||
private HTableDescriptor getTableDescriptor(final TableName tableName)
|
||||
private TableDescriptor getTableDescriptor(final TableName tableName)
|
||||
throws FileNotFoundException, IOException {
|
||||
return this.services.getTableDescriptors().get(tableName);
|
||||
}
|
||||
|
|
|
@ -23,11 +23,11 @@ import java.util.concurrent.TimeUnit;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ScheduledChore;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.locking.LockManager;
|
||||
import org.apache.hadoop.hbase.master.locking.LockProcedure;
|
||||
import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
|
||||
|
@ -61,9 +61,9 @@ public class ExpiredMobFileCleanerChore extends ScheduledChore {
|
|||
protected void chore() {
|
||||
try {
|
||||
TableDescriptors htds = master.getTableDescriptors();
|
||||
Map<String, HTableDescriptor> map = htds.getAll();
|
||||
for (HTableDescriptor htd : map.values()) {
|
||||
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
|
||||
Map<String, TableDescriptor> map = htds.getAll();
|
||||
for (TableDescriptor htd : map.values()) {
|
||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||
if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) {
|
||||
// clean only for mob-enabled column.
|
||||
// obtain a read table lock before cleaning, synchronize with MobFileCompactionChore.
|
||||
|
|
|
@ -41,6 +41,7 @@ import java.util.concurrent.Future;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Function;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import javax.servlet.ServletException;
|
||||
|
@ -60,10 +61,8 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
|
|||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
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.MasterNotRunningException;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
|
@ -77,9 +76,12 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
|
|||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
|
@ -590,11 +592,9 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
return connector.getLocalPort();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TableDescriptors getFsTableDescriptors() throws IOException {
|
||||
return super.getFsTableDescriptors();
|
||||
protected Function<TableDescriptorBuilder, TableDescriptorBuilder> getMetaTableObserver() {
|
||||
return builder -> builder.setRegionReplication(conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM));
|
||||
}
|
||||
|
||||
/**
|
||||
* For compatibility, if failed with regionserver credentials, try the master one
|
||||
*/
|
||||
|
@ -761,9 +761,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
|
||||
// enable table descriptors cache
|
||||
this.tableDescriptors.setCacheOn();
|
||||
// set the META's descriptor to the correct replication
|
||||
this.tableDescriptors.get(TableName.META_TABLE_NAME).setRegionReplication(
|
||||
conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM));
|
||||
|
||||
// warm-up HTDs cache on master initialization
|
||||
if (preLoadTableDescriptors) {
|
||||
status.setStatus("Pre-loading table descriptors");
|
||||
|
@ -1501,7 +1499,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
return false;
|
||||
}
|
||||
|
||||
HTableDescriptor tblDesc = getTableDescriptors().get(table);
|
||||
TableDescriptor tblDesc = getTableDescriptors().get(table);
|
||||
if (table.isSystemTable() || (tblDesc != null &&
|
||||
!tblDesc.isNormalizationEnabled())) {
|
||||
LOG.debug("Skipping normalization for table: " + table + ", as it's either system"
|
||||
|
@ -1712,34 +1710,34 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
|
||||
@Override
|
||||
public long createTable(
|
||||
final HTableDescriptor hTableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final byte [][] splitKeys,
|
||||
final long nonceGroup,
|
||||
final long nonce) throws IOException {
|
||||
checkInitialized();
|
||||
|
||||
String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
|
||||
String namespace = tableDescriptor.getTableName().getNamespaceAsString();
|
||||
this.clusterSchemaService.getNamespace(namespace);
|
||||
|
||||
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
|
||||
sanityCheckTableDescriptor(hTableDescriptor);
|
||||
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(tableDescriptor, splitKeys);
|
||||
sanityCheckTableDescriptor(tableDescriptor);
|
||||
|
||||
return MasterProcedureUtil.submitProcedure(
|
||||
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||
@Override
|
||||
protected void run() throws IOException {
|
||||
getMaster().getMasterCoprocessorHost().preCreateTable(hTableDescriptor, newRegions);
|
||||
getMaster().getMasterCoprocessorHost().preCreateTable(tableDescriptor, newRegions);
|
||||
|
||||
LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
|
||||
LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
|
||||
|
||||
// TODO: We can handle/merge duplicate requests, and differentiate the case of
|
||||
// TableExistsException by saying if the schema is the same or not.
|
||||
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
|
||||
submitProcedure(new CreateTableProcedure(
|
||||
procedureExecutor.getEnvironment(), hTableDescriptor, newRegions, latch));
|
||||
procedureExecutor.getEnvironment(), tableDescriptor, newRegions, latch));
|
||||
latch.await();
|
||||
|
||||
getMaster().getMasterCoprocessorHost().postCreateTable(hTableDescriptor, newRegions);
|
||||
getMaster().getMasterCoprocessorHost().postCreateTable(tableDescriptor, newRegions);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1750,25 +1748,25 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long createSystemTable(final HTableDescriptor hTableDescriptor) throws IOException {
|
||||
public long createSystemTable(final TableDescriptor tableDescriptor) throws IOException {
|
||||
if (isStopped()) {
|
||||
throw new MasterNotRunningException();
|
||||
}
|
||||
|
||||
TableName tableName = hTableDescriptor.getTableName();
|
||||
TableName tableName = tableDescriptor.getTableName();
|
||||
if (!(tableName.isSystemTable())) {
|
||||
throw new IllegalArgumentException(
|
||||
"Only system table creation can use this createSystemTable API");
|
||||
}
|
||||
|
||||
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, null);
|
||||
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(tableDescriptor, null);
|
||||
|
||||
LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
|
||||
LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
|
||||
|
||||
// This special create table is called locally to master. Therefore, no RPC means no need
|
||||
// to use nonce to detect duplicated RPC call.
|
||||
long procId = this.procedureExecutor.submitProcedure(
|
||||
new CreateTableProcedure(procedureExecutor.getEnvironment(), hTableDescriptor, newRegions));
|
||||
new CreateTableProcedure(procedureExecutor.getEnvironment(), tableDescriptor, newRegions));
|
||||
|
||||
return procId;
|
||||
}
|
||||
|
@ -1778,7 +1776,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
* values (compression, etc) work. Throws an exception if something is wrong.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void sanityCheckTableDescriptor(final HTableDescriptor htd) throws IOException {
|
||||
private void sanityCheckTableDescriptor(final TableDescriptor htd) throws IOException {
|
||||
final String CONF_KEY = "hbase.table.sanity.checks";
|
||||
boolean logWarn = false;
|
||||
if (!conf.getBoolean(CONF_KEY, true)) {
|
||||
|
@ -1848,7 +1846,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
|
||||
}
|
||||
|
||||
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||
if (hcd.getTimeToLive() <= 0) {
|
||||
String message = "TTL for column family " + hcd.getNameAsString() + " must be positive.";
|
||||
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
|
||||
|
@ -1869,7 +1867,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
}
|
||||
// max versions already being checked
|
||||
|
||||
// HBASE-13776 Setting illegal versions for HColumnDescriptor
|
||||
// HBASE-13776 Setting illegal versions for ColumnFamilyDescriptor
|
||||
// does not throw IllegalArgumentException
|
||||
// check minVersions <= maxVerions
|
||||
if (hcd.getMinVersions() > hcd.getMaxVersions()) {
|
||||
|
@ -1893,7 +1891,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
}
|
||||
}
|
||||
|
||||
private void checkReplicationScope(HColumnDescriptor hcd) throws IOException{
|
||||
private void checkReplicationScope(ColumnFamilyDescriptor hcd) throws IOException{
|
||||
// check replication scope
|
||||
WALProtos.ScopeType scop = WALProtos.ScopeType.valueOf(hcd.getScope());
|
||||
if (scop == null) {
|
||||
|
@ -1905,7 +1903,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
}
|
||||
}
|
||||
|
||||
private void checkCompactionPolicy(Configuration conf, HTableDescriptor htd)
|
||||
private void checkCompactionPolicy(Configuration conf, TableDescriptor htd)
|
||||
throws IOException {
|
||||
// FIFO compaction has some requirements
|
||||
// Actually FCP ignores periodic major compactions
|
||||
|
@ -1925,7 +1923,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
blockingFileCount = conf.getInt(HStore.BLOCKING_STOREFILES_KEY, blockingFileCount);
|
||||
}
|
||||
|
||||
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||
String compactionPolicy =
|
||||
hcd.getConfigurationValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY);
|
||||
if (compactionPolicy == null) {
|
||||
|
@ -1938,7 +1936,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
String message = null;
|
||||
|
||||
// 1. Check TTL
|
||||
if (hcd.getTimeToLive() == HColumnDescriptor.DEFAULT_TTL) {
|
||||
if (hcd.getTimeToLive() == ColumnFamilyDescriptorBuilder.DEFAULT_TTL) {
|
||||
message = "Default TTL is not supported for FIFO compaction";
|
||||
throw new IOException(message);
|
||||
}
|
||||
|
@ -2040,36 +2038,36 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
}, getServerName().toShortString() + ".masterManager"));
|
||||
}
|
||||
|
||||
private void checkCompression(final HTableDescriptor htd)
|
||||
private void checkCompression(final TableDescriptor htd)
|
||||
throws IOException {
|
||||
if (!this.masterCheckCompression) return;
|
||||
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||
checkCompression(hcd);
|
||||
}
|
||||
}
|
||||
|
||||
private void checkCompression(final HColumnDescriptor hcd)
|
||||
private void checkCompression(final ColumnFamilyDescriptor hcd)
|
||||
throws IOException {
|
||||
if (!this.masterCheckCompression) return;
|
||||
CompressionTest.testCompression(hcd.getCompressionType());
|
||||
CompressionTest.testCompression(hcd.getCompactionCompressionType());
|
||||
}
|
||||
|
||||
private void checkEncryption(final Configuration conf, final HTableDescriptor htd)
|
||||
private void checkEncryption(final Configuration conf, final TableDescriptor htd)
|
||||
throws IOException {
|
||||
if (!this.masterCheckEncryption) return;
|
||||
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||
checkEncryption(conf, hcd);
|
||||
}
|
||||
}
|
||||
|
||||
private void checkEncryption(final Configuration conf, final HColumnDescriptor hcd)
|
||||
private void checkEncryption(final Configuration conf, final ColumnFamilyDescriptor hcd)
|
||||
throws IOException {
|
||||
if (!this.masterCheckEncryption) return;
|
||||
EncryptionTest.testEncryption(conf, hcd.getEncryptionType(), hcd.getEncryptionKey());
|
||||
}
|
||||
|
||||
private void checkClassLoading(final Configuration conf, final HTableDescriptor htd)
|
||||
private void checkClassLoading(final Configuration conf, final TableDescriptor htd)
|
||||
throws IOException {
|
||||
RegionSplitPolicy.getSplitPolicyClass(htd, conf);
|
||||
RegionCoprocessorHost.testTableCoprocessorAttrs(conf, htd);
|
||||
|
@ -2143,7 +2141,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
@Override
|
||||
public long addColumn(
|
||||
final TableName tableName,
|
||||
final HColumnDescriptor columnDescriptor,
|
||||
final ColumnFamilyDescriptor columnDescriptor,
|
||||
final long nonceGroup,
|
||||
final long nonce)
|
||||
throws IOException {
|
||||
|
@ -2179,7 +2177,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
@Override
|
||||
public long modifyColumn(
|
||||
final TableName tableName,
|
||||
final HColumnDescriptor descriptor,
|
||||
final ColumnFamilyDescriptor descriptor,
|
||||
final long nonceGroup,
|
||||
final long nonce)
|
||||
throws IOException {
|
||||
|
@ -2373,7 +2371,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long modifyTable(final TableName tableName, final HTableDescriptor descriptor,
|
||||
public long modifyTable(final TableName tableName, final TableDescriptor descriptor,
|
||||
final long nonceGroup, final long nonce) throws IOException {
|
||||
checkInitialized();
|
||||
sanityCheckTableDescriptor(descriptor);
|
||||
|
@ -3127,7 +3125,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
throws IOException {
|
||||
if (tableNameList == null || tableNameList.isEmpty()) {
|
||||
// request for all TableDescriptors
|
||||
Collection<HTableDescriptor> allHtds;
|
||||
Collection<TableDescriptor> allHtds;
|
||||
if (namespace != null && namespace.length() > 0) {
|
||||
// Do a check on the namespace existence. Will fail if does not exist.
|
||||
this.clusterSchemaService.getNamespace(namespace);
|
||||
|
@ -3135,7 +3133,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
} else {
|
||||
allHtds = tableDescriptors.getAll().values();
|
||||
}
|
||||
for (HTableDescriptor desc: allHtds) {
|
||||
for (TableDescriptor desc: allHtds) {
|
||||
if (tableStateManager.isTablePresent(desc.getTableName())
|
||||
&& (includeSysTables || !desc.getTableName().isSystemTable())) {
|
||||
htds.add(desc);
|
||||
|
@ -3144,7 +3142,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
} else {
|
||||
for (TableName s: tableNameList) {
|
||||
if (tableStateManager.isTablePresent(s)) {
|
||||
HTableDescriptor desc = tableDescriptors.get(s);
|
||||
TableDescriptor desc = tableDescriptors.get(s);
|
||||
if (desc != null) {
|
||||
htds.add(desc);
|
||||
}
|
||||
|
@ -3249,7 +3247,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
* @param allFiles Whether add all mob files into the compaction.
|
||||
*/
|
||||
public void requestMobCompaction(TableName tableName,
|
||||
List<HColumnDescriptor> columns, boolean allFiles) throws IOException {
|
||||
List<ColumnFamilyDescriptor> columns, boolean allFiles) throws IOException {
|
||||
mobCompactThread.requestMobCompaction(conf, fs, tableName, columns, allFiles);
|
||||
}
|
||||
|
||||
|
|
|
@ -28,13 +28,15 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.ClusterId;
|
||||
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.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
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.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
|
||||
|
@ -387,10 +389,8 @@ public class MasterFileSystem {
|
|||
// not make it in first place. Turn off block caching for bootstrap.
|
||||
// Enable after.
|
||||
HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
HTableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
|
||||
setInfoFamilyCachingForMeta(metaDescriptor, false);
|
||||
HRegion meta = HRegion.createHRegion(metaHRI, rd, c, metaDescriptor, null);
|
||||
setInfoFamilyCachingForMeta(metaDescriptor, true);
|
||||
TableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
|
||||
HRegion meta = HRegion.createHRegion(metaHRI, rd, c, setInfoFamilyCachingForMeta(metaDescriptor, false), null);
|
||||
meta.close();
|
||||
} catch (IOException e) {
|
||||
e = e instanceof RemoteException ?
|
||||
|
@ -403,13 +403,17 @@ public class MasterFileSystem {
|
|||
/**
|
||||
* Enable in memory caching for hbase:meta
|
||||
*/
|
||||
public static void setInfoFamilyCachingForMeta(HTableDescriptor metaDescriptor, final boolean b) {
|
||||
for (HColumnDescriptor hcd: metaDescriptor.getColumnFamilies()) {
|
||||
public static TableDescriptor setInfoFamilyCachingForMeta(TableDescriptor metaDescriptor, final boolean b) {
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(metaDescriptor);
|
||||
for (ColumnFamilyDescriptor hcd: metaDescriptor.getColumnFamilies()) {
|
||||
if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
|
||||
hcd.setBlockCacheEnabled(b);
|
||||
hcd.setInMemory(b);
|
||||
builder.modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(hcd)
|
||||
.setBlockCacheEnabled(b)
|
||||
.setInMemory(b)
|
||||
.build());
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName)
|
||||
|
|
|
@ -31,9 +31,9 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.master.locking.LockManager;
|
||||
import org.apache.hadoop.hbase.master.locking.LockProcedure;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
|
@ -79,7 +79,7 @@ public class MasterMobCompactionThread {
|
|||
* @param allFiles Whether add all mob files into the compaction.
|
||||
*/
|
||||
public void requestMobCompaction(Configuration conf, FileSystem fs, TableName tableName,
|
||||
List<HColumnDescriptor> columns, boolean allFiles) throws IOException {
|
||||
List<ColumnFamilyDescriptor> columns, boolean allFiles) throws IOException {
|
||||
master.reportMobCompactionStart(tableName);
|
||||
try {
|
||||
masterMobPool.execute(new CompactionRunner(fs, tableName, columns,
|
||||
|
@ -102,11 +102,11 @@ public class MasterMobCompactionThread {
|
|||
private class CompactionRunner implements Runnable {
|
||||
private FileSystem fs;
|
||||
private TableName tableName;
|
||||
private List<HColumnDescriptor> hcds;
|
||||
private List<ColumnFamilyDescriptor> hcds;
|
||||
private boolean allFiles;
|
||||
private ExecutorService pool;
|
||||
|
||||
public CompactionRunner(FileSystem fs, TableName tableName, List<HColumnDescriptor> hcds,
|
||||
public CompactionRunner(FileSystem fs, TableName tableName, List<ColumnFamilyDescriptor> hcds,
|
||||
boolean allFiles, ExecutorService pool) {
|
||||
super();
|
||||
this.fs = fs;
|
||||
|
@ -123,7 +123,7 @@ public class MasterMobCompactionThread {
|
|||
MobUtils.getTableLockName(tableName), LockProcedure.LockType.EXCLUSIVE,
|
||||
this.getClass().getName() + ": mob compaction");
|
||||
try {
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
for (ColumnFamilyDescriptor hcd : hcds) {
|
||||
MobUtils.doMobCompaction(conf, fs, tableName, hcd, pool, allFiles, lock);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -31,10 +31,8 @@ import java.util.Set;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
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.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.ProcedureInfo;
|
||||
|
@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
|
@ -363,7 +362,7 @@ public class MasterRpcServices extends RSRpcServices
|
|||
try {
|
||||
long procId = master.addColumn(
|
||||
ProtobufUtil.toTableName(req.getTableName()),
|
||||
ProtobufUtil.convertToHColumnDesc(req.getColumnFamilies()),
|
||||
ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()),
|
||||
req.getNonceGroup(),
|
||||
req.getNonce());
|
||||
if (procId == -1) {
|
||||
|
@ -439,11 +438,11 @@ public class MasterRpcServices extends RSRpcServices
|
|||
@Override
|
||||
public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
|
||||
throws ServiceException {
|
||||
HTableDescriptor hTableDescriptor = ProtobufUtil.convertToHTableDesc(req.getTableSchema());
|
||||
TableDescriptor tableDescriptor = ProtobufUtil.toTableDescriptor(req.getTableSchema());
|
||||
byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
|
||||
try {
|
||||
long procId =
|
||||
master.createTable(hTableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce());
|
||||
master.createTable(tableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce());
|
||||
return CreateTableResponse.newBuilder().setProcId(procId).build();
|
||||
} catch (IOException ioe) {
|
||||
throw new ServiceException(ioe);
|
||||
|
@ -865,7 +864,7 @@ public class MasterRpcServices extends RSRpcServices
|
|||
if (descriptors != null && descriptors.size() > 0) {
|
||||
// Add the table descriptors to the response
|
||||
for (TableDescriptor htd: descriptors) {
|
||||
builder.addTableSchema(ProtobufUtil.convertToTableSchema(htd));
|
||||
builder.addTableSchema(ProtobufUtil.toTableSchema(htd));
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
|
@ -1118,7 +1117,7 @@ public class MasterRpcServices extends RSRpcServices
|
|||
ListTableDescriptorsByNamespaceResponse.newBuilder();
|
||||
for (TableDescriptor htd : master
|
||||
.listTableDescriptorsByNamespace(request.getNamespaceName())) {
|
||||
b.addTableSchema(ProtobufUtil.convertToTableSchema(htd));
|
||||
b.addTableSchema(ProtobufUtil.toTableSchema(htd));
|
||||
}
|
||||
return b.build();
|
||||
} catch (IOException e) {
|
||||
|
@ -1147,7 +1146,7 @@ public class MasterRpcServices extends RSRpcServices
|
|||
try {
|
||||
long procId = master.modifyColumn(
|
||||
ProtobufUtil.toTableName(req.getTableName()),
|
||||
ProtobufUtil.convertToHColumnDesc(req.getColumnFamilies()),
|
||||
ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()),
|
||||
req.getNonceGroup(),
|
||||
req.getNonce());
|
||||
if (procId == -1) {
|
||||
|
@ -1181,7 +1180,7 @@ public class MasterRpcServices extends RSRpcServices
|
|||
try {
|
||||
long procId = master.modifyTable(
|
||||
ProtobufUtil.toTableName(req.getTableName()),
|
||||
ProtobufUtil.convertToHTableDesc(req.getTableSchema()),
|
||||
ProtobufUtil.toTableDescriptor(req.getTableSchema()),
|
||||
req.getNonceGroup(),
|
||||
req.getNonce());
|
||||
return ModifyTableResponse.newBuilder().setProcId(procId).build();
|
||||
|
@ -1532,12 +1531,12 @@ public class MasterRpcServices extends RSRpcServices
|
|||
throw new DoNotRetryIOException("Table " + tableName + " is not enabled");
|
||||
}
|
||||
boolean allFiles = false;
|
||||
List<HColumnDescriptor> compactedColumns = new ArrayList<>();
|
||||
HColumnDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies();
|
||||
List<ColumnFamilyDescriptor> compactedColumns = new ArrayList<>();
|
||||
ColumnFamilyDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies();
|
||||
byte[] family = null;
|
||||
if (request.hasFamily()) {
|
||||
family = request.getFamily().toByteArray();
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
for (ColumnFamilyDescriptor hcd : hcds) {
|
||||
if (Bytes.equals(family, hcd.getName())) {
|
||||
if (!hcd.isMobEnabled()) {
|
||||
LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family");
|
||||
|
@ -1548,7 +1547,7 @@ public class MasterRpcServices extends RSRpcServices
|
|||
}
|
||||
}
|
||||
} else {
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
for (ColumnFamilyDescriptor hcd : hcds) {
|
||||
if (hcd.isMobEnabled()) {
|
||||
compactedColumns.add(hcd);
|
||||
}
|
||||
|
|
|
@ -21,9 +21,7 @@ package org.apache.hadoop.hbase.master;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ProcedureInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
@ -32,6 +30,7 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
|
@ -159,17 +158,17 @@ public interface MasterServices extends Server {
|
|||
* a single region is created.
|
||||
*/
|
||||
long createTable(
|
||||
final HTableDescriptor desc,
|
||||
final TableDescriptor desc,
|
||||
final byte[][] splitKeys,
|
||||
final long nonceGroup,
|
||||
final long nonce) throws IOException;
|
||||
|
||||
/**
|
||||
* Create a system table using the given table definition.
|
||||
* @param hTableDescriptor The system table definition
|
||||
* @param tableDescriptor The system table definition
|
||||
* a single region is created.
|
||||
*/
|
||||
long createSystemTable(final HTableDescriptor hTableDescriptor) throws IOException;
|
||||
long createSystemTable(final TableDescriptor tableDescriptor) throws IOException;
|
||||
|
||||
/**
|
||||
* Delete a table
|
||||
|
@ -207,7 +206,7 @@ public interface MasterServices extends Server {
|
|||
*/
|
||||
long modifyTable(
|
||||
final TableName tableName,
|
||||
final HTableDescriptor descriptor,
|
||||
final TableDescriptor descriptor,
|
||||
final long nonceGroup,
|
||||
final long nonce)
|
||||
throws IOException;
|
||||
|
@ -247,7 +246,7 @@ public interface MasterServices extends Server {
|
|||
*/
|
||||
long addColumn(
|
||||
final TableName tableName,
|
||||
final HColumnDescriptor column,
|
||||
final ColumnFamilyDescriptor column,
|
||||
final long nonceGroup,
|
||||
final long nonce)
|
||||
throws IOException;
|
||||
|
@ -262,7 +261,7 @@ public interface MasterServices extends Server {
|
|||
*/
|
||||
long modifyColumn(
|
||||
final TableName tableName,
|
||||
final HColumnDescriptor descriptor,
|
||||
final ColumnFamilyDescriptor descriptor,
|
||||
final long nonceGroup,
|
||||
final long nonce)
|
||||
throws IOException;
|
||||
|
|
|
@ -24,11 +24,11 @@ import java.util.concurrent.TimeUnit;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ScheduledChore;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.locking.LockManager;
|
||||
import org.apache.hadoop.hbase.master.locking.LockProcedure;
|
||||
|
@ -55,8 +55,8 @@ public class MobCompactionChore extends ScheduledChore {
|
|||
protected void chore() {
|
||||
try {
|
||||
TableDescriptors htds = master.getTableDescriptors();
|
||||
Map<String, HTableDescriptor> map = htds.getAll();
|
||||
for (HTableDescriptor htd : map.values()) {
|
||||
Map<String, TableDescriptor> map = htds.getAll();
|
||||
for (TableDescriptor htd : map.values()) {
|
||||
if (!master.getTableStateManager().isTableState(htd.getTableName(),
|
||||
TableState.State.ENABLED)) {
|
||||
continue;
|
||||
|
@ -66,7 +66,7 @@ public class MobCompactionChore extends ScheduledChore {
|
|||
final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
|
||||
MobUtils.getTableLockName(htd.getTableName()), LockProcedure.LockType.EXCLUSIVE,
|
||||
this.getClass().getName() + ": mob compaction");
|
||||
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||
if (!hcd.isMobEnabled()) {
|
||||
continue;
|
||||
}
|
||||
|
|
|
@ -24,12 +24,12 @@ import java.util.Set;
|
|||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
|
||||
import edu.umd.cs.findbugs.annotations.NonNull;
|
||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -198,7 +198,7 @@ public class TableStateManager {
|
|||
|
||||
public static void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
|
||||
throws IOException {
|
||||
final Map<String, HTableDescriptor> allDescriptors =
|
||||
final Map<String, TableDescriptor> allDescriptors =
|
||||
tableDescriptors.getAllDescriptors();
|
||||
final Map<String, TableState> states = new HashMap<>();
|
||||
MetaTableAccessor.fullScanTables(connection, new MetaTableAccessor.Visitor() {
|
||||
|
@ -210,7 +210,7 @@ public class TableStateManager {
|
|||
return true;
|
||||
}
|
||||
});
|
||||
for (Map.Entry<String, HTableDescriptor> entry : allDescriptors.entrySet()) {
|
||||
for (Map.Entry<String, TableDescriptor> entry : allDescriptors.entrySet()) {
|
||||
String table = entry.getKey();
|
||||
if (table.equals(TableName.META_TABLE_NAME.getNameAsString()))
|
||||
continue;
|
||||
|
|
|
@ -31,18 +31,18 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.MetaMutationAnnotation;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.master.CatalogJanitor;
|
||||
|
@ -603,10 +603,10 @@ public class MergeTableRegionsProcedure
|
|||
throws IOException {
|
||||
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
|
||||
final Configuration conf = env.getMasterConfiguration();
|
||||
final HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
|
||||
final TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
|
||||
|
||||
for (String family: regionFs.getFamilies()) {
|
||||
final HColumnDescriptor hcd = htd.getFamily(family.getBytes());
|
||||
final ColumnFamilyDescriptor hcd = htd.getColumnFamily(family.getBytes());
|
||||
final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
|
||||
|
||||
if (storeFiles != null && storeFiles.size() > 0) {
|
||||
|
@ -682,7 +682,7 @@ public class MergeTableRegionsProcedure
|
|||
}
|
||||
|
||||
private int getRegionReplication(final MasterProcedureEnv env) throws IOException {
|
||||
final HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
|
||||
final TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
|
||||
return htd.getRegionReplication();
|
||||
}
|
||||
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
|
@ -221,7 +221,7 @@ public class RegionStateStore {
|
|||
// ============================================================================================
|
||||
public void splitRegion(final HRegionInfo parent, final HRegionInfo hriA,
|
||||
final HRegionInfo hriB, final ServerName serverName) throws IOException {
|
||||
final HTableDescriptor htd = getTableDescriptor(parent.getTable());
|
||||
final TableDescriptor htd = getTableDescriptor(parent.getTable());
|
||||
MetaTableAccessor.splitRegion(master.getConnection(), parent, hriA, hriB, serverName,
|
||||
getRegionReplication(htd), hasSerialReplicationScope(htd));
|
||||
}
|
||||
|
@ -231,7 +231,7 @@ public class RegionStateStore {
|
|||
// ============================================================================================
|
||||
public void mergeRegions(final HRegionInfo parent, final HRegionInfo hriA,
|
||||
final HRegionInfo hriB, final ServerName serverName) throws IOException {
|
||||
final HTableDescriptor htd = getTableDescriptor(parent.getTable());
|
||||
final TableDescriptor htd = getTableDescriptor(parent.getTable());
|
||||
MetaTableAccessor.mergeRegions(master.getConnection(), parent, hriA, hriB, serverName,
|
||||
getRegionReplication(htd), EnvironmentEdgeManager.currentTime(),
|
||||
hasSerialReplicationScope(htd));
|
||||
|
@ -255,15 +255,15 @@ public class RegionStateStore {
|
|||
return hasSerialReplicationScope(getTableDescriptor(tableName));
|
||||
}
|
||||
|
||||
private boolean hasSerialReplicationScope(final HTableDescriptor htd) {
|
||||
private boolean hasSerialReplicationScope(final TableDescriptor htd) {
|
||||
return (htd != null)? htd.hasSerialReplicationScope(): false;
|
||||
}
|
||||
|
||||
private int getRegionReplication(final HTableDescriptor htd) {
|
||||
private int getRegionReplication(final TableDescriptor htd) {
|
||||
return (htd != null) ? htd.getRegionReplication() : 1;
|
||||
}
|
||||
|
||||
private HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
|
||||
private TableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
|
||||
return master.getTableDescriptors().get(tableName);
|
||||
}
|
||||
|
||||
|
|
|
@ -34,10 +34,10 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -194,7 +194,7 @@ class RegionLocationFinder {
|
|||
*/
|
||||
protected HDFSBlocksDistribution internalGetTopBlockLocation(HRegionInfo region) {
|
||||
try {
|
||||
HTableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
|
||||
TableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
|
||||
if (tableDescriptor != null) {
|
||||
HDFSBlocksDistribution blocksDistribution =
|
||||
HRegion.computeHDFSBlocksDistribution(getConf(), tableDescriptor, region);
|
||||
|
@ -209,14 +209,14 @@ class RegionLocationFinder {
|
|||
}
|
||||
|
||||
/**
|
||||
* return HTableDescriptor for a given tableName
|
||||
* return TableDescriptor for a given tableName
|
||||
*
|
||||
* @param tableName the table name
|
||||
* @return HTableDescriptor
|
||||
* @return TableDescriptor
|
||||
* @throws IOException
|
||||
*/
|
||||
protected HTableDescriptor getTableDescriptor(TableName tableName) throws IOException {
|
||||
HTableDescriptor tableDescriptor = null;
|
||||
protected TableDescriptor getTableDescriptor(TableName tableName) throws IOException {
|
||||
TableDescriptor tableDescriptor = null;
|
||||
try {
|
||||
if (this.services != null && this.services.getTableDescriptors() != null) {
|
||||
tableDescriptor = this.services.getTableDescriptors().get(tableName);
|
||||
|
|
|
@ -27,17 +27,17 @@ import java.util.Set;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ScheduledChore;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -63,11 +63,11 @@ public class ReplicationMetaCleaner extends ScheduledChore {
|
|||
@Override
|
||||
protected void chore() {
|
||||
try {
|
||||
Map<String, HTableDescriptor> tables = master.getTableDescriptors().getAllDescriptors();
|
||||
Map<String, TableDescriptor> tables = master.getTableDescriptors().getAllDescriptors();
|
||||
Map<String, Set<String>> serialTables = new HashMap<>();
|
||||
for (Map.Entry<String, HTableDescriptor> entry : tables.entrySet()) {
|
||||
for (Map.Entry<String, TableDescriptor> entry : tables.entrySet()) {
|
||||
boolean hasSerialScope = false;
|
||||
for (HColumnDescriptor column : entry.getValue().getFamilies()) {
|
||||
for (ColumnFamilyDescriptor column : entry.getValue().getColumnFamilies()) {
|
||||
if (column.getScope() == HConstants.REPLICATION_SCOPE_SERIAL) {
|
||||
hasSerialScope = true;
|
||||
break;
|
||||
|
|
|
@ -25,12 +25,13 @@ import java.util.List;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
|
||||
|
@ -45,30 +46,30 @@ public class AddColumnFamilyProcedure
|
|||
private static final Log LOG = LogFactory.getLog(AddColumnFamilyProcedure.class);
|
||||
|
||||
private TableName tableName;
|
||||
private HTableDescriptor unmodifiedHTableDescriptor;
|
||||
private HColumnDescriptor cfDescriptor;
|
||||
private TableDescriptor unmodifiedTableDescriptor;
|
||||
private ColumnFamilyDescriptor cfDescriptor;
|
||||
|
||||
private List<HRegionInfo> regionInfoList;
|
||||
private Boolean traceEnabled;
|
||||
|
||||
public AddColumnFamilyProcedure() {
|
||||
super();
|
||||
this.unmodifiedHTableDescriptor = null;
|
||||
this.unmodifiedTableDescriptor = null;
|
||||
this.regionInfoList = null;
|
||||
this.traceEnabled = null;
|
||||
}
|
||||
|
||||
public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||
final HColumnDescriptor cfDescriptor) throws IOException {
|
||||
final ColumnFamilyDescriptor cfDescriptor) throws IOException {
|
||||
this(env, tableName, cfDescriptor, null);
|
||||
}
|
||||
|
||||
public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||
final HColumnDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
|
||||
final ColumnFamilyDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
|
||||
super(env, latch);
|
||||
this.tableName = tableName;
|
||||
this.cfDescriptor = cfDescriptor;
|
||||
this.unmodifiedHTableDescriptor = null;
|
||||
this.unmodifiedTableDescriptor = null;
|
||||
this.regionInfoList = null;
|
||||
this.traceEnabled = null;
|
||||
}
|
||||
|
@ -172,10 +173,10 @@ public class AddColumnFamilyProcedure
|
|||
MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
|
||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||
.setColumnfamilySchema(ProtobufUtil.convertToColumnFamilySchema(cfDescriptor));
|
||||
if (unmodifiedHTableDescriptor != null) {
|
||||
.setColumnfamilySchema(ProtobufUtil.toColumnFamilySchema(cfDescriptor));
|
||||
if (unmodifiedTableDescriptor != null) {
|
||||
addCFMsg
|
||||
.setUnmodifiedTableSchema(ProtobufUtil.convertToTableSchema(unmodifiedHTableDescriptor));
|
||||
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
|
||||
}
|
||||
|
||||
addCFMsg.build().writeDelimitedTo(stream);
|
||||
|
@ -189,9 +190,9 @@ public class AddColumnFamilyProcedure
|
|||
MasterProcedureProtos.AddColumnFamilyStateData.parseDelimitedFrom(stream);
|
||||
setUser(MasterProcedureUtil.toUserInfo(addCFMsg.getUserInfo()));
|
||||
tableName = ProtobufUtil.toTableName(addCFMsg.getTableName());
|
||||
cfDescriptor = ProtobufUtil.convertToHColumnDesc(addCFMsg.getColumnfamilySchema());
|
||||
cfDescriptor = ProtobufUtil.toColumnFamilyDescriptor(addCFMsg.getColumnfamilySchema());
|
||||
if (addCFMsg.hasUnmodifiedTableSchema()) {
|
||||
unmodifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(addCFMsg.getUnmodifiedTableSchema());
|
||||
unmodifiedTableDescriptor = ProtobufUtil.toTableDescriptor(addCFMsg.getUnmodifiedTableSchema());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -229,11 +230,11 @@ public class AddColumnFamilyProcedure
|
|||
checkTableModifiable(env);
|
||||
|
||||
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
|
||||
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
if (unmodifiedHTableDescriptor == null) {
|
||||
throw new IOException("HTableDescriptor missing for " + tableName);
|
||||
unmodifiedTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
if (unmodifiedTableDescriptor == null) {
|
||||
throw new IOException("TableDescriptor missing for " + tableName);
|
||||
}
|
||||
if (unmodifiedHTableDescriptor.hasFamily(cfDescriptor.getName())) {
|
||||
if (unmodifiedTableDescriptor.hasColumnFamily(cfDescriptor.getName())) {
|
||||
throw new InvalidFamilyOperationException("Column family '" + getColumnFamilyName()
|
||||
+ "' in table '" + tableName + "' already exists so cannot be added");
|
||||
}
|
||||
|
@ -258,17 +259,18 @@ public class AddColumnFamilyProcedure
|
|||
// Update table descriptor
|
||||
LOG.info("AddColumn. Table = " + tableName + " HCD = " + cfDescriptor.toString());
|
||||
|
||||
HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
|
||||
if (htd.hasFamily(cfDescriptor.getName())) {
|
||||
if (htd.hasColumnFamily(cfDescriptor.getName())) {
|
||||
// It is possible to reach this situation, as we could already add the column family
|
||||
// to table descriptor, but the master failover happens before we complete this state.
|
||||
// We should be able to handle running this function multiple times without causing problem.
|
||||
return;
|
||||
}
|
||||
|
||||
htd.addFamily(cfDescriptor);
|
||||
env.getMasterServices().getTableDescriptors().add(htd);
|
||||
env.getMasterServices().getTableDescriptors().add(
|
||||
TableDescriptorBuilder.newBuilder(htd)
|
||||
.addColumnFamily(cfDescriptor).build());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -277,14 +279,14 @@ public class AddColumnFamilyProcedure
|
|||
* @throws IOException
|
||||
**/
|
||||
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
|
||||
HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
if (htd.hasFamily(cfDescriptor.getName())) {
|
||||
TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
if (htd.hasColumnFamily(cfDescriptor.getName())) {
|
||||
// Remove the column family from file system and update the table descriptor to
|
||||
// the before-add-column-family-state
|
||||
MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(env, tableName,
|
||||
getRegionInfoList(env), cfDescriptor.getName(), cfDescriptor.isMobEnabled());
|
||||
|
||||
env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
|
||||
env.getMasterServices().getTableDescriptors().add(unmodifiedTableDescriptor);
|
||||
|
||||
// Make sure regions are opened after table descriptor is updated.
|
||||
//reOpenAllRegionsIfTableIsOnline(env);
|
||||
|
|
|
@ -33,11 +33,12 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
|
@ -67,7 +68,7 @@ public class CloneSnapshotProcedure
|
|||
extends AbstractStateMachineTableProcedure<CloneSnapshotState> {
|
||||
private static final Log LOG = LogFactory.getLog(CloneSnapshotProcedure.class);
|
||||
|
||||
private HTableDescriptor hTableDescriptor;
|
||||
private TableDescriptor tableDescriptor;
|
||||
private SnapshotDescription snapshot;
|
||||
private boolean restoreAcl;
|
||||
private List<HRegionInfo> newRegions = null;
|
||||
|
@ -85,21 +86,21 @@ public class CloneSnapshotProcedure
|
|||
}
|
||||
|
||||
public CloneSnapshotProcedure(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor, final SnapshotDescription snapshot) {
|
||||
this(env, hTableDescriptor, snapshot, false);
|
||||
final TableDescriptor tableDescriptor, final SnapshotDescription snapshot) {
|
||||
this(env, tableDescriptor, snapshot, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param env MasterProcedureEnv
|
||||
* @param hTableDescriptor the table to operate on
|
||||
* @param tableDescriptor the table to operate on
|
||||
* @param snapshot snapshot to clone from
|
||||
*/
|
||||
public CloneSnapshotProcedure(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor, final SnapshotDescription snapshot,
|
||||
final TableDescriptor tableDescriptor, final SnapshotDescription snapshot,
|
||||
final boolean restoreAcl) {
|
||||
super(env);
|
||||
this.hTableDescriptor = hTableDescriptor;
|
||||
this.tableDescriptor = tableDescriptor;
|
||||
this.snapshot = snapshot;
|
||||
this.restoreAcl = restoreAcl;
|
||||
|
||||
|
@ -121,7 +122,7 @@ public class CloneSnapshotProcedure
|
|||
Configuration conf = env.getMasterServices().getConfiguration();
|
||||
if (restoreAcl && snapshot.hasUsersAndPermissions() && snapshot.getUsersAndPermissions() != null
|
||||
&& SnapshotDescriptionUtils.isSecurityAvailable(conf)) {
|
||||
RestoreSnapshotHelper.restoreSnapshotAcl(snapshot, hTableDescriptor.getTableName(), conf);
|
||||
RestoreSnapshotHelper.restoreSnapshotAcl(snapshot, tableDescriptor.getTableName(), conf);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -141,7 +142,7 @@ public class CloneSnapshotProcedure
|
|||
setNextState(CloneSnapshotState.CLONE_SNAPSHOT_WRITE_FS_LAYOUT);
|
||||
break;
|
||||
case CLONE_SNAPSHOT_WRITE_FS_LAYOUT:
|
||||
newRegions = createFilesystemLayout(env, hTableDescriptor, newRegions);
|
||||
newRegions = createFilesystemLayout(env, tableDescriptor, newRegions);
|
||||
setNextState(CloneSnapshotState.CLONE_SNAPSHOT_ADD_TO_META);
|
||||
break;
|
||||
case CLONE_SNAPSHOT_ADD_TO_META:
|
||||
|
@ -224,7 +225,7 @@ public class CloneSnapshotProcedure
|
|||
|
||||
@Override
|
||||
public TableName getTableName() {
|
||||
return hTableDescriptor.getTableName();
|
||||
return tableDescriptor.getTableName();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -250,7 +251,7 @@ public class CloneSnapshotProcedure
|
|||
MasterProcedureProtos.CloneSnapshotStateData.newBuilder()
|
||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||
.setSnapshot(this.snapshot)
|
||||
.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
|
||||
.setTableSchema(ProtobufUtil.toTableSchema(tableDescriptor));
|
||||
if (newRegions != null) {
|
||||
for (HRegionInfo hri: newRegions) {
|
||||
cloneSnapshotMsg.addRegionInfo(HRegionInfo.convert(hri));
|
||||
|
@ -281,7 +282,7 @@ public class CloneSnapshotProcedure
|
|||
MasterProcedureProtos.CloneSnapshotStateData.parseDelimitedFrom(stream);
|
||||
setUser(MasterProcedureUtil.toUserInfo(cloneSnapshotMsg.getUserInfo()));
|
||||
snapshot = cloneSnapshotMsg.getSnapshot();
|
||||
hTableDescriptor = ProtobufUtil.convertToHTableDesc(cloneSnapshotMsg.getTableSchema());
|
||||
tableDescriptor = ProtobufUtil.toTableDescriptor(cloneSnapshotMsg.getTableSchema());
|
||||
if (cloneSnapshotMsg.getRegionInfoCount() == 0) {
|
||||
newRegions = null;
|
||||
} else {
|
||||
|
@ -341,7 +342,7 @@ public class CloneSnapshotProcedure
|
|||
|
||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||
if (cpHost != null) {
|
||||
cpHost.preCreateTableAction(hTableDescriptor, null, getUser());
|
||||
cpHost.preCreateTableAction(tableDescriptor, null, getUser());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -357,7 +358,7 @@ public class CloneSnapshotProcedure
|
|||
if (cpHost != null) {
|
||||
final HRegionInfo[] regions = (newRegions == null) ? null :
|
||||
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
||||
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions, getUser());
|
||||
cpHost.postCompletedCreateTableAction(tableDescriptor, regions, getUser());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -368,9 +369,9 @@ public class CloneSnapshotProcedure
|
|||
*/
|
||||
private List<HRegionInfo> createFilesystemLayout(
|
||||
final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final List<HRegionInfo> newRegions) throws IOException {
|
||||
return createFsLayout(env, hTableDescriptor, newRegions, new CreateHdfsRegions() {
|
||||
return createFsLayout(env, tableDescriptor, newRegions, new CreateHdfsRegions() {
|
||||
@Override
|
||||
public List<HRegionInfo> createHdfsRegions(
|
||||
final MasterProcedureEnv env,
|
||||
|
@ -390,7 +391,7 @@ public class CloneSnapshotProcedure
|
|||
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
|
||||
SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshot);
|
||||
RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
|
||||
conf, fs, manifest, hTableDescriptor, tableRootDir, monitorException, monitorStatus);
|
||||
conf, fs, manifest, tableDescriptor, tableRootDir, monitorException, monitorStatus);
|
||||
RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
|
||||
|
||||
// Clone operation should not have stuff to restore or remove
|
||||
|
@ -429,7 +430,7 @@ public class CloneSnapshotProcedure
|
|||
*/
|
||||
private List<HRegionInfo> createFsLayout(
|
||||
final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
List<HRegionInfo> newRegions,
|
||||
final CreateHdfsRegions hdfsRegionHandler) throws IOException {
|
||||
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
|
||||
|
@ -437,17 +438,17 @@ public class CloneSnapshotProcedure
|
|||
|
||||
// 1. Create Table Descriptor
|
||||
// using a copy of descriptor, table will be created enabling first
|
||||
HTableDescriptor underConstruction = new HTableDescriptor(hTableDescriptor);
|
||||
final Path tempTableDir = FSUtils.getTableDir(tempdir, hTableDescriptor.getTableName());
|
||||
final Path tempTableDir = FSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
|
||||
((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
|
||||
.createTableDescriptorForTableDirectory(tempTableDir, underConstruction, false);
|
||||
.createTableDescriptorForTableDirectory(tempTableDir,
|
||||
TableDescriptorBuilder.newBuilder(tableDescriptor).build(), false);
|
||||
|
||||
// 2. Create Regions
|
||||
newRegions = hdfsRegionHandler.createHdfsRegions(
|
||||
env, tempdir, hTableDescriptor.getTableName(), newRegions);
|
||||
env, tempdir, tableDescriptor.getTableName(), newRegions);
|
||||
|
||||
// 3. Move Table temp directory to the hbase root location
|
||||
CreateTableProcedure.moveTempDirectoryToHBaseRoot(env, hTableDescriptor, tempTableDir);
|
||||
CreateTableProcedure.moveTempDirectoryToHBaseRoot(env, tableDescriptor, tempTableDir);
|
||||
|
||||
return newRegions;
|
||||
}
|
||||
|
@ -458,11 +459,11 @@ public class CloneSnapshotProcedure
|
|||
* @throws IOException
|
||||
*/
|
||||
private void addRegionsToMeta(final MasterProcedureEnv env) throws IOException {
|
||||
newRegions = CreateTableProcedure.addTableToMeta(env, hTableDescriptor, newRegions);
|
||||
newRegions = CreateTableProcedure.addTableToMeta(env, tableDescriptor, newRegions);
|
||||
|
||||
RestoreSnapshotHelper.RestoreMetaChanges metaChanges =
|
||||
new RestoreSnapshotHelper.RestoreMetaChanges(
|
||||
hTableDescriptor, parentsToChildrenPairMap);
|
||||
tableDescriptor, parentsToChildrenPairMap);
|
||||
metaChanges.updateMetaParentRegions(env.getMasterServices().getConnection(), newRegions);
|
||||
}
|
||||
|
||||
|
|
|
@ -30,12 +30,12 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
|
@ -55,7 +55,7 @@ public class CreateTableProcedure
|
|||
extends AbstractStateMachineTableProcedure<CreateTableState> {
|
||||
private static final Log LOG = LogFactory.getLog(CreateTableProcedure.class);
|
||||
|
||||
private HTableDescriptor hTableDescriptor;
|
||||
private TableDescriptor tableDescriptor;
|
||||
private List<HRegionInfo> newRegions;
|
||||
|
||||
public CreateTableProcedure() {
|
||||
|
@ -64,15 +64,15 @@ public class CreateTableProcedure
|
|||
}
|
||||
|
||||
public CreateTableProcedure(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions) {
|
||||
this(env, hTableDescriptor, newRegions, null);
|
||||
final TableDescriptor tableDescriptor, final HRegionInfo[] newRegions) {
|
||||
this(env, tableDescriptor, newRegions, null);
|
||||
}
|
||||
|
||||
public CreateTableProcedure(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
|
||||
final TableDescriptor tableDescriptor, final HRegionInfo[] newRegions,
|
||||
final ProcedurePrepareLatch syncLatch) {
|
||||
super(env, syncLatch);
|
||||
this.hTableDescriptor = hTableDescriptor;
|
||||
this.tableDescriptor = tableDescriptor;
|
||||
this.newRegions = newRegions != null ? Lists.newArrayList(newRegions) : null;
|
||||
}
|
||||
|
||||
|
@ -98,11 +98,11 @@ public class CreateTableProcedure
|
|||
setNextState(CreateTableState.CREATE_TABLE_WRITE_FS_LAYOUT);
|
||||
break;
|
||||
case CREATE_TABLE_WRITE_FS_LAYOUT:
|
||||
newRegions = createFsLayout(env, hTableDescriptor, newRegions);
|
||||
newRegions = createFsLayout(env, tableDescriptor, newRegions);
|
||||
setNextState(CreateTableState.CREATE_TABLE_ADD_TO_META);
|
||||
break;
|
||||
case CREATE_TABLE_ADD_TO_META:
|
||||
newRegions = addTableToMeta(env, hTableDescriptor, newRegions);
|
||||
newRegions = addTableToMeta(env, tableDescriptor, newRegions);
|
||||
setNextState(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS);
|
||||
break;
|
||||
case CREATE_TABLE_ASSIGN_REGIONS:
|
||||
|
@ -174,7 +174,7 @@ public class CreateTableProcedure
|
|||
|
||||
@Override
|
||||
public TableName getTableName() {
|
||||
return hTableDescriptor.getTableName();
|
||||
return tableDescriptor.getTableName();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -189,7 +189,7 @@ public class CreateTableProcedure
|
|||
MasterProcedureProtos.CreateTableStateData.Builder state =
|
||||
MasterProcedureProtos.CreateTableStateData.newBuilder()
|
||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||
.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
|
||||
.setTableSchema(ProtobufUtil.toTableSchema(tableDescriptor));
|
||||
if (newRegions != null) {
|
||||
for (HRegionInfo hri: newRegions) {
|
||||
state.addRegionInfo(HRegionInfo.convert(hri));
|
||||
|
@ -205,7 +205,7 @@ public class CreateTableProcedure
|
|||
MasterProcedureProtos.CreateTableStateData state =
|
||||
MasterProcedureProtos.CreateTableStateData.parseDelimitedFrom(stream);
|
||||
setUser(MasterProcedureUtil.toUserInfo(state.getUserInfo()));
|
||||
hTableDescriptor = ProtobufUtil.convertToHTableDesc(state.getTableSchema());
|
||||
tableDescriptor = ProtobufUtil.toTableDescriptor(state.getTableSchema());
|
||||
if (state.getRegionInfoCount() == 0) {
|
||||
newRegions = null;
|
||||
} else {
|
||||
|
@ -235,7 +235,7 @@ public class CreateTableProcedure
|
|||
}
|
||||
|
||||
// check that we have at least 1 CF
|
||||
if (hTableDescriptor.getColumnFamilyCount() == 0) {
|
||||
if (tableDescriptor.getColumnFamilyCount() == 0) {
|
||||
setFailure("master-create-table", new DoNotRetryIOException("Table " +
|
||||
getTableName().toString() + " should have at least one column family."));
|
||||
return false;
|
||||
|
@ -256,7 +256,7 @@ public class CreateTableProcedure
|
|||
if (cpHost != null) {
|
||||
final HRegionInfo[] regions = newRegions == null ? null :
|
||||
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
||||
cpHost.preCreateTableAction(hTableDescriptor, regions, getUser());
|
||||
cpHost.preCreateTableAction(tableDescriptor, regions, getUser());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -266,7 +266,7 @@ public class CreateTableProcedure
|
|||
if (cpHost != null) {
|
||||
final HRegionInfo[] regions = (newRegions == null) ? null :
|
||||
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
||||
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions, getUser());
|
||||
cpHost.postCompletedCreateTableAction(tableDescriptor, regions, getUser());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -277,9 +277,9 @@ public class CreateTableProcedure
|
|||
}
|
||||
|
||||
protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor, final List<HRegionInfo> newRegions)
|
||||
final TableDescriptor tableDescriptor, final List<HRegionInfo> newRegions)
|
||||
throws IOException {
|
||||
return createFsLayout(env, hTableDescriptor, newRegions, new CreateHdfsRegions() {
|
||||
return createFsLayout(env, tableDescriptor, newRegions, new CreateHdfsRegions() {
|
||||
@Override
|
||||
public List<HRegionInfo> createHdfsRegions(final MasterProcedureEnv env,
|
||||
final Path tableRootDir, final TableName tableName,
|
||||
|
@ -287,40 +287,40 @@ public class CreateTableProcedure
|
|||
HRegionInfo[] regions = newRegions != null ?
|
||||
newRegions.toArray(new HRegionInfo[newRegions.size()]) : null;
|
||||
return ModifyRegionUtils.createRegions(env.getMasterConfiguration(),
|
||||
tableRootDir, hTableDescriptor, regions, null);
|
||||
tableRootDir, tableDescriptor, regions, null);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor, List<HRegionInfo> newRegions,
|
||||
final TableDescriptor tableDescriptor, List<HRegionInfo> newRegions,
|
||||
final CreateHdfsRegions hdfsRegionHandler) throws IOException {
|
||||
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
|
||||
final Path tempdir = mfs.getTempDir();
|
||||
|
||||
// 1. Create Table Descriptor
|
||||
// using a copy of descriptor, table will be created enabling first
|
||||
final Path tempTableDir = FSUtils.getTableDir(tempdir, hTableDescriptor.getTableName());
|
||||
final Path tempTableDir = FSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
|
||||
((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
|
||||
.createTableDescriptorForTableDirectory(
|
||||
tempTableDir, hTableDescriptor, false);
|
||||
tempTableDir, tableDescriptor, false);
|
||||
|
||||
// 2. Create Regions
|
||||
newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir,
|
||||
hTableDescriptor.getTableName(), newRegions);
|
||||
tableDescriptor.getTableName(), newRegions);
|
||||
|
||||
// 3. Move Table temp directory to the hbase root location
|
||||
moveTempDirectoryToHBaseRoot(env, hTableDescriptor, tempTableDir);
|
||||
moveTempDirectoryToHBaseRoot(env, tableDescriptor, tempTableDir);
|
||||
|
||||
return newRegions;
|
||||
}
|
||||
|
||||
protected static void moveTempDirectoryToHBaseRoot(
|
||||
final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final Path tempTableDir) throws IOException {
|
||||
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
|
||||
final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), hTableDescriptor.getTableName());
|
||||
final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableDescriptor.getTableName());
|
||||
FileSystem fs = mfs.getFileSystem();
|
||||
if (!fs.delete(tableDir, true) && fs.exists(tableDir)) {
|
||||
throw new IOException("Couldn't delete " + tableDir);
|
||||
|
@ -332,20 +332,20 @@ public class CreateTableProcedure
|
|||
}
|
||||
|
||||
protected static List<HRegionInfo> addTableToMeta(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final List<HRegionInfo> regions) throws IOException {
|
||||
assert (regions != null && regions.size() > 0) : "expected at least 1 region, got " + regions;
|
||||
|
||||
ProcedureSyncWait.waitMetaRegions(env);
|
||||
|
||||
// Add replicas if needed
|
||||
List<HRegionInfo> newRegions = addReplicas(env, hTableDescriptor, regions);
|
||||
List<HRegionInfo> newRegions = addReplicas(env, tableDescriptor, regions);
|
||||
|
||||
// Add regions to META
|
||||
addRegionsToMeta(env, hTableDescriptor, newRegions);
|
||||
addRegionsToMeta(env, tableDescriptor, newRegions);
|
||||
|
||||
// Setup replication for region replicas if needed
|
||||
if (hTableDescriptor.getRegionReplication() > 1) {
|
||||
if (tableDescriptor.getRegionReplication() > 1) {
|
||||
ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration());
|
||||
}
|
||||
return newRegions;
|
||||
|
@ -354,14 +354,14 @@ public class CreateTableProcedure
|
|||
/**
|
||||
* Create any replicas for the regions (the default replicas that was
|
||||
* already created is passed to the method)
|
||||
* @param hTableDescriptor descriptor to use
|
||||
* @param tableDescriptor descriptor to use
|
||||
* @param regions default replicas
|
||||
* @return the combined list of default and non-default replicas
|
||||
*/
|
||||
private static List<HRegionInfo> addReplicas(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final List<HRegionInfo> regions) {
|
||||
int numRegionReplicas = hTableDescriptor.getRegionReplication() - 1;
|
||||
int numRegionReplicas = tableDescriptor.getRegionReplication() - 1;
|
||||
if (numRegionReplicas <= 0) {
|
||||
return regions;
|
||||
}
|
||||
|
@ -394,10 +394,10 @@ public class CreateTableProcedure
|
|||
* Add the specified set of regions to the hbase:meta table.
|
||||
*/
|
||||
private static void addRegionsToMeta(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final List<HRegionInfo> regionInfos) throws IOException {
|
||||
MetaTableAccessor.addRegionsToMeta(env.getMasterServices().getConnection(),
|
||||
regionInfos, hTableDescriptor.getRegionReplication());
|
||||
regionInfos, tableDescriptor.getRegionReplication());
|
||||
}
|
||||
|
||||
protected static void updateTableDescCache(final MasterProcedureEnv env,
|
||||
|
|
|
@ -26,10 +26,11 @@ import java.util.List;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
@ -45,7 +46,7 @@ public class DeleteColumnFamilyProcedure
|
|||
extends AbstractStateMachineTableProcedure<DeleteColumnFamilyState> {
|
||||
private static final Log LOG = LogFactory.getLog(DeleteColumnFamilyProcedure.class);
|
||||
|
||||
private HTableDescriptor unmodifiedHTableDescriptor;
|
||||
private TableDescriptor unmodifiedTableDescriptor;
|
||||
private TableName tableName;
|
||||
private byte [] familyName;
|
||||
private boolean hasMob;
|
||||
|
@ -55,7 +56,7 @@ public class DeleteColumnFamilyProcedure
|
|||
|
||||
public DeleteColumnFamilyProcedure() {
|
||||
super();
|
||||
this.unmodifiedHTableDescriptor = null;
|
||||
this.unmodifiedTableDescriptor = null;
|
||||
this.regionInfoList = null;
|
||||
this.traceEnabled = null;
|
||||
}
|
||||
|
@ -70,7 +71,7 @@ public class DeleteColumnFamilyProcedure
|
|||
super(env, latch);
|
||||
this.tableName = tableName;
|
||||
this.familyName = familyName;
|
||||
this.unmodifiedHTableDescriptor = null;
|
||||
this.unmodifiedTableDescriptor = null;
|
||||
this.regionInfoList = null;
|
||||
this.traceEnabled = null;
|
||||
}
|
||||
|
@ -179,9 +180,9 @@ public class DeleteColumnFamilyProcedure
|
|||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||
.setColumnfamilyName(UnsafeByteOperations.unsafeWrap(familyName));
|
||||
if (unmodifiedHTableDescriptor != null) {
|
||||
if (unmodifiedTableDescriptor != null) {
|
||||
deleteCFMsg
|
||||
.setUnmodifiedTableSchema(ProtobufUtil.convertToTableSchema(unmodifiedHTableDescriptor));
|
||||
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
|
||||
}
|
||||
|
||||
deleteCFMsg.build().writeDelimitedTo(stream);
|
||||
|
@ -197,7 +198,7 @@ public class DeleteColumnFamilyProcedure
|
|||
familyName = deleteCFMsg.getColumnfamilyName().toByteArray();
|
||||
|
||||
if (deleteCFMsg.hasUnmodifiedTableSchema()) {
|
||||
unmodifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(deleteCFMsg.getUnmodifiedTableSchema());
|
||||
unmodifiedTableDescriptor = ProtobufUtil.toTableDescriptor(deleteCFMsg.getUnmodifiedTableSchema());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -235,22 +236,22 @@ public class DeleteColumnFamilyProcedure
|
|||
checkTableModifiable(env);
|
||||
|
||||
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
|
||||
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
if (unmodifiedHTableDescriptor == null) {
|
||||
throw new IOException("HTableDescriptor missing for " + tableName);
|
||||
unmodifiedTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
if (unmodifiedTableDescriptor == null) {
|
||||
throw new IOException("TableDescriptor missing for " + tableName);
|
||||
}
|
||||
if (!unmodifiedHTableDescriptor.hasFamily(familyName)) {
|
||||
if (!unmodifiedTableDescriptor.hasColumnFamily(familyName)) {
|
||||
throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
|
||||
+ "' does not exist, so it cannot be deleted");
|
||||
}
|
||||
|
||||
if (unmodifiedHTableDescriptor.getColumnFamilyCount() == 1) {
|
||||
if (unmodifiedTableDescriptor.getColumnFamilyCount() == 1) {
|
||||
throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
|
||||
+ "' is the only column family in the table, so it cannot be deleted");
|
||||
}
|
||||
|
||||
// whether mob family
|
||||
hasMob = unmodifiedHTableDescriptor.getFamily(familyName).isMobEnabled();
|
||||
hasMob = unmodifiedTableDescriptor.getColumnFamily(familyName).isMobEnabled();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -272,17 +273,17 @@ public class DeleteColumnFamilyProcedure
|
|||
// Update table descriptor
|
||||
LOG.info("DeleteColumn. Table = " + tableName + " family = " + getColumnFamilyName());
|
||||
|
||||
HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
|
||||
if (!htd.hasFamily(familyName)) {
|
||||
if (!htd.hasColumnFamily(familyName)) {
|
||||
// It is possible to reach this situation, as we could already delete the column family
|
||||
// from table descriptor, but the master failover happens before we complete this state.
|
||||
// We should be able to handle running this function multiple times without causing problem.
|
||||
return;
|
||||
}
|
||||
|
||||
htd.removeFamily(familyName);
|
||||
env.getMasterServices().getTableDescriptors().add(htd);
|
||||
env.getMasterServices().getTableDescriptors().add(
|
||||
TableDescriptorBuilder.newBuilder(htd).removeColumnFamily(familyName).build());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -291,7 +292,7 @@ public class DeleteColumnFamilyProcedure
|
|||
* @throws IOException
|
||||
**/
|
||||
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
|
||||
env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
|
||||
env.getMasterServices().getTableDescriptors().add(unmodifiedTableDescriptor);
|
||||
|
||||
// Make sure regions are opened after table descriptor is updated.
|
||||
//reOpenAllRegionsIfTableIsOnline(env);
|
||||
|
|
|
@ -24,11 +24,12 @@ import java.io.OutputStream;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
|
||||
|
@ -43,28 +44,28 @@ public class ModifyColumnFamilyProcedure
|
|||
private static final Log LOG = LogFactory.getLog(ModifyColumnFamilyProcedure.class);
|
||||
|
||||
private TableName tableName;
|
||||
private HTableDescriptor unmodifiedHTableDescriptor;
|
||||
private HColumnDescriptor cfDescriptor;
|
||||
private TableDescriptor unmodifiedtableDescriptor;
|
||||
private ColumnFamilyDescriptor cfDescriptor;
|
||||
|
||||
private Boolean traceEnabled;
|
||||
|
||||
public ModifyColumnFamilyProcedure() {
|
||||
super();
|
||||
this.unmodifiedHTableDescriptor = null;
|
||||
this.unmodifiedtableDescriptor = null;
|
||||
this.traceEnabled = null;
|
||||
}
|
||||
|
||||
public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||
final HColumnDescriptor cfDescriptor) {
|
||||
final ColumnFamilyDescriptor cfDescriptor) {
|
||||
this(env, tableName, cfDescriptor, null);
|
||||
}
|
||||
|
||||
public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||
final HColumnDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
|
||||
final ColumnFamilyDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
|
||||
super(env, latch);
|
||||
this.tableName = tableName;
|
||||
this.cfDescriptor = cfDescriptor;
|
||||
this.unmodifiedHTableDescriptor = null;
|
||||
this.unmodifiedtableDescriptor = null;
|
||||
this.traceEnabled = null;
|
||||
}
|
||||
|
||||
|
@ -165,10 +166,10 @@ public class ModifyColumnFamilyProcedure
|
|||
MasterProcedureProtos.ModifyColumnFamilyStateData.newBuilder()
|
||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||
.setColumnfamilySchema(ProtobufUtil.convertToColumnFamilySchema(cfDescriptor));
|
||||
if (unmodifiedHTableDescriptor != null) {
|
||||
.setColumnfamilySchema(ProtobufUtil.toColumnFamilySchema(cfDescriptor));
|
||||
if (unmodifiedtableDescriptor != null) {
|
||||
modifyCFMsg
|
||||
.setUnmodifiedTableSchema(ProtobufUtil.convertToTableSchema(unmodifiedHTableDescriptor));
|
||||
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedtableDescriptor));
|
||||
}
|
||||
|
||||
modifyCFMsg.build().writeDelimitedTo(stream);
|
||||
|
@ -182,9 +183,9 @@ public class ModifyColumnFamilyProcedure
|
|||
MasterProcedureProtos.ModifyColumnFamilyStateData.parseDelimitedFrom(stream);
|
||||
setUser(MasterProcedureUtil.toUserInfo(modifyCFMsg.getUserInfo()));
|
||||
tableName = ProtobufUtil.toTableName(modifyCFMsg.getTableName());
|
||||
cfDescriptor = ProtobufUtil.convertToHColumnDesc(modifyCFMsg.getColumnfamilySchema());
|
||||
cfDescriptor = ProtobufUtil.toColumnFamilyDescriptor(modifyCFMsg.getColumnfamilySchema());
|
||||
if (modifyCFMsg.hasUnmodifiedTableSchema()) {
|
||||
unmodifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(modifyCFMsg.getUnmodifiedTableSchema());
|
||||
unmodifiedtableDescriptor = ProtobufUtil.toTableDescriptor(modifyCFMsg.getUnmodifiedTableSchema());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -221,11 +222,11 @@ public class ModifyColumnFamilyProcedure
|
|||
// Checks whether the table is allowed to be modified.
|
||||
checkTableModifiable(env);
|
||||
|
||||
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
if (unmodifiedHTableDescriptor == null) {
|
||||
throw new IOException("HTableDescriptor missing for " + tableName);
|
||||
unmodifiedtableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
if (unmodifiedtableDescriptor == null) {
|
||||
throw new IOException("TableDescriptor missing for " + tableName);
|
||||
}
|
||||
if (!unmodifiedHTableDescriptor.hasFamily(cfDescriptor.getName())) {
|
||||
if (!unmodifiedtableDescriptor.hasColumnFamily(cfDescriptor.getName())) {
|
||||
throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
|
||||
+ "' does not exist, so it cannot be modified");
|
||||
}
|
||||
|
@ -250,9 +251,9 @@ public class ModifyColumnFamilyProcedure
|
|||
// Update table descriptor
|
||||
LOG.info("ModifyColumnFamily. Table = " + tableName + " HCD = " + cfDescriptor.toString());
|
||||
|
||||
HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
htd.modifyFamily(cfDescriptor);
|
||||
env.getMasterServices().getTableDescriptors().add(htd);
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(env.getMasterServices().getTableDescriptors().get(tableName));
|
||||
builder.modifyColumnFamily(cfDescriptor);
|
||||
env.getMasterServices().getTableDescriptors().add(builder.build());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -261,7 +262,7 @@ public class ModifyColumnFamilyProcedure
|
|||
* @throws IOException
|
||||
**/
|
||||
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
|
||||
env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
|
||||
env.getMasterServices().getTableDescriptors().add(unmodifiedtableDescriptor);
|
||||
|
||||
// Make sure regions are opened after table descriptor is updated.
|
||||
//reOpenAllRegionsIfTableIsOnline(env);
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
|
@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
@ -52,8 +52,8 @@ public class ModifyTableProcedure
|
|||
extends AbstractStateMachineTableProcedure<ModifyTableState> {
|
||||
private static final Log LOG = LogFactory.getLog(ModifyTableProcedure.class);
|
||||
|
||||
private HTableDescriptor unmodifiedHTableDescriptor = null;
|
||||
private HTableDescriptor modifiedHTableDescriptor;
|
||||
private TableDescriptor unmodifiedTableDescriptor = null;
|
||||
private TableDescriptor modifiedTableDescriptor;
|
||||
private boolean deleteColumnFamilyInModify;
|
||||
|
||||
private List<HRegionInfo> regionInfoList;
|
||||
|
@ -64,19 +64,19 @@ public class ModifyTableProcedure
|
|||
initilize();
|
||||
}
|
||||
|
||||
public ModifyTableProcedure(final MasterProcedureEnv env, final HTableDescriptor htd) {
|
||||
public ModifyTableProcedure(final MasterProcedureEnv env, final TableDescriptor htd) {
|
||||
this(env, htd, null);
|
||||
}
|
||||
|
||||
public ModifyTableProcedure(final MasterProcedureEnv env, final HTableDescriptor htd,
|
||||
public ModifyTableProcedure(final MasterProcedureEnv env, final TableDescriptor htd,
|
||||
final ProcedurePrepareLatch latch) {
|
||||
super(env, latch);
|
||||
initilize();
|
||||
this.modifiedHTableDescriptor = htd;
|
||||
this.modifiedTableDescriptor = htd;
|
||||
}
|
||||
|
||||
private void initilize() {
|
||||
this.unmodifiedHTableDescriptor = null;
|
||||
this.unmodifiedTableDescriptor = null;
|
||||
this.regionInfoList = null;
|
||||
this.traceEnabled = null;
|
||||
this.deleteColumnFamilyInModify = false;
|
||||
|
@ -104,7 +104,7 @@ public class ModifyTableProcedure
|
|||
setNextState(ModifyTableState.MODIFY_TABLE_REMOVE_REPLICA_COLUMN);
|
||||
break;
|
||||
case MODIFY_TABLE_REMOVE_REPLICA_COLUMN:
|
||||
updateReplicaColumnsIfNeeded(env, unmodifiedHTableDescriptor, modifiedHTableDescriptor);
|
||||
updateReplicaColumnsIfNeeded(env, unmodifiedTableDescriptor, modifiedTableDescriptor);
|
||||
if (deleteColumnFamilyInModify) {
|
||||
setNextState(ModifyTableState.MODIFY_TABLE_DELETE_FS_LAYOUT);
|
||||
} else {
|
||||
|
@ -112,7 +112,7 @@ public class ModifyTableProcedure
|
|||
}
|
||||
break;
|
||||
case MODIFY_TABLE_DELETE_FS_LAYOUT:
|
||||
deleteFromFs(env, unmodifiedHTableDescriptor, modifiedHTableDescriptor);
|
||||
deleteFromFs(env, unmodifiedTableDescriptor, modifiedTableDescriptor);
|
||||
setNextState(ModifyTableState.MODIFY_TABLE_POST_OPERATION);
|
||||
break;
|
||||
case MODIFY_TABLE_POST_OPERATION:
|
||||
|
@ -191,12 +191,12 @@ public class ModifyTableProcedure
|
|||
MasterProcedureProtos.ModifyTableStateData.Builder modifyTableMsg =
|
||||
MasterProcedureProtos.ModifyTableStateData.newBuilder()
|
||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||
.setModifiedTableSchema(ProtobufUtil.convertToTableSchema(modifiedHTableDescriptor))
|
||||
.setModifiedTableSchema(ProtobufUtil.toTableSchema(modifiedTableDescriptor))
|
||||
.setDeleteColumnFamilyInModify(deleteColumnFamilyInModify);
|
||||
|
||||
if (unmodifiedHTableDescriptor != null) {
|
||||
if (unmodifiedTableDescriptor != null) {
|
||||
modifyTableMsg
|
||||
.setUnmodifiedTableSchema(ProtobufUtil.convertToTableSchema(unmodifiedHTableDescriptor));
|
||||
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
|
||||
}
|
||||
|
||||
modifyTableMsg.build().writeDelimitedTo(stream);
|
||||
|
@ -209,18 +209,18 @@ public class ModifyTableProcedure
|
|||
MasterProcedureProtos.ModifyTableStateData modifyTableMsg =
|
||||
MasterProcedureProtos.ModifyTableStateData.parseDelimitedFrom(stream);
|
||||
setUser(MasterProcedureUtil.toUserInfo(modifyTableMsg.getUserInfo()));
|
||||
modifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(modifyTableMsg.getModifiedTableSchema());
|
||||
modifiedTableDescriptor = ProtobufUtil.toTableDescriptor(modifyTableMsg.getModifiedTableSchema());
|
||||
deleteColumnFamilyInModify = modifyTableMsg.getDeleteColumnFamilyInModify();
|
||||
|
||||
if (modifyTableMsg.hasUnmodifiedTableSchema()) {
|
||||
unmodifiedHTableDescriptor =
|
||||
ProtobufUtil.convertToHTableDesc(modifyTableMsg.getUnmodifiedTableSchema());
|
||||
unmodifiedTableDescriptor =
|
||||
ProtobufUtil.toTableDescriptor(modifyTableMsg.getUnmodifiedTableSchema());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableName getTableName() {
|
||||
return modifiedHTableDescriptor.getTableName();
|
||||
return modifiedTableDescriptor.getTableName();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -240,27 +240,27 @@ public class ModifyTableProcedure
|
|||
}
|
||||
|
||||
// check that we have at least 1 CF
|
||||
if (modifiedHTableDescriptor.getColumnFamilyCount() == 0) {
|
||||
if (modifiedTableDescriptor.getColumnFamilyCount() == 0) {
|
||||
throw new DoNotRetryIOException("Table " + getTableName().toString() +
|
||||
" should have at least one column family.");
|
||||
}
|
||||
|
||||
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
|
||||
this.unmodifiedHTableDescriptor =
|
||||
this.unmodifiedTableDescriptor =
|
||||
env.getMasterServices().getTableDescriptors().get(getTableName());
|
||||
|
||||
if (env.getMasterServices().getTableStateManager()
|
||||
.isTableState(getTableName(), TableState.State.ENABLED)) {
|
||||
if (modifiedHTableDescriptor.getRegionReplication() != unmodifiedHTableDescriptor
|
||||
if (modifiedTableDescriptor.getRegionReplication() != unmodifiedTableDescriptor
|
||||
.getRegionReplication()) {
|
||||
throw new IOException("REGION_REPLICATION change is not supported for enabled tables");
|
||||
}
|
||||
}
|
||||
|
||||
// Find out whether all column families in unmodifiedHTableDescriptor also exists in
|
||||
// the modifiedHTableDescriptor. This is to determine whether we are safe to rollback.
|
||||
final Set<byte[]> oldFamilies = unmodifiedHTableDescriptor.getFamiliesKeys();
|
||||
final Set<byte[]> newFamilies = modifiedHTableDescriptor.getFamiliesKeys();
|
||||
// Find out whether all column families in unmodifiedTableDescriptor also exists in
|
||||
// the modifiedTableDescriptor. This is to determine whether we are safe to rollback.
|
||||
final Set<byte[]> oldFamilies = unmodifiedTableDescriptor.getColumnFamilyNames();
|
||||
final Set<byte[]> newFamilies = modifiedTableDescriptor.getColumnFamilyNames();
|
||||
for (byte[] familyName : oldFamilies) {
|
||||
if (!newFamilies.contains(familyName)) {
|
||||
this.deleteColumnFamilyInModify = true;
|
||||
|
@ -287,7 +287,7 @@ public class ModifyTableProcedure
|
|||
* @throws IOException
|
||||
**/
|
||||
private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
|
||||
env.getMasterServices().getTableDescriptors().add(modifiedHTableDescriptor);
|
||||
env.getMasterServices().getTableDescriptors().add(modifiedTableDescriptor);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -296,10 +296,10 @@ public class ModifyTableProcedure
|
|||
* @throws IOException
|
||||
**/
|
||||
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
|
||||
env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
|
||||
env.getMasterServices().getTableDescriptors().add(unmodifiedTableDescriptor);
|
||||
|
||||
// delete any new column families from the modifiedHTableDescriptor.
|
||||
deleteFromFs(env, modifiedHTableDescriptor, unmodifiedHTableDescriptor);
|
||||
// delete any new column families from the modifiedTableDescriptor.
|
||||
deleteFromFs(env, modifiedTableDescriptor, unmodifiedTableDescriptor);
|
||||
|
||||
// Make sure regions are opened after table descriptor is updated.
|
||||
//reOpenAllRegionsIfTableIsOnline(env);
|
||||
|
@ -312,18 +312,17 @@ public class ModifyTableProcedure
|
|||
* @throws IOException
|
||||
*/
|
||||
private void deleteFromFs(final MasterProcedureEnv env,
|
||||
final HTableDescriptor oldHTableDescriptor, final HTableDescriptor newHTableDescriptor)
|
||||
final TableDescriptor oldTableDescriptor, final TableDescriptor newTableDescriptor)
|
||||
throws IOException {
|
||||
final Set<byte[]> oldFamilies = oldHTableDescriptor.getFamiliesKeys();
|
||||
final Set<byte[]> newFamilies = newHTableDescriptor.getFamiliesKeys();
|
||||
final Set<byte[]> oldFamilies = oldTableDescriptor.getColumnFamilyNames();
|
||||
final Set<byte[]> newFamilies = newTableDescriptor.getColumnFamilyNames();
|
||||
for (byte[] familyName : oldFamilies) {
|
||||
if (!newFamilies.contains(familyName)) {
|
||||
MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(
|
||||
env,
|
||||
getTableName(),
|
||||
getRegionInfoList(env),
|
||||
familyName,
|
||||
oldHTableDescriptor.getFamily(familyName).isMobEnabled());
|
||||
familyName, oldTableDescriptor.getColumnFamily(familyName).isMobEnabled());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -335,10 +334,10 @@ public class ModifyTableProcedure
|
|||
*/
|
||||
private void updateReplicaColumnsIfNeeded(
|
||||
final MasterProcedureEnv env,
|
||||
final HTableDescriptor oldHTableDescriptor,
|
||||
final HTableDescriptor newHTableDescriptor) throws IOException {
|
||||
final int oldReplicaCount = oldHTableDescriptor.getRegionReplication();
|
||||
final int newReplicaCount = newHTableDescriptor.getRegionReplication();
|
||||
final TableDescriptor oldTableDescriptor,
|
||||
final TableDescriptor newTableDescriptor) throws IOException {
|
||||
final int oldReplicaCount = oldTableDescriptor.getRegionReplication();
|
||||
final int newReplicaCount = newTableDescriptor.getRegionReplication();
|
||||
|
||||
if (newReplicaCount < oldReplicaCount) {
|
||||
Set<byte[]> tableRows = new HashSet<>();
|
||||
|
@ -402,10 +401,10 @@ public class ModifyTableProcedure
|
|||
if (cpHost != null) {
|
||||
switch (state) {
|
||||
case MODIFY_TABLE_PRE_OPERATION:
|
||||
cpHost.preModifyTableAction(getTableName(), modifiedHTableDescriptor, getUser());
|
||||
cpHost.preModifyTableAction(getTableName(), modifiedTableDescriptor, getUser());
|
||||
break;
|
||||
case MODIFY_TABLE_POST_OPERATION:
|
||||
cpHost.postCompletedModifyTableAction(getTableName(), modifiedHTableDescriptor,getUser());
|
||||
cpHost.postCompletedModifyTableAction(getTableName(), modifiedTableDescriptor,getUser());
|
||||
break;
|
||||
default:
|
||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||
|
|
|
@ -33,12 +33,12 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
|
@ -61,7 +61,7 @@ public class RestoreSnapshotProcedure
|
|||
extends AbstractStateMachineTableProcedure<RestoreSnapshotState> {
|
||||
private static final Log LOG = LogFactory.getLog(RestoreSnapshotProcedure.class);
|
||||
|
||||
private HTableDescriptor modifiedHTableDescriptor;
|
||||
private TableDescriptor modifiedTableDescriptor;
|
||||
private List<HRegionInfo> regionsToRestore = null;
|
||||
private List<HRegionInfo> regionsToRemove = null;
|
||||
private List<HRegionInfo> regionsToAdd = null;
|
||||
|
@ -82,24 +82,24 @@ public class RestoreSnapshotProcedure
|
|||
}
|
||||
|
||||
public RestoreSnapshotProcedure(final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor, final SnapshotDescription snapshot) {
|
||||
this(env, hTableDescriptor, snapshot, false);
|
||||
final TableDescriptor tableDescriptor, final SnapshotDescription snapshot) {
|
||||
this(env, tableDescriptor, snapshot, false);
|
||||
}
|
||||
/**
|
||||
* Constructor
|
||||
* @param env MasterProcedureEnv
|
||||
* @param hTableDescriptor the table to operate on
|
||||
* @param tableDescriptor the table to operate on
|
||||
* @param snapshot snapshot to restore from
|
||||
* @throws IOException
|
||||
*/
|
||||
public RestoreSnapshotProcedure(
|
||||
final MasterProcedureEnv env,
|
||||
final HTableDescriptor hTableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final SnapshotDescription snapshot,
|
||||
final boolean restoreAcl) {
|
||||
super(env);
|
||||
// This is the new schema we are going to write out as this modification.
|
||||
this.modifiedHTableDescriptor = hTableDescriptor;
|
||||
this.modifiedTableDescriptor = tableDescriptor;
|
||||
// Snapshot information
|
||||
this.snapshot = snapshot;
|
||||
this.restoreAcl = restoreAcl;
|
||||
|
@ -204,7 +204,7 @@ public class RestoreSnapshotProcedure
|
|||
|
||||
@Override
|
||||
public TableName getTableName() {
|
||||
return modifiedHTableDescriptor.getTableName();
|
||||
return modifiedTableDescriptor.getTableName();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -236,7 +236,7 @@ public class RestoreSnapshotProcedure
|
|||
MasterProcedureProtos.RestoreSnapshotStateData.newBuilder()
|
||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||
.setSnapshot(this.snapshot)
|
||||
.setModifiedTableSchema(ProtobufUtil.convertToTableSchema(modifiedHTableDescriptor));
|
||||
.setModifiedTableSchema(ProtobufUtil.toTableSchema(modifiedTableDescriptor));
|
||||
|
||||
if (regionsToRestore != null) {
|
||||
for (HRegionInfo hri: regionsToRestore) {
|
||||
|
@ -278,8 +278,8 @@ public class RestoreSnapshotProcedure
|
|||
MasterProcedureProtos.RestoreSnapshotStateData.parseDelimitedFrom(stream);
|
||||
setUser(MasterProcedureUtil.toUserInfo(restoreSnapshotMsg.getUserInfo()));
|
||||
snapshot = restoreSnapshotMsg.getSnapshot();
|
||||
modifiedHTableDescriptor =
|
||||
ProtobufUtil.convertToHTableDesc(restoreSnapshotMsg.getModifiedTableSchema());
|
||||
modifiedTableDescriptor =
|
||||
ProtobufUtil.toTableDescriptor(restoreSnapshotMsg.getModifiedTableSchema());
|
||||
|
||||
if (restoreSnapshotMsg.getRegionInfoForRestoreCount() == 0) {
|
||||
regionsToRestore = null;
|
||||
|
@ -333,7 +333,7 @@ public class RestoreSnapshotProcedure
|
|||
env.getMasterServices().checkTableModifiable(tableName);
|
||||
|
||||
// Check that we have at least 1 CF
|
||||
if (modifiedHTableDescriptor.getColumnFamilyCount() == 0) {
|
||||
if (modifiedTableDescriptor.getColumnFamilyCount() == 0) {
|
||||
throw new DoNotRetryIOException("Table " + getTableName().toString() +
|
||||
" should have at least one column family.");
|
||||
}
|
||||
|
@ -363,7 +363,7 @@ public class RestoreSnapshotProcedure
|
|||
* @throws IOException
|
||||
**/
|
||||
private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
|
||||
env.getMasterServices().getTableDescriptors().add(modifiedHTableDescriptor);
|
||||
env.getMasterServices().getTableDescriptors().add(modifiedTableDescriptor);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -386,7 +386,7 @@ public class RestoreSnapshotProcedure
|
|||
env.getMasterServices().getConfiguration(),
|
||||
fs,
|
||||
manifest,
|
||||
modifiedHTableDescriptor,
|
||||
modifiedTableDescriptor,
|
||||
rootDir,
|
||||
monitorException,
|
||||
getMonitorStatus());
|
||||
|
@ -440,19 +440,19 @@ public class RestoreSnapshotProcedure
|
|||
MetaTableAccessor.addRegionsToMeta(
|
||||
conn,
|
||||
regionsToAdd,
|
||||
modifiedHTableDescriptor.getRegionReplication());
|
||||
modifiedTableDescriptor.getRegionReplication());
|
||||
}
|
||||
|
||||
if (regionsToRestore != null) {
|
||||
MetaTableAccessor.overwriteRegions(
|
||||
conn,
|
||||
regionsToRestore,
|
||||
modifiedHTableDescriptor.getRegionReplication());
|
||||
modifiedTableDescriptor.getRegionReplication());
|
||||
}
|
||||
|
||||
RestoreSnapshotHelper.RestoreMetaChanges metaChanges =
|
||||
new RestoreSnapshotHelper.RestoreMetaChanges(
|
||||
modifiedHTableDescriptor, parentsToChildrenPairMap);
|
||||
modifiedTableDescriptor, parentsToChildrenPairMap);
|
||||
metaChanges.updateMetaParentRegions(conn, regionsToAdd);
|
||||
|
||||
// At this point the restore is complete.
|
||||
|
|
|
@ -28,11 +28,11 @@ import java.util.List;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.exceptions.HBaseException;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
@ -48,7 +48,7 @@ public class TruncateTableProcedure
|
|||
|
||||
private boolean preserveSplits;
|
||||
private List<HRegionInfo> regions;
|
||||
private HTableDescriptor hTableDescriptor;
|
||||
private TableDescriptor tableDescriptor;
|
||||
private TableName tableName;
|
||||
|
||||
public TruncateTableProcedure() {
|
||||
|
@ -95,7 +95,7 @@ public class TruncateTableProcedure
|
|||
setNextState(TruncateTableState.TRUNCATE_TABLE_REMOVE_FROM_META);
|
||||
break;
|
||||
case TRUNCATE_TABLE_REMOVE_FROM_META:
|
||||
hTableDescriptor = env.getMasterServices().getTableDescriptors()
|
||||
tableDescriptor = env.getMasterServices().getTableDescriptors()
|
||||
.get(tableName);
|
||||
DeleteTableProcedure.deleteFromMeta(env, getTableName(), regions);
|
||||
DeleteTableProcedure.deleteAssignmentState(env, getTableName());
|
||||
|
@ -105,26 +105,26 @@ public class TruncateTableProcedure
|
|||
DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
|
||||
if (!preserveSplits) {
|
||||
// if we are not preserving splits, generate a new single region
|
||||
regions = Arrays.asList(ModifyRegionUtils.createHRegionInfos(hTableDescriptor, null));
|
||||
regions = Arrays.asList(ModifyRegionUtils.createHRegionInfos(tableDescriptor, null));
|
||||
} else {
|
||||
regions = recreateRegionInfo(regions);
|
||||
}
|
||||
setNextState(TruncateTableState.TRUNCATE_TABLE_CREATE_FS_LAYOUT);
|
||||
break;
|
||||
case TRUNCATE_TABLE_CREATE_FS_LAYOUT:
|
||||
regions = CreateTableProcedure.createFsLayout(env, hTableDescriptor, regions);
|
||||
regions = CreateTableProcedure.createFsLayout(env, tableDescriptor, regions);
|
||||
CreateTableProcedure.updateTableDescCache(env, getTableName());
|
||||
setNextState(TruncateTableState.TRUNCATE_TABLE_ADD_TO_META);
|
||||
break;
|
||||
case TRUNCATE_TABLE_ADD_TO_META:
|
||||
regions = CreateTableProcedure.addTableToMeta(env, hTableDescriptor, regions);
|
||||
regions = CreateTableProcedure.addTableToMeta(env, tableDescriptor, regions);
|
||||
setNextState(TruncateTableState.TRUNCATE_TABLE_ASSIGN_REGIONS);
|
||||
break;
|
||||
case TRUNCATE_TABLE_ASSIGN_REGIONS:
|
||||
CreateTableProcedure.setEnablingState(env, getTableName());
|
||||
addChildProcedure(env.getAssignmentManager().createAssignProcedures(regions));
|
||||
setNextState(TruncateTableState.TRUNCATE_TABLE_POST_OPERATION);
|
||||
hTableDescriptor = null;
|
||||
tableDescriptor = null;
|
||||
regions = null;
|
||||
break;
|
||||
case TRUNCATE_TABLE_POST_OPERATION:
|
||||
|
@ -216,8 +216,8 @@ public class TruncateTableProcedure
|
|||
MasterProcedureProtos.TruncateTableStateData.newBuilder()
|
||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||
.setPreserveSplits(preserveSplits);
|
||||
if (hTableDescriptor != null) {
|
||||
state.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
|
||||
if (tableDescriptor != null) {
|
||||
state.setTableSchema(ProtobufUtil.toTableSchema(tableDescriptor));
|
||||
} else {
|
||||
state.setTableName(ProtobufUtil.toProtoTableName(tableName));
|
||||
}
|
||||
|
@ -237,8 +237,8 @@ public class TruncateTableProcedure
|
|||
MasterProcedureProtos.TruncateTableStateData.parseDelimitedFrom(stream);
|
||||
setUser(MasterProcedureUtil.toUserInfo(state.getUserInfo()));
|
||||
if (state.hasTableSchema()) {
|
||||
hTableDescriptor = ProtobufUtil.convertToHTableDesc(state.getTableSchema());
|
||||
tableName = hTableDescriptor.getTableName();
|
||||
tableDescriptor = ProtobufUtil.toTableDescriptor(state.getTableSchema());
|
||||
tableName = tableDescriptor.getTableName();
|
||||
} else {
|
||||
tableName = ProtobufUtil.toTableName(state.getTableName());
|
||||
}
|
||||
|
|
|
@ -30,9 +30,9 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
@ -137,16 +137,16 @@ public final class MasterSnapshotVerifier {
|
|||
* @param manifest snapshot manifest to inspect
|
||||
*/
|
||||
private void verifyTableInfo(final SnapshotManifest manifest) throws IOException {
|
||||
HTableDescriptor htd = manifest.getTableDescriptor();
|
||||
TableDescriptor htd = manifest.getTableDescriptor();
|
||||
if (htd == null) {
|
||||
throw new CorruptedSnapshotException("Missing Table Descriptor",
|
||||
ProtobufUtil.createSnapshotDesc(snapshot));
|
||||
}
|
||||
|
||||
if (!htd.getNameAsString().equals(snapshot.getTable())) {
|
||||
if (!htd.getTableName().getNameAsString().equals(snapshot.getTable())) {
|
||||
throw new CorruptedSnapshotException(
|
||||
"Invalid Table Descriptor. Expected " + snapshot.getTable() + " name, got "
|
||||
+ htd.getNameAsString(), ProtobufUtil.createSnapshotDesc(snapshot));
|
||||
+ htd.getTableName().getNameAsString(), ProtobufUtil.createSnapshotDesc(snapshot));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -38,12 +38,13 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
|
@ -555,7 +556,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
cleanupSentinels();
|
||||
|
||||
// check to see if the table exists
|
||||
HTableDescriptor desc = null;
|
||||
TableDescriptor desc = null;
|
||||
try {
|
||||
desc = master.getTableDescriptors().get(
|
||||
TableName.valueOf(snapshot.getTable()));
|
||||
|
@ -679,10 +680,10 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
* @throws IOException
|
||||
*/
|
||||
private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName,
|
||||
final SnapshotDescription snapshot, final HTableDescriptor snapshotTableDesc,
|
||||
final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc,
|
||||
final NonceKey nonceKey, final boolean restoreAcl) throws IOException {
|
||||
MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName, snapshotTableDesc);
|
||||
TableDescriptor htd = TableDescriptorBuilder.copy(tableName, snapshotTableDesc);
|
||||
if (cpHost != null) {
|
||||
cpHost.preCloneSnapshot(reqSnapshot, htd);
|
||||
}
|
||||
|
@ -707,14 +708,14 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
* The operation will fail if the destination table has a snapshot or restore in progress.
|
||||
*
|
||||
* @param snapshot Snapshot Descriptor
|
||||
* @param hTableDescriptor Table Descriptor of the table to create
|
||||
* @param tableDescriptor Table Descriptor of the table to create
|
||||
* @param nonceKey unique identifier to prevent duplicated RPC
|
||||
* @return procId the ID of the clone snapshot procedure
|
||||
*/
|
||||
synchronized long cloneSnapshot(final SnapshotDescription snapshot,
|
||||
final HTableDescriptor hTableDescriptor, final NonceKey nonceKey, final boolean restoreAcl)
|
||||
final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl)
|
||||
throws HBaseSnapshotException {
|
||||
TableName tableName = hTableDescriptor.getTableName();
|
||||
TableName tableName = tableDescriptor.getTableName();
|
||||
|
||||
// make sure we aren't running a snapshot on the same table
|
||||
if (isTakingSnapshot(tableName)) {
|
||||
|
@ -729,7 +730,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
try {
|
||||
long procId = master.getMasterProcedureExecutor().submitProcedure(
|
||||
new CloneSnapshotProcedure(master.getMasterProcedureExecutor().getEnvironment(),
|
||||
hTableDescriptor, snapshot, restoreAcl),
|
||||
tableDescriptor, snapshot, restoreAcl),
|
||||
nonceKey);
|
||||
this.restoreTableToProcIdMap.put(tableName, procId);
|
||||
return procId;
|
||||
|
@ -765,7 +766,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
SnapshotDescription snapshot = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
|
||||
SnapshotManifest manifest = SnapshotManifest.open(master.getConfiguration(), fs,
|
||||
snapshotDir, snapshot);
|
||||
HTableDescriptor snapshotTableDesc = manifest.getTableDescriptor();
|
||||
TableDescriptor snapshotTableDesc = manifest.getTableDescriptor();
|
||||
TableName tableName = TableName.valueOf(reqSnapshot.getTable());
|
||||
|
||||
// stop tracking "abandoned" handlers
|
||||
|
@ -799,7 +800,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
* @throws IOException
|
||||
*/
|
||||
private long restoreSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName,
|
||||
final SnapshotDescription snapshot, final HTableDescriptor snapshotTableDesc,
|
||||
final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc,
|
||||
final NonceKey nonceKey, final boolean restoreAcl) throws IOException {
|
||||
MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
|
||||
|
||||
|
@ -836,15 +837,15 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
* Restore the specified snapshot. The restore will fail if the destination table has a snapshot
|
||||
* or restore in progress.
|
||||
* @param snapshot Snapshot Descriptor
|
||||
* @param hTableDescriptor Table Descriptor
|
||||
* @param tableDescriptor Table Descriptor
|
||||
* @param nonceKey unique identifier to prevent duplicated RPC
|
||||
* @param restoreAcl true to restore acl of snapshot
|
||||
* @return procId the ID of the restore snapshot procedure
|
||||
*/
|
||||
private synchronized long restoreSnapshot(final SnapshotDescription snapshot,
|
||||
final HTableDescriptor hTableDescriptor, final NonceKey nonceKey, final boolean restoreAcl)
|
||||
final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl)
|
||||
throws HBaseSnapshotException {
|
||||
final TableName tableName = hTableDescriptor.getTableName();
|
||||
final TableName tableName = tableDescriptor.getTableName();
|
||||
|
||||
// make sure we aren't running a snapshot on the same table
|
||||
if (isTakingSnapshot(tableName)) {
|
||||
|
@ -859,7 +860,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
try {
|
||||
long procId = master.getMasterProcedureExecutor().submitProcedure(
|
||||
new RestoreSnapshotProcedure(master.getMasterProcedureExecutor().getEnvironment(),
|
||||
hTableDescriptor, snapshot, restoreAcl),
|
||||
tableDescriptor, snapshot, restoreAcl),
|
||||
nonceKey);
|
||||
this.restoreTableToProcIdMap.put(tableName, procId);
|
||||
return procId;
|
||||
|
|
|
@ -33,9 +33,9 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
|
||||
|
@ -89,7 +89,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
|
|||
protected final SnapshotManifest snapshotManifest;
|
||||
protected final SnapshotManager snapshotManager;
|
||||
|
||||
protected HTableDescriptor htd;
|
||||
protected TableDescriptor htd;
|
||||
|
||||
/**
|
||||
* @param snapshot descriptor of the snapshot to take
|
||||
|
@ -124,12 +124,12 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
|
|||
"Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable);
|
||||
}
|
||||
|
||||
private HTableDescriptor loadTableDescriptor()
|
||||
private TableDescriptor loadTableDescriptor()
|
||||
throws FileNotFoundException, IOException {
|
||||
HTableDescriptor htd =
|
||||
TableDescriptor htd =
|
||||
this.master.getTableDescriptors().get(snapshotTable);
|
||||
if (htd == null) {
|
||||
throw new IOException("HTableDescriptor missing for " + snapshotTable);
|
||||
throw new IOException("TableDescriptor missing for " + snapshotTable);
|
||||
}
|
||||
return htd;
|
||||
}
|
||||
|
|
|
@ -26,15 +26,15 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
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.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -59,7 +59,7 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
|
|||
* @param tableName The current table name.
|
||||
* @param family The current family.
|
||||
*/
|
||||
public void cleanExpiredMobFiles(String tableName, HColumnDescriptor family) throws IOException {
|
||||
public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family) throws IOException {
|
||||
Configuration conf = getConf();
|
||||
TableName tn = TableName.valueOf(tableName);
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
|
@ -98,8 +98,8 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
|
|||
Connection connection = ConnectionFactory.createConnection(getConf());
|
||||
Admin admin = connection.getAdmin();
|
||||
try {
|
||||
HTableDescriptor htd = admin.getTableDescriptor(tn);
|
||||
HColumnDescriptor family = htd.getFamily(Bytes.toBytes(familyName));
|
||||
TableDescriptor htd = admin.listTableDescriptor(tn);
|
||||
ColumnFamilyDescriptor family = htd.getColumnFamily(Bytes.toBytes(familyName));
|
||||
if (family == null || !family.isMobEnabled()) {
|
||||
throw new IOException("Column family " + familyName + " is not a MOB column family");
|
||||
}
|
||||
|
|
|
@ -44,10 +44,8 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
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.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
|
@ -57,6 +55,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.crypto.Encryption;
|
||||
|
@ -286,7 +285,7 @@ public final class MobUtils {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void cleanExpiredMobFiles(FileSystem fs, Configuration conf, TableName tableName,
|
||||
HColumnDescriptor columnDescriptor, CacheConfig cacheConfig, long current)
|
||||
ColumnFamilyDescriptor columnDescriptor, CacheConfig cacheConfig, long current)
|
||||
throws IOException {
|
||||
long timeToLive = columnDescriptor.getTimeToLive();
|
||||
if (Integer.MAX_VALUE == timeToLive) {
|
||||
|
@ -519,7 +518,7 @@ public final class MobUtils {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||
ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||
Compression.Algorithm compression, String startKey, CacheConfig cacheConfig,
|
||||
Encryption.Context cryptoContext, boolean isCompaction)
|
||||
throws IOException {
|
||||
|
@ -543,7 +542,7 @@ public final class MobUtils {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static StoreFileWriter createRefFileWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, Path basePath, long maxKeyCount, CacheConfig cacheConfig,
|
||||
ColumnFamilyDescriptor family, Path basePath, long maxKeyCount, CacheConfig cacheConfig,
|
||||
Encryption.Context cryptoContext, boolean isCompaction)
|
||||
throws IOException {
|
||||
return createWriter(conf, fs, family,
|
||||
|
@ -570,7 +569,7 @@ public final class MobUtils {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||
ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||
Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig,
|
||||
Encryption.Context cryptoContext, boolean isCompaction)
|
||||
throws IOException {
|
||||
|
@ -596,7 +595,7 @@ public final class MobUtils {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static StoreFileWriter createDelFileWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||
ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||
Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig,
|
||||
Encryption.Context cryptoContext)
|
||||
throws IOException {
|
||||
|
@ -623,7 +622,7 @@ public final class MobUtils {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount,
|
||||
ColumnFamilyDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount,
|
||||
Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext,
|
||||
boolean isCompaction)
|
||||
throws IOException {
|
||||
|
@ -797,7 +796,7 @@ public final class MobUtils {
|
|||
* @param allFiles Whether add all mob files into the compaction.
|
||||
*/
|
||||
public static void doMobCompaction(Configuration conf, FileSystem fs, TableName tableName,
|
||||
HColumnDescriptor hcd, ExecutorService pool, boolean allFiles, LockManager.MasterLock lock)
|
||||
ColumnFamilyDescriptor hcd, ExecutorService pool, boolean allFiles, LockManager.MasterLock lock)
|
||||
throws IOException {
|
||||
String className = conf.get(MobConstants.MOB_COMPACTOR_CLASS_KEY,
|
||||
PartitionedMobCompactor.class.getName());
|
||||
|
@ -805,7 +804,7 @@ public final class MobUtils {
|
|||
MobCompactor compactor = null;
|
||||
try {
|
||||
compactor = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {
|
||||
Configuration.class, FileSystem.class, TableName.class, HColumnDescriptor.class,
|
||||
Configuration.class, FileSystem.class, TableName.class, ColumnFamilyDescriptor.class,
|
||||
ExecutorService.class }, new Object[] { conf, fs, tableName, hcd, pool });
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Unable to load configured mob file compactor '" + className + "'", e);
|
||||
|
@ -857,9 +856,9 @@ public final class MobUtils {
|
|||
* @param htd The current table descriptor.
|
||||
* @return Whether this table has mob-enabled columns.
|
||||
*/
|
||||
public static boolean hasMobColumns(HTableDescriptor htd) {
|
||||
HColumnDescriptor[] hcds = htd.getColumnFamilies();
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
public static boolean hasMobColumns(TableDescriptor htd) {
|
||||
ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies();
|
||||
for (ColumnFamilyDescriptor hcd : hcds) {
|
||||
if (hcd.isMobEnabled()) {
|
||||
return true;
|
||||
}
|
||||
|
@ -899,7 +898,7 @@ public final class MobUtils {
|
|||
* @param fileDate The date string parsed from the mob file name.
|
||||
* @return True if the mob file is expired.
|
||||
*/
|
||||
public static boolean isMobFileExpired(HColumnDescriptor column, long current, String fileDate) {
|
||||
public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long current, String fileDate) {
|
||||
if (column.getMinVersions() > 0) {
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -27,9 +27,9 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
|
@ -42,14 +42,14 @@ public abstract class MobCompactor {
|
|||
protected FileSystem fs;
|
||||
protected Configuration conf;
|
||||
protected TableName tableName;
|
||||
protected HColumnDescriptor column;
|
||||
protected ColumnFamilyDescriptor column;
|
||||
|
||||
protected Path mobTableDir;
|
||||
protected Path mobFamilyDir;
|
||||
protected ExecutorService pool;
|
||||
|
||||
public MobCompactor(Configuration conf, FileSystem fs, TableName tableName,
|
||||
HColumnDescriptor column, ExecutorService pool) {
|
||||
ColumnFamilyDescriptor column, ExecutorService pool) {
|
||||
this.conf = conf;
|
||||
this.fs = fs;
|
||||
this.tableName = tableName;
|
||||
|
|
|
@ -45,13 +45,13 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.TagUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
|
||||
|
@ -109,7 +109,7 @@ public class PartitionedMobCompactor extends MobCompactor {
|
|||
private Encryption.Context cryptoContext = Encryption.Context.NONE;
|
||||
|
||||
public PartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName,
|
||||
HColumnDescriptor column, ExecutorService pool) throws IOException {
|
||||
ColumnFamilyDescriptor column, ExecutorService pool) throws IOException {
|
||||
super(conf, fs, tableName, column, pool);
|
||||
mergeableSize = conf.getLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
|
||||
MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);
|
||||
|
|
|
@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
|
|||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.mapreduce.JobUtil;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
|
@ -109,13 +109,13 @@ public class CompactionTool extends Configured implements Tool {
|
|||
if (isFamilyDir(fs, path)) {
|
||||
Path regionDir = path.getParent();
|
||||
Path tableDir = regionDir.getParent();
|
||||
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
|
||||
compactStoreFiles(tableDir, htd, hri,
|
||||
path.getName(), compactOnce, major);
|
||||
} else if (isRegionDir(fs, path)) {
|
||||
Path tableDir = path.getParent();
|
||||
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
compactRegion(tableDir, htd, path, compactOnce, major);
|
||||
} else if (isTableDir(fs, path)) {
|
||||
compactTable(path, compactOnce, major);
|
||||
|
@ -127,13 +127,13 @@ public class CompactionTool extends Configured implements Tool {
|
|||
|
||||
private void compactTable(final Path tableDir, final boolean compactOnce, final boolean major)
|
||||
throws IOException {
|
||||
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
|
||||
compactRegion(tableDir, htd, regionDir, compactOnce, major);
|
||||
}
|
||||
}
|
||||
|
||||
private void compactRegion(final Path tableDir, final HTableDescriptor htd,
|
||||
private void compactRegion(final Path tableDir, final TableDescriptor htd,
|
||||
final Path regionDir, final boolean compactOnce, final boolean major)
|
||||
throws IOException {
|
||||
HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
|
||||
|
@ -147,7 +147,7 @@ public class CompactionTool extends Configured implements Tool {
|
|||
* If the compact once flag is not specified, execute the compaction until
|
||||
* no more compactions are needed. Uses the Configuration settings provided.
|
||||
*/
|
||||
private void compactStoreFiles(final Path tableDir, final HTableDescriptor htd,
|
||||
private void compactStoreFiles(final Path tableDir, final TableDescriptor htd,
|
||||
final HRegionInfo hri, final String familyName, final boolean compactOnce,
|
||||
final boolean major) throws IOException {
|
||||
HStore store = getStore(conf, fs, tableDir, htd, hri, familyName, tmpDir);
|
||||
|
@ -177,7 +177,7 @@ public class CompactionTool extends Configured implements Tool {
|
|||
* the store dir to compact as source.
|
||||
*/
|
||||
private static HStore getStore(final Configuration conf, final FileSystem fs,
|
||||
final Path tableDir, final HTableDescriptor htd, final HRegionInfo hri,
|
||||
final Path tableDir, final TableDescriptor htd, final HRegionInfo hri,
|
||||
final String familyName, final Path tempDir) throws IOException {
|
||||
HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, hri) {
|
||||
@Override
|
||||
|
@ -186,7 +186,7 @@ public class CompactionTool extends Configured implements Tool {
|
|||
}
|
||||
};
|
||||
HRegion region = new HRegion(regionFs, null, conf, htd, null);
|
||||
return new HStore(region, htd.getFamily(Bytes.toBytes(familyName)), conf);
|
||||
return new HStore(region, htd.getColumnFamily(Bytes.toBytes(familyName)), conf);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -41,12 +41,12 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
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.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.Reference;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -340,8 +340,8 @@ public class HRegionFileSystem {
|
|||
* @return true if region has reference file
|
||||
* @throws IOException
|
||||
*/
|
||||
public boolean hasReferences(final HTableDescriptor htd) throws IOException {
|
||||
for (HColumnDescriptor family : htd.getFamilies()) {
|
||||
public boolean hasReferences(final TableDescriptor htd) throws IOException {
|
||||
for (ColumnFamilyDescriptor family : htd.getColumnFamilies()) {
|
||||
if (hasReferences(family.getNameAsString())) {
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -50,6 +50,7 @@ import java.util.concurrent.CountDownLatch;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.function.Function;
|
||||
|
||||
import javax.management.MalformedObjectNameException;
|
||||
import javax.management.ObjectName;
|
||||
|
@ -89,6 +90,7 @@ import org.apache.hadoop.hbase.client.Connection;
|
|||
import org.apache.hadoop.hbase.client.ConnectionUtils;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.locking.EntityLock;
|
||||
import org.apache.hadoop.hbase.client.locking.LockServiceClient;
|
||||
import org.apache.hadoop.hbase.conf.ConfigurationManager;
|
||||
|
@ -700,7 +702,11 @@ public class HRegionServer extends HasThread implements
|
|||
|
||||
protected TableDescriptors getFsTableDescriptors() throws IOException {
|
||||
return new FSTableDescriptors(this.conf,
|
||||
this.fs, this.rootDir, !canUpdateTableDescriptor(), false);
|
||||
this.fs, this.rootDir, !canUpdateTableDescriptor(), false, getMetaTableObserver());
|
||||
}
|
||||
|
||||
protected Function<TableDescriptorBuilder, TableDescriptorBuilder> getMetaTableObserver() {
|
||||
return null;
|
||||
}
|
||||
|
||||
protected void setInitLatch(CountDownLatch latch) {
|
||||
|
|
|
@ -58,7 +58,6 @@ import org.apache.hadoop.hbase.DroppedSnapshotException;
|
|||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MultiActionResultTooLarge;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
@ -77,6 +76,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
|||
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.VersionInfoUtil;
|
||||
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
|
||||
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
|
||||
|
@ -1859,7 +1859,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
|
||||
OpenRegionResponse.Builder builder = OpenRegionResponse.newBuilder();
|
||||
final int regionCount = request.getOpenInfoCount();
|
||||
final Map<TableName, HTableDescriptor> htds = new HashMap<>(regionCount);
|
||||
final Map<TableName, TableDescriptor> htds = new HashMap<>(regionCount);
|
||||
final boolean isBulkAssign = regionCount > 1;
|
||||
try {
|
||||
checkOpen();
|
||||
|
@ -1898,7 +1898,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
|
||||
for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {
|
||||
final HRegionInfo region = HRegionInfo.convert(regionOpenInfo.getRegion());
|
||||
HTableDescriptor htd;
|
||||
TableDescriptor htd;
|
||||
try {
|
||||
String encodedName = region.getEncodedName();
|
||||
byte[] encodedNameBytes = region.getEncodedNameAsBytes();
|
||||
|
@ -2020,7 +2020,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
|
||||
RegionInfo regionInfo = request.getRegionInfo();
|
||||
final HRegionInfo region = HRegionInfo.convert(regionInfo);
|
||||
HTableDescriptor htd;
|
||||
TableDescriptor htd;
|
||||
WarmupRegionResponse response = WarmupRegionResponse.getDefaultInstance();
|
||||
|
||||
try {
|
||||
|
|
|
@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.regionserver.handler;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
|
||||
|
@ -33,8 +33,8 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
|||
@InterfaceAudience.Private
|
||||
public class OpenMetaHandler extends OpenRegionHandler {
|
||||
public OpenMetaHandler(final Server server,
|
||||
final RegionServerServices rsServices, HRegionInfo regionInfo,
|
||||
final HTableDescriptor htd, long masterSystemTime) {
|
||||
final RegionServerServices rsServices, HRegionInfo regionInfo,
|
||||
final TableDescriptor htd, long masterSystemTime) {
|
||||
super(server, rsServices, regionInfo, htd, masterSystemTime, EventType.M_RS_OPEN_META);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,11 +19,10 @@
|
|||
package org.apache.hadoop.hbase.regionserver.handler;
|
||||
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
|
||||
/**
|
||||
|
@ -34,7 +33,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
|||
@InterfaceAudience.Private
|
||||
public class OpenPriorityRegionHandler extends OpenRegionHandler {
|
||||
public OpenPriorityRegionHandler(Server server, RegionServerServices rsServices,
|
||||
HRegionInfo regionInfo, HTableDescriptor htd, long masterSystemTime) {
|
||||
HRegionInfo regionInfo, TableDescriptor htd, long masterSystemTime) {
|
||||
super(server, rsServices, regionInfo, htd, masterSystemTime,
|
||||
EventType.M_RS_OPEN_PRIORITY_REGION);
|
||||
}
|
||||
|
|
|
@ -25,8 +25,8 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||
|
@ -48,18 +48,18 @@ public class OpenRegionHandler extends EventHandler {
|
|||
protected final RegionServerServices rsServices;
|
||||
|
||||
private final HRegionInfo regionInfo;
|
||||
private final HTableDescriptor htd;
|
||||
private final TableDescriptor htd;
|
||||
private final long masterSystemTime;
|
||||
|
||||
public OpenRegionHandler(final Server server,
|
||||
final RegionServerServices rsServices, HRegionInfo regionInfo,
|
||||
HTableDescriptor htd, long masterSystemTime) {
|
||||
TableDescriptor htd, long masterSystemTime) {
|
||||
this(server, rsServices, regionInfo, htd, masterSystemTime, EventType.M_RS_OPEN_REGION);
|
||||
}
|
||||
|
||||
protected OpenRegionHandler(final Server server,
|
||||
final RegionServerServices rsServices, final HRegionInfo regionInfo,
|
||||
final HTableDescriptor htd, long masterSystemTime, EventType eventType) {
|
||||
final RegionServerServices rsServices, final HRegionInfo regionInfo,
|
||||
final TableDescriptor htd, long masterSystemTime, EventType eventType) {
|
||||
super(server, eventType);
|
||||
this.rsServices = rsServices;
|
||||
this.regionInfo = regionInfo;
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.HBaseIOException;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -55,6 +54,7 @@ import org.apache.hadoop.hbase.client.RegionAdminServiceCallable;
|
|||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.RetryingCallable;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
|
||||
|
@ -399,7 +399,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
|
|||
if (requiresReplication == null) {
|
||||
// check if the table requires memstore replication
|
||||
// some unit-test drop the table, so we should do a bypass check and always replicate.
|
||||
HTableDescriptor htd = tableDescriptors.get(tableName);
|
||||
TableDescriptor htd = tableDescriptors.get(tableName);
|
||||
requiresReplication = htd == null || htd.hasRegionMemstoreReplication();
|
||||
memstoreReplicationEnabled.put(tableName, requiresReplication);
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ import java.util.Set;
|
|||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -43,7 +44,6 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
|
@ -125,7 +125,7 @@ public class RestoreSnapshotHelper {
|
|||
private final SnapshotDescription snapshotDesc;
|
||||
private final TableName snapshotTable;
|
||||
|
||||
private final HTableDescriptor tableDesc;
|
||||
private final TableDescriptor tableDesc;
|
||||
private final Path rootDir;
|
||||
private final Path tableDir;
|
||||
|
||||
|
@ -136,7 +136,7 @@ public class RestoreSnapshotHelper {
|
|||
public RestoreSnapshotHelper(final Configuration conf,
|
||||
final FileSystem fs,
|
||||
final SnapshotManifest manifest,
|
||||
final HTableDescriptor tableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final Path rootDir,
|
||||
final ForeignExceptionDispatcher monitor,
|
||||
final MonitoredTask status) {
|
||||
|
@ -146,7 +146,7 @@ public class RestoreSnapshotHelper {
|
|||
public RestoreSnapshotHelper(final Configuration conf,
|
||||
final FileSystem fs,
|
||||
final SnapshotManifest manifest,
|
||||
final HTableDescriptor tableDescriptor,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final Path rootDir,
|
||||
final ForeignExceptionDispatcher monitor,
|
||||
final MonitoredTask status,
|
||||
|
@ -265,18 +265,18 @@ public class RestoreSnapshotHelper {
|
|||
*/
|
||||
public static class RestoreMetaChanges {
|
||||
private final Map<String, Pair<String, String> > parentsMap;
|
||||
private final HTableDescriptor htd;
|
||||
private final TableDescriptor htd;
|
||||
|
||||
private List<HRegionInfo> regionsToRestore = null;
|
||||
private List<HRegionInfo> regionsToRemove = null;
|
||||
private List<HRegionInfo> regionsToAdd = null;
|
||||
|
||||
public RestoreMetaChanges(HTableDescriptor htd, Map<String, Pair<String, String> > parentsMap) {
|
||||
public RestoreMetaChanges(TableDescriptor htd, Map<String, Pair<String, String> > parentsMap) {
|
||||
this.parentsMap = parentsMap;
|
||||
this.htd = htd;
|
||||
}
|
||||
|
||||
public HTableDescriptor getTableDescriptor() {
|
||||
public TableDescriptor getTableDescriptor() {
|
||||
return htd;
|
||||
}
|
||||
|
||||
|
|
|
@ -36,10 +36,10 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -76,7 +76,7 @@ public final class SnapshotManifest {
|
|||
|
||||
private List<SnapshotRegionManifest> regionManifests;
|
||||
private SnapshotDescription desc;
|
||||
private HTableDescriptor htd;
|
||||
private TableDescriptor htd;
|
||||
|
||||
private final ForeignExceptionSnare monitor;
|
||||
private final Configuration conf;
|
||||
|
@ -119,7 +119,7 @@ public final class SnapshotManifest {
|
|||
/**
|
||||
* Return a SnapshotManifest instance with the information already loaded in-memory.
|
||||
* SnapshotManifest manifest = SnapshotManifest.open(...)
|
||||
* HTableDescriptor htd = manifest.getTableDescriptor()
|
||||
* TableDescriptor htd = manifest.getTableDescriptor()
|
||||
* for (SnapshotRegionManifest regionManifest: manifest.getRegionManifests())
|
||||
* hri = regionManifest.getRegionInfo()
|
||||
* for (regionManifest.getFamilyFiles())
|
||||
|
@ -136,7 +136,7 @@ public final class SnapshotManifest {
|
|||
/**
|
||||
* Add the table descriptor to the snapshot manifest
|
||||
*/
|
||||
public void addTableDescriptor(final HTableDescriptor htd) throws IOException {
|
||||
public void addTableDescriptor(final TableDescriptor htd) throws IOException {
|
||||
this.htd = htd;
|
||||
}
|
||||
|
||||
|
@ -182,7 +182,7 @@ public final class SnapshotManifest {
|
|||
LOG.debug("Creating references for mob files");
|
||||
|
||||
Path mobRegionPath = MobUtils.getMobRegionPath(conf, regionInfo.getTable());
|
||||
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||
// 2.1. build the snapshot reference for the store if it's a mob store
|
||||
if (!hcd.isMobEnabled()) {
|
||||
continue;
|
||||
|
@ -377,7 +377,7 @@ public final class SnapshotManifest {
|
|||
case SnapshotManifestV2.DESCRIPTOR_VERSION: {
|
||||
SnapshotDataManifest dataManifest = readDataManifest();
|
||||
if (dataManifest != null) {
|
||||
htd = ProtobufUtil.convertToHTableDesc(dataManifest.getTableSchema());
|
||||
htd = ProtobufUtil.toTableDescriptor(dataManifest.getTableSchema());
|
||||
regionManifests = dataManifest.getRegionManifestsList();
|
||||
} else {
|
||||
// Compatibility, load the v1 regions
|
||||
|
@ -429,7 +429,7 @@ public final class SnapshotManifest {
|
|||
/**
|
||||
* Get the table descriptor from the Snapshot
|
||||
*/
|
||||
public HTableDescriptor getTableDescriptor() {
|
||||
public TableDescriptor getTableDescriptor() {
|
||||
return this.htd;
|
||||
}
|
||||
|
||||
|
@ -485,7 +485,7 @@ public final class SnapshotManifest {
|
|||
}
|
||||
|
||||
SnapshotDataManifest.Builder dataManifestBuilder = SnapshotDataManifest.newBuilder();
|
||||
dataManifestBuilder.setTableSchema(ProtobufUtil.convertToTableSchema(htd));
|
||||
dataManifestBuilder.setTableSchema(ProtobufUtil.toTableSchema(htd));
|
||||
|
||||
if (v1Regions != null && v1Regions.size() > 0) {
|
||||
dataManifestBuilder.addAllRegionManifests(v1Regions);
|
||||
|
|
|
@ -24,11 +24,10 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.function.Function;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.primitives.Ints;
|
||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
||||
import org.apache.commons.lang.NotImplementedException;
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -40,17 +39,19 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
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.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.primitives.Ints;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableInfoMissingException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
|
||||
/**
|
||||
* Implementation of {@link TableDescriptors} that reads descriptors from the
|
||||
|
@ -79,10 +80,14 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
private volatile boolean usecache;
|
||||
private volatile boolean fsvisited;
|
||||
|
||||
@VisibleForTesting long cachehits = 0;
|
||||
@VisibleForTesting long invocations = 0;
|
||||
@VisibleForTesting
|
||||
long cachehits = 0;
|
||||
@VisibleForTesting
|
||||
long invocations = 0;
|
||||
|
||||
/** The file name prefix used to store HTD in HDFS */
|
||||
/**
|
||||
* The file name prefix used to store HTD in HDFS
|
||||
*/
|
||||
static final String TABLEINFO_FILE_PREFIX = ".tableinfo";
|
||||
static final String TABLEINFO_DIR = ".tabledesc";
|
||||
static final String TMP_DIR = ".tmp";
|
||||
|
@ -90,12 +95,12 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
// This cache does not age out the old stuff. Thinking is that the amount
|
||||
// of data we keep up in here is so small, no need to do occasional purge.
|
||||
// TODO.
|
||||
private final Map<TableName, HTableDescriptor> cache = new ConcurrentHashMap<>();
|
||||
private final Map<TableName, TableDescriptor> cache = new ConcurrentHashMap<>();
|
||||
|
||||
/**
|
||||
* Table descriptor for <code>hbase:meta</code> catalog table
|
||||
*/
|
||||
private final HTableDescriptor metaTableDescriptor;
|
||||
private final TableDescriptor metaTableDescriptor;
|
||||
|
||||
/**
|
||||
* Construct a FSTableDescriptors instance using the hbase root dir of the given
|
||||
|
@ -107,91 +112,112 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
}
|
||||
|
||||
public FSTableDescriptors(final Configuration conf, final FileSystem fs, final Path rootdir)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
this(conf, fs, rootdir, false, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param fsreadonly True if we are read-only when it comes to filesystem
|
||||
* operations; i.e. on remove, we do not do delete in fs.
|
||||
* operations; i.e. on remove, we do not do delete in fs.
|
||||
*/
|
||||
public FSTableDescriptors(final Configuration conf, final FileSystem fs,
|
||||
final Path rootdir, final boolean fsreadonly, final boolean usecache) throws IOException {
|
||||
super();
|
||||
final Path rootdir, final boolean fsreadonly, final boolean usecache) throws IOException {
|
||||
this(conf, fs, rootdir, fsreadonly, usecache, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param fsreadonly True if we are read-only when it comes to filesystem
|
||||
* operations; i.e. on remove, we do not do delete in fs.
|
||||
* @param metaObserver Used by HMaster. It need to modify the META_REPLICAS_NUM for meta table descriptor.
|
||||
* see HMaster#finishActiveMasterInitialization
|
||||
* TODO: This is a workaround. Should remove this ugly code...
|
||||
*/
|
||||
public FSTableDescriptors(final Configuration conf, final FileSystem fs,
|
||||
final Path rootdir, final boolean fsreadonly, final boolean usecache,
|
||||
Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
|
||||
this.fs = fs;
|
||||
this.rootdir = rootdir;
|
||||
this.fsreadonly = fsreadonly;
|
||||
this.usecache = usecache;
|
||||
|
||||
this.metaTableDescriptor = createMetaTableDescriptor(conf);
|
||||
this.metaTableDescriptor = metaObserver == null ? createMetaTableDescriptor(conf)
|
||||
: metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static HTableDescriptor createMetaTableDescriptor(final Configuration conf)
|
||||
public static TableDescriptorBuilder createMetaTableDescriptorBuilder(final Configuration conf) throws IOException {
|
||||
return TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY)
|
||||
.setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
|
||||
HConstants.DEFAULT_HBASE_META_VERSIONS))
|
||||
.setInMemory(true)
|
||||
.setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
|
||||
HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true)
|
||||
.build())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.REPLICATION_BARRIER_FAMILY)
|
||||
.setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
|
||||
HConstants.DEFAULT_HBASE_META_VERSIONS))
|
||||
.setInMemory(true)
|
||||
.setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
|
||||
HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true)
|
||||
.build())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.REPLICATION_POSITION_FAMILY)
|
||||
.setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
|
||||
HConstants.DEFAULT_HBASE_META_VERSIONS))
|
||||
.setInMemory(true)
|
||||
.setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
|
||||
HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true)
|
||||
.build())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.REPLICATION_META_FAMILY)
|
||||
.setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
|
||||
HConstants.DEFAULT_HBASE_META_VERSIONS))
|
||||
.setInMemory(true)
|
||||
.setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
|
||||
HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true)
|
||||
.build())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.TABLE_FAMILY)
|
||||
// Ten is arbitrary number. Keep versions to help debugging.
|
||||
.setMaxVersions(10)
|
||||
.setInMemory(true)
|
||||
.setBlocksize(8 * 1024)
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true)
|
||||
.build())
|
||||
.addCoprocessor("org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
|
||||
null, Coprocessor.PRIORITY_SYSTEM, null);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static TableDescriptor createMetaTableDescriptor(final Configuration conf)
|
||||
throws IOException {
|
||||
return new HTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
|
||||
.addColumnFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)
|
||||
.setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
|
||||
HConstants.DEFAULT_HBASE_META_VERSIONS))
|
||||
.setInMemory(true)
|
||||
.setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
|
||||
HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true))
|
||||
.addColumnFamily(new HColumnDescriptor(HConstants.REPLICATION_BARRIER_FAMILY)
|
||||
.setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
|
||||
HConstants.DEFAULT_HBASE_META_VERSIONS))
|
||||
.setInMemory(true)
|
||||
.setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
|
||||
HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true))
|
||||
.addColumnFamily(new HColumnDescriptor(HConstants.REPLICATION_POSITION_FAMILY)
|
||||
.setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
|
||||
HConstants.DEFAULT_HBASE_META_VERSIONS))
|
||||
.setInMemory(true)
|
||||
.setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
|
||||
HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true))
|
||||
.addColumnFamily(new HColumnDescriptor(HConstants.REPLICATION_META_FAMILY)
|
||||
.setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
|
||||
HConstants.DEFAULT_HBASE_META_VERSIONS))
|
||||
.setInMemory(true)
|
||||
.setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
|
||||
HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true))
|
||||
.addColumnFamily(new HColumnDescriptor(HConstants.TABLE_FAMILY)
|
||||
// Ten is arbitrary number. Keep versions to help debugging.
|
||||
.setMaxVersions(10)
|
||||
.setInMemory(true)
|
||||
.setBlocksize(8 * 1024)
|
||||
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
|
||||
// Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
// Enable cache of data blocks in L1 if more than one caching tier deployed:
|
||||
// e.g. if using CombinedBlockCache (BucketCache).
|
||||
.setCacheDataInL1(true))
|
||||
.addCoprocessor("org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
|
||||
null, Coprocessor.PRIORITY_SYSTEM, null)
|
||||
.build());
|
||||
return createMetaTableDescriptorBuilder(conf).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -219,7 +245,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
*/
|
||||
@Override
|
||||
@Nullable
|
||||
public HTableDescriptor get(final TableName tablename)
|
||||
public TableDescriptor get(final TableName tablename)
|
||||
throws IOException {
|
||||
invocations++;
|
||||
if (TableName.META_TABLE_NAME.equals(tablename)) {
|
||||
|
@ -234,13 +260,13 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
|
||||
if (usecache) {
|
||||
// Look in cache of descriptors.
|
||||
HTableDescriptor cachedtdm = this.cache.get(tablename);
|
||||
TableDescriptor cachedtdm = this.cache.get(tablename);
|
||||
if (cachedtdm != null) {
|
||||
cachehits++;
|
||||
return cachedtdm;
|
||||
}
|
||||
}
|
||||
HTableDescriptor tdmt = null;
|
||||
TableDescriptor tdmt = null;
|
||||
try {
|
||||
tdmt = getTableDescriptorFromFs(fs, rootdir, tablename);
|
||||
} catch (NullPointerException e) {
|
||||
|
@ -264,21 +290,21 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* Returns a map from table name to table descriptor for all tables.
|
||||
*/
|
||||
@Override
|
||||
public Map<String, HTableDescriptor> getAllDescriptors()
|
||||
public Map<String, TableDescriptor> getAllDescriptors()
|
||||
throws IOException {
|
||||
Map<String, HTableDescriptor> tds = new TreeMap<>();
|
||||
Map<String, TableDescriptor> tds = new TreeMap<>();
|
||||
|
||||
if (fsvisited && usecache) {
|
||||
for (Map.Entry<TableName, HTableDescriptor> entry: this.cache.entrySet()) {
|
||||
for (Map.Entry<TableName, TableDescriptor> entry: this.cache.entrySet()) {
|
||||
tds.put(entry.getKey().toString(), entry.getValue());
|
||||
}
|
||||
// add hbase:meta to the response
|
||||
tds.put(this.metaTableDescriptor.getNameAsString(), metaTableDescriptor);
|
||||
tds.put(this.metaTableDescriptor.getTableName().getNameAsString(), metaTableDescriptor);
|
||||
} else {
|
||||
LOG.debug("Fetching table descriptors from the filesystem.");
|
||||
boolean allvisited = true;
|
||||
for (Path d : FSUtils.getTableDirs(fs, rootdir)) {
|
||||
HTableDescriptor htd = null;
|
||||
TableDescriptor htd = null;
|
||||
try {
|
||||
htd = get(FSUtils.getTableName(d));
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
|
@ -301,10 +327,10 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* Returns a map from table name to table descriptor for all tables.
|
||||
*/
|
||||
@Override
|
||||
public Map<String, HTableDescriptor> getAll() throws IOException {
|
||||
Map<String, HTableDescriptor> htds = new TreeMap<>();
|
||||
Map<String, HTableDescriptor> allDescriptors = getAllDescriptors();
|
||||
for (Map.Entry<String, HTableDescriptor> entry : allDescriptors
|
||||
public Map<String, TableDescriptor> getAll() throws IOException {
|
||||
Map<String, TableDescriptor> htds = new TreeMap<>();
|
||||
Map<String, TableDescriptor> allDescriptors = getAllDescriptors();
|
||||
for (Map.Entry<String, TableDescriptor> entry : allDescriptors
|
||||
.entrySet()) {
|
||||
htds.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
|
@ -316,13 +342,13 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* @see #get(org.apache.hadoop.hbase.TableName)
|
||||
*/
|
||||
@Override
|
||||
public Map<String, HTableDescriptor> getByNamespace(String name)
|
||||
public Map<String, TableDescriptor> getByNamespace(String name)
|
||||
throws IOException {
|
||||
Map<String, HTableDescriptor> htds = new TreeMap<>();
|
||||
Map<String, TableDescriptor> htds = new TreeMap<>();
|
||||
List<Path> tableDirs =
|
||||
FSUtils.getLocalTableDirs(fs, FSUtils.getNamespaceDir(rootdir, name));
|
||||
for (Path d: tableDirs) {
|
||||
HTableDescriptor htd = null;
|
||||
TableDescriptor htd = null;
|
||||
try {
|
||||
htd = get(FSUtils.getTableName(d));
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
|
@ -340,7 +366,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* and updates the local cache with it.
|
||||
*/
|
||||
@Override
|
||||
public void add(HTableDescriptor htd) throws IOException {
|
||||
public void add(TableDescriptor htd) throws IOException {
|
||||
if (fsreadonly) {
|
||||
throw new NotImplementedException("Cannot add a table descriptor - in read only mode");
|
||||
}
|
||||
|
@ -351,7 +377,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tableName.getNameAsString())) {
|
||||
throw new NotImplementedException(
|
||||
"Cannot add a table descriptor for a reserved subdirectory name: "
|
||||
+ htd.getNameAsString());
|
||||
+ htd.getTableName().getNameAsString());
|
||||
}
|
||||
updateTableDescriptor(htd);
|
||||
}
|
||||
|
@ -362,7 +388,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* from the FileSystem.
|
||||
*/
|
||||
@Override
|
||||
public HTableDescriptor remove(final TableName tablename)
|
||||
public TableDescriptor remove(final TableName tablename)
|
||||
throws IOException {
|
||||
if (fsreadonly) {
|
||||
throw new NotImplementedException("Cannot remove a table descriptor - in read only mode");
|
||||
|
@ -373,7 +399,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
throw new IOException("Failed delete of " + tabledir.toString());
|
||||
}
|
||||
}
|
||||
HTableDescriptor descriptor = this.cache.remove(tablename);
|
||||
TableDescriptor descriptor = this.cache.remove(tablename);
|
||||
return descriptor;
|
||||
}
|
||||
|
||||
|
@ -557,7 +583,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* if it exists, bypassing the local cache.
|
||||
* Returns null if it's not found.
|
||||
*/
|
||||
public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs,
|
||||
public static TableDescriptor getTableDescriptorFromFs(FileSystem fs,
|
||||
Path hbaseRootDir, TableName tableName) throws IOException {
|
||||
Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
|
||||
return getTableDescriptorFromFs(fs, tableDir);
|
||||
|
@ -568,7 +594,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* directly from the file system if it exists.
|
||||
* @throws TableInfoMissingException if there is no descriptor
|
||||
*/
|
||||
public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs, Path tableDir)
|
||||
public static TableDescriptor getTableDescriptorFromFs(FileSystem fs, Path tableDir)
|
||||
throws IOException {
|
||||
FileStatus status = getTableInfoPath(fs, tableDir, false);
|
||||
if (status == null) {
|
||||
|
@ -577,7 +603,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
return readTableDescriptor(fs, status);
|
||||
}
|
||||
|
||||
private static HTableDescriptor readTableDescriptor(FileSystem fs, FileStatus status)
|
||||
private static TableDescriptor readTableDescriptor(FileSystem fs, FileStatus status)
|
||||
throws IOException {
|
||||
int len = Ints.checkedCast(status.getLen());
|
||||
byte [] content = new byte[len];
|
||||
|
@ -587,9 +613,9 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
} finally {
|
||||
fsDataInputStream.close();
|
||||
}
|
||||
HTableDescriptor htd = null;
|
||||
TableDescriptor htd = null;
|
||||
try {
|
||||
htd = HTableDescriptor.parseFrom(content);
|
||||
htd = TableDescriptorBuilder.parseFrom(content);
|
||||
} catch (DeserializationException e) {
|
||||
throw new IOException("content=" + Bytes.toShort(content), e);
|
||||
}
|
||||
|
@ -601,7 +627,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* @throws IOException Thrown if failed update.
|
||||
* @throws NotImplementedException if in read only mode
|
||||
*/
|
||||
@VisibleForTesting Path updateTableDescriptor(HTableDescriptor td)
|
||||
@VisibleForTesting Path updateTableDescriptor(TableDescriptor td)
|
||||
throws IOException {
|
||||
if (fsreadonly) {
|
||||
throw new NotImplementedException("Cannot update a table descriptor - in read only mode");
|
||||
|
@ -663,7 +689,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* @return Descriptor file or null if we failed write.
|
||||
*/
|
||||
private static Path writeTableDescriptor(final FileSystem fs,
|
||||
final HTableDescriptor htd, final Path tableDir,
|
||||
final TableDescriptor htd, final Path tableDir,
|
||||
final FileStatus currentDescriptorFile)
|
||||
throws IOException {
|
||||
// Get temporary dir into which we'll first write a file to avoid half-written file phenomenon.
|
||||
|
@ -718,42 +744,42 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
return tableInfoDirPath;
|
||||
}
|
||||
|
||||
private static void writeTD(final FileSystem fs, final Path p, final HTableDescriptor htd)
|
||||
private static void writeTD(final FileSystem fs, final Path p, final TableDescriptor htd)
|
||||
throws IOException {
|
||||
FSDataOutputStream out = fs.create(p, false);
|
||||
try {
|
||||
// We used to write this file out as a serialized HTD Writable followed by two '\n's and then
|
||||
// the toString version of HTD. Now we just write out the pb serialization.
|
||||
out.write(htd.toByteArray());
|
||||
out.write(TableDescriptorBuilder.toByteArray(htd));
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create new HTableDescriptor in HDFS. Happens when we are creating table.
|
||||
* Create new TableDescriptor in HDFS. Happens when we are creating table.
|
||||
* Used by tests.
|
||||
* @return True if we successfully created file.
|
||||
*/
|
||||
public boolean createTableDescriptor(HTableDescriptor htd) throws IOException {
|
||||
public boolean createTableDescriptor(TableDescriptor htd) throws IOException {
|
||||
return createTableDescriptor(htd, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create new HTableDescriptor in HDFS. Happens when we are creating table. If
|
||||
* Create new TableDescriptor in HDFS. Happens when we are creating table. If
|
||||
* forceCreation is true then even if previous table descriptor is present it
|
||||
* will be overwritten
|
||||
*
|
||||
* @return True if we successfully created file.
|
||||
*/
|
||||
public boolean createTableDescriptor(HTableDescriptor htd, boolean forceCreation)
|
||||
public boolean createTableDescriptor(TableDescriptor htd, boolean forceCreation)
|
||||
throws IOException {
|
||||
Path tableDir = getTableDir(htd.getTableName());
|
||||
return createTableDescriptorForTableDirectory(tableDir, htd, forceCreation);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new HTableDescriptor in HDFS in the specified table directory. Happens when we create
|
||||
* Create a new TableDescriptor in HDFS in the specified table directory. Happens when we create
|
||||
* a new table or snapshot a table.
|
||||
* @param tableDir table directory under which we should write the file
|
||||
* @param htd description of the table to write
|
||||
|
@ -764,7 +790,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* @throws IOException if a filesystem error occurs
|
||||
*/
|
||||
public boolean createTableDescriptorForTableDirectory(Path tableDir,
|
||||
HTableDescriptor htd, boolean forceCreation) throws IOException {
|
||||
TableDescriptor htd, boolean forceCreation) throws IOException {
|
||||
if (fsreadonly) {
|
||||
throw new NotImplementedException("Cannot create a table descriptor - in read only mode");
|
||||
}
|
||||
|
|
|
@ -17,6 +17,10 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
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.shaded.com.google.common.base.Joiner;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
|
||||
|
@ -84,11 +88,9 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
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.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
|
@ -939,7 +941,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
TableName tableName = hi.getTableName();
|
||||
TableInfo tableInfo = tablesInfo.get(tableName);
|
||||
Preconditions.checkNotNull(tableInfo, "Table '" + tableName + "' not present!");
|
||||
HTableDescriptor template = tableInfo.getHTD();
|
||||
TableDescriptor template = tableInfo.getHTD();
|
||||
|
||||
// find min and max key values
|
||||
Pair<byte[],byte[]> orphanRegionRange = null;
|
||||
|
@ -1200,17 +1202,17 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
*/
|
||||
private void reportTablesInFlux() {
|
||||
AtomicInteger numSkipped = new AtomicInteger(0);
|
||||
HTableDescriptor[] allTables = getTables(numSkipped);
|
||||
TableDescriptor[] allTables = getTables(numSkipped);
|
||||
errors.print("Number of Tables: " + allTables.length);
|
||||
if (details) {
|
||||
if (numSkipped.get() > 0) {
|
||||
errors.detail("Number of Tables in flux: " + numSkipped.get());
|
||||
}
|
||||
for (HTableDescriptor td : allTables) {
|
||||
for (TableDescriptor td : allTables) {
|
||||
errors.detail(" Table: " + td.getTableName() + "\t" +
|
||||
(td.isReadOnly() ? "ro" : "rw") + "\t" +
|
||||
(td.isMetaRegion() ? "META" : " ") + "\t" +
|
||||
" families: " + td.getFamilies().size());
|
||||
" families: " + td.getColumnFamilyCount());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1314,7 +1316,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
modTInfo = new TableInfo(tableName);
|
||||
tablesInfo.put(tableName, modTInfo);
|
||||
try {
|
||||
HTableDescriptor htd =
|
||||
TableDescriptor htd =
|
||||
FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName);
|
||||
modTInfo.htds.add(htd);
|
||||
} catch (IOException ioe) {
|
||||
|
@ -1361,17 +1363,17 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
* To fabricate a .tableinfo file with following contents<br>
|
||||
* 1. the correct tablename <br>
|
||||
* 2. the correct colfamily list<br>
|
||||
* 3. the default properties for both {@link HTableDescriptor} and {@link HColumnDescriptor}<br>
|
||||
* 3. the default properties for both {@link TableDescriptor} and {@link ColumnFamilyDescriptor}<br>
|
||||
* @throws IOException
|
||||
*/
|
||||
private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName tableName,
|
||||
Set<String> columns) throws IOException {
|
||||
if (columns ==null || columns.isEmpty()) return false;
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
for (String columnfamimly : columns) {
|
||||
htd.addFamily(new HColumnDescriptor(columnfamimly));
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(columnfamimly));
|
||||
}
|
||||
fstd.createTableDescriptor(htd, true);
|
||||
fstd.createTableDescriptor(builder.build(), true);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -1396,7 +1398,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
* 2. else create a default .tableinfo file with following items<br>
|
||||
* 2.1 the correct tablename <br>
|
||||
* 2.2 the correct colfamily list<br>
|
||||
* 2.3 the default properties for both {@link HTableDescriptor} and {@link HColumnDescriptor}<br>
|
||||
* 2.3 the default properties for both {@link TableDescriptor} and {@link ColumnFamilyDescriptor}<br>
|
||||
* @throws IOException
|
||||
*/
|
||||
public void fixOrphanTables() throws IOException {
|
||||
|
@ -1404,7 +1406,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
|
||||
List<TableName> tmpList = new ArrayList<>(orphanTableDirs.keySet().size());
|
||||
tmpList.addAll(orphanTableDirs.keySet());
|
||||
HTableDescriptor[] htds = getHTableDescriptors(tmpList);
|
||||
TableDescriptor[] htds = getTableDescriptors(tmpList);
|
||||
Iterator<Entry<TableName, Set<String>>> iter =
|
||||
orphanTableDirs.entrySet().iterator();
|
||||
int j = 0;
|
||||
|
@ -1417,7 +1419,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
LOG.info("Trying to fix orphan table error: " + tableName);
|
||||
if (j < htds.length) {
|
||||
if (tableName.equals(htds[j].getTableName())) {
|
||||
HTableDescriptor htd = htds[j];
|
||||
TableDescriptor htd = htds[j];
|
||||
LOG.info("fixing orphan table: " + tableName + " from cache");
|
||||
fstd.createTableDescriptor(htd, true);
|
||||
j++;
|
||||
|
@ -1426,7 +1428,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
} else {
|
||||
if (fabricateTableInfo(fstd, tableName, entry.getValue())) {
|
||||
LOG.warn("fixing orphan table: " + tableName + " with a default .tableinfo file");
|
||||
LOG.warn("Strongly recommend to modify the HTableDescriptor if necessary for: " + tableName);
|
||||
LOG.warn("Strongly recommend to modify the TableDescriptor if necessary for: " + tableName);
|
||||
iter.remove();
|
||||
} else {
|
||||
LOG.error("Unable to create default .tableinfo for " + tableName + " while missing column family information");
|
||||
|
@ -1463,7 +1465,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
Path rootdir = FSUtils.getRootDir(getConf());
|
||||
Configuration c = getConf();
|
||||
HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
HTableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
|
||||
TableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
|
||||
MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, false);
|
||||
// The WAL subsystem will use the default rootDir rather than the passed in rootDir
|
||||
// unless I pass along via the conf.
|
||||
|
@ -2646,8 +2648,8 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
* regions reported for the table, but table dir is there in hdfs
|
||||
*/
|
||||
private void loadTableInfosForTablesWithNoRegion() throws IOException {
|
||||
Map<String, HTableDescriptor> allTables = new FSTableDescriptors(getConf()).getAll();
|
||||
for (HTableDescriptor htd : allTables.values()) {
|
||||
Map<String, TableDescriptor> allTables = new FSTableDescriptors(getConf()).getAll();
|
||||
for (TableDescriptor htd : allTables.values()) {
|
||||
if (checkMetaOnly && !htd.isMetaTable()) {
|
||||
continue;
|
||||
}
|
||||
|
@ -2770,8 +2772,8 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
// region split calculator
|
||||
final RegionSplitCalculator<HbckInfo> sc = new RegionSplitCalculator<>(cmp);
|
||||
|
||||
// Histogram of different HTableDescriptors found. Ideally there is only one!
|
||||
final Set<HTableDescriptor> htds = new HashSet<>();
|
||||
// Histogram of different TableDescriptors found. Ideally there is only one!
|
||||
final Set<TableDescriptor> htds = new HashSet<>();
|
||||
|
||||
// key = start split, values = set of splits in problem group
|
||||
final Multimap<byte[], HbckInfo> overlapGroups =
|
||||
|
@ -2788,9 +2790,9 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
/**
|
||||
* @return descriptor common to all regions. null if are none or multiple!
|
||||
*/
|
||||
private HTableDescriptor getHTD() {
|
||||
private TableDescriptor getHTD() {
|
||||
if (htds.size() == 1) {
|
||||
return (HTableDescriptor)htds.toArray()[0];
|
||||
return (TableDescriptor)htds.toArray()[0];
|
||||
} else {
|
||||
LOG.error("None/Multiple table descriptors found for table '"
|
||||
+ tableName + "' regions: " + htds);
|
||||
|
@ -2960,7 +2962,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
"First region should start with an empty key. Creating a new " +
|
||||
"region and regioninfo in HDFS to plug the hole.",
|
||||
getTableInfo(), next);
|
||||
HTableDescriptor htd = getTableInfo().getHTD();
|
||||
TableDescriptor htd = getTableInfo().getHTD();
|
||||
// from special EMPTY_START_ROW to next region's startKey
|
||||
HRegionInfo newRegion = new HRegionInfo(htd.getTableName(),
|
||||
HConstants.EMPTY_START_ROW, next.getStartKey());
|
||||
|
@ -2977,7 +2979,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY,
|
||||
"Last region should end with an empty key. Creating a new "
|
||||
+ "region and regioninfo in HDFS to plug the hole.", getTableInfo());
|
||||
HTableDescriptor htd = getTableInfo().getHTD();
|
||||
TableDescriptor htd = getTableInfo().getHTD();
|
||||
// from curEndKey to EMPTY_START_ROW
|
||||
HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), curEndKey,
|
||||
HConstants.EMPTY_START_ROW);
|
||||
|
@ -3001,7 +3003,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
+ Bytes.toStringBinary(holeStopKey)
|
||||
+ ". Creating a new regioninfo and region "
|
||||
+ "dir in hdfs to plug the hole.");
|
||||
HTableDescriptor htd = getTableInfo().getHTD();
|
||||
TableDescriptor htd = getTableInfo().getHTD();
|
||||
HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), holeStartKey, holeStopKey);
|
||||
HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
|
||||
LOG.info("Plugged hole by creating new empty region: "+ newRegion + " " +region);
|
||||
|
@ -3202,7 +3204,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
}
|
||||
|
||||
// create new empty container region.
|
||||
HTableDescriptor htd = getTableInfo().getHTD();
|
||||
TableDescriptor htd = getTableInfo().getHTD();
|
||||
// from start key to end Key
|
||||
HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), range.getFirst(),
|
||||
range.getSecond());
|
||||
|
@ -3503,7 +3505,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
* @return tables that have not been modified recently
|
||||
* @throws IOException if an error is encountered
|
||||
*/
|
||||
HTableDescriptor[] getTables(AtomicInteger numSkipped) {
|
||||
TableDescriptor[] getTables(AtomicInteger numSkipped) {
|
||||
List<TableName> tableNames = new ArrayList<>();
|
||||
long now = EnvironmentEdgeManager.currentTime();
|
||||
|
||||
|
@ -3520,19 +3522,19 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
}
|
||||
}
|
||||
}
|
||||
return getHTableDescriptors(tableNames);
|
||||
return getTableDescriptors(tableNames);
|
||||
}
|
||||
|
||||
HTableDescriptor[] getHTableDescriptors(List<TableName> tableNames) {
|
||||
HTableDescriptor[] htd = new HTableDescriptor[0];
|
||||
LOG.info("getHTableDescriptors == tableNames => " + tableNames);
|
||||
TableDescriptor[] getTableDescriptors(List<TableName> tableNames) {
|
||||
LOG.info("getTableDescriptors == tableNames => " + tableNames);
|
||||
try (Connection conn = ConnectionFactory.createConnection(getConf());
|
||||
Admin admin = conn.getAdmin()) {
|
||||
htd = admin.getTableDescriptorsByTableName(tableNames);
|
||||
List<TableDescriptor> tds = admin.listTableDescriptors(tableNames);
|
||||
return tds.toArray(new TableDescriptor[tds.size()]);
|
||||
} catch (IOException e) {
|
||||
LOG.debug("Exception getting table descriptors", e);
|
||||
}
|
||||
return htd;
|
||||
return new TableDescriptor[0];
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Connection;
|
|||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -188,7 +188,7 @@ public class HBaseFsckRepair {
|
|||
* Creates, flushes, and closes a new region.
|
||||
*/
|
||||
public static HRegion createHDFSRegionDir(Configuration conf,
|
||||
HRegionInfo hri, HTableDescriptor htd) throws IOException {
|
||||
HRegionInfo hri, TableDescriptor htd) throws IOException {
|
||||
// Create HRegion
|
||||
Path root = FSUtils.getRootDir(conf);
|
||||
HRegion region = HRegion.createHRegion(hri, root, conf, htd, null);
|
||||
|
|
|
@ -39,9 +39,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
|
||||
/**
|
||||
* Utility methods for interacting with the regions.
|
||||
|
@ -61,13 +60,13 @@ public abstract class ModifyRegionUtils {
|
|||
void editRegion(final HRegionInfo region) throws IOException;
|
||||
}
|
||||
|
||||
public static HRegionInfo[] createHRegionInfos(HTableDescriptor hTableDescriptor,
|
||||
public static HRegionInfo[] createHRegionInfos(TableDescriptor tableDescriptor,
|
||||
byte[][] splitKeys) {
|
||||
long regionId = System.currentTimeMillis();
|
||||
HRegionInfo[] hRegionInfos = null;
|
||||
if (splitKeys == null || splitKeys.length == 0) {
|
||||
hRegionInfos = new HRegionInfo[]{
|
||||
new HRegionInfo(hTableDescriptor.getTableName(), null, null, false, regionId)
|
||||
new HRegionInfo(tableDescriptor.getTableName(), null, null, false, regionId)
|
||||
};
|
||||
} else {
|
||||
int numRegions = splitKeys.length + 1;
|
||||
|
@ -77,7 +76,7 @@ public abstract class ModifyRegionUtils {
|
|||
for (int i = 0; i < numRegions; i++) {
|
||||
endKey = (i == splitKeys.length) ? null : splitKeys[i];
|
||||
hRegionInfos[i] =
|
||||
new HRegionInfo(hTableDescriptor.getTableName(), startKey, endKey,
|
||||
new HRegionInfo(tableDescriptor.getTableName(), startKey, endKey,
|
||||
false, regionId);
|
||||
startKey = endKey;
|
||||
}
|
||||
|
@ -91,20 +90,20 @@ public abstract class ModifyRegionUtils {
|
|||
*
|
||||
* @param conf {@link Configuration}
|
||||
* @param rootDir Root directory for HBase instance
|
||||
* @param hTableDescriptor description of the table
|
||||
* @param tableDescriptor description of the table
|
||||
* @param newRegions {@link HRegionInfo} that describes the regions to create
|
||||
* @param task {@link RegionFillTask} custom code to populate region after creation
|
||||
* @throws IOException
|
||||
*/
|
||||
public static List<HRegionInfo> createRegions(final Configuration conf, final Path rootDir,
|
||||
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
|
||||
final TableDescriptor tableDescriptor, final HRegionInfo[] newRegions,
|
||||
final RegionFillTask task) throws IOException {
|
||||
if (newRegions == null) return null;
|
||||
int regionNumber = newRegions.length;
|
||||
ThreadPoolExecutor exec = getRegionOpenAndInitThreadPool(conf,
|
||||
"RegionOpenAndInitThread-" + hTableDescriptor.getTableName(), regionNumber);
|
||||
"RegionOpenAndInitThread-" + tableDescriptor.getTableName(), regionNumber);
|
||||
try {
|
||||
return createRegions(exec, conf, rootDir, hTableDescriptor, newRegions, task);
|
||||
return createRegions(exec, conf, rootDir, tableDescriptor, newRegions, task);
|
||||
} finally {
|
||||
exec.shutdownNow();
|
||||
}
|
||||
|
@ -117,15 +116,15 @@ public abstract class ModifyRegionUtils {
|
|||
* @param exec Thread Pool Executor
|
||||
* @param conf {@link Configuration}
|
||||
* @param rootDir Root directory for HBase instance
|
||||
* @param hTableDescriptor description of the table
|
||||
* @param tableDescriptor description of the table
|
||||
* @param newRegions {@link HRegionInfo} that describes the regions to create
|
||||
* @param task {@link RegionFillTask} custom code to populate region after creation
|
||||
* @throws IOException
|
||||
*/
|
||||
public static List<HRegionInfo> createRegions(final ThreadPoolExecutor exec,
|
||||
final Configuration conf, final Path rootDir,
|
||||
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
|
||||
final RegionFillTask task) throws IOException {
|
||||
final Configuration conf, final Path rootDir,
|
||||
final TableDescriptor tableDescriptor, final HRegionInfo[] newRegions,
|
||||
final RegionFillTask task) throws IOException {
|
||||
if (newRegions == null) return null;
|
||||
int regionNumber = newRegions.length;
|
||||
CompletionService<HRegionInfo> completionService = new ExecutorCompletionService<>(exec);
|
||||
|
@ -134,7 +133,7 @@ public abstract class ModifyRegionUtils {
|
|||
completionService.submit(new Callable<HRegionInfo>() {
|
||||
@Override
|
||||
public HRegionInfo call() throws IOException {
|
||||
return createRegion(conf, rootDir, hTableDescriptor, newRegion, task);
|
||||
return createRegion(conf, rootDir, tableDescriptor, newRegion, task);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -156,20 +155,20 @@ public abstract class ModifyRegionUtils {
|
|||
* Create new set of regions on the specified file-system.
|
||||
* @param conf {@link Configuration}
|
||||
* @param rootDir Root directory for HBase instance
|
||||
* @param hTableDescriptor description of the table
|
||||
* @param tableDescriptor description of the table
|
||||
* @param newRegion {@link HRegionInfo} that describes the region to create
|
||||
* @param task {@link RegionFillTask} custom code to populate region after creation
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HRegionInfo createRegion(final Configuration conf, final Path rootDir,
|
||||
final HTableDescriptor hTableDescriptor, final HRegionInfo newRegion,
|
||||
final TableDescriptor tableDescriptor, final HRegionInfo newRegion,
|
||||
final RegionFillTask task) throws IOException {
|
||||
// 1. Create HRegion
|
||||
// The WAL subsystem will use the default rootDir rather than the passed in rootDir
|
||||
// unless I pass along via the conf.
|
||||
Configuration confForWAL = new Configuration(conf);
|
||||
confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
|
||||
HRegion region = HRegion.createHRegion(newRegion, rootDir, conf, hTableDescriptor, null, false);
|
||||
HRegion region = HRegion.createHRegion(newRegion, rootDir, conf, tableDescriptor, null, false);
|
||||
try {
|
||||
// 2. Custom user code to interact with the created region
|
||||
if (task != null) {
|
||||
|
|
|
@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.client.Delete;
|
|||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
|
@ -466,10 +467,20 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
|
||||
/**
|
||||
* @return META table descriptor
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #getMetaDescriptor()}
|
||||
*/
|
||||
@Deprecated
|
||||
public HTableDescriptor getMetaTableDescriptor() {
|
||||
return new ImmutableHTableDescriptor(getMetaTableDescriptorBuilder().build());
|
||||
}
|
||||
|
||||
/**
|
||||
* @return META table descriptor
|
||||
*/
|
||||
public TableDescriptorBuilder getMetaTableDescriptorBuilder() {
|
||||
try {
|
||||
return new FSTableDescriptors(conf).get(TableName.META_TABLE_NAME);
|
||||
return FSTableDescriptors.createMetaTableDescriptorBuilder(conf);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Unable to create META table descriptor", e);
|
||||
}
|
||||
|
|
|
@ -25,10 +25,13 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.junit.*;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
|
@ -46,9 +49,9 @@ public class TestFSTableDescriptorForceCreation {
|
|||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
Path rootdir = new Path(UTIL.getDataTestDir(), name);
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
|
||||
assertTrue("Should create new table descriptor", fstd.createTableDescriptor(htd, false));
|
||||
assertTrue("Should create new table descriptor",
|
||||
fstd.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build(), false));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -59,7 +62,7 @@ public class TestFSTableDescriptorForceCreation {
|
|||
// Cleanup old tests if any detritus laying around.
|
||||
Path rootdir = new Path(UTIL.getDataTestDir(), name);
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
|
||||
fstd.add(htd);
|
||||
assertFalse("Should not create new table descriptor", fstd.createTableDescriptor(htd, false));
|
||||
}
|
||||
|
@ -71,7 +74,7 @@ public class TestFSTableDescriptorForceCreation {
|
|||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
Path rootdir = new Path(UTIL.getDataTestDir(), name);
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
|
||||
fstd.createTableDescriptor(htd, false);
|
||||
assertTrue("Should create new table descriptor",
|
||||
fstd.createTableDescriptor(htd, true));
|
||||
|
|
|
@ -24,6 +24,8 @@ import java.io.IOException;
|
|||
|
||||
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.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
|
@ -152,22 +154,22 @@ public class TestHColumnDescriptorDefaultVersions {
|
|||
Admin admin = TEST_UTIL.getAdmin();
|
||||
|
||||
// Verify descriptor from master
|
||||
HTableDescriptor htd = admin.getTableDescriptor(tableName);
|
||||
HColumnDescriptor[] hcds = htd.getColumnFamilies();
|
||||
TableDescriptor htd = admin.listTableDescriptor(tableName);
|
||||
ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies();
|
||||
verifyHColumnDescriptor(expected, hcds, tableName, families);
|
||||
|
||||
// Verify descriptor from HDFS
|
||||
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
|
||||
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
|
||||
HTableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
hcds = td.getColumnFamilies();
|
||||
verifyHColumnDescriptor(expected, hcds, tableName, families);
|
||||
}
|
||||
|
||||
private void verifyHColumnDescriptor(int expected, final HColumnDescriptor[] hcds,
|
||||
private void verifyHColumnDescriptor(int expected, final ColumnFamilyDescriptor[] hcds,
|
||||
final TableName tableName,
|
||||
final byte[]... families) {
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
for (ColumnFamilyDescriptor hcd : hcds) {
|
||||
assertEquals(expected, hcd.getMaxVersions());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,14 +38,11 @@ import java.util.regex.Pattern;
|
|||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
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.TableNotEnabledException;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.LoadBalancer;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
|
@ -751,7 +748,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
// Verify descriptor from HDFS
|
||||
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
|
||||
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
|
||||
HTableDescriptor td =
|
||||
TableDescriptor td =
|
||||
FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
verifyTableDescriptor(td, tableName, families);
|
||||
}
|
||||
|
|
|
@ -23,15 +23,14 @@ import java.util.List;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ChoreService;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ProcedureInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
|
@ -75,7 +74,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
|
|||
|
||||
@Override
|
||||
public long createTable(
|
||||
final HTableDescriptor desc,
|
||||
final TableDescriptor desc,
|
||||
final byte[][] splitKeys,
|
||||
final long nonceGroup,
|
||||
final long nonce) throws IOException {
|
||||
|
@ -84,7 +83,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long createSystemTable(final HTableDescriptor hTableDescriptor) throws IOException {
|
||||
public long createSystemTable(final TableDescriptor tableDescriptor) throws IOException {
|
||||
return -1;
|
||||
}
|
||||
|
||||
|
@ -267,7 +266,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
|
|||
@Override
|
||||
public long modifyTable(
|
||||
final TableName tableName,
|
||||
final HTableDescriptor descriptor,
|
||||
final TableDescriptor descriptor,
|
||||
final long nonceGroup,
|
||||
final long nonce) throws IOException {
|
||||
return -1;
|
||||
|
@ -290,13 +289,13 @@ public class MockNoopMasterServices implements MasterServices, Server {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long addColumn(final TableName tableName, final HColumnDescriptor columnDescriptor,
|
||||
public long addColumn(final TableName tableName, final ColumnFamilyDescriptor columnDescriptor,
|
||||
final long nonceGroup, final long nonce) throws IOException {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long modifyColumn(final TableName tableName, final HColumnDescriptor descriptor,
|
||||
public long modifyColumn(final TableName tableName, final ColumnFamilyDescriptor descriptor,
|
||||
final long nonceGroup, final long nonce) throws IOException {
|
||||
return -1;
|
||||
}
|
||||
|
|
|
@ -24,19 +24,19 @@ import java.util.NavigableMap;
|
|||
import java.util.SortedSet;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerLoad;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.YouAreDeadException;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.master.LoadBalancer;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
|
@ -300,36 +300,36 @@ public class MockMasterServices extends MockNoopMasterServices {
|
|||
public TableDescriptors getTableDescriptors() {
|
||||
return new TableDescriptors() {
|
||||
@Override
|
||||
public HTableDescriptor remove(TableName tablename) throws IOException {
|
||||
public TableDescriptor remove(TableName tablename) throws IOException {
|
||||
// noop
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, HTableDescriptor> getAll() throws IOException {
|
||||
public Map<String, TableDescriptor> getAll() throws IOException {
|
||||
// noop
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override public Map<String, HTableDescriptor> getAllDescriptors() throws IOException {
|
||||
@Override public Map<String, TableDescriptor> getAllDescriptors() throws IOException {
|
||||
// noop
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HTableDescriptor get(TableName tablename) throws IOException {
|
||||
HTableDescriptor htd = new HTableDescriptor(tablename);
|
||||
htd.addFamily(new HColumnDescriptor(DEFAULT_COLUMN_FAMILY_NAME));
|
||||
return htd;
|
||||
public TableDescriptor get(TableName tablename) throws IOException {
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tablename);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(DEFAULT_COLUMN_FAMILY_NAME));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException {
|
||||
public Map<String, TableDescriptor> getByNamespace(String name) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(HTableDescriptor htd) throws IOException {
|
||||
public void add(TableDescriptor htd) throws IOException {
|
||||
// noop
|
||||
}
|
||||
|
||||
|
|
|
@ -33,20 +33,23 @@ import org.apache.commons.logging.LogFactory;
|
|||
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.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
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.Connection;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MasterMetaBootstrap;
|
||||
|
@ -61,6 +64,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
|
|||
import org.apache.hadoop.hbase.util.MD5Hash;
|
||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class MasterProcedureTestingUtility {
|
||||
private static final Log LOG = LogFactory.getLog(MasterProcedureTestingUtility.class);
|
||||
|
||||
|
@ -136,17 +140,17 @@ public class MasterProcedureTestingUtility {
|
|||
// ==========================================================================
|
||||
// Table Helpers
|
||||
// ==========================================================================
|
||||
public static HTableDescriptor createHTD(final TableName tableName, final String... family) {
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
public static TableDescriptor createHTD(final TableName tableName, final String... family) {
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
for (int i = 0; i < family.length; ++i) {
|
||||
htd.addFamily(new HColumnDescriptor(family[i]));
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(family[i]));
|
||||
}
|
||||
return htd;
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
public static HRegionInfo[] createTable(final ProcedureExecutor<MasterProcedureEnv> procExec,
|
||||
final TableName tableName, final byte[][] splitKeys, String... family) throws IOException {
|
||||
HTableDescriptor htd = createHTD(tableName, family);
|
||||
TableDescriptor htd = createHTD(tableName, family);
|
||||
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
|
||||
long procId = ProcedureTestingUtility.submitAndWait(procExec,
|
||||
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
|
||||
|
@ -194,12 +198,12 @@ public class MasterProcedureTestingUtility {
|
|||
assertEquals(regions.length, countMetaRegions(master, tableName));
|
||||
|
||||
// check htd
|
||||
HTableDescriptor htd = master.getTableDescriptors().get(tableName);
|
||||
TableDescriptor htd = master.getTableDescriptors().get(tableName);
|
||||
assertTrue("table descriptor not found", htd != null);
|
||||
for (int i = 0; i < family.length; ++i) {
|
||||
assertTrue("family not found " + family[i], htd.getFamily(Bytes.toBytes(family[i])) != null);
|
||||
assertTrue("family not found " + family[i], htd.getColumnFamily(Bytes.toBytes(family[i])) != null);
|
||||
}
|
||||
assertEquals(family.length, htd.getFamilies().size());
|
||||
assertEquals(family.length, htd.getColumnFamilyCount());
|
||||
}
|
||||
|
||||
public static void validateTableDeletion(
|
||||
|
@ -267,18 +271,18 @@ public class MasterProcedureTestingUtility {
|
|||
|
||||
public static void validateColumnFamilyAddition(final HMaster master, final TableName tableName,
|
||||
final String family) throws IOException {
|
||||
HTableDescriptor htd = master.getTableDescriptors().get(tableName);
|
||||
TableDescriptor htd = master.getTableDescriptors().get(tableName);
|
||||
assertTrue(htd != null);
|
||||
|
||||
assertTrue(htd.hasFamily(family.getBytes()));
|
||||
assertTrue(htd.hasColumnFamily(family.getBytes()));
|
||||
}
|
||||
|
||||
public static void validateColumnFamilyDeletion(final HMaster master, final TableName tableName,
|
||||
final String family) throws IOException {
|
||||
// verify htd
|
||||
HTableDescriptor htd = master.getTableDescriptors().get(tableName);
|
||||
TableDescriptor htd = master.getTableDescriptors().get(tableName);
|
||||
assertTrue(htd != null);
|
||||
assertFalse(htd.hasFamily(family.getBytes()));
|
||||
assertFalse(htd.hasColumnFamily(family.getBytes()));
|
||||
|
||||
// verify fs
|
||||
final FileSystem fs = master.getMasterFileSystem().getFileSystem();
|
||||
|
@ -290,13 +294,13 @@ public class MasterProcedureTestingUtility {
|
|||
}
|
||||
|
||||
public static void validateColumnFamilyModification(final HMaster master,
|
||||
final TableName tableName, final String family, HColumnDescriptor columnDescriptor)
|
||||
final TableName tableName, final String family, ColumnFamilyDescriptor columnDescriptor)
|
||||
throws IOException {
|
||||
HTableDescriptor htd = master.getTableDescriptors().get(tableName);
|
||||
TableDescriptor htd = master.getTableDescriptors().get(tableName);
|
||||
assertTrue(htd != null);
|
||||
|
||||
HColumnDescriptor hcfd = htd.getFamily(family.getBytes());
|
||||
assertTrue(hcfd.equals(columnDescriptor));
|
||||
ColumnFamilyDescriptor hcfd = htd.getColumnFamily(family.getBytes());
|
||||
assertEquals(0, ColumnFamilyDescriptor.COMPARATOR.compare(hcfd, columnDescriptor));
|
||||
}
|
||||
|
||||
public static void loadData(final Connection connection, final TableName tableName,
|
||||
|
|
|
@ -22,9 +22,11 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||
|
@ -76,10 +78,11 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
// create table with 0 families will fail
|
||||
final HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName);
|
||||
final TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(MasterProcedureTestingUtility.createHTD(tableName));
|
||||
|
||||
// disable sanity check
|
||||
htd.setConfiguration("hbase.table.sanity.checks", Boolean.FALSE.toString());
|
||||
builder.setConfiguration("hbase.table.sanity.checks", Boolean.FALSE.toString());
|
||||
TableDescriptor htd = builder.build();
|
||||
final HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
|
||||
|
||||
long procId =
|
||||
|
@ -96,7 +99,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
|||
public void testCreateExisting() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
final HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f");
|
||||
final TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f");
|
||||
final HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
|
||||
|
||||
// create the table
|
||||
|
@ -125,7 +128,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
|||
|
||||
// Start the Create procedure && kill the executor
|
||||
byte[][] splitKeys = null;
|
||||
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
|
||||
TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
|
||||
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
|
||||
long procId = procExec.submitProcedure(
|
||||
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
|
||||
|
@ -138,18 +141,21 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
|||
@Test(timeout=90000)
|
||||
public void testRollbackAndDoubleExecution() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
testRollbackAndDoubleExecution(MasterProcedureTestingUtility.createHTD(tableName, F1, F2));
|
||||
testRollbackAndDoubleExecution(TableDescriptorBuilder.newBuilder(MasterProcedureTestingUtility.createHTD(tableName, F1, F2)));
|
||||
}
|
||||
|
||||
@Test(timeout=90000)
|
||||
public void testRollbackAndDoubleExecutionOnMobTable() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, F1, F2);
|
||||
htd.getFamily(Bytes.toBytes(F1)).setMobEnabled(true);
|
||||
testRollbackAndDoubleExecution(htd);
|
||||
TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, F1, F2);
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(htd)
|
||||
.modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(htd.getColumnFamily(Bytes.toBytes(F1)))
|
||||
.setMobEnabled(true)
|
||||
.build());
|
||||
testRollbackAndDoubleExecution(builder);
|
||||
}
|
||||
|
||||
private void testRollbackAndDoubleExecution(HTableDescriptor htd) throws Exception {
|
||||
private void testRollbackAndDoubleExecution(TableDescriptorBuilder builder) throws Exception {
|
||||
// create the table
|
||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
||||
|
@ -158,7 +164,8 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
|||
final byte[][] splitKeys = new byte[][] {
|
||||
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
|
||||
};
|
||||
htd.setRegionReplication(3);
|
||||
builder.setRegionReplication(3);
|
||||
TableDescriptor htd = builder.build();
|
||||
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
|
||||
long procId = procExec.submitProcedure(
|
||||
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
|
||||
|
@ -181,9 +188,9 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
|||
splitKeys[i] = Bytes.toBytes(String.format("%08d", i));
|
||||
}
|
||||
|
||||
final HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(
|
||||
final TableDescriptor htd = MasterProcedureTestingUtility.createHTD(
|
||||
TableName.valueOf("TestMRegions"), F1, F2);
|
||||
UTIL.getHBaseAdmin().createTableAsync(htd, splitKeys)
|
||||
UTIL.getAdmin().createTableAsync(htd, splitKeys)
|
||||
.get(10, java.util.concurrent.TimeUnit.HOURS);
|
||||
LOG.info("TABLE CREATED");
|
||||
}
|
||||
|
|
|
@ -27,8 +27,8 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||
|
@ -112,7 +112,7 @@ public class TestMasterFailoverWithProcedures {
|
|||
|
||||
// Start the Create procedure && kill the executor
|
||||
byte[][] splitKeys = null;
|
||||
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
|
||||
TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
|
||||
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
|
||||
long procId = procExec.submitProcedure(
|
||||
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
|
||||
|
|
|
@ -31,8 +31,8 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
|
@ -135,7 +135,7 @@ public class TestMasterProcedureWalLease {
|
|||
backupStore3.recoverLease();
|
||||
|
||||
// Try to trigger a command on the master (WAL lease expired on the active one)
|
||||
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(TableName.valueOf(name.getMethodName()), "f");
|
||||
TableDescriptor htd = MasterProcedureTestingUtility.createHTD(TableName.valueOf(name.getMethodName()), "f");
|
||||
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
|
||||
LOG.debug("submit proc");
|
||||
try {
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
|
@ -274,14 +275,14 @@ public class TestTableDescriptorModificationFromClient {
|
|||
// Verify descriptor from HDFS
|
||||
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
|
||||
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
|
||||
HTableDescriptor td =
|
||||
TableDescriptor td =
|
||||
FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
verifyTableDescriptor(td, tableName, families);
|
||||
}
|
||||
|
||||
private void verifyTableDescriptor(final HTableDescriptor htd,
|
||||
private void verifyTableDescriptor(final TableDescriptor htd,
|
||||
final TableName tableName, final byte[]... families) {
|
||||
Set<byte[]> htdFamilies = htd.getFamiliesKeys();
|
||||
Set<byte[]> htdFamilies = htd.getColumnFamilyNames();
|
||||
assertEquals(tableName, htd.getTableName());
|
||||
assertEquals(families.length, htdFamilies.size());
|
||||
for (byte[] familyName: families) {
|
||||
|
|
|
@ -54,6 +54,7 @@ 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.MobCompactPartitionPolicy;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
@ -514,7 +515,7 @@ public class TestPartitionedMobCompactor {
|
|||
CacheConfig cacheConfig = null;
|
||||
|
||||
MyPartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName,
|
||||
HColumnDescriptor column, ExecutorService pool, final int delPartitionSize,
|
||||
ColumnFamilyDescriptor column, ExecutorService pool, final int delPartitionSize,
|
||||
final CacheConfig cacheConf, final int PartitionsIncludeDelFiles)
|
||||
throws IOException {
|
||||
super(conf, fs, tableName, column, pool);
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.Durability;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.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;
|
||||
|
@ -79,10 +80,11 @@ public class TestGetClosestAtOrBefore {
|
|||
FileSystem filesystem = FileSystem.get(conf);
|
||||
Path rootdir = UTIL.getDataTestDirOnTestFS();
|
||||
// Up flush size else we bind up when we use default catalog flush of 16k.
|
||||
UTIL.getMetaTableDescriptor().setMemStoreFlushSize(64 * 1024 * 1024);
|
||||
TableDescriptorBuilder metaBuilder = UTIL.getMetaTableDescriptorBuilder()
|
||||
.setMemStoreFlushSize(64 * 1024 * 1024);
|
||||
|
||||
Region mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO,
|
||||
rootdir, this.conf, UTIL.getMetaTableDescriptor());
|
||||
rootdir, this.conf, metaBuilder.build());
|
||||
try {
|
||||
// Write rows for three tables 'A', 'B', and 'C'.
|
||||
for (char c = 'A'; c < 'D'; c++) {
|
||||
|
|
|
@ -39,22 +39,22 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
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.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
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.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MasterRpcServices;
|
||||
|
@ -207,7 +207,7 @@ public class TestRegionMergeTransactionOnCluster {
|
|||
List<Pair<HRegionInfo, ServerName>> tableRegions = MetaTableAccessor
|
||||
.getTableRegionsAndLocations(MASTER.getConnection(), tableName);
|
||||
HRegionInfo mergedRegionInfo = tableRegions.get(0).getFirst();
|
||||
HTableDescriptor tableDescriptor = MASTER.getTableDescriptors().get(
|
||||
TableDescriptor tableDescriptor = MASTER.getTableDescriptors().get(
|
||||
tableName);
|
||||
Result mergedRegionResult = MetaTableAccessor.getRegionResult(
|
||||
MASTER.getConnection(), mergedRegionInfo.getRegionName());
|
||||
|
@ -231,11 +231,11 @@ public class TestRegionMergeTransactionOnCluster {
|
|||
assertTrue(fs.exists(regionAdir));
|
||||
assertTrue(fs.exists(regionBdir));
|
||||
|
||||
HColumnDescriptor[] columnFamilies = tableDescriptor.getColumnFamilies();
|
||||
ColumnFamilyDescriptor[] columnFamilies = tableDescriptor.getColumnFamilies();
|
||||
HRegionFileSystem hrfs = new HRegionFileSystem(
|
||||
TEST_UTIL.getConfiguration(), fs, tabledir, mergedRegionInfo);
|
||||
int count = 0;
|
||||
for(HColumnDescriptor colFamily : columnFamilies) {
|
||||
for(ColumnFamilyDescriptor colFamily : columnFamilies) {
|
||||
count += hrfs.getStoreFiles(colFamily.getName()).size();
|
||||
}
|
||||
ADMIN.compactRegion(mergedRegionInfo.getRegionName());
|
||||
|
@ -244,7 +244,7 @@ public class TestRegionMergeTransactionOnCluster {
|
|||
long timeout = System.currentTimeMillis() + waitTime;
|
||||
int newcount = 0;
|
||||
while (System.currentTimeMillis() < timeout) {
|
||||
for(HColumnDescriptor colFamily : columnFamilies) {
|
||||
for(ColumnFamilyDescriptor colFamily : columnFamilies) {
|
||||
newcount += hrfs.getStoreFiles(colFamily.getName()).size();
|
||||
}
|
||||
if(newcount > count) {
|
||||
|
@ -263,7 +263,7 @@ public class TestRegionMergeTransactionOnCluster {
|
|||
}
|
||||
while (System.currentTimeMillis() < timeout) {
|
||||
int newcount1 = 0;
|
||||
for(HColumnDescriptor colFamily : columnFamilies) {
|
||||
for(ColumnFamilyDescriptor colFamily : columnFamilies) {
|
||||
newcount1 += hrfs.getStoreFiles(colFamily.getName()).size();
|
||||
}
|
||||
if(newcount1 <= 1) {
|
||||
|
|
|
@ -26,13 +26,13 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
|
@ -261,7 +261,7 @@ public class TestRegionServerNoMaster {
|
|||
hri.getEncodedNameAsBytes()));
|
||||
|
||||
// Let's start the open handler
|
||||
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
|
||||
TableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
|
||||
|
||||
getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd, -1));
|
||||
|
||||
|
|
|
@ -36,20 +36,20 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||
import org.apache.hadoop.hbase.Waiter.Predicate;
|
||||
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.RetriesExhaustedWithDetailsException;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
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.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
|
@ -652,31 +652,30 @@ public class SecureTestUtil {
|
|||
|
||||
public static Table createTable(HBaseTestingUtility testUtil, TableName tableName,
|
||||
byte[][] families) throws Exception {
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
for (byte[] family : families) {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
htd.addFamily(hcd);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
|
||||
}
|
||||
createTable(testUtil, testUtil.getAdmin(), htd);
|
||||
return testUtil.getConnection().getTable(htd.getTableName());
|
||||
createTable(testUtil, testUtil.getAdmin(), builder.build());
|
||||
return testUtil.getConnection().getTable(tableName);
|
||||
}
|
||||
|
||||
public static void createTable(HBaseTestingUtility testUtil, HTableDescriptor htd)
|
||||
public static void createTable(HBaseTestingUtility testUtil, TableDescriptor htd)
|
||||
throws Exception {
|
||||
createTable(testUtil, testUtil.getAdmin(), htd);
|
||||
}
|
||||
|
||||
public static void createTable(HBaseTestingUtility testUtil, HTableDescriptor htd,
|
||||
public static void createTable(HBaseTestingUtility testUtil, TableDescriptor htd,
|
||||
byte[][] splitKeys) throws Exception {
|
||||
createTable(testUtil, testUtil.getAdmin(), htd, splitKeys);
|
||||
}
|
||||
|
||||
public static void createTable(HBaseTestingUtility testUtil, Admin admin, HTableDescriptor htd)
|
||||
public static void createTable(HBaseTestingUtility testUtil, Admin admin, TableDescriptor htd)
|
||||
throws Exception {
|
||||
createTable(testUtil, admin, htd, null);
|
||||
}
|
||||
|
||||
public static void createTable(HBaseTestingUtility testUtil, Admin admin, HTableDescriptor htd,
|
||||
public static void createTable(HBaseTestingUtility testUtil, Admin admin, TableDescriptor htd,
|
||||
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.
|
||||
|
|
|
@ -24,18 +24,20 @@ import java.io.IOException;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
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.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Assert;
|
||||
|
||||
public class MobSnapshotTestingUtils {
|
||||
|
@ -60,15 +62,17 @@ public class MobSnapshotTestingUtils {
|
|||
private static void createMobTable(final HBaseTestingUtility util,
|
||||
final TableName tableName, final byte[][] splitKeys, int regionReplication,
|
||||
final byte[]... families) throws IOException, InterruptedException {
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.setRegionReplication(regionReplication);
|
||||
TableDescriptorBuilder builder
|
||||
= TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setRegionReplication(regionReplication);
|
||||
for (byte[] family : families) {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(0L);
|
||||
htd.addFamily(hcd);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(family)
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(0L)
|
||||
.build());
|
||||
}
|
||||
util.getAdmin().createTable(htd, splitKeys);
|
||||
util.getAdmin().createTable(builder.build(), splitKeys);
|
||||
SnapshotTestingUtils.waitForTableToBeOnline(util, tableName);
|
||||
assertEquals((splitKeys.length + 1) * regionReplication, util
|
||||
.getAdmin().getTableRegions(tableName).size());
|
||||
|
@ -80,29 +84,29 @@ public class MobSnapshotTestingUtils {
|
|||
* @param util
|
||||
* @param tableName
|
||||
* @param families
|
||||
* @return An HTable instance for the created table.
|
||||
* @return An Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Table createMobTable(final HBaseTestingUtility util,
|
||||
final TableName tableName, final byte[]... families) throws IOException {
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
for (byte[] family : families) {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
// Disable blooms (they are on by default as of 0.95) but we disable them
|
||||
// here because
|
||||
// tests have hard coded counts of what to expect in block cache, etc.,
|
||||
// and blooms being
|
||||
// on is interfering.
|
||||
hcd.setBloomFilterType(BloomType.NONE);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(0L);
|
||||
htd.addFamily(hcd);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family)
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(0L)
|
||||
.build());
|
||||
}
|
||||
util.getAdmin().createTable(htd);
|
||||
util.getAdmin().createTable(builder.build());
|
||||
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait
|
||||
// until they are assigned
|
||||
util.waitUntilAllRegionsAssigned(htd.getTableName());
|
||||
return ConnectionFactory.createConnection(util.getConfiguration()).getTable(htd.getTableName());
|
||||
util.waitUntilAllRegionsAssigned(tableName);
|
||||
return ConnectionFactory.createConnection(util.getConfiguration()).getTable(tableName);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -146,13 +150,14 @@ public class MobSnapshotTestingUtils {
|
|||
}
|
||||
|
||||
@Override
|
||||
public HTableDescriptor createHtd(final String tableName) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(0L);
|
||||
htd.addFamily(hcd);
|
||||
return htd;
|
||||
public TableDescriptor createHtd(final String tableName) {
|
||||
return TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(Bytes.toBytes(TEST_FAMILY))
|
||||
.setMobEnabled(true)
|
||||
.setMobThreshold(0L)
|
||||
.build())
|
||||
.build();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,36 +40,35 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
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.TableNotEnabledException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.SnapshotType;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
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.Table;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.client.SnapshotType;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.client.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.FSVisitor;
|
||||
|
@ -492,7 +491,7 @@ public final class SnapshotTestingUtils {
|
|||
public static class SnapshotBuilder {
|
||||
private final RegionData[] tableRegions;
|
||||
private final SnapshotProtos.SnapshotDescription desc;
|
||||
private final HTableDescriptor htd;
|
||||
private final TableDescriptor htd;
|
||||
private final Configuration conf;
|
||||
private final FileSystem fs;
|
||||
private final Path rootDir;
|
||||
|
@ -500,7 +499,7 @@ public final class SnapshotTestingUtils {
|
|||
private int snapshotted = 0;
|
||||
|
||||
public SnapshotBuilder(final Configuration conf, final FileSystem fs,
|
||||
final Path rootDir, final HTableDescriptor htd,
|
||||
final Path rootDir, final TableDescriptor htd,
|
||||
final SnapshotProtos.SnapshotDescription desc, final RegionData[] tableRegions)
|
||||
throws IOException {
|
||||
this.fs = fs;
|
||||
|
@ -514,7 +513,7 @@ public final class SnapshotTestingUtils {
|
|||
.createTableDescriptorForTableDirectory(snapshotDir, htd, false);
|
||||
}
|
||||
|
||||
public HTableDescriptor getTableDescriptor() {
|
||||
public TableDescriptor getTableDescriptor() {
|
||||
return this.htd;
|
||||
}
|
||||
|
||||
|
@ -680,11 +679,11 @@ public final class SnapshotTestingUtils {
|
|||
|
||||
private SnapshotBuilder createSnapshot(final String snapshotName, final String tableName,
|
||||
final int numRegions, final int version) throws IOException {
|
||||
HTableDescriptor htd = createHtd(tableName);
|
||||
TableDescriptor htd = createHtd(tableName);
|
||||
RegionData[] regions = createTable(htd, numRegions);
|
||||
|
||||
SnapshotProtos.SnapshotDescription desc = SnapshotProtos.SnapshotDescription.newBuilder()
|
||||
.setTable(htd.getNameAsString())
|
||||
.setTable(htd.getTableName().getNameAsString())
|
||||
.setName(snapshotName)
|
||||
.setVersion(version)
|
||||
.build();
|
||||
|
@ -694,13 +693,13 @@ public final class SnapshotTestingUtils {
|
|||
return new SnapshotBuilder(conf, fs, rootDir, htd, desc, regions);
|
||||
}
|
||||
|
||||
public HTableDescriptor createHtd(final String tableName) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
|
||||
htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
|
||||
return htd;
|
||||
public TableDescriptor createHtd(final String tableName) {
|
||||
return TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY))
|
||||
.build();
|
||||
}
|
||||
|
||||
private RegionData[] createTable(final HTableDescriptor htd, final int nregions)
|
||||
private RegionData[] createTable(final TableDescriptor htd, final int nregions)
|
||||
throws IOException {
|
||||
Path tableDir = FSUtils.getTableDir(rootDir, htd.getTableName());
|
||||
new FSTableDescriptors(conf).createTableDescriptorForTableDirectory(tableDir, htd, false);
|
||||
|
@ -766,14 +765,15 @@ public final class SnapshotTestingUtils {
|
|||
public static void createTable(final HBaseTestingUtility util, final TableName tableName,
|
||||
int regionReplication, int nRegions, final byte[]... families)
|
||||
throws IOException, InterruptedException {
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.setRegionReplication(regionReplication);
|
||||
TableDescriptorBuilder builder
|
||||
= TableDescriptorBuilder
|
||||
.newBuilder(tableName)
|
||||
.setRegionReplication(regionReplication);
|
||||
for (byte[] family : families) {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
htd.addFamily(hcd);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
|
||||
}
|
||||
byte[][] splitKeys = getSplitKeys(nRegions);
|
||||
util.createTable(htd, splitKeys);
|
||||
util.createTable(builder.build(), splitKeys);
|
||||
assertEquals((splitKeys.length + 1) * regionReplication,
|
||||
util.getAdmin().getTableRegions(tableName).size());
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -104,11 +104,11 @@ public class TestRestoreSnapshotHelper {
|
|||
builder.addRegionV2();
|
||||
builder.addRegionV1();
|
||||
Path snapshotDir = builder.commit();
|
||||
HTableDescriptor htd = builder.getTableDescriptor();
|
||||
TableDescriptor htd = builder.getTableDescriptor();
|
||||
SnapshotDescription desc = builder.getSnapshotDescription();
|
||||
|
||||
// Test clone a snapshot
|
||||
HTableDescriptor htdClone = snapshotMock.createHtd("testtb-clone");
|
||||
TableDescriptor htdClone = snapshotMock.createHtd("testtb-clone");
|
||||
testRestore(snapshotDir, desc, htdClone);
|
||||
verifyRestore(rootDir, htd, htdClone);
|
||||
|
||||
|
@ -118,13 +118,13 @@ public class TestRestoreSnapshotHelper {
|
|||
.setTable("testtb-clone")
|
||||
.build();
|
||||
Path cloneDir = FSUtils.getTableDir(rootDir, htdClone.getTableName());
|
||||
HTableDescriptor htdClone2 = snapshotMock.createHtd("testtb-clone2");
|
||||
TableDescriptor htdClone2 = snapshotMock.createHtd("testtb-clone2");
|
||||
testRestore(cloneDir, cloneDesc, htdClone2);
|
||||
verifyRestore(rootDir, htd, htdClone2);
|
||||
}
|
||||
|
||||
private void verifyRestore(final Path rootDir, final HTableDescriptor sourceHtd,
|
||||
final HTableDescriptor htdClone) throws IOException {
|
||||
private void verifyRestore(final Path rootDir, final TableDescriptor sourceHtd,
|
||||
final TableDescriptor htdClone) throws IOException {
|
||||
List<String> files = SnapshotTestingUtils.listHFileNames(fs,
|
||||
FSUtils.getTableDir(rootDir, htdClone.getTableName()));
|
||||
assertEquals(12, files.size());
|
||||
|
@ -148,7 +148,7 @@ public class TestRestoreSnapshotHelper {
|
|||
* @param htdClone The HTableDescriptor of the table to restore/clone.
|
||||
*/
|
||||
private void testRestore(final Path snapshotDir, final SnapshotDescription sd,
|
||||
final HTableDescriptor htdClone) throws IOException {
|
||||
final TableDescriptor htdClone) throws IOException {
|
||||
LOG.debug("pre-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir);
|
||||
FSUtils.logFileSystemState(fs, rootDir, LOG);
|
||||
|
||||
|
@ -164,7 +164,7 @@ public class TestRestoreSnapshotHelper {
|
|||
* Initialize the restore helper, based on the snapshot and table information provided.
|
||||
*/
|
||||
private RestoreSnapshotHelper getRestoreHelper(final Path rootDir, final Path snapshotDir,
|
||||
final SnapshotDescription sd, final HTableDescriptor htdClone) throws IOException {
|
||||
final SnapshotDescription sd, final TableDescriptor htdClone) throws IOException {
|
||||
ForeignExceptionDispatcher monitor = Mockito.mock(ForeignExceptionDispatcher.class);
|
||||
MonitoredTask status = Mockito.mock(MonitoredTask.class);
|
||||
|
||||
|
|
|
@ -28,9 +28,9 @@ 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.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
|
||||
|
@ -129,7 +129,7 @@ public class TestSnapshotManifest {
|
|||
SnapshotRegionManifest.Builder dataRegionManifestBuilder =
|
||||
SnapshotRegionManifest.newBuilder();
|
||||
|
||||
for (HColumnDescriptor hcd: builder.getTableDescriptor().getFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd: builder.getTableDescriptor().getColumnFamilies()) {
|
||||
SnapshotRegionManifest.FamilyFiles.Builder family =
|
||||
SnapshotRegionManifest.FamilyFiles.newBuilder();
|
||||
family.setFamilyName(UnsafeByteOperations.unsafeWrap(hcd.getName()));
|
||||
|
@ -150,7 +150,7 @@ public class TestSnapshotManifest {
|
|||
}
|
||||
|
||||
dataManifestBuilder
|
||||
.setTableSchema(ProtobufUtil.convertToTableSchema(builder.getTableDescriptor()));
|
||||
.setTableSchema(ProtobufUtil.toTableSchema(builder.getTableDescriptor()));
|
||||
|
||||
SnapshotDataManifest dataManifest = dataManifestBuilder.build();
|
||||
return writeDataManifest(dataManifest);
|
||||
|
@ -163,7 +163,7 @@ public class TestSnapshotManifest {
|
|||
SnapshotRegionManifest.Builder dataRegionManifestBuilder = SnapshotRegionManifest.newBuilder();
|
||||
dataRegionManifestBuilder.setRegionInfo(HRegionInfo.convert(regionInfo));
|
||||
|
||||
for (HColumnDescriptor hcd: builder.getTableDescriptor().getFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd: builder.getTableDescriptor().getColumnFamilies()) {
|
||||
SnapshotRegionManifest.FamilyFiles.Builder family =
|
||||
SnapshotRegionManifest.FamilyFiles.newBuilder();
|
||||
family.setFamilyName(UnsafeByteOperations.unsafeWrap(hcd.getName()));
|
||||
|
|
|
@ -40,11 +40,12 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
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.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
|
@ -78,7 +79,7 @@ public class TestFSTableDescriptors {
|
|||
@Test
|
||||
public void testCreateAndUpdate() throws IOException {
|
||||
Path testdir = UTIL.getDataTestDir(name.getMethodName());
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
|
||||
assertTrue(fstd.createTableDescriptor(htd));
|
||||
|
@ -98,7 +99,7 @@ public class TestFSTableDescriptors {
|
|||
@Test
|
||||
public void testSequenceIdAdvancesOnTableInfo() throws IOException {
|
||||
Path testdir = UTIL.getDataTestDir(name.getMethodName());
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
|
||||
Path p0 = fstd.updateTableDescriptor(htd);
|
||||
|
@ -118,7 +119,7 @@ public class TestFSTableDescriptors {
|
|||
assertTrue(!fs.exists(p2));
|
||||
int i3 = FSTableDescriptors.getTableInfoSequenceId(p3);
|
||||
assertTrue(i3 == i2 + 1);
|
||||
HTableDescriptor descriptor = fstd.get(htd.getTableName());
|
||||
TableDescriptor descriptor = fstd.get(htd.getTableName());
|
||||
assertEquals(descriptor, htd);
|
||||
}
|
||||
|
||||
|
@ -161,7 +162,7 @@ public class TestFSTableDescriptors {
|
|||
// Cleanup old tests if any detrius laying around.
|
||||
Path rootdir = new Path(UTIL.getDataTestDir(), name);
|
||||
TableDescriptors htds = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
|
||||
htds.add(htd);
|
||||
assertNotNull(htds.remove(htd.getTableName()));
|
||||
assertNull(htds.remove(htd.getTableName()));
|
||||
|
@ -170,11 +171,11 @@ public class TestFSTableDescriptors {
|
|||
@Test public void testReadingHTDFromFS() throws IOException {
|
||||
final String name = this.name.getMethodName();
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
|
||||
Path rootdir = UTIL.getDataTestDir(name);
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
fstd.createTableDescriptor(htd);
|
||||
HTableDescriptor td2 =
|
||||
TableDescriptor td2 =
|
||||
FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName());
|
||||
assertTrue(htd.equals(td2));
|
||||
}
|
||||
|
@ -184,25 +185,25 @@ public class TestFSTableDescriptors {
|
|||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
Path rootdir = UTIL.getDataTestDir(name);
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
|
||||
Path descriptorFile = fstd.updateTableDescriptor(htd);
|
||||
try (FSDataOutputStream out = fs.create(descriptorFile, true)) {
|
||||
out.write(htd.toByteArray());
|
||||
out.write(TableDescriptorBuilder.toByteArray(htd));
|
||||
}
|
||||
FSTableDescriptors fstd2 = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor td2 = fstd2.get(htd.getTableName());
|
||||
TableDescriptor td2 = fstd2.get(htd.getTableName());
|
||||
assertEquals(htd, td2);
|
||||
FileStatus descriptorFile2 =
|
||||
FSTableDescriptors.getTableInfoPath(fs, fstd2.getTableDir(htd.getTableName()));
|
||||
byte[] buffer = htd.toByteArray();
|
||||
byte[] buffer = TableDescriptorBuilder.toByteArray(htd);
|
||||
try (FSDataInputStream in = fs.open(descriptorFile2.getPath())) {
|
||||
in.readFully(buffer);
|
||||
}
|
||||
HTableDescriptor td3 = HTableDescriptor.parseFrom(buffer);
|
||||
TableDescriptor td3 = TableDescriptorBuilder.parseFrom(buffer);
|
||||
assertEquals(htd, td3);
|
||||
}
|
||||
|
||||
@Test public void testHTableDescriptors()
|
||||
@Test public void testTableDescriptors()
|
||||
throws IOException, InterruptedException {
|
||||
final String name = this.name.getMethodName();
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
|
@ -210,7 +211,7 @@ public class TestFSTableDescriptors {
|
|||
Path rootdir = new Path(UTIL.getDataTestDir(), name);
|
||||
FSTableDescriptors htds = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir) {
|
||||
@Override
|
||||
public HTableDescriptor get(TableName tablename)
|
||||
public TableDescriptor get(TableName tablename)
|
||||
throws TableExistsException, FileNotFoundException, IOException {
|
||||
LOG.info(tablename + ", cachehits=" + this.cachehits);
|
||||
return super.get(tablename);
|
||||
|
@ -219,9 +220,7 @@ public class TestFSTableDescriptors {
|
|||
final int count = 10;
|
||||
// Write out table infos.
|
||||
for (int i = 0; i < count; i++) {
|
||||
HTableDescriptor htd = new HTableDescriptor(
|
||||
new HTableDescriptor(TableName.valueOf(name + i)));
|
||||
htds.createTableDescriptor(htd);
|
||||
htds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build());
|
||||
}
|
||||
|
||||
for (int i = 0; i < count; i++) {
|
||||
|
@ -232,9 +231,9 @@ public class TestFSTableDescriptors {
|
|||
}
|
||||
// Update the table infos
|
||||
for (int i = 0; i < count; i++) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
|
||||
htd.addFamily(new HColumnDescriptor("" + i));
|
||||
htds.updateTableDescriptor(htd);
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i));
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of("" + i));
|
||||
htds.updateTableDescriptor(builder.build());
|
||||
}
|
||||
// Wait a while so mod time we write is for sure different.
|
||||
Thread.sleep(100);
|
||||
|
@ -250,7 +249,7 @@ public class TestFSTableDescriptors {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testHTableDescriptorsNoCache()
|
||||
public void testTableDescriptorsNoCache()
|
||||
throws IOException, InterruptedException {
|
||||
final String name = this.name.getMethodName();
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
|
@ -261,8 +260,7 @@ public class TestFSTableDescriptors {
|
|||
final int count = 10;
|
||||
// Write out table infos.
|
||||
for (int i = 0; i < count; i++) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
|
||||
htds.createTableDescriptor(htd);
|
||||
htds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build());
|
||||
}
|
||||
|
||||
for (int i = 0; i < 2 * count; i++) {
|
||||
|
@ -270,14 +268,14 @@ public class TestFSTableDescriptors {
|
|||
}
|
||||
// Update the table infos
|
||||
for (int i = 0; i < count; i++) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
|
||||
htd.addFamily(new HColumnDescriptor("" + i));
|
||||
htds.updateTableDescriptor(htd);
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i));
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of("" + i));
|
||||
htds.updateTableDescriptor(builder.build());
|
||||
}
|
||||
for (int i = 0; i < count; i++) {
|
||||
assertNotNull("Expected HTD, got null instead", htds.get(TableName.valueOf(name + i)));
|
||||
assertTrue("Column Family " + i + " missing",
|
||||
htds.get(TableName.valueOf(name + i)).hasFamily(Bytes.toBytes("" + i)));
|
||||
htds.get(TableName.valueOf(name + i)).hasColumnFamily(Bytes.toBytes("" + i)));
|
||||
}
|
||||
assertEquals(count * 4, htds.invocations);
|
||||
assertEquals("expected=0, actual=" + htds.cachehits, 0, htds.cachehits);
|
||||
|
@ -294,12 +292,10 @@ public class TestFSTableDescriptors {
|
|||
final int count = 4;
|
||||
// Write out table infos.
|
||||
for (int i = 0; i < count; i++) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
|
||||
htds.createTableDescriptor(htd);
|
||||
htds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build());
|
||||
}
|
||||
// add hbase:meta
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
|
||||
htds.createTableDescriptor(htd);
|
||||
htds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build());
|
||||
|
||||
assertEquals("getAll() didn't return all TableDescriptors, expected: " +
|
||||
(count + 1) + " got: " + htds.getAll().size(),
|
||||
|
@ -321,8 +317,7 @@ public class TestFSTableDescriptors {
|
|||
final int count = 10;
|
||||
// Write out table infos via non-cached FSTableDescriptors
|
||||
for (int i = 0; i < count; i++) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
|
||||
nonchtds.createTableDescriptor(htd);
|
||||
nonchtds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build());
|
||||
}
|
||||
|
||||
// Calls to getAll() won't increase the cache counter, do per table.
|
||||
|
@ -333,15 +328,15 @@ public class TestFSTableDescriptors {
|
|||
assertTrue(nonchtds.getAll().size() == chtds.getAll().size());
|
||||
|
||||
// add a new entry for hbase:meta
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build();
|
||||
nonchtds.createTableDescriptor(htd);
|
||||
|
||||
// hbase:meta will only increase the cachehit by 1
|
||||
assertTrue(nonchtds.getAll().size() == chtds.getAll().size());
|
||||
|
||||
for (Map.Entry entry: nonchtds.getAll().entrySet()) {
|
||||
for (Map.Entry<String, TableDescriptor> entry: nonchtds.getAll().entrySet()) {
|
||||
String t = (String) entry.getKey();
|
||||
HTableDescriptor nchtd = (HTableDescriptor) entry.getValue();
|
||||
TableDescriptor nchtd = entry.getValue();
|
||||
assertTrue("expected " + htd.toString() +
|
||||
" got: " + chtds.get(TableName.valueOf(t)).toString(),
|
||||
(nchtd.equals(chtds.get(TableName.valueOf(t)))));
|
||||
|
@ -366,7 +361,7 @@ public class TestFSTableDescriptors {
|
|||
// Cleanup old tests if any detrius laying around.
|
||||
Path rootdir = new Path(UTIL.getDataTestDir(), name);
|
||||
TableDescriptors htds = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
|
||||
htds.add(htd);
|
||||
htds.add(htd);
|
||||
htds.add(htd);
|
||||
|
@ -415,12 +410,14 @@ public class TestFSTableDescriptors {
|
|||
@Test
|
||||
public void testCreateTableDescriptorUpdatesIfExistsAlready() throws IOException {
|
||||
Path testdir = UTIL.getDataTestDir(name.getMethodName());
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
|
||||
assertTrue(fstd.createTableDescriptor(htd));
|
||||
assertFalse(fstd.createTableDescriptor(htd));
|
||||
htd.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue"));
|
||||
htd = TableDescriptorBuilder.newBuilder(htd)
|
||||
.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue"))
|
||||
.build();
|
||||
assertTrue(fstd.createTableDescriptor(htd)); //this will re-create
|
||||
Path tableDir = fstd.getTableDir(htd.getTableName());
|
||||
Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR);
|
||||
|
@ -443,10 +440,10 @@ public class TestFSTableDescriptors {
|
|||
}
|
||||
|
||||
@Override
|
||||
public HTableDescriptor get(TableName tablename)
|
||||
public TableDescriptor get(TableName tablename)
|
||||
throws TableExistsException, FileNotFoundException, IOException {
|
||||
LOG.info((super.isUsecache() ? "Cached" : "Non-Cached") +
|
||||
" HTableDescriptor.get() on " + tablename + ", cachehits=" + this.cachehits);
|
||||
" TableDescriptor.get() on " + tablename + ", cachehits=" + this.cachehits);
|
||||
return super.get(tablename);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue