HBASE-13016 Clean up remnants of table states stored in table descriptors (Andrey Stepachev)

This commit is contained in:
stack 2015-02-11 11:08:47 -08:00
parent a0f2bc07b2
commit dec0ec2290
6 changed files with 61 additions and 564 deletions

View File

@ -22,7 +22,6 @@ 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.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
/**
@ -94,28 +93,17 @@ public class TableState {
}
private final long timestamp;
private final TableName tableName;
private final State state;
/**
* Create instance of TableState.
* @param tableName name of the table
* @param state table state
*/
public TableState(TableName tableName, State state, long timestamp) {
public TableState(TableName tableName, State state) {
this.tableName = tableName;
this.state = state;
this.timestamp = timestamp;
}
/**
* Create instance of TableState with current timestamp
*
* @param tableName table for which state is created
* @param state state of the table
*/
public TableState(TableName tableName, State state) {
this(tableName, state, System.currentTimeMillis());
}
/**
@ -125,15 +113,6 @@ public class TableState {
return state;
}
/**
* Timestamp of table state
*
* @return milliseconds
*/
public long getTimestamp() {
return timestamp;
}
/**
* Table name for state
*
@ -172,10 +151,7 @@ public class TableState {
*/
public HBaseProtos.TableState convert() {
return HBaseProtos.TableState.newBuilder()
.setState(this.state.convert())
.setTable(ProtobufUtil.toProtoTableName(this.tableName)) // set for backward compatibility
.setTimestamp(this.timestamp)
.build();
.setState(this.state.convert()).build();
}
/**
@ -187,7 +163,7 @@ public class TableState {
*/
public static TableState convert(TableName tableName, HBaseProtos.TableState tableState) {
TableState.State state = State.convert(tableState.getState());
return new TableState(tableName, state, tableState.getTimestamp());
return new TableState(tableName, state);
}
public static TableState parseFrom(TableName tableName, byte[] bytes)
@ -220,7 +196,6 @@ public class TableState {
TableState that = (TableState) o;
if (timestamp != that.timestamp) return false;
if (state != that.state) return false;
if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null)
return false;
@ -230,8 +205,7 @@ public class TableState {
@Override
public int hashCode() {
int result = (int) (timestamp ^ (timestamp >>> 32));
result = 31 * result + (tableName != null ? tableName.hashCode() : 0);
int result = (tableName != null ? tableName.hashCode() : 0);
result = 31 * result + (state != null ? state.hashCode() : 0);
return result;
}
@ -239,7 +213,6 @@ public class TableState {
@Override
public String toString() {
return "TableState{" +
"timestamp=" + timestamp +
", tableName=" + tableName +
", state=" + state +
'}';

View File

@ -2551,30 +2551,6 @@ public final class HBaseProtos {
* </pre>
*/
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState();
// required .TableName table = 2;
/**
* <code>required .TableName table = 2;</code>
*/
boolean hasTable();
/**
* <code>required .TableName table = 2;</code>
*/
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable();
/**
* <code>required .TableName table = 2;</code>
*/
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder();
// optional uint64 timestamp = 3;
/**
* <code>optional uint64 timestamp = 3;</code>
*/
boolean hasTimestamp();
/**
* <code>optional uint64 timestamp = 3;</code>
*/
long getTimestamp();
}
/**
* Protobuf type {@code TableState}
@ -2642,24 +2618,6 @@ public final class HBaseProtos {
}
break;
}
case 18: {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
if (((bitField0_ & 0x00000002) == 0x00000002)) {
subBuilder = table_.toBuilder();
}
table_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
if (subBuilder != null) {
subBuilder.mergeFrom(table_);
table_ = subBuilder.buildPartial();
}
bitField0_ |= 0x00000002;
break;
}
case 24: {
bitField0_ |= 0x00000004;
timestamp_ = input.readUInt64();
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -2828,48 +2786,8 @@ public final class HBaseProtos {
return state_;
}
// required .TableName table = 2;
public static final int TABLE_FIELD_NUMBER = 2;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName table_;
/**
* <code>required .TableName table = 2;</code>
*/
public boolean hasTable() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
* <code>required .TableName table = 2;</code>
*/
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable() {
return table_;
}
/**
* <code>required .TableName table = 2;</code>
*/
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder() {
return table_;
}
// optional uint64 timestamp = 3;
public static final int TIMESTAMP_FIELD_NUMBER = 3;
private long timestamp_;
/**
* <code>optional uint64 timestamp = 3;</code>
*/
public boolean hasTimestamp() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
/**
* <code>optional uint64 timestamp = 3;</code>
*/
public long getTimestamp() {
return timestamp_;
}
private void initFields() {
state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
timestamp_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -2880,14 +2798,6 @@ public final class HBaseProtos {
memoizedIsInitialized = 0;
return false;
}
if (!hasTable()) {
memoizedIsInitialized = 0;
return false;
}
if (!getTable().isInitialized()) {
memoizedIsInitialized = 0;
return false;
}
memoizedIsInitialized = 1;
return true;
}
@ -2898,12 +2808,6 @@ public final class HBaseProtos {
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeEnum(1, state_.getNumber());
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
output.writeMessage(2, table_);
}
if (((bitField0_ & 0x00000004) == 0x00000004)) {
output.writeUInt64(3, timestamp_);
}
getUnknownFields().writeTo(output);
}
@ -2917,14 +2821,6 @@ public final class HBaseProtos {
size += com.google.protobuf.CodedOutputStream
.computeEnumSize(1, state_.getNumber());
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(2, table_);
}
if (((bitField0_ & 0x00000004) == 0x00000004)) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(3, timestamp_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@ -2953,16 +2849,6 @@ public final class HBaseProtos {
result = result &&
(getState() == other.getState());
}
result = result && (hasTable() == other.hasTable());
if (hasTable()) {
result = result && getTable()
.equals(other.getTable());
}
result = result && (hasTimestamp() == other.hasTimestamp());
if (hasTimestamp()) {
result = result && (getTimestamp()
== other.getTimestamp());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@ -2980,14 +2866,6 @@ public final class HBaseProtos {
hash = (37 * hash) + STATE_FIELD_NUMBER;
hash = (53 * hash) + hashEnum(getState());
}
if (hasTable()) {
hash = (37 * hash) + TABLE_FIELD_NUMBER;
hash = (53 * hash) + getTable().hashCode();
}
if (hasTimestamp()) {
hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getTimestamp());
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@ -3093,7 +2971,6 @@ public final class HBaseProtos {
}
private void maybeForceBuilderInitialization() {
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
getTableFieldBuilder();
}
}
private static Builder create() {
@ -3104,14 +2981,6 @@ public final class HBaseProtos {
super.clear();
state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
bitField0_ = (bitField0_ & ~0x00000001);
if (tableBuilder_ == null) {
table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
} else {
tableBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000002);
timestamp_ = 0L;
bitField0_ = (bitField0_ & ~0x00000004);
return this;
}
@ -3144,18 +3013,6 @@ public final class HBaseProtos {
to_bitField0_ |= 0x00000001;
}
result.state_ = state_;
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
to_bitField0_ |= 0x00000002;
}
if (tableBuilder_ == null) {
result.table_ = table_;
} else {
result.table_ = tableBuilder_.build();
}
if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
to_bitField0_ |= 0x00000004;
}
result.timestamp_ = timestamp_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -3175,12 +3032,6 @@ public final class HBaseProtos {
if (other.hasState()) {
setState(other.getState());
}
if (other.hasTable()) {
mergeTable(other.getTable());
}
if (other.hasTimestamp()) {
setTimestamp(other.getTimestamp());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@ -3190,14 +3041,6 @@ public final class HBaseProtos {
return false;
}
if (!hasTable()) {
return false;
}
if (!getTable().isInitialized()) {
return false;
}
return true;
}
@ -3272,156 +3115,6 @@ public final class HBaseProtos {
return this;
}
// required .TableName table = 2;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableBuilder_;
/**
* <code>required .TableName table = 2;</code>
*/
public boolean hasTable() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
* <code>required .TableName table = 2;</code>
*/
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable() {
if (tableBuilder_ == null) {
return table_;
} else {
return tableBuilder_.getMessage();
}
}
/**
* <code>required .TableName table = 2;</code>
*/
public Builder setTable(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
if (tableBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
}
table_ = value;
onChanged();
} else {
tableBuilder_.setMessage(value);
}
bitField0_ |= 0x00000002;
return this;
}
/**
* <code>required .TableName table = 2;</code>
*/
public Builder setTable(
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
if (tableBuilder_ == null) {
table_ = builderForValue.build();
onChanged();
} else {
tableBuilder_.setMessage(builderForValue.build());
}
bitField0_ |= 0x00000002;
return this;
}
/**
* <code>required .TableName table = 2;</code>
*/
public Builder mergeTable(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
if (tableBuilder_ == null) {
if (((bitField0_ & 0x00000002) == 0x00000002) &&
table_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
table_ =
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(table_).mergeFrom(value).buildPartial();
} else {
table_ = value;
}
onChanged();
} else {
tableBuilder_.mergeFrom(value);
}
bitField0_ |= 0x00000002;
return this;
}
/**
* <code>required .TableName table = 2;</code>
*/
public Builder clearTable() {
if (tableBuilder_ == null) {
table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
onChanged();
} else {
tableBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000002);
return this;
}
/**
* <code>required .TableName table = 2;</code>
*/
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableBuilder() {
bitField0_ |= 0x00000002;
onChanged();
return getTableFieldBuilder().getBuilder();
}
/**
* <code>required .TableName table = 2;</code>
*/
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder() {
if (tableBuilder_ != null) {
return tableBuilder_.getMessageOrBuilder();
} else {
return table_;
}
}
/**
* <code>required .TableName table = 2;</code>
*/
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>
getTableFieldBuilder() {
if (tableBuilder_ == null) {
tableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
table_,
getParentForChildren(),
isClean());
table_ = null;
}
return tableBuilder_;
}
// optional uint64 timestamp = 3;
private long timestamp_ ;
/**
* <code>optional uint64 timestamp = 3;</code>
*/
public boolean hasTimestamp() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
/**
* <code>optional uint64 timestamp = 3;</code>
*/
public long getTimestamp() {
return timestamp_;
}
/**
* <code>optional uint64 timestamp = 3;</code>
*/
public Builder setTimestamp(long value) {
bitField0_ |= 0x00000004;
timestamp_ = value;
onChanged();
return this;
}
/**
* <code>optional uint64 timestamp = 3;</code>
*/
public Builder clearTimestamp() {
bitField0_ = (bitField0_ & ~0x00000004);
timestamp_ = 0L;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:TableState)
}
@ -3449,16 +3142,6 @@ public final class HBaseProtos {
* <code>required .TableSchema schema = 1;</code>
*/
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getSchemaOrBuilder();
// optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
/**
* <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
*/
@java.lang.Deprecated boolean hasState();
/**
* <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
*/
@java.lang.Deprecated org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState();
}
/**
* Protobuf type {@code TableDescriptor}
@ -3528,17 +3211,6 @@ public final class HBaseProtos {
bitField0_ |= 0x00000001;
break;
}
case 16: {
int rawValue = input.readEnum();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.valueOf(rawValue);
if (value == null) {
unknownFields.mergeVarintField(2, rawValue);
} else {
bitField0_ |= 0x00000002;
state_ = value;
}
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -3601,25 +3273,8 @@ public final class HBaseProtos {
return schema_;
}
// optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
public static final int STATE_FIELD_NUMBER = 2;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_;
/**
* <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
*/
@java.lang.Deprecated public boolean hasState() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
* <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
*/
@java.lang.Deprecated public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
return state_;
}
private void initFields() {
schema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -3644,9 +3299,6 @@ public final class HBaseProtos {
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeMessage(1, schema_);
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
output.writeEnum(2, state_.getNumber());
}
getUnknownFields().writeTo(output);
}
@ -3660,10 +3312,6 @@ public final class HBaseProtos {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, schema_);
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
size += com.google.protobuf.CodedOutputStream
.computeEnumSize(2, state_.getNumber());
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@ -3692,11 +3340,6 @@ public final class HBaseProtos {
result = result && getSchema()
.equals(other.getSchema());
}
result = result && (hasState() == other.hasState());
if (hasState()) {
result = result &&
(getState() == other.getState());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@ -3714,10 +3357,6 @@ public final class HBaseProtos {
hash = (37 * hash) + SCHEMA_FIELD_NUMBER;
hash = (53 * hash) + getSchema().hashCode();
}
if (hasState()) {
hash = (37 * hash) + STATE_FIELD_NUMBER;
hash = (53 * hash) + hashEnum(getState());
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@ -3838,8 +3477,6 @@ public final class HBaseProtos {
schemaBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000001);
state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
bitField0_ = (bitField0_ & ~0x00000002);
return this;
}
@ -3876,10 +3513,6 @@ public final class HBaseProtos {
} else {
result.schema_ = schemaBuilder_.build();
}
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
to_bitField0_ |= 0x00000002;
}
result.state_ = state_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -3899,9 +3532,6 @@ public final class HBaseProtos {
if (other.hasSchema()) {
mergeSchema(other.getSchema());
}
if (other.hasState()) {
setState(other.getState());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@ -4054,42 +3684,6 @@ public final class HBaseProtos {
return schemaBuilder_;
}
// optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
/**
* <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
*/
@java.lang.Deprecated public boolean hasState() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
* <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
*/
@java.lang.Deprecated public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
return state_;
}
/**
* <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
*/
@java.lang.Deprecated public Builder setState(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00000002;
state_ = value;
onChanged();
return this;
}
/**
* <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
*/
@java.lang.Deprecated public Builder clearState() {
bitField0_ = (bitField0_ & ~0x00000002);
state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:TableDescriptor)
}
@ -18193,56 +17787,53 @@ public final class HBaseProtos {
"Name\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPai" +
"r\022,\n\017column_families\030\003 \003(\0132\023.ColumnFamil" +
"ySchema\022&\n\rconfiguration\030\004 \003(\0132\017.NameStr" +
"ingPair\"\235\001\n\nTableState\022 \n\005state\030\001 \002(\0162\021." +
"TableState.State\022\031\n\005table\030\002 \002(\0132\n.TableN" +
"ame\022\021\n\ttimestamp\030\003 \001(\004\"?\n\005State\022\013\n\007ENABL" +
"ED\020\000\022\014\n\010DISABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENA",
"BLING\020\003\"^\n\017TableDescriptor\022\034\n\006schema\030\001 \002" +
"(\0132\014.TableSchema\022-\n\005state\030\002 \001(\0162\021.TableS" +
"tate.State:\007ENABLEDB\002\030\001\"o\n\022ColumnFamilyS" +
"chema\022\014\n\004name\030\001 \002(\014\022#\n\nattributes\030\002 \003(\0132" +
"\017.BytesBytesPair\022&\n\rconfiguration\030\003 \003(\0132" +
"\017.NameStringPair\"\232\001\n\nRegionInfo\022\021\n\tregio" +
"n_id\030\001 \002(\004\022\036\n\ntable_name\030\002 \002(\0132\n.TableNa" +
"me\022\021\n\tstart_key\030\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017" +
"\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\022\025\n\nreplic" +
"a_id\030\007 \001(\005:\0010\"1\n\014FavoredNodes\022!\n\014favored",
"_node\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpeci" +
"fier\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.Reg" +
"ionSpecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023Regio" +
"nSpecifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCOD" +
"ED_REGION_NAME\020\002\"%\n\tTimeRange\022\014\n\004from\030\001 " +
"\001(\004\022\n\n\002to\030\002 \001(\004\"A\n\nServerName\022\021\n\thost_na" +
"me\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\022\n\nstart_code\030\003 \001" +
"(\004\"\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameS" +
"tringPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\"," +
"\n\rNameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002",
" \001(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n" +
"\006second\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030\001" +
" \001(\t\022\r\n\005value\030\002 \001(\003\"\314\001\n\023SnapshotDescript" +
"ion\022\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcrea" +
"tion_time\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.Snaps" +
"hotDescription.Type:\005FLUSH\022\017\n\007version\030\005 " +
"\001(\005\022\r\n\005owner\030\006 \001(\t\".\n\004Type\022\014\n\010DISABLED\020\000" +
"\022\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"}\n\024ProcedureD" +
"escription\022\021\n\tsignature\030\001 \002(\t\022\020\n\010instanc" +
"e\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022&\n\rcon",
"figuration\030\004 \003(\0132\017.NameStringPair\"\n\n\010Emp" +
"tyMsg\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDo" +
"ubleMsg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rBigDecima" +
"lMsg\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016" +
"least_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 " +
"\002(\004\"K\n\023NamespaceDescriptor\022\014\n\004name\030\001 \002(\014" +
"\022&\n\rconfiguration\030\002 \003(\0132\017.NameStringPair" +
"\"$\n\020RegionServerInfo\022\020\n\010infoPort\030\001 \001(\005*r" +
"\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL" +
"\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_",
"OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010Ti" +
"meUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECONDS\020" +
"\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINU" +
"TES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.apache" +
".hadoop.hbase.protobuf.generatedB\013HBaseP" +
"rotosH\001\240\001\001"
"ingPair\"o\n\nTableState\022 \n\005state\030\001 \002(\0162\021.T" +
"ableState.State\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n" +
"\010DISABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003" +
"\"/\n\017TableDescriptor\022\034\n\006schema\030\001 \002(\0132\014.Ta",
"bleSchema\"o\n\022ColumnFamilySchema\022\014\n\004name\030" +
"\001 \002(\014\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPa" +
"ir\022&\n\rconfiguration\030\003 \003(\0132\017.NameStringPa" +
"ir\"\232\001\n\nRegionInfo\022\021\n\tregion_id\030\001 \002(\004\022\036\n\n" +
"table_name\030\002 \002(\0132\n.TableName\022\021\n\tstart_ke" +
"y\030\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017\n\007offline\030\005 \001(" +
"\010\022\r\n\005split\030\006 \001(\010\022\025\n\nreplica_id\030\007 \001(\005:\0010\"" +
"1\n\014FavoredNodes\022!\n\014favored_node\030\001 \003(\0132\013." +
"ServerName\"\225\001\n\017RegionSpecifier\0222\n\004type\030\001" +
" \002(\0162$.RegionSpecifier.RegionSpecifierTy",
"pe\022\r\n\005value\030\002 \002(\014\"?\n\023RegionSpecifierType" +
"\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCODED_REGION_NAME" +
"\020\002\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(" +
"\004\"A\n\nServerName\022\021\n\thost_name\030\001 \002(\t\022\014\n\004po" +
"rt\030\002 \001(\r\022\022\n\nstart_code\030\003 \001(\004\"\033\n\013Coproces" +
"sor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004n" +
"ame\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPai" +
"r\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesB" +
"ytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\"" +
",\n\rNameInt64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030",
"\002 \001(\003\"\314\001\n\023SnapshotDescription\022\014\n\004name\030\001 " +
"\002(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(" +
"\003:\0010\022.\n\004type\030\004 \001(\0162\031.SnapshotDescription" +
".Type:\005FLUSH\022\017\n\007version\030\005 \001(\005\022\r\n\005owner\030\006" +
" \001(\t\".\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\022\r\n" +
"\tSKIPFLUSH\020\002\"}\n\024ProcedureDescription\022\021\n\t" +
"signature\030\001 \002(\t\022\020\n\010instance\030\002 \001(\t\022\030\n\rcre" +
"ation_time\030\003 \001(\003:\0010\022&\n\rconfiguration\030\004 \003" +
"(\0132\017.NameStringPair\"\n\n\010EmptyMsg\"\033\n\007LongM" +
"sg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDoubleMsg\022\022\n\ndou",
"ble_msg\030\001 \002(\001\"\'\n\rBigDecimalMsg\022\026\n\016bigdec" +
"imal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016least_sig_bits" +
"\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 \002(\004\"K\n\023Namespa" +
"ceDescriptor\022\014\n\004name\030\001 \002(\014\022&\n\rconfigurat" +
"ion\030\002 \003(\0132\017.NameStringPair\"$\n\020RegionServ" +
"erInfo\022\020\n\010infoPort\030\001 \001(\005*r\n\013CompareType\022" +
"\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022" +
"\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007" +
"GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUnit\022\017\n\013NANO" +
"SECONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n\014MILLISECO",
"NDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOURS" +
"\020\006\022\010\n\004DAYS\020\007B>\n*org.apache.hadoop.hbase." +
"protobuf.generatedB\013HBaseProtosH\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -18266,13 +17857,13 @@ public final class HBaseProtos {
internal_static_TableState_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_TableState_descriptor,
new java.lang.String[] { "State", "Table", "Timestamp", });
new java.lang.String[] { "State", });
internal_static_TableDescriptor_descriptor =
getDescriptor().getMessageTypes().get(3);
internal_static_TableDescriptor_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_TableDescriptor_descriptor,
new java.lang.String[] { "Schema", "State", });
new java.lang.String[] { "Schema", });
internal_static_ColumnFamilySchema_descriptor =
getDescriptor().getMessageTypes().get(4);
internal_static_ColumnFamilySchema_fieldAccessorTable = new

View File

@ -55,14 +55,11 @@ message TableState {
}
// This is the table's state.
required State state = 1;
required TableName table = 2 [deprecated = true];
optional uint64 timestamp = 3;
}
/** On HDFS representation of table state. */
message TableDescriptor {
required TableSchema schema = 1;
optional TableState.State state = 2 [ default = ENABLED, deprecated = true ];
}
/**

View File

@ -17,14 +17,12 @@
*/
package org.apache.hadoop.hbase;
import javax.annotation.Nullable;
import java.io.IOException;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@ -36,26 +34,6 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
@InterfaceAudience.Private
public class TableDescriptor {
private HTableDescriptor hTableDescriptor;
/**
* Don't use, state was moved to meta, use MetaTableAccessor instead
* @deprecated state was moved to meta
*/
@Deprecated
@Nullable
private TableState.State tableState;
/**
* Creates TableDescriptor with all fields.
* @param hTableDescriptor HTableDescriptor to use
* @param tableState table state
* @deprecated state was moved to meta
*/
@Deprecated
public TableDescriptor(HTableDescriptor hTableDescriptor,
@Nullable TableState.State tableState) {
this.hTableDescriptor = hTableDescriptor;
this.tableState = tableState;
}
/**
* Creates TableDescriptor with Enabled table.
@ -63,7 +41,7 @@ public class TableDescriptor {
*/
@VisibleForTesting
public TableDescriptor(HTableDescriptor hTableDescriptor) {
this(hTableDescriptor, TableState.State.ENABLED);
this.hTableDescriptor = hTableDescriptor;
}
/**
@ -78,25 +56,6 @@ public class TableDescriptor {
this.hTableDescriptor = hTableDescriptor;
}
/**
* @return table state
* @deprecated state was moved to meta
*/
@Deprecated
@Nullable
public TableState.State getTableState() {
return tableState;
}
/**
* @param tableState state to set for table
* @deprecated state was moved to meta
*/
@Deprecated
public void setTableState(@Nullable TableState.State tableState) {
this.tableState = tableState;
}
/**
* Convert to PB.
*/
@ -104,8 +63,6 @@ public class TableDescriptor {
public HBaseProtos.TableDescriptor convert() {
HBaseProtos.TableDescriptor.Builder builder = HBaseProtos.TableDescriptor.newBuilder()
.setSchema(hTableDescriptor.convert());
if (tableState!= null)
builder.setState(tableState.convert());
return builder.build();
}
@ -113,11 +70,7 @@ public class TableDescriptor {
* Convert from PB
*/
public static TableDescriptor convert(HBaseProtos.TableDescriptor proto) {
HTableDescriptor hTableDescriptor = HTableDescriptor.convert(proto.getSchema());
TableState.State state = proto.hasState()?
TableState.State.convert(proto.getState())
:null;
return new TableDescriptor(hTableDescriptor, state);
return new TableDescriptor(HTableDescriptor.convert(proto.getSchema()));
}
/**
@ -158,23 +111,18 @@ public class TableDescriptor {
if (hTableDescriptor != null ?
!hTableDescriptor.equals(that.hTableDescriptor) :
that.hTableDescriptor != null) return false;
if (tableState != that.tableState) return false;
return true;
}
@Override
public int hashCode() {
int result = hTableDescriptor != null ? hTableDescriptor.hashCode() : 0;
result = 31 * result + (tableState != null ? tableState.hashCode() : 0);
return result;
return hTableDescriptor != null ? hTableDescriptor.hashCode() : 0;
}
@Override
public String toString() {
return "TableDescriptor{" +
"hTableDescriptor=" + hTableDescriptor +
", tableState=" + tableState +
'}';
}

View File

@ -214,20 +214,9 @@ public class TableStateManager {
if (table.equals(TableName.META_TABLE_NAME.getNameAsString()))
continue;
if (!states.containsKey(table)) {
LOG.warn("Found table without state " + table);
TableDescriptor td = entry.getValue();
TableState.State tds = td.getTableState();
if (tds != null) {
LOG.warn("Found table with state in descriptor, using that state");
MetaTableAccessor.updateTableState(connection, TableName.valueOf(table), tds);
LOG.warn("Updating table descriptor");
td.setTableState(null);
tableDescriptors.add(td);
} else {
LOG.warn("Found table with no state in descriptor, assuming ENABLED");
MetaTableAccessor.updateTableState(connection, TableName.valueOf(table),
TableState.State.ENABLED);
}
LOG.warn("Found table with no state, assuming ENABLED");
MetaTableAccessor.updateTableState(connection, TableName.valueOf(table),
TableState.State.ENABLED);
}
}
}

View File

@ -47,11 +47,10 @@ public class TestTableDescriptor {
htd.setDurability(Durability.ASYNC_WAL);
htd.setReadOnly(true);
htd.setRegionReplication(2);
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
TableDescriptor td = new TableDescriptor(htd);
byte[] bytes = td.toByteArray();
TableDescriptor deserializedTd = TableDescriptor.parseFrom(bytes);
assertEquals(td, deserializedTd);
assertEquals(td.getHTableDescriptor(), deserializedTd.getHTableDescriptor());
assertEquals(td.getTableState(), deserializedTd.getTableState());
}
}