HBASE-15605 Remove PB references from HCD and HTD for 2.0 (Ram)

This commit is contained in:
Vasudevan 2016-04-11 14:12:07 +05:30
parent 80df1cb7b6
commit a395922ad5
19 changed files with 162 additions and 167 deletions

View File

@ -32,11 +32,8 @@ import org.apache.hadoop.hbase.exceptions.HBaseException;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PrettyPrinter;
import org.apache.hadoop.hbase.util.PrettyPrinter.Unit;
@ -296,13 +293,6 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
*/
private int cachedMaxVersions = UNINITIALIZED;
/**
* Default constructor. Must be present for PB deserializations.
*/
private HColumnDescriptor() {
this.name = null;
}
/**
* Construct a column descriptor specifying only the family name
* The other attributes are defaulted.
@ -1075,8 +1065,9 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
* @return This instance serialized with pb with pb magic prefix
* @see #parseFrom(byte[])
*/
public byte [] toByteArray() {
return ProtobufUtil.prependPBMagic(convert().toByteArray());
public byte[] toByteArray() {
return ProtobufUtil
.prependPBMagic(ProtobufUtil.convertToColumnFamilySchema(this).toByteArray());
}
/**
@ -1096,47 +1087,7 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
} catch (IOException e) {
throw new DeserializationException(e);
}
return convert(cfs);
}
/**
* @param cfs
* @return An {@link HColumnDescriptor} made from the passed in <code>cfs</code>
*/
public static HColumnDescriptor convert(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();
hcd.name = 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;
}
/**
* @return Convert this instance to a the pb column family type
*/
public ColumnFamilySchema convert() {
ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
builder.setName(ByteStringer.wrap(getName()));
for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
aBuilder.setFirst(ByteStringer.wrap(e.getKey().get()));
aBuilder.setSecond(ByteStringer.wrap(e.getValue().get()));
builder.addAttributes(aBuilder.build());
}
for (Map.Entry<String, String> e : this.configuration.entrySet()) {
NameStringPair.Builder aBuilder = NameStringPair.newBuilder();
aBuilder.setName(e.getKey());
aBuilder.setValue(e.getValue());
builder.addConfiguration(aBuilder.build());
}
return builder.build();
return ProtobufUtil.convertToHColumnDesc(cfs);
}
/**

View File

@ -41,13 +41,9 @@ import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -1519,8 +1515,8 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
* @return This instance serialized with pb with pb magic prefix
* @see #parseFrom(byte[])
*/
public byte [] toByteArray() {
return ProtobufUtil.prependPBMagic(convert().toByteArray());
public byte[] toByteArray() {
return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToTableSchema(this).toByteArray());
}
/**
@ -1544,54 +1540,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
} catch (IOException e) {
throw new DeserializationException(e);
}
return convert(ts);
}
/**
* @return Convert the current {@link HTableDescriptor} into a pb TableSchema instance.
*/
public TableSchema convert() {
TableSchema.Builder builder = TableSchema.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(getTableName()));
for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
aBuilder.setFirst(ByteStringer.wrap(e.getKey().get()));
aBuilder.setSecond(ByteStringer.wrap(e.getValue().get()));
builder.addAttributes(aBuilder.build());
}
for (HColumnDescriptor hcd: getColumnFamilies()) {
builder.addColumnFamilies(hcd.convert());
}
for (Map.Entry<String, String> e : this.configuration.entrySet()) {
NameStringPair.Builder aBuilder = NameStringPair.newBuilder();
aBuilder.setName(e.getKey());
aBuilder.setValue(e.getValue());
builder.addConfiguration(aBuilder.build());
}
return builder.build();
}
/**
* @param ts A pb TableSchema instance.
* @return An {@link HTableDescriptor} made from the passed in pb <code>ts</code>.
*/
public static HTableDescriptor convert(final TableSchema ts) {
List<ColumnFamilySchema> list = ts.getColumnFamiliesList();
HColumnDescriptor [] hcds = new HColumnDescriptor[list.size()];
int index = 0;
for (ColumnFamilySchema cfs: list) {
hcds[index++] = HColumnDescriptor.convert(cfs);
}
HTableDescriptor htd = new HTableDescriptor(
ProtobufUtil.toTableName(ts.getTableName()),
hcds);
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;
return ProtobufUtil.convertToHTableDesc(ts);
}
/**

View File

@ -2241,7 +2241,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
master.close();
}
if (!htds.getTableSchemaList().isEmpty()) {
return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
}
throw new TableNotFoundException(tableName.getNameAsString());
}

View File

@ -420,7 +420,7 @@ public class HBaseAdmin implements Admin {
htds = master.getTableDescriptors(controller, req);
if (!htds.getTableSchemaList().isEmpty()) {
return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
}
return null;
}
@ -2033,7 +2033,7 @@ public class HBaseAdmin implements Admin {
HTableDescriptor[] res = new HTableDescriptor[list.size()];
for(int i=0; i < list.size(); i++) {
res[i] = HTableDescriptor.convert(list.get(i));
res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
}
return res;
}

View File

@ -17,21 +17,6 @@
*/
package org.apache.hadoop.hbase.protobuf;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import com.google.common.net.HostAndPort;
import com.google.protobuf.ByteString;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.Message;
import com.google.protobuf.Parser;
import com.google.protobuf.RpcChannel;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
@ -54,11 +39,14 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
.RegionSpecifierType.REGION_NAME;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
@ -126,10 +114,14 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
@ -171,11 +163,10 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.token.Token;
import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
.RegionSpecifierType.REGION_NAME;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import com.google.common.net.HostAndPort;
import com.google.protobuf.ByteString;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.InvalidProtocolBufferException;
@ -394,7 +385,7 @@ public final class ProtobufUtil {
HTableDescriptor[] ret = new HTableDescriptor[proto.getTableSchemaCount()];
for (int i = 0; i < proto.getTableSchemaCount(); ++i) {
ret[i] = HTableDescriptor.convert(proto.getTableSchema(i));
ret[i] = convertToHTableDesc(proto.getTableSchema(i));
}
return ret;
}
@ -3313,4 +3304,97 @@ public final class ProtobufUtil {
.addAllServers(hostports)
.addAllTables(tables).build();
}
/**
* Converts an HColumnDescriptor to ColumnFamilySchema
* @param hcd the HColummnDescriptor
* @return Convert this instance to a the pb column family type
*/
public static ColumnFamilySchema convertToColumnFamilySchema(HColumnDescriptor hcd) {
ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
builder.setName(ByteStringer.wrap(hcd.getName()));
for (Map.Entry<Bytes, Bytes> e : hcd.getValues().entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
aBuilder.setFirst(ByteStringer.wrap(e.getKey().get()));
aBuilder.setSecond(ByteStringer.wrap(e.getValue().get()));
builder.addAttributes(aBuilder.build());
}
for (Map.Entry<String, String> e : hcd.getConfiguration().entrySet()) {
NameStringPair.Builder aBuilder = NameStringPair.newBuilder();
aBuilder.setName(e.getKey());
aBuilder.setValue(e.getValue());
builder.addConfiguration(aBuilder.build());
}
return builder.build();
}
/**
* Converts a ColumnFamilySchema to HColumnDescriptor
* @param cfs the ColumnFamilySchema
* @return An {@link HColumnDescriptor} made from the passed in <code>cfs</code>
*/
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 an HTableDescriptor to TableSchema
* @param htd the HTableDescriptor
* @return Convert the current {@link HTableDescriptor} into a pb TableSchema instance.
*/
public static TableSchema convertToTableSchema(HTableDescriptor htd) {
TableSchema.Builder builder = TableSchema.newBuilder();
builder.setTableName(toProtoTableName(htd.getTableName()));
for (Map.Entry<Bytes, Bytes> e : htd.getValues().entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
aBuilder.setFirst(ByteStringer.wrap(e.getKey().get()));
aBuilder.setSecond(ByteStringer.wrap(e.getValue().get()));
builder.addAttributes(aBuilder.build());
}
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
builder.addColumnFamilies(convertToColumnFamilySchema(hcd));
}
for (Map.Entry<String, String> e : htd.getConfiguration().entrySet()) {
NameStringPair.Builder aBuilder = NameStringPair.newBuilder();
aBuilder.setName(e.getKey());
aBuilder.setValue(e.getValue());
builder.addConfiguration(aBuilder.build());
}
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>.
*/
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;
}
}

View File

@ -1034,7 +1034,7 @@ public final class RequestConverter {
final long nonce) {
AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
builder.setColumnFamilies(column.convert());
builder.setColumnFamilies(ProtobufUtil.convertToColumnFamilySchema(column));
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
@ -1074,7 +1074,7 @@ public final class RequestConverter {
final long nonce) {
ModifyColumnRequest.Builder builder = ModifyColumnRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setColumnFamilies(column.convert());
builder.setColumnFamilies(ProtobufUtil.convertToColumnFamilySchema(column));
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
@ -1236,7 +1236,7 @@ public final class RequestConverter {
final long nonceGroup,
final long nonce) {
CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
builder.setTableSchema(hTableDesc.convert());
builder.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDesc));
if (splitKeys != null) {
for (byte [] splitKey : splitKeys) {
builder.addSplitKeys(ByteStringer.wrap(splitKey));
@ -1262,7 +1262,7 @@ public final class RequestConverter {
final long nonce) {
ModifyTableRequest.Builder builder = ModifyTableRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setTableSchema(hTableDesc.convert());
builder.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDesc));
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();

View File

@ -61,7 +61,7 @@ public class TableDescriptor {
@SuppressWarnings("deprecation")
public HBaseProtos.TableDescriptor convert() {
HBaseProtos.TableDescriptor.Builder builder = HBaseProtos.TableDescriptor.newBuilder()
.setSchema(hTableDescriptor.convert());
.setSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
return builder.build();
}
@ -69,7 +69,7 @@ public class TableDescriptor {
* Convert from PB
*/
public static TableDescriptor convert(HBaseProtos.TableDescriptor proto) {
return new TableDescriptor(HTableDescriptor.convert(proto.getSchema()));
return new TableDescriptor(ProtobufUtil.convertToHTableDesc(proto.getSchema()));
}
/**

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.IsolationLevel;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@ -141,7 +142,7 @@ public class TableSnapshotInputFormatImpl {
@Override
public void write(DataOutput out) throws IOException {
TableSnapshotRegionSplit.Builder builder = TableSnapshotRegionSplit.newBuilder()
.setTable(htd.convert())
.setTable(ProtobufUtil.convertToTableSchema(htd))
.setRegion(HRegionInfo.convert(regionInfo));
for (String location : locations) {
@ -168,7 +169,7 @@ public class TableSnapshotInputFormatImpl {
byte[] buf = new byte[len];
in.readFully(buf);
TableSnapshotRegionSplit split = TableSnapshotRegionSplit.PARSER.parseFrom(buf);
this.htd = HTableDescriptor.convert(split.getTable());
this.htd = ProtobufUtil.convertToHTableDesc(split.getTable());
this.regionInfo = HRegionInfo.convert(split.getRegion());
List<String> locationsList = split.getLocationsList();
this.locations = locationsList.toArray(new String[locationsList.size()]);

View File

@ -298,7 +298,7 @@ public class MasterRpcServices extends RSRpcServices
try {
long procId = master.addColumn(
ProtobufUtil.toTableName(req.getTableName()),
HColumnDescriptor.convert(req.getColumnFamilies()),
ProtobufUtil.convertToHColumnDesc(req.getColumnFamilies()),
req.getNonceGroup(),
req.getNonce());
if (procId == -1) {
@ -373,7 +373,7 @@ public class MasterRpcServices extends RSRpcServices
@Override
public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
throws ServiceException {
HTableDescriptor hTableDescriptor = HTableDescriptor.convert(req.getTableSchema());
HTableDescriptor hTableDescriptor = ProtobufUtil.convertToHTableDesc(req.getTableSchema());
byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
try {
long procId =
@ -807,7 +807,7 @@ public class MasterRpcServices extends RSRpcServices
if (descriptors != null && descriptors.size() > 0) {
// Add the table descriptors to the response
for (HTableDescriptor htd: descriptors) {
builder.addTableSchema(htd.convert());
builder.addTableSchema(ProtobufUtil.convertToTableSchema(htd));
}
}
return builder.build();
@ -1032,7 +1032,7 @@ public class MasterRpcServices extends RSRpcServices
ListTableDescriptorsByNamespaceResponse.newBuilder();
for (HTableDescriptor htd : master
.listTableDescriptorsByNamespace(request.getNamespaceName())) {
b.addTableSchema(htd.convert());
b.addTableSchema(ProtobufUtil.convertToTableSchema(htd));
}
return b.build();
} catch (IOException e) {
@ -1061,7 +1061,7 @@ public class MasterRpcServices extends RSRpcServices
try {
long procId = master.modifyColumn(
ProtobufUtil.toTableName(req.getTableName()),
HColumnDescriptor.convert(req.getColumnFamilies()),
ProtobufUtil.convertToHColumnDesc(req.getColumnFamilies()),
req.getNonceGroup(),
req.getNonce());
if (procId == -1) {
@ -1095,7 +1095,7 @@ public class MasterRpcServices extends RSRpcServices
try {
long procId = master.modifyTable(
ProtobufUtil.toTableName(req.getTableName()),
HTableDescriptor.convert(req.getTableSchema()),
ProtobufUtil.convertToHTableDesc(req.getTableSchema()),
req.getNonceGroup(),
req.getNonce());
return ModifyTableResponse.newBuilder().setProcId(procId).build();

View File

@ -201,9 +201,10 @@ public class AddColumnFamilyProcedure
MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setColumnfamilySchema(cfDescriptor.convert());
.setColumnfamilySchema(ProtobufUtil.convertToColumnFamilySchema(cfDescriptor));
if (unmodifiedHTableDescriptor != null) {
addCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
addCFMsg
.setUnmodifiedTableSchema(ProtobufUtil.convertToTableSchema(unmodifiedHTableDescriptor));
}
addCFMsg.build().writeDelimitedTo(stream);
@ -217,9 +218,9 @@ public class AddColumnFamilyProcedure
MasterProcedureProtos.AddColumnFamilyStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(addCFMsg.getUserInfo());
tableName = ProtobufUtil.toTableName(addCFMsg.getTableName());
cfDescriptor = HColumnDescriptor.convert(addCFMsg.getColumnfamilySchema());
cfDescriptor = ProtobufUtil.convertToHColumnDesc(addCFMsg.getColumnfamilySchema());
if (addCFMsg.hasUnmodifiedTableSchema()) {
unmodifiedHTableDescriptor = HTableDescriptor.convert(addCFMsg.getUnmodifiedTableSchema());
unmodifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(addCFMsg.getUnmodifiedTableSchema());
}
}

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateHdfsR
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
@ -268,7 +269,7 @@ public class CloneSnapshotProcedure
MasterProcedureProtos.CloneSnapshotStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
.setSnapshot(this.snapshot)
.setTableSchema(hTableDescriptor.convert());
.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
if (newRegions != null) {
for (HRegionInfo hri: newRegions) {
cloneSnapshotMsg.addRegionInfo(HRegionInfo.convert(hri));
@ -299,7 +300,7 @@ public class CloneSnapshotProcedure
MasterProcedureProtos.CloneSnapshotStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(cloneSnapshotMsg.getUserInfo());
snapshot = cloneSnapshotMsg.getSnapshot();
hTableDescriptor = HTableDescriptor.convert(cloneSnapshotMsg.getTableSchema());
hTableDescriptor = ProtobufUtil.convertToHTableDesc(cloneSnapshotMsg.getTableSchema());
if (cloneSnapshotMsg.getRegionInfoCount() == 0) {
newRegions = null;
} else {

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
@ -238,7 +239,7 @@ public class CreateTableProcedure
MasterProcedureProtos.CreateTableStateData.Builder state =
MasterProcedureProtos.CreateTableStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
.setTableSchema(hTableDescriptor.convert());
.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
if (newRegions != null) {
for (HRegionInfo hri: newRegions) {
state.addRegionInfo(HRegionInfo.convert(hri));
@ -254,7 +255,7 @@ public class CreateTableProcedure
MasterProcedureProtos.CreateTableStateData state =
MasterProcedureProtos.CreateTableStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
hTableDescriptor = HTableDescriptor.convert(state.getTableSchema());
hTableDescriptor = ProtobufUtil.convertToHTableDesc(state.getTableSchema());
if (state.getRegionInfoCount() == 0) {
newRegions = null;
} else {

View File

@ -220,7 +220,8 @@ public class DeleteColumnFamilyProcedure
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setColumnfamilyName(ByteStringer.wrap(familyName));
if (unmodifiedHTableDescriptor != null) {
deleteCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
deleteCFMsg
.setUnmodifiedTableSchema(ProtobufUtil.convertToTableSchema(unmodifiedHTableDescriptor));
}
deleteCFMsg.build().writeDelimitedTo(stream);
@ -236,7 +237,7 @@ public class DeleteColumnFamilyProcedure
familyName = deleteCFMsg.getColumnfamilyName().toByteArray();
if (deleteCFMsg.hasUnmodifiedTableSchema()) {
unmodifiedHTableDescriptor = HTableDescriptor.convert(deleteCFMsg.getUnmodifiedTableSchema());
unmodifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(deleteCFMsg.getUnmodifiedTableSchema());
}
}

View File

@ -198,9 +198,10 @@ public class ModifyColumnFamilyProcedure
MasterProcedureProtos.ModifyColumnFamilyStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setColumnfamilySchema(cfDescriptor.convert());
.setColumnfamilySchema(ProtobufUtil.convertToColumnFamilySchema(cfDescriptor));
if (unmodifiedHTableDescriptor != null) {
modifyCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
modifyCFMsg
.setUnmodifiedTableSchema(ProtobufUtil.convertToTableSchema(unmodifiedHTableDescriptor));
}
modifyCFMsg.build().writeDelimitedTo(stream);
@ -214,9 +215,9 @@ public class ModifyColumnFamilyProcedure
MasterProcedureProtos.ModifyColumnFamilyStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(modifyCFMsg.getUserInfo());
tableName = ProtobufUtil.toTableName(modifyCFMsg.getTableName());
cfDescriptor = HColumnDescriptor.convert(modifyCFMsg.getColumnfamilySchema());
cfDescriptor = ProtobufUtil.convertToHColumnDesc(modifyCFMsg.getColumnfamilySchema());
if (modifyCFMsg.hasUnmodifiedTableSchema()) {
unmodifiedHTableDescriptor = HTableDescriptor.convert(modifyCFMsg.getUnmodifiedTableSchema());
unmodifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(modifyCFMsg.getUnmodifiedTableSchema());
}
}

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableState;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@ -231,11 +232,12 @@ public class ModifyTableProcedure
MasterProcedureProtos.ModifyTableStateData.Builder modifyTableMsg =
MasterProcedureProtos.ModifyTableStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
.setModifiedTableSchema(modifiedHTableDescriptor.convert())
.setModifiedTableSchema(ProtobufUtil.convertToTableSchema(modifiedHTableDescriptor))
.setDeleteColumnFamilyInModify(deleteColumnFamilyInModify);
if (unmodifiedHTableDescriptor != null) {
modifyTableMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
modifyTableMsg
.setUnmodifiedTableSchema(ProtobufUtil.convertToTableSchema(unmodifiedHTableDescriptor));
}
modifyTableMsg.build().writeDelimitedTo(stream);
@ -248,12 +250,12 @@ public class ModifyTableProcedure
MasterProcedureProtos.ModifyTableStateData modifyTableMsg =
MasterProcedureProtos.ModifyTableStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(modifyTableMsg.getUserInfo());
modifiedHTableDescriptor = HTableDescriptor.convert(modifyTableMsg.getModifiedTableSchema());
modifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(modifyTableMsg.getModifiedTableSchema());
deleteColumnFamilyInModify = modifyTableMsg.getDeleteColumnFamilyInModify();
if (modifyTableMsg.hasUnmodifiedTableSchema()) {
unmodifiedHTableDescriptor =
HTableDescriptor.convert(modifyTableMsg.getUnmodifiedTableSchema());
ProtobufUtil.convertToHTableDesc(modifyTableMsg.getUnmodifiedTableSchema());
}
}

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
@ -235,7 +236,7 @@ public class RestoreSnapshotProcedure
MasterProcedureProtos.RestoreSnapshotStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
.setSnapshot(this.snapshot)
.setModifiedTableSchema(modifiedHTableDescriptor.convert());
.setModifiedTableSchema(ProtobufUtil.convertToTableSchema(modifiedHTableDescriptor));
if (regionsToRestore != null) {
for (HRegionInfo hri: regionsToRestore) {
@ -278,7 +279,7 @@ public class RestoreSnapshotProcedure
user = MasterProcedureUtil.toUserInfo(restoreSnapshotMsg.getUserInfo());
snapshot = restoreSnapshotMsg.getSnapshot();
modifiedHTableDescriptor =
HTableDescriptor.convert(restoreSnapshotMsg.getModifiedTableSchema());
ProtobufUtil.convertToHTableDesc(restoreSnapshotMsg.getModifiedTableSchema());
if (restoreSnapshotMsg.getRegionInfoForRestoreCount() == 0) {
regionsToRestore = null;

View File

@ -210,7 +210,7 @@ public class TruncateTableProcedure
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
.setPreserveSplits(preserveSplits);
if (hTableDescriptor != null) {
state.setTableSchema(hTableDescriptor.convert());
state.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
} else {
state.setTableName(ProtobufUtil.toProtoTableName(tableName));
}
@ -230,7 +230,7 @@ public class TruncateTableProcedure
MasterProcedureProtos.TruncateTableStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
if (state.hasTableSchema()) {
hTableDescriptor = HTableDescriptor.convert(state.getTableSchema());
hTableDescriptor = ProtobufUtil.convertToHTableDesc(state.getTableSchema());
tableName = hTableDescriptor.getTableName();
} else {
tableName = ProtobufUtil.toTableName(state.getTableName());

View File

@ -360,7 +360,7 @@ public final class SnapshotManifest {
case SnapshotManifestV2.DESCRIPTOR_VERSION: {
SnapshotDataManifest dataManifest = readDataManifest();
if (dataManifest != null) {
htd = HTableDescriptor.convert(dataManifest.getTableSchema());
htd = ProtobufUtil.convertToHTableDesc(dataManifest.getTableSchema());
regionManifests = dataManifest.getRegionManifestsList();
} else {
// Compatibility, load the v1 regions
@ -465,7 +465,7 @@ public final class SnapshotManifest {
}
SnapshotDataManifest.Builder dataManifestBuilder = SnapshotDataManifest.newBuilder();
dataManifestBuilder.setTableSchema(htd.convert());
dataManifestBuilder.setTableSchema(ProtobufUtil.convertToTableSchema(htd));
if (v1Regions != null && v1Regions.size() > 0) {
dataManifestBuilder.addAllRegionManifests(v1Regions);

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@ -104,7 +105,8 @@ public class TestSnapshotManifest {
startKey = stopKey;
}
dataManifestBuilder.setTableSchema(builder.getTableDescriptor().convert());
dataManifestBuilder
.setTableSchema(ProtobufUtil.convertToTableSchema(builder.getTableDescriptor()));
SnapshotDataManifest dataManifest = dataManifestBuilder.build();
writeDataManifest(dataManifest);