HBASE-7767 Get rid of ZKTable, and table enable/disable state in ZK (Andrey Stepachev)

This commit is contained in:
stack 2014-09-15 09:34:10 -07:00
parent f8c3a5b156
commit 3cc5d19039
68 changed files with 4515 additions and 1494 deletions

View File

@ -169,6 +169,11 @@ class ConnectionAdapter implements ClusterConnection {
return wrappedConnection.isTableAvailable(tableName, splitKeys); return wrappedConnection.isTableAvailable(tableName, splitKeys);
} }
@Override
public TableState getTableState(TableName tableName) throws IOException {
return wrappedConnection.getTableState(tableName);
}
@Override @Override
public HTableDescriptor[] listTables() throws IOException { public HTableDescriptor[] listTables() throws IOException {
return wrappedConnection.listTables(); return wrappedConnection.listTables();
@ -435,4 +440,4 @@ class ConnectionAdapter implements ClusterConnection {
public AsyncProcess getAsyncProcess() { public AsyncProcess getAsyncProcess() {
return wrappedConnection.getAsyncProcess(); return wrappedConnection.getAsyncProcess();
} }
} }

View File

@ -176,6 +176,8 @@ import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
/** /**
* An internal, A non-instantiable class that manages creation of {@link HConnection}s. * An internal, A non-instantiable class that manages creation of {@link HConnection}s.
*/ */
@ -893,7 +895,7 @@ class ConnectionManager {
@Override @Override
public boolean isTableEnabled(TableName tableName) throws IOException { public boolean isTableEnabled(TableName tableName) throws IOException {
return this.registry.isTableOnlineState(tableName, true); return getTableState(tableName).inStates(TableState.State.ENABLED);
} }
@Override @Override
@ -903,7 +905,7 @@ class ConnectionManager {
@Override @Override
public boolean isTableDisabled(TableName tableName) throws IOException { public boolean isTableDisabled(TableName tableName) throws IOException {
return this.registry.isTableOnlineState(tableName, false); return getTableState(tableName).inStates(TableState.State.DISABLED);
} }
@Override @Override
@ -1992,6 +1994,13 @@ class ConnectionManager {
return stub.listTableNamesByNamespace(controller, request); return stub.listTableNamesByNamespace(controller, request);
} }
@Override
public GetTableStateResponse getTableState(
RpcController controller, GetTableStateRequest request)
throws ServiceException {
return stub.getTableState(controller, request);
}
@Override @Override
public void close() { public void close() {
release(this.mss); release(this.mss);
@ -2498,6 +2507,20 @@ class ConnectionManager {
throws IOException { throws IOException {
return getHTableDescriptor(TableName.valueOf(tableName)); return getHTableDescriptor(TableName.valueOf(tableName));
} }
@Override
public TableState getTableState(TableName tableName) throws IOException {
MasterKeepAliveConnection master = getKeepAliveMasterService();
try {
GetTableStateResponse resp = master.getTableState(null,
RequestConverter.buildGetTableStateRequest(tableName));
return TableState.convert(resp.getTableState());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
} finally {
master.close();
}
}
} }
/** /**

View File

@ -207,6 +207,13 @@ public interface HConnection extends Abortable, Closeable {
@Deprecated @Deprecated
boolean isTableDisabled(byte[] tableName) throws IOException; boolean isTableDisabled(byte[] tableName) throws IOException;
/**
* Retrieve TableState, represent current table state.
* @param tableName table state for
* @return state of the table
*/
public TableState getTableState(TableName tableName) throws IOException;
/** /**
* @param tableName table name * @param tableName table name
* @return true if all regions of the table are available, false otherwise * @return true if all regions of the table are available, false otherwise
@ -576,4 +583,4 @@ public interface HConnection extends Abortable, Closeable {
* @deprecated internal method, do not use thru HConnection */ * @deprecated internal method, do not use thru HConnection */
@Deprecated @Deprecated
public NonceGenerator getNonceGenerator(); public NonceGenerator getNonceGenerator();
} }

View File

@ -43,15 +43,9 @@ interface Registry {
*/ */
String getClusterId(); String getClusterId();
/**
* @param enabled Return true if table is enabled
* @throws IOException
*/
boolean isTableOnlineState(TableName tableName, boolean enabled) throws IOException;
/** /**
* @return Count of 'running' regionservers * @return Count of 'running' regionservers
* @throws IOException * @throws IOException
*/ */
int getCurrentNrHRS() throws IOException; int getCurrentNrHRS() throws IOException;
} }

View File

@ -0,0 +1,203 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
/**
* Represents table state.
*/
@InterfaceAudience.Private
public class TableState {
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static enum State {
ENABLED,
DISABLED,
DISABLING,
ENABLING;
/**
* Covert from PB version of State
*
* @param state convert from
* @return POJO
*/
public static State convert(HBaseProtos.TableState.State state) {
State ret;
switch (state) {
case ENABLED:
ret = State.ENABLED;
break;
case DISABLED:
ret = State.DISABLED;
break;
case DISABLING:
ret = State.DISABLING;
break;
case ENABLING:
ret = State.ENABLING;
break;
default:
throw new IllegalStateException(state.toString());
}
return ret;
}
/**
* Covert to PB version of State
*
* @return PB
*/
public HBaseProtos.TableState.State convert() {
HBaseProtos.TableState.State state;
switch (this) {
case ENABLED:
state = HBaseProtos.TableState.State.ENABLED;
break;
case DISABLED:
state = HBaseProtos.TableState.State.DISABLED;
break;
case DISABLING:
state = HBaseProtos.TableState.State.DISABLING;
break;
case ENABLING:
state = HBaseProtos.TableState.State.ENABLING;
break;
default:
throw new IllegalStateException(this.toString());
}
return state;
}
}
private final long timestamp;
private final TableName tableName;
private final State state;
/**
* Create instance of TableState.
* @param state table state
*/
public TableState(TableName tableName, State state, long timestamp) {
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());
}
/**
* @return table state
*/
public State getState() {
return state;
}
/**
* Timestamp of table state
*
* @return milliseconds
*/
public long getTimestamp() {
return timestamp;
}
/**
* Table name for state
*
* @return milliseconds
*/
public TableName getTableName() {
return tableName;
}
/**
* Check that table in given states
* @param state state
* @return true if satisfies
*/
public boolean inStates(State state) {
return this.state.equals(state);
}
/**
* Check that table in given states
* @param states state list
* @return true if satisfies
*/
public boolean inStates(State... states) {
for (State s : states) {
if (s.equals(this.state))
return true;
}
return false;
}
/**
* Covert to PB version of TableState
* @return PB
*/
public HBaseProtos.TableState convert() {
return HBaseProtos.TableState.newBuilder()
.setState(this.state.convert())
.setTable(ProtobufUtil.toProtoTableName(this.tableName))
.setTimestamp(this.timestamp)
.build();
}
/**
* Covert from PB version of TableState
* @param tableState convert from
* @return POJO
*/
public static TableState convert(HBaseProtos.TableState tableState) {
TableState.State state = State.convert(tableState.getState());
return new TableState(ProtobufUtil.toTableName(tableState.getTable()),
state, tableState.getTimestamp());
}
/**
* Static version of state checker
* @param state desired
* @param target equals to any of
* @return true if satisfies
*/
public static boolean isInStates(State state, State... target) {
for (State tableState : target) {
if (state.equals(tableState))
return true;
}
return false;
}
}

View File

@ -18,18 +18,17 @@
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -97,24 +96,6 @@ class ZooKeeperRegistry implements Registry {
return this.clusterId; return this.clusterId;
} }
@Override
public boolean isTableOnlineState(TableName tableName, boolean enabled)
throws IOException {
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
try {
if (enabled) {
return ZKTableStateClientSideReader.isEnabledTable(zkw, tableName);
}
return ZKTableStateClientSideReader.isDisabledTable(zkw, tableName);
} catch (KeeperException e) {
throw new IOException("Enable/Disable failed", e);
} catch (InterruptedException e) {
throw new InterruptedIOException();
} finally {
zkw.close();
}
}
@Override @Override
public int getCurrentNrHRS() throws IOException { public int getCurrentNrHRS() throws IOException {
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher(); ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
@ -128,4 +109,4 @@ class ZooKeeperRegistry implements Registry {
zkw.close(); zkw.close();
} }
} }
} }

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.protobuf;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -106,6 +107,8 @@ import org.apache.hadoop.hbase.util.Pair;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
/** /**
* Helper utility to build protocol buffer requests, * Helper utility to build protocol buffer requests,
* or build components for protocol buffer requests. * or build components for protocol buffer requests.
@ -1176,6 +1179,19 @@ public final class RequestConverter {
return builder.build(); return builder.build();
} }
/**
* Creates a protocol buffer GetTableStateRequest
*
* @param tableName table to get request for
* @return a GetTableStateRequest
*/
public static GetTableStateRequest buildGetTableStateRequest(
final TableName tableName) {
return GetTableStateRequest.newBuilder()
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.build();
}
/** /**
* Creates a protocol buffer GetTableDescriptorsRequest for a single table * Creates a protocol buffer GetTableDescriptorsRequest for a single table
* *

View File

@ -1,168 +0,0 @@
/**
* Copyright The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.zookeeper;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.zookeeper.KeeperException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
/**
* Non-instantiable class that provides helper functions to learn
* about HBase table state for code running on client side (hence, not having
* access to consensus context).
*
* Doesn't cache any table state, just goes directly to ZooKeeper.
* TODO: decouple this class from ZooKeeper.
*/
@InterfaceAudience.Private
public class ZKTableStateClientSideReader {
private ZKTableStateClientSideReader() {}
/**
* Go to zookeeper and see if state of table is {@code ZooKeeperProtos.Table.State#DISABLED}.
* This method does not use cache.
* This method is for clients other than AssignmentManager
* @param zkw ZooKeeperWatcher instance to use
* @param tableName table we're checking
* @return True if table is enabled.
* @throws KeeperException
*/
public static boolean isDisabledTable(final ZooKeeperWatcher zkw,
final TableName tableName)
throws KeeperException, InterruptedException {
ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
return isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
}
/**
* Go to zookeeper and see if state of table is {@code ZooKeeperProtos.Table.State#ENABLED}.
* This method does not use cache.
* This method is for clients other than AssignmentManager
* @param zkw ZooKeeperWatcher instance to use
* @param tableName table we're checking
* @return True if table is enabled.
* @throws KeeperException
*/
public static boolean isEnabledTable(final ZooKeeperWatcher zkw,
final TableName tableName)
throws KeeperException, InterruptedException {
return getTableState(zkw, tableName) == ZooKeeperProtos.Table.State.ENABLED;
}
/**
* Go to zookeeper and see if state of table is {@code ZooKeeperProtos.Table.State#DISABLING}
* of {@code ZooKeeperProtos.Table.State#DISABLED}.
* This method does not use cache.
* This method is for clients other than AssignmentManager.
* @param zkw ZooKeeperWatcher instance to use
* @param tableName table we're checking
* @return True if table is enabled.
* @throws KeeperException
*/
public static boolean isDisablingOrDisabledTable(final ZooKeeperWatcher zkw,
final TableName tableName)
throws KeeperException, InterruptedException {
ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
return isTableState(ZooKeeperProtos.Table.State.DISABLING, state) ||
isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
}
/**
* Gets a list of all the tables set as disabled in zookeeper.
* @return Set of disabled tables, empty Set if none
* @throws KeeperException
*/
public static Set<TableName> getDisabledTables(ZooKeeperWatcher zkw)
throws KeeperException, InterruptedException {
Set<TableName> disabledTables = new HashSet<TableName>();
List<String> children =
ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
for (String child: children) {
TableName tableName =
TableName.valueOf(child);
ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
if (state == ZooKeeperProtos.Table.State.DISABLED) disabledTables.add(tableName);
}
return disabledTables;
}
/**
* Gets a list of all the tables set as disabled in zookeeper.
* @return Set of disabled tables, empty Set if none
* @throws KeeperException
*/
public static Set<TableName> getDisabledOrDisablingTables(ZooKeeperWatcher zkw)
throws KeeperException, InterruptedException {
Set<TableName> disabledTables = new HashSet<TableName>();
List<String> children =
ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
for (String child: children) {
TableName tableName =
TableName.valueOf(child);
ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
if (state == ZooKeeperProtos.Table.State.DISABLED ||
state == ZooKeeperProtos.Table.State.DISABLING)
disabledTables.add(tableName);
}
return disabledTables;
}
static boolean isTableState(final ZooKeeperProtos.Table.State expectedState,
final ZooKeeperProtos.Table.State currentState) {
return currentState != null && currentState.equals(expectedState);
}
/**
* @param zkw ZooKeeperWatcher instance to use
* @param tableName table we're checking
* @return Null or {@link ZooKeeperProtos.Table.State} found in znode.
* @throws KeeperException
*/
static ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,
final TableName tableName)
throws KeeperException, InterruptedException {
String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
byte [] data = ZKUtil.getData(zkw, znode);
if (data == null || data.length <= 0) return null;
try {
ProtobufUtil.expectPBMagicPrefix(data);
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
int magicLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build();
return t.getState();
} catch (InvalidProtocolBufferException e) {
KeeperException ke = new KeeperException.DataInconsistencyException();
ke.initCause(e);
throw ke;
} catch (DeserializationException e) {
throw ZKUtil.convert(e);
}
}
}

View File

@ -94,6 +94,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
// znode containing the current cluster state // znode containing the current cluster state
public String clusterStateZNode; public String clusterStateZNode;
// znode used for table disabling/enabling // znode used for table disabling/enabling
@Deprecated
public String tableZNode; public String tableZNode;
// znode containing the unique cluster ID // znode containing the unique cluster ID
public String clusterIdZNode; public String clusterIdZNode;

View File

@ -129,12 +129,6 @@ public class TestClientNoCluster extends Configured implements Tool {
return HConstants.CLUSTER_ID_DEFAULT; return HConstants.CLUSTER_ID_DEFAULT;
} }
@Override
public boolean isTableOnlineState(TableName tableName, boolean enabled)
throws IOException {
return enabled;
}
@Override @Override
public int getCurrentNrHRS() throws IOException { public int getCurrentNrHRS() throws IOException {
return 1; return 1;
@ -814,4 +808,4 @@ public class TestClientNoCluster extends Configured implements Tool {
public static void main(String[] args) throws Exception { public static void main(String[] args) throws Exception {
System.exit(ToolRunner.run(HBaseConfiguration.create(), new TestClientNoCluster(), args)); System.exit(ToolRunner.run(HBaseConfiguration.create(), new TestClientNoCluster(), args));
} }
} }

View File

@ -3242,12 +3242,12 @@ public final class ZooKeeperProtos {
// @@protoc_insertion_point(class_scope:SplitLogTask) // @@protoc_insertion_point(class_scope:SplitLogTask)
} }
public interface TableOrBuilder public interface DeprecatedTableStateOrBuilder
extends com.google.protobuf.MessageOrBuilder { extends com.google.protobuf.MessageOrBuilder {
// required .Table.State state = 1 [default = ENABLED]; // required .DeprecatedTableState.State state = 1 [default = ENABLED];
/** /**
* <code>required .Table.State state = 1 [default = ENABLED];</code> * <code>required .DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -3257,7 +3257,7 @@ public final class ZooKeeperProtos {
*/ */
boolean hasState(); boolean hasState();
/** /**
* <code>required .Table.State state = 1 [default = ENABLED];</code> * <code>required .DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -3265,32 +3265,33 @@ public final class ZooKeeperProtos {
* for more. * for more.
* </pre> * </pre>
*/ */
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState(); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State getState();
} }
/** /**
* Protobuf type {@code Table} * Protobuf type {@code DeprecatedTableState}
* *
* <pre> * <pre>
** **
* The znode that holds state of table. * The znode that holds state of table.
* Deprected, table state is stored in table descriptor on HDFS.
* </pre> * </pre>
*/ */
public static final class Table extends public static final class DeprecatedTableState extends
com.google.protobuf.GeneratedMessage com.google.protobuf.GeneratedMessage
implements TableOrBuilder { implements DeprecatedTableStateOrBuilder {
// Use Table.newBuilder() to construct. // Use DeprecatedTableState.newBuilder() to construct.
private Table(com.google.protobuf.GeneratedMessage.Builder<?> builder) { private DeprecatedTableState(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
super(builder); super(builder);
this.unknownFields = builder.getUnknownFields(); this.unknownFields = builder.getUnknownFields();
} }
private Table(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } private DeprecatedTableState(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
private static final Table defaultInstance; private static final DeprecatedTableState defaultInstance;
public static Table getDefaultInstance() { public static DeprecatedTableState getDefaultInstance() {
return defaultInstance; return defaultInstance;
} }
public Table getDefaultInstanceForType() { public DeprecatedTableState getDefaultInstanceForType() {
return defaultInstance; return defaultInstance;
} }
@ -3300,7 +3301,7 @@ public final class ZooKeeperProtos {
getUnknownFields() { getUnknownFields() {
return this.unknownFields; return this.unknownFields;
} }
private Table( private DeprecatedTableState(
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
@ -3325,7 +3326,7 @@ public final class ZooKeeperProtos {
} }
case 8: { case 8: {
int rawValue = input.readEnum(); int rawValue = input.readEnum();
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State value = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.valueOf(rawValue); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State value = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.valueOf(rawValue);
if (value == null) { if (value == null) {
unknownFields.mergeVarintField(1, rawValue); unknownFields.mergeVarintField(1, rawValue);
} else { } else {
@ -3348,33 +3349,33 @@ public final class ZooKeeperProtos {
} }
public static final com.google.protobuf.Descriptors.Descriptor public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() { getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Table_descriptor; return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_DeprecatedTableState_descriptor;
} }
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() { internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Table_fieldAccessorTable return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_DeprecatedTableState_fieldAccessorTable
.ensureFieldAccessorsInitialized( .ensureFieldAccessorsInitialized(
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder.class); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.Builder.class);
} }
public static com.google.protobuf.Parser<Table> PARSER = public static com.google.protobuf.Parser<DeprecatedTableState> PARSER =
new com.google.protobuf.AbstractParser<Table>() { new com.google.protobuf.AbstractParser<DeprecatedTableState>() {
public Table parsePartialFrom( public DeprecatedTableState parsePartialFrom(
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return new Table(input, extensionRegistry); return new DeprecatedTableState(input, extensionRegistry);
} }
}; };
@java.lang.Override @java.lang.Override
public com.google.protobuf.Parser<Table> getParserForType() { public com.google.protobuf.Parser<DeprecatedTableState> getParserForType() {
return PARSER; return PARSER;
} }
/** /**
* Protobuf enum {@code Table.State} * Protobuf enum {@code DeprecatedTableState.State}
* *
* <pre> * <pre>
* Table's current state * Table's current state
@ -3452,7 +3453,7 @@ public final class ZooKeeperProtos {
} }
public static final com.google.protobuf.Descriptors.EnumDescriptor public static final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptor() { getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDescriptor().getEnumTypes().get(0); return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.getDescriptor().getEnumTypes().get(0);
} }
private static final State[] VALUES = values(); private static final State[] VALUES = values();
@ -3474,15 +3475,15 @@ public final class ZooKeeperProtos {
this.value = value; this.value = value;
} }
// @@protoc_insertion_point(enum_scope:Table.State) // @@protoc_insertion_point(enum_scope:DeprecatedTableState.State)
} }
private int bitField0_; private int bitField0_;
// required .Table.State state = 1 [default = ENABLED]; // required .DeprecatedTableState.State state = 1 [default = ENABLED];
public static final int STATE_FIELD_NUMBER = 1; public static final int STATE_FIELD_NUMBER = 1;
private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State state_; private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State state_;
/** /**
* <code>required .Table.State state = 1 [default = ENABLED];</code> * <code>required .DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -3494,7 +3495,7 @@ public final class ZooKeeperProtos {
return ((bitField0_ & 0x00000001) == 0x00000001); return ((bitField0_ & 0x00000001) == 0x00000001);
} }
/** /**
* <code>required .Table.State state = 1 [default = ENABLED];</code> * <code>required .DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -3502,12 +3503,12 @@ public final class ZooKeeperProtos {
* for more. * for more.
* </pre> * </pre>
*/ */
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State getState() {
return state_; return state_;
} }
private void initFields() { private void initFields() {
state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED; state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.ENABLED;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -3558,10 +3559,10 @@ public final class ZooKeeperProtos {
if (obj == this) { if (obj == this) {
return true; return true;
} }
if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table)) { if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState)) {
return super.equals(obj); return super.equals(obj);
} }
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table) obj; org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState) obj;
boolean result = true; boolean result = true;
result = result && (hasState() == other.hasState()); result = result && (hasState() == other.hasState());
@ -3591,53 +3592,53 @@ public final class ZooKeeperProtos {
return hash; return hash;
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
com.google.protobuf.ByteString data) com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data); return PARSER.parseFrom(data);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
com.google.protobuf.ByteString data, com.google.protobuf.ByteString data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry); return PARSER.parseFrom(data, extensionRegistry);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom(byte[] data) public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data); return PARSER.parseFrom(data);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
byte[] data, byte[] data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry); return PARSER.parseFrom(data, extensionRegistry);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom(java.io.InputStream input) public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(java.io.InputStream input)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseFrom(input); return PARSER.parseFrom(input);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
java.io.InputStream input, java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseFrom(input, extensionRegistry); return PARSER.parseFrom(input, extensionRegistry);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseDelimitedFrom(java.io.InputStream input) public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseDelimitedFrom(input); return PARSER.parseDelimitedFrom(input);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseDelimitedFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseDelimitedFrom(
java.io.InputStream input, java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseDelimitedFrom(input, extensionRegistry); return PARSER.parseDelimitedFrom(input, extensionRegistry);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
com.google.protobuf.CodedInputStream input) com.google.protobuf.CodedInputStream input)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseFrom(input); return PARSER.parseFrom(input);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
@ -3646,7 +3647,7 @@ public final class ZooKeeperProtos {
public static Builder newBuilder() { return Builder.create(); } public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); } public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table prototype) { public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState prototype) {
return newBuilder().mergeFrom(prototype); return newBuilder().mergeFrom(prototype);
} }
public Builder toBuilder() { return newBuilder(this); } public Builder toBuilder() { return newBuilder(this); }
@ -3658,29 +3659,30 @@ public final class ZooKeeperProtos {
return builder; return builder;
} }
/** /**
* Protobuf type {@code Table} * Protobuf type {@code DeprecatedTableState}
* *
* <pre> * <pre>
** **
* The znode that holds state of table. * The znode that holds state of table.
* Deprected, table state is stored in table descriptor on HDFS.
* </pre> * </pre>
*/ */
public static final class Builder extends public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder> com.google.protobuf.GeneratedMessage.Builder<Builder>
implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableOrBuilder { implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableStateOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() { getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Table_descriptor; return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_DeprecatedTableState_descriptor;
} }
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() { internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Table_fieldAccessorTable return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_DeprecatedTableState_fieldAccessorTable
.ensureFieldAccessorsInitialized( .ensureFieldAccessorsInitialized(
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder.class); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.Builder.class);
} }
// Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.newBuilder() // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.newBuilder()
private Builder() { private Builder() {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -3700,7 +3702,7 @@ public final class ZooKeeperProtos {
public Builder clear() { public Builder clear() {
super.clear(); super.clear();
state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED; state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.ENABLED;
bitField0_ = (bitField0_ & ~0x00000001); bitField0_ = (bitField0_ & ~0x00000001);
return this; return this;
} }
@ -3711,23 +3713,23 @@ public final class ZooKeeperProtos {
public com.google.protobuf.Descriptors.Descriptor public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() { getDescriptorForType() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Table_descriptor; return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_DeprecatedTableState_descriptor;
} }
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table getDefaultInstanceForType() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState getDefaultInstanceForType() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.getDefaultInstance();
} }
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table build() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState build() {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table result = buildPartial(); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState result = buildPartial();
if (!result.isInitialized()) { if (!result.isInitialized()) {
throw newUninitializedMessageException(result); throw newUninitializedMessageException(result);
} }
return result; return result;
} }
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table buildPartial() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState buildPartial() {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table(this); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState(this);
int from_bitField0_ = bitField0_; int from_bitField0_ = bitField0_;
int to_bitField0_ = 0; int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) { if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@ -3740,16 +3742,16 @@ public final class ZooKeeperProtos {
} }
public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table) { if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState) {
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table)other); return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState)other);
} else { } else {
super.mergeFrom(other); super.mergeFrom(other);
return this; return this;
} }
} }
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table other) { public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState other) {
if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance()) return this; if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.getDefaultInstance()) return this;
if (other.hasState()) { if (other.hasState()) {
setState(other.getState()); setState(other.getState());
} }
@ -3769,11 +3771,11 @@ public final class ZooKeeperProtos {
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parsedMessage = null; org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parsedMessage = null;
try { try {
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table) e.getUnfinishedMessage(); parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState) e.getUnfinishedMessage();
throw e; throw e;
} finally { } finally {
if (parsedMessage != null) { if (parsedMessage != null) {
@ -3784,10 +3786,10 @@ public final class ZooKeeperProtos {
} }
private int bitField0_; private int bitField0_;
// required .Table.State state = 1 [default = ENABLED]; // required .DeprecatedTableState.State state = 1 [default = ENABLED];
private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED; private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.ENABLED;
/** /**
* <code>required .Table.State state = 1 [default = ENABLED];</code> * <code>required .DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -3799,7 +3801,7 @@ public final class ZooKeeperProtos {
return ((bitField0_ & 0x00000001) == 0x00000001); return ((bitField0_ & 0x00000001) == 0x00000001);
} }
/** /**
* <code>required .Table.State state = 1 [default = ENABLED];</code> * <code>required .DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -3807,11 +3809,11 @@ public final class ZooKeeperProtos {
* for more. * for more.
* </pre> * </pre>
*/ */
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State getState() {
return state_; return state_;
} }
/** /**
* <code>required .Table.State state = 1 [default = ENABLED];</code> * <code>required .DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -3819,7 +3821,7 @@ public final class ZooKeeperProtos {
* for more. * for more.
* </pre> * </pre>
*/ */
public Builder setState(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State value) { public Builder setState(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State value) {
if (value == null) { if (value == null) {
throw new NullPointerException(); throw new NullPointerException();
} }
@ -3829,7 +3831,7 @@ public final class ZooKeeperProtos {
return this; return this;
} }
/** /**
* <code>required .Table.State state = 1 [default = ENABLED];</code> * <code>required .DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -3839,20 +3841,20 @@ public final class ZooKeeperProtos {
*/ */
public Builder clearState() { public Builder clearState() {
bitField0_ = (bitField0_ & ~0x00000001); bitField0_ = (bitField0_ & ~0x00000001);
state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED; state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.ENABLED;
onChanged(); onChanged();
return this; return this;
} }
// @@protoc_insertion_point(builder_scope:Table) // @@protoc_insertion_point(builder_scope:DeprecatedTableState)
} }
static { static {
defaultInstance = new Table(true); defaultInstance = new DeprecatedTableState(true);
defaultInstance.initFields(); defaultInstance.initFields();
} }
// @@protoc_insertion_point(class_scope:Table) // @@protoc_insertion_point(class_scope:DeprecatedTableState)
} }
public interface ReplicationPeerOrBuilder public interface ReplicationPeerOrBuilder
@ -9512,10 +9514,10 @@ public final class ZooKeeperProtos {
com.google.protobuf.GeneratedMessage.FieldAccessorTable com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_SplitLogTask_fieldAccessorTable; internal_static_SplitLogTask_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor private static com.google.protobuf.Descriptors.Descriptor
internal_static_Table_descriptor; internal_static_DeprecatedTableState_descriptor;
private static private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_Table_fieldAccessorTable; internal_static_DeprecatedTableState_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor private static com.google.protobuf.Descriptors.Descriptor
internal_static_ReplicationPeer_descriptor; internal_static_ReplicationPeer_descriptor;
private static private static
@ -9573,27 +9575,28 @@ public final class ZooKeeperProtos {
"UNASSIGNED\020\000\022\t\n\005OWNED\020\001\022\014\n\010RESIGNED\020\002\022\010\n" + "UNASSIGNED\020\000\022\t\n\005OWNED\020\001\022\014\n\010RESIGNED\020\002\022\010\n" +
"\004DONE\020\003\022\007\n\003ERR\020\004\">\n\014RecoveryMode\022\013\n\007UNKN" + "\004DONE\020\003\022\007\n\003ERR\020\004\">\n\014RecoveryMode\022\013\n\007UNKN" +
"OWN\020\000\022\021\n\rLOG_SPLITTING\020\001\022\016\n\nLOG_REPLAY\020\002" + "OWN\020\000\022\021\n\rLOG_SPLITTING\020\001\022\016\n\nLOG_REPLAY\020\002" +
"\"n\n\005Table\022$\n\005state\030\001 \002(\0162\014.Table.State:\007" + "\"\214\001\n\024DeprecatedTableState\0223\n\005state\030\001 \002(\016" +
"ENABLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISABLE" + "2\033.DeprecatedTableState.State:\007ENABLED\"?" +
"D\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"\215\001\n\017Rep" + "\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001\022\r\n\tDI" +
"licationPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027repl" + "SABLING\020\002\022\014\n\010ENABLING\020\003\"\215\001\n\017ReplicationP" +
"icationEndpointImpl\030\002 \001(\t\022\035\n\004data\030\003 \003(\0132" + "eer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027replicationEn" +
"\017.BytesBytesPair\022&\n\rconfiguration\030\004 \003(\0132" + "dpointImpl\030\002 \001(\t\022\035\n\004data\030\003 \003(\0132\017.BytesBy" +
"\017.NameStringPair\"^\n\020ReplicationState\022&\n\005", "tesPair\022&\n\rconfiguration\030\004 \003(\0132\017.NameStr",
"state\030\001 \002(\0162\027.ReplicationState.State\"\"\n\005" + "ingPair\"^\n\020ReplicationState\022&\n\005state\030\001 \002" +
"State\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001\"+\n\027Repl" + "(\0162\027.ReplicationState.State\"\"\n\005State\022\013\n\007" +
"icationHLogPosition\022\020\n\010position\030\001 \002(\003\"%\n" + "ENABLED\020\000\022\014\n\010DISABLED\020\001\"+\n\027ReplicationHL" +
"\017ReplicationLock\022\022\n\nlock_owner\030\001 \002(\t\"\230\001\n" + "ogPosition\022\020\n\010position\030\001 \002(\003\"%\n\017Replicat" +
"\tTableLock\022\036\n\ntable_name\030\001 \001(\0132\n.TableNa" + "ionLock\022\022\n\nlock_owner\030\001 \002(\t\"\230\001\n\tTableLoc" +
"me\022\037\n\nlock_owner\030\002 \001(\0132\013.ServerName\022\021\n\tt" + "k\022\036\n\ntable_name\030\001 \001(\0132\n.TableName\022\037\n\nloc" +
"hread_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(\010\022\017\n\007pur" + "k_owner\030\002 \001(\0132\013.ServerName\022\021\n\tthread_id\030" +
"pose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003\";\n\017Store" + "\003 \001(\003\022\021\n\tis_shared\030\004 \001(\010\022\017\n\007purpose\030\005 \001(" +
"SequenceId\022\023\n\013family_name\030\001 \002(\014\022\023\n\013seque" + "\t\022\023\n\013create_time\030\006 \001(\003\";\n\017StoreSequenceI" +
"nce_id\030\002 \002(\004\"g\n\026RegionStoreSequenceIds\022 ", "d\022\023\n\013family_name\030\001 \002(\014\022\023\n\013sequence_id\030\002 ",
"\n\030last_flushed_sequence_id\030\001 \002(\004\022+\n\021stor" + "\002(\004\"g\n\026RegionStoreSequenceIds\022 \n\030last_fl" +
"e_sequence_id\030\002 \003(\0132\020.StoreSequenceIdBE\n" + "ushed_sequence_id\030\001 \002(\004\022+\n\021store_sequenc" +
"*org.apache.hadoop.hbase.protobuf.genera" + "e_id\030\002 \003(\0132\020.StoreSequenceIdBE\n*org.apac" +
"tedB\017ZooKeeperProtosH\001\210\001\001\240\001\001" "he.hadoop.hbase.protobuf.generatedB\017ZooK" +
"eeperProtosH\001\210\001\001\240\001\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -9624,11 +9627,11 @@ public final class ZooKeeperProtos {
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_SplitLogTask_descriptor, internal_static_SplitLogTask_descriptor,
new java.lang.String[] { "State", "ServerName", "Mode", }); new java.lang.String[] { "State", "ServerName", "Mode", });
internal_static_Table_descriptor = internal_static_DeprecatedTableState_descriptor =
getDescriptor().getMessageTypes().get(4); getDescriptor().getMessageTypes().get(4);
internal_static_Table_fieldAccessorTable = new internal_static_DeprecatedTableState_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_Table_descriptor, internal_static_DeprecatedTableState_descriptor,
new java.lang.String[] { "State", }); new java.lang.String[] { "State", });
internal_static_ReplicationPeer_descriptor = internal_static_ReplicationPeer_descriptor =
getDescriptor().getMessageTypes().get(5); getDescriptor().getMessageTypes().get(5);

View File

@ -44,6 +44,27 @@ message TableSchema {
repeated NameStringPair configuration = 4; repeated NameStringPair configuration = 4;
} }
/** Denotes state of the table */
message TableState {
// Table's current state
enum State {
ENABLED = 0;
DISABLED = 1;
DISABLING = 2;
ENABLING = 3;
}
// This is the table's state.
required State state = 1;
required TableName table = 2;
optional uint64 timestamp = 3;
}
/** On HDFS representation of table state. */
message TableDescriptor {
required TableSchema schema = 1;
optional TableState.State state = 2 [ default = ENABLED ];
}
/** /**
* Column Family Schema * Column Family Schema
* Inspired by the rest ColumSchemaMessage * Inspired by the rest ColumSchemaMessage

View File

@ -326,6 +326,14 @@ message GetTableNamesResponse {
repeated TableName table_names = 1; repeated TableName table_names = 1;
} }
message GetTableStateRequest {
required TableName table_name = 1;
}
message GetTableStateResponse {
required TableState table_state = 1;
}
message GetClusterStatusRequest { message GetClusterStatusRequest {
} }
@ -565,4 +573,8 @@ service MasterService {
/** returns a list of tables for a given namespace*/ /** returns a list of tables for a given namespace*/
rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest) rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest)
returns(ListTableNamesByNamespaceResponse); returns(ListTableNamesByNamespaceResponse);
/** returns table state */
rpc GetTableState(GetTableStateRequest)
returns(GetTableStateResponse);
} }

View File

@ -87,8 +87,9 @@ message SplitLogTask {
/** /**
* The znode that holds state of table. * The znode that holds state of table.
* Deprected, table state is stored in table descriptor on HDFS.
*/ */
message Table { message DeprecatedTableState {
// Table's current state // Table's current state
enum State { enum State {
ENABLED = 0; ENABLED = 0;

View File

@ -55,12 +55,4 @@ public interface CoordinatedStateManager {
* @return instance of Server coordinated state manager runs within * @return instance of Server coordinated state manager runs within
*/ */
Server getServer(); Server getServer();
/**
* Returns implementation of TableStateManager.
* @throws InterruptedException if operation is interrupted
* @throws CoordinatedStateException if error happens in underlying coordination mechanism
*/
TableStateManager getTableStateManager() throws InterruptedException,
CoordinatedStateException;
} }

View File

@ -0,0 +1,154 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.IOException;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.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;
/**
* Class represents table state on HDFS.
*/
@InterfaceAudience.Private
public class TableDescriptor {
private HTableDescriptor hTableDescriptor;
private TableState.State tableState;
/**
* Creates TableDescriptor with all fields.
* @param hTableDescriptor HTableDescriptor to use
* @param tableState table state
*/
public TableDescriptor(HTableDescriptor hTableDescriptor,
TableState.State tableState) {
this.hTableDescriptor = hTableDescriptor;
this.tableState = tableState;
}
/**
* Creates TableDescriptor with Enabled table.
* @param hTableDescriptor HTableDescriptor to use
*/
@VisibleForTesting
public TableDescriptor(HTableDescriptor hTableDescriptor) {
this(hTableDescriptor, TableState.State.ENABLED);
}
/**
* Associated HTableDescriptor
* @return instance of HTableDescriptor
*/
public HTableDescriptor getHTableDescriptor() {
return hTableDescriptor;
}
public void setHTableDescriptor(HTableDescriptor hTableDescriptor) {
this.hTableDescriptor = hTableDescriptor;
}
public TableState.State getTableState() {
return tableState;
}
public void setTableState(TableState.State tableState) {
this.tableState = tableState;
}
/**
* Convert to PB.
*/
public HBaseProtos.TableDescriptor convert() {
return HBaseProtos.TableDescriptor.newBuilder()
.setSchema(hTableDescriptor.convert())
.setState(tableState.convert())
.build();
}
/**
* Convert from PB
*/
public static TableDescriptor convert(HBaseProtos.TableDescriptor proto) {
HTableDescriptor hTableDescriptor = HTableDescriptor.convert(proto.getSchema());
TableState.State state = TableState.State.convert(proto.getState());
return new TableDescriptor(hTableDescriptor, state);
}
/**
* @return This instance serialized with pb with pb magic prefix
* @see #parseFrom(byte[])
*/
public byte [] toByteArray() {
return ProtobufUtil.prependPBMagic(convert().toByteArray());
}
/**
* @param bytes A pb serialized {@link TableDescriptor} instance with pb magic prefix
* @see #toByteArray()
*/
public static TableDescriptor parseFrom(final byte [] bytes)
throws DeserializationException, IOException {
if (!ProtobufUtil.isPBMagicPrefix(bytes)) {
throw new DeserializationException("Expected PB encoded TableDescriptor");
}
int pblen = ProtobufUtil.lengthOfPBMagic();
HBaseProtos.TableDescriptor.Builder builder = HBaseProtos.TableDescriptor.newBuilder();
HBaseProtos.TableDescriptor ts;
try {
ts = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
} catch (InvalidProtocolBufferException e) {
throw new DeserializationException(e);
}
return convert(ts);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
TableDescriptor that = (TableDescriptor) o;
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;
}
@Override
public String toString() {
return "TableDescriptor{" +
"hTableDescriptor=" + hTableDescriptor +
", tableState=" + tableState +
'}';
}
}

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/** /**
* Get, remove and modify table descriptors. * Get, remove and modify table descriptors.
@ -37,6 +36,14 @@ public interface TableDescriptors {
HTableDescriptor get(final TableName tableName) HTableDescriptor get(final TableName tableName)
throws IOException; throws IOException;
/**
* @param tableName
* @return TableDescriptor for tablename
* @throws IOException
*/
TableDescriptor getDescriptor(final TableName tableName)
throws IOException;
/** /**
* Get Map of all NamespaceDescriptors for a given namespace. * Get Map of all NamespaceDescriptors for a given namespace.
* @return Map of all descriptors. * @return Map of all descriptors.
@ -54,6 +61,15 @@ public interface TableDescriptors {
Map<String, HTableDescriptor> getAll() Map<String, HTableDescriptor> getAll()
throws IOException; 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;
/** /**
* Add or update descriptor * Add or update descriptor
* @param htd Descriptor to set into TableDescriptors * @param htd Descriptor to set into TableDescriptors
@ -62,6 +78,14 @@ public interface TableDescriptors {
void add(final HTableDescriptor htd) void add(final HTableDescriptor htd)
throws IOException; throws IOException;
/**
* Add or update descriptor
* @param htd Descriptor to set into TableDescriptors
* @throws IOException
*/
void add(final TableDescriptor htd)
throws IOException;
/** /**
* @param tablename * @param tablename
* @return Instance of table descriptor or null if none found. * @return Instance of table descriptor or null if none found.

View File

@ -1,115 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import java.io.InterruptedIOException;
import java.util.Set;
/**
* Helper class for table state management for operations running inside
* RegionServer or HMaster.
* Depending on implementation, fetches information from HBase system table,
* local data store, ZooKeeper ensemble or somewhere else.
* Code running on client side (with no coordinated state context) shall instead use
* {@link org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader}
*/
@InterfaceAudience.Private
public interface TableStateManager {
/**
* Sets the table into desired state. Fails silently if the table is already in this state.
* @param tableName table to process
* @param state new state of this table
* @throws CoordinatedStateException if error happened when trying to set table state
*/
void setTableState(TableName tableName, ZooKeeperProtos.Table.State state)
throws CoordinatedStateException;
/**
* Sets the specified table into the newState, but only if the table is already in
* one of the possibleCurrentStates (otherwise no operation is performed).
* @param tableName table to process
* @param newState new state for the table
* @param states table should be in one of these states for the operation
* to be performed
* @throws CoordinatedStateException if error happened while performing operation
* @return true if operation succeeded, false otherwise
*/
boolean setTableStateIfInStates(TableName tableName, ZooKeeperProtos.Table.State newState,
ZooKeeperProtos.Table.State... states)
throws CoordinatedStateException;
/**
* Sets the specified table into the newState, but only if the table is NOT in
* one of the possibleCurrentStates (otherwise no operation is performed).
* @param tableName table to process
* @param newState new state for the table
* @param states table should NOT be in one of these states for the operation
* to be performed
* @throws CoordinatedStateException if error happened while performing operation
* @return true if operation succeeded, false otherwise
*/
boolean setTableStateIfNotInStates(TableName tableName, ZooKeeperProtos.Table.State newState,
ZooKeeperProtos.Table.State... states)
throws CoordinatedStateException;
/**
* @return true if the table is in any one of the listed states, false otherwise.
*/
boolean isTableState(TableName tableName, ZooKeeperProtos.Table.State... states);
/**
* Mark table as deleted. Fails silently if the table is not currently marked as disabled.
* @param tableName table to be deleted
* @throws CoordinatedStateException if error happened while performing operation
*/
void setDeletedTable(TableName tableName) throws CoordinatedStateException;
/**
* Checks if table is present.
*
* @param tableName table we're checking
* @return true if the table is present, false otherwise
*/
boolean isTablePresent(TableName tableName);
/**
* @return set of tables which are in any one of the listed states, empty Set if none
*/
Set<TableName> getTablesInStates(ZooKeeperProtos.Table.State... states)
throws InterruptedIOException, CoordinatedStateException;
/**
* If the table is found in the given state the in-memory state is removed. This
* helps in cases where CreateTable is to be retried by the client in case of
* failures. If deletePermanentState is true - the flag kept permanently is
* also reset.
*
* @param tableName table we're working on
* @param states if table isn't in any one of these states, operation aborts
* @param deletePermanentState if true, reset the permanent flag
* @throws CoordinatedStateException if error happened in underlying coordination engine
*/
void checkAndRemoveTableState(TableName tableName, ZooKeeperProtos.Table.State states,
boolean deletePermanentState)
throws CoordinatedStateException;
}

View File

@ -118,4 +118,4 @@ public class CoprocessorHConnection extends HConnectionImplementation {
public NonceGenerator getNonceGenerator() { public NonceGenerator getNonceGenerator() {
return NO_NONCE_GEN; // don't use nonces for coprocessor connection return NO_NONCE_GEN; // don't use nonces for coprocessor connection
} }
} }

View File

@ -18,10 +18,8 @@
package org.apache.hadoop.hbase.coordination; package org.apache.hadoop.hbase.coordination;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableStateManager;
/** /**
* Base class for {@link org.apache.hadoop.hbase.CoordinatedStateManager} implementations. * Base class for {@link org.apache.hadoop.hbase.CoordinatedStateManager} implementations.
@ -49,10 +47,6 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan
return null; return null;
} }
@Override
public abstract TableStateManager getTableStateManager() throws InterruptedException,
CoordinatedStateException;
/** /**
* Method to retrieve coordination for split log worker * Method to retrieve coordination for split log worker
*/ */
@ -61,4 +55,4 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan
* Method to retrieve coordination for split log manager * Method to retrieve coordination for split log manager
*/ */
public abstract SplitLogManagerCoordination getSplitLogManagerCoordination(); public abstract SplitLogManagerCoordination getSplitLogManagerCoordination();
} }

View File

@ -18,14 +18,9 @@
package org.apache.hadoop.hbase.coordination; package org.apache.hadoop.hbase.coordination;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/** /**
* ZooKeeper-based implementation of {@link org.apache.hadoop.hbase.CoordinatedStateManager}. * ZooKeeper-based implementation of {@link org.apache.hadoop.hbase.CoordinatedStateManager}.
@ -51,16 +46,6 @@ public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
return server; return server;
} }
@Override
public TableStateManager getTableStateManager() throws InterruptedException,
CoordinatedStateException {
try {
return new ZKTableStateManager(server.getZooKeeper());
} catch (KeeperException e) {
throw new CoordinatedStateException(e);
}
}
@Override @Override
public SplitLogWorkerCoordination getSplitLogWorkerCoordination() { public SplitLogWorkerCoordination getSplitLogWorkerCoordination() {
return splitLogWorkerCoordination; return splitLogWorkerCoordination;

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.master; package org.apache.hadoop.hbase.master;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -47,7 +46,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
@ -60,9 +58,9 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
@ -76,8 +74,6 @@ import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
import org.apache.hadoop.hbase.master.handler.EnableTableHandler; import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLog;
@ -207,14 +203,14 @@ public class AssignmentManager {
* @param service Executor service * @param service Executor service
* @param metricsMaster metrics manager * @param metricsMaster metrics manager
* @param tableLockManager TableLock manager * @param tableLockManager TableLock manager
* @throws CoordinatedStateException
* @throws IOException * @throws IOException
*/ */
public AssignmentManager(Server server, ServerManager serverManager, public AssignmentManager(Server server, ServerManager serverManager,
final LoadBalancer balancer, final LoadBalancer balancer,
final ExecutorService service, MetricsMaster metricsMaster, final ExecutorService service, MetricsMaster metricsMaster,
final TableLockManager tableLockManager) final TableLockManager tableLockManager,
throws IOException, CoordinatedStateException { final TableStateManager tableStateManager)
throws IOException {
this.server = server; this.server = server;
this.serverManager = serverManager; this.serverManager = serverManager;
this.executorService = service; this.executorService = service;
@ -226,15 +222,9 @@ public class AssignmentManager {
this.shouldAssignRegionsWithFavoredNodes = conf.getClass( this.shouldAssignRegionsWithFavoredNodes = conf.getClass(
HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals( HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals(
FavoredNodeLoadBalancer.class); FavoredNodeLoadBalancer.class);
try {
if (server.getCoordinatedStateManager() != null) { this.tableStateManager = tableStateManager;
this.tableStateManager = server.getCoordinatedStateManager().getTableStateManager();
} else {
this.tableStateManager = null;
}
} catch (InterruptedException e) {
throw new InterruptedIOException();
}
// This is the max attempts, not retries, so it should be at least 1. // This is the max attempts, not retries, so it should be at least 1.
this.maximumAttempts = Math.max(1, this.maximumAttempts = Math.max(1,
this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10)); this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10));
@ -392,10 +382,9 @@ public class AssignmentManager {
* @throws IOException * @throws IOException
* @throws KeeperException * @throws KeeperException
* @throws InterruptedException * @throws InterruptedException
* @throws CoordinatedStateException
*/ */
void joinCluster() throws IOException, void joinCluster() throws IOException,
KeeperException, InterruptedException, CoordinatedStateException { KeeperException, InterruptedException {
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
// Concurrency note: In the below the accesses on regionsInTransition are // Concurrency note: In the below the accesses on regionsInTransition are
// outside of a synchronization block where usually all accesses to RIT are // outside of a synchronization block where usually all accesses to RIT are
@ -430,10 +419,9 @@ public class AssignmentManager {
* Map of dead servers and their regions. Can be null. * Map of dead servers and their regions. Can be null.
* @throws IOException * @throws IOException
* @throws InterruptedException * @throws InterruptedException
* @throws CoordinatedStateException
*/ */
boolean processDeadServersAndRegionsInTransition(final Set<ServerName> deadServers) boolean processDeadServersAndRegionsInTransition(final Set<ServerName> deadServers)
throws IOException, InterruptedException, CoordinatedStateException { throws IOException, InterruptedException {
boolean failover = !serverManager.getDeadServers().isEmpty(); boolean failover = !serverManager.getDeadServers().isEmpty();
if (failover) { if (failover) {
// This may not be a failover actually, especially if meta is on this master. // This may not be a failover actually, especially if meta is on this master.
@ -502,8 +490,8 @@ public class AssignmentManager {
if (!failover) { if (!failover) {
disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates( disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING, TableState.State.DISABLED, TableState.State.DISABLING,
ZooKeeperProtos.Table.State.ENABLING); TableState.State.ENABLING);
// Clean re/start, mark all user regions closed before reassignment // Clean re/start, mark all user regions closed before reassignment
allRegions = regionStates.closeAllUserRegions( allRegions = regionStates.closeAllUserRegions(
@ -1011,7 +999,7 @@ public class AssignmentManager {
// will not be in ENABLING or ENABLED state. // will not be in ENABLING or ENABLED state.
TableName tableName = region.getTable(); TableName tableName = region.getTable();
if (!tableStateManager.isTableState(tableName, if (!tableStateManager.isTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED, ZooKeeperProtos.Table.State.ENABLING)) { TableState.State.ENABLED, TableState.State.ENABLING)) {
LOG.debug("Setting table " + tableName + " to ENABLED state."); LOG.debug("Setting table " + tableName + " to ENABLED state.");
setEnabledTable(tableName); setEnabledTable(tableName);
} }
@ -1147,8 +1135,8 @@ public class AssignmentManager {
private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) { private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
if (this.tableStateManager.isTableState(region.getTable(), if (this.tableStateManager.isTableState(region.getTable(),
ZooKeeperProtos.Table.State.DISABLED, TableState.State.DISABLED,
ZooKeeperProtos.Table.State.DISABLING) || replicasToClose.contains(region)) { TableState.State.DISABLING) || replicasToClose.contains(region)) {
LOG.info("Table " + region.getTable() + " is disabled or disabling;" LOG.info("Table " + region.getTable() + " is disabled or disabling;"
+ " skipping assign of " + region.getRegionNameAsString()); + " skipping assign of " + region.getRegionNameAsString());
offlineDisabledRegion(region); offlineDisabledRegion(region);
@ -1480,7 +1468,7 @@ public class AssignmentManager {
for (HRegionInfo hri : regionsFromMetaScan) { for (HRegionInfo hri : regionsFromMetaScan) {
TableName tableName = hri.getTable(); TableName tableName = hri.getTable();
if (!tableStateManager.isTableState(tableName, if (!tableStateManager.isTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
setEnabledTable(tableName); setEnabledTable(tableName);
} }
} }
@ -1525,14 +1513,14 @@ public class AssignmentManager {
* @throws IOException * @throws IOException
*/ */
Set<ServerName> rebuildUserRegions() throws Set<ServerName> rebuildUserRegions() throws
IOException, KeeperException, CoordinatedStateException { IOException, KeeperException {
Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates( Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates(
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.ENABLING); TableState.State.DISABLED, TableState.State.ENABLING);
Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates( Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
ZooKeeperProtos.Table.State.DISABLED, TableState.State.DISABLED,
ZooKeeperProtos.Table.State.DISABLING, TableState.State.DISABLING,
ZooKeeperProtos.Table.State.ENABLING); TableState.State.ENABLING);
// Region assignment from META // Region assignment from META
List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getShortCircuitConnection()); List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getShortCircuitConnection());
@ -1597,7 +1585,7 @@ public class AssignmentManager {
// this will be used in rolling restarts // this will be used in rolling restarts
if (!disabledOrDisablingOrEnabling.contains(tableName) if (!disabledOrDisablingOrEnabling.contains(tableName)
&& !getTableStateManager().isTableState(tableName, && !getTableStateManager().isTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
setEnabledTable(tableName); setEnabledTable(tableName);
} }
} }
@ -1614,9 +1602,9 @@ public class AssignmentManager {
* @throws IOException * @throws IOException
*/ */
private void recoverTableInDisablingState() private void recoverTableInDisablingState()
throws KeeperException, IOException, CoordinatedStateException { throws KeeperException, IOException {
Set<TableName> disablingTables = Set<TableName> disablingTables =
tableStateManager.getTablesInStates(ZooKeeperProtos.Table.State.DISABLING); tableStateManager.getTablesInStates(TableState.State.DISABLING);
if (disablingTables.size() != 0) { if (disablingTables.size() != 0) {
for (TableName tableName : disablingTables) { for (TableName tableName : disablingTables) {
// Recover by calling DisableTableHandler // Recover by calling DisableTableHandler
@ -1638,9 +1626,9 @@ public class AssignmentManager {
* @throws IOException * @throws IOException
*/ */
private void recoverTableInEnablingState() private void recoverTableInEnablingState()
throws KeeperException, IOException, CoordinatedStateException { throws KeeperException, IOException {
Set<TableName> enablingTables = tableStateManager. Set<TableName> enablingTables = tableStateManager.
getTablesInStates(ZooKeeperProtos.Table.State.ENABLING); getTablesInStates(TableState.State.ENABLING);
if (enablingTables.size() != 0) { if (enablingTables.size() != 0) {
for (TableName tableName : enablingTables) { for (TableName tableName : enablingTables) {
// Recover by calling EnableTableHandler // Recover by calling EnableTableHandler
@ -1675,7 +1663,7 @@ public class AssignmentManager {
if (!serverManager.isServerOnline(regionState.getServerName())) { if (!serverManager.isServerOnline(regionState.getServerName())) {
continue; // SSH will handle it continue; // SSH will handle it
} }
State state = regionState.getState(); RegionState.State state = regionState.getState();
LOG.info("Processing " + regionState); LOG.info("Processing " + regionState);
switch (state) { switch (state) {
case CLOSED: case CLOSED:
@ -1809,7 +1797,7 @@ public class AssignmentManager {
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
LOG.warn("Failed to unassign " LOG.warn("Failed to unassign "
+ hri.getRegionNameAsString() + " since interrupted", ie); + hri.getRegionNameAsString() + " since interrupted", ie);
regionStates.updateRegionState(hri, State.FAILED_CLOSE); regionStates.updateRegionState(hri, RegionState.State.FAILED_CLOSE);
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
return; return;
} }
@ -1987,7 +1975,7 @@ public class AssignmentManager {
it.remove(); it.remove();
} else { } else {
if (tableStateManager.isTableState(hri.getTable(), if (tableStateManager.isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
regionStates.regionOffline(hri); regionStates.regionOffline(hri);
it.remove(); it.remove();
continue; continue;
@ -2009,7 +1997,7 @@ public class AssignmentManager {
HRegionInfo hri = plan.getRegionInfo(); HRegionInfo hri = plan.getRegionInfo();
TableName tableName = hri.getTable(); TableName tableName = hri.getTable();
if (tableStateManager.isTableState(tableName, if (tableStateManager.isTableState(tableName,
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
LOG.info("Ignored moving region of disabling/disabled table " LOG.info("Ignored moving region of disabling/disabled table "
+ tableName); + tableName);
return; return;
@ -2043,8 +2031,8 @@ public class AssignmentManager {
protected void setEnabledTable(TableName tableName) { protected void setEnabledTable(TableName tableName) {
try { try {
this.tableStateManager.setTableState(tableName, this.tableStateManager.setTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED); TableState.State.ENABLED);
} catch (CoordinatedStateException e) { } catch (IOException e) {
// here we can abort as it is the start up flow // here we can abort as it is the start up flow
String errorMsg = "Unable to ensure that the table " + tableName String errorMsg = "Unable to ensure that the table " + tableName
+ " will be" + " enabled because of a ZooKeeper issue"; + " will be" + " enabled because of a ZooKeeper issue";
@ -2087,8 +2075,9 @@ public class AssignmentManager {
if (regionState != null) { if (regionState != null) {
// When there are more than one region server a new RS is selected as the // When there are more than one region server a new RS is selected as the
// destination and the same is updated in the region plan. (HBASE-5546) // destination and the same is updated in the region plan. (HBASE-5546)
if (getTableStateManager().isTableState(hri.getTable(), Table.State.DISABLED, if (getTableStateManager().isTableState(hri.getTable(),
Table.State.DISABLING) || replicasToClose.contains(hri)) { TableState.State.DISABLED, TableState.State.DISABLING) ||
replicasToClose.contains(hri)) {
offlineDisabledRegion(hri); offlineDisabledRegion(hri);
return null; return null;
} }
@ -2131,7 +2120,7 @@ public class AssignmentManager {
// reset the count, if any // reset the count, if any
failedOpenTracker.remove(hri.getEncodedName()); failedOpenTracker.remove(hri.getEncodedName());
if (getTableStateManager().isTableState(hri.getTable(), if (getTableStateManager().isTableState(hri.getTable(),
Table.State.DISABLED, Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(hri); invokeUnAssign(hri);
} }
return null; return null;
@ -2149,8 +2138,8 @@ public class AssignmentManager {
if (current == null || !current.isOnServer(serverName) || !current.isClosing()) { if (current == null || !current.isOnServer(serverName) || !current.isClosing()) {
return hri.getShortNameToLog() + " is not closing on " + serverName; return hri.getShortNameToLog() + " is not closing on " + serverName;
} }
if (getTableStateManager().isTableState(hri.getTable(), Table.State.DISABLED, if (getTableStateManager().isTableState(hri.getTable(), TableState.State.DISABLED,
Table.State.DISABLING) || replicasToClose.contains(hri)) { TableState.State.DISABLING) || replicasToClose.contains(hri)) {
offlineDisabledRegion(hri); offlineDisabledRegion(hri);
return null; return null;
} }
@ -2263,7 +2252,7 @@ public class AssignmentManager {
// User could disable the table before master knows the new region. // User could disable the table before master knows the new region.
if (getTableStateManager().isTableState(hri.getTable(), if (getTableStateManager().isTableState(hri.getTable(),
Table.State.DISABLED, Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(a); invokeUnAssign(a);
invokeUnAssign(b); invokeUnAssign(b);
} else { } else {
@ -2308,7 +2297,7 @@ public class AssignmentManager {
regionOffline(a); regionOffline(a);
regionOffline(b); regionOffline(b);
if (getTableStateManager().isTableState(hri.getTable(), if (getTableStateManager().isTableState(hri.getTable(),
Table.State.DISABLED, Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(hri); invokeUnAssign(hri);
} }
return null; return null;
@ -2419,7 +2408,7 @@ public class AssignmentManager {
// User could disable the table before master knows the new region. // User could disable the table before master knows the new region.
if (getTableStateManager().isTableState(hri.getTable(), if (getTableStateManager().isTableState(hri.getTable(),
Table.State.DISABLED, Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(hri); invokeUnAssign(hri);
} else { } else {
Callable<Object> mergeReplicasCallable = new Callable<Object>() { Callable<Object> mergeReplicasCallable = new Callable<Object>() {
@ -2464,7 +2453,7 @@ public class AssignmentManager {
regionOffline(hri); regionOffline(hri);
if (getTableStateManager().isTableState(hri.getTable(), if (getTableStateManager().isTableState(hri.getTable(),
Table.State.DISABLED, Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(a); invokeUnAssign(a);
invokeUnAssign(b); invokeUnAssign(b);
} }

View File

@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.executor.ExecutorType; import org.apache.hadoop.hbase.executor.ExecutorType;
@ -100,7 +101,6 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost; import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager; import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.regionserver.RSRpcServices;
@ -114,6 +114,7 @@ import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.util.ZKDataMigrator;
import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker; import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker; import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
@ -224,6 +225,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// monitor for distributed procedures // monitor for distributed procedures
MasterProcedureManagerHost mpmHost; MasterProcedureManagerHost mpmHost;
// handle table states
private TableStateManager tableStateManager;
/** flag used in test cases in order to simulate RS failures during master initialization */ /** flag used in test cases in order to simulate RS failures during master initialization */
private volatile boolean initializationBeforeMetaAssignment = false; private volatile boolean initializationBeforeMetaAssignment = false;
@ -409,7 +413,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
this.loadBalancerTracker.start(); this.loadBalancerTracker.start();
this.assignmentManager = new AssignmentManager(this, serverManager, this.assignmentManager = new AssignmentManager(this, serverManager,
this.balancer, this.service, this.metricsMaster, this.balancer, this.service, this.metricsMaster,
this.tableLockManager); this.tableLockManager, tableStateManager);
this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
this.serverManager); this.serverManager);
@ -436,6 +440,14 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
this.mpmHost.register(new MasterFlushTableProcedureManager()); this.mpmHost.register(new MasterFlushTableProcedureManager());
this.mpmHost.loadProcedures(conf); this.mpmHost.loadProcedures(conf);
this.mpmHost.initialize(this, this.metricsMaster); this.mpmHost.initialize(this, this.metricsMaster);
// migrating existent table state from zk
for (Map.Entry<TableName, TableState.State> entry : ZKDataMigrator
.queryForTableStates(getZooKeeper()).entrySet()) {
LOG.info("Converting state from zk to new states:" + entry);
tableStateManager.setTableState(entry.getKey(), entry.getValue());
}
ZKUtil.deleteChildrenRecursively(getZooKeeper(), getZooKeeper().tableZNode);
} }
/** /**
@ -490,6 +502,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Invalidate all write locks held previously // Invalidate all write locks held previously
this.tableLockManager.reapWriteLocks(); this.tableLockManager.reapWriteLocks();
this.tableStateManager = new TableStateManager(this);
this.tableStateManager.start();
status.setStatus("Initializing ZK system trackers"); status.setStatus("Initializing ZK system trackers");
initializeZKBasedSystemTrackers(); initializeZKBasedSystemTrackers();
@ -737,8 +752,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
} }
private void enableMeta(TableName metaTableName) { private void enableMeta(TableName metaTableName) {
if (!this.assignmentManager.getTableStateManager().isTableState(metaTableName, if (!this.tableStateManager.isTableState(metaTableName,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
this.assignmentManager.setEnabledTable(metaTableName); this.assignmentManager.setEnabledTable(metaTableName);
} }
} }
@ -777,6 +792,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return this.fileSystemManager; return this.fileSystemManager;
} }
@Override
public TableStateManager getTableStateManager() {
return tableStateManager;
}
/* /*
* Start up all services. If any of these threads gets an unhandled exception * Start up all services. If any of these threads gets an unhandled exception
* then they just die with a logged message. This should be fine because * then they just die with a logged message. This should be fine because
@ -1452,7 +1472,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
throw new TableNotFoundException(tableName); throw new TableNotFoundException(tableName);
} }
if (!getAssignmentManager().getTableStateManager(). if (!getAssignmentManager().getTableStateManager().
isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED)) { isTableState(tableName, TableState.State.DISABLED)) {
throw new TableNotDisabledException(tableName); throw new TableNotDisabledException(tableName);
} }
} }

View File

@ -43,8 +43,10 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
@ -454,7 +456,9 @@ public class MasterFileSystem {
} }
// Create tableinfo-s for hbase:meta if not already there. // Create tableinfo-s for hbase:meta if not already there.
new FSTableDescriptors(fs, rd).createTableDescriptor(HTableDescriptor.META_TABLEDESC); // assume, created table descriptor is for enabling table
new FSTableDescriptors(fs, rd).createTableDescriptor(
new TableDescriptor(HTableDescriptor.META_TABLEDESC, TableState.State.ENABLING));
return rd; return rd;
} }

View File

@ -37,8 +37,10 @@ import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.MergeRegionException; import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
@ -848,7 +850,7 @@ public class MasterRpcServices extends RSRpcServices
public GetTableNamesResponse getTableNames(RpcController controller, public GetTableNamesResponse getTableNames(RpcController controller,
GetTableNamesRequest req) throws ServiceException { GetTableNamesRequest req) throws ServiceException {
try { try {
master.checkInitialized(); master.checkServiceStarted();
Collection<HTableDescriptor> descriptors = master.getTableDescriptors().getAll().values(); Collection<HTableDescriptor> descriptors = master.getTableDescriptors().getAll().values();
GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder(); GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder();
for (HTableDescriptor descriptor: descriptors) { for (HTableDescriptor descriptor: descriptors) {
@ -863,6 +865,25 @@ public class MasterRpcServices extends RSRpcServices
} }
} }
@Override
public MasterProtos.GetTableStateResponse getTableState(RpcController controller,
MasterProtos.GetTableStateRequest request) throws ServiceException {
try {
master.checkServiceStarted();
TableName tableName = ProtobufUtil.toTableName(request.getTableName());
TableState.State state = master.getTableStateManager()
.getTableState(tableName);
if (state == null)
throw new TableNotFoundException(tableName);
MasterProtos.GetTableStateResponse.Builder builder =
MasterProtos.GetTableStateResponse.newBuilder();
builder.setTableState(new TableState(tableName, state).convert());
return builder.build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
@Override @Override
public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c, public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c,
IsCatalogJanitorEnabledRequest req) throws ServiceException { IsCatalogJanitorEnabledRequest req) throws ServiceException {

View File

@ -65,6 +65,11 @@ public interface MasterServices extends Server {
*/ */
TableLockManager getTableLockManager(); TableLockManager getTableLockManager();
/**
* @return Master's instance of {@link TableStateManager}
*/
TableStateManager getTableStateManager();
/** /**
* @return Master's instance of {@link MasterCoprocessorHost} * @return Master's instance of {@link MasterCoprocessorHost}
*/ */

View File

@ -29,6 +29,8 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.TreeMap; import java.util.TreeMap;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -40,16 +42,12 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
/** /**
* Region state accountant. It holds the states of all regions in the memory. * Region state accountant. It holds the states of all regions in the memory.
* In normal scenario, it should match the meta table and the true region states. * In normal scenario, it should match the meta table and the true region states.
@ -509,7 +507,7 @@ public class RegionStates {
if (oldServerName != null && serverHoldings.containsKey(oldServerName) if (oldServerName != null && serverHoldings.containsKey(oldServerName)
&& (newState == State.MERGED || newState == State.SPLIT && (newState == State.MERGED || newState == State.SPLIT
|| hri.isMetaRegion() || tableStateManager.isTableState(hri.getTable(), || hri.isMetaRegion() || tableStateManager.isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING))) { TableState.State.DISABLED, TableState.State.DISABLING))) {
// Offline the region only if it's merged/split, or the table is disabled/disabling. // Offline the region only if it's merged/split, or the table is disabled/disabling.
// Otherwise, offline it from this server only when it is online on a different server. // Otherwise, offline it from this server only when it is online on a different server.
LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName); LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
@ -937,8 +935,8 @@ public class RegionStates {
* Update a region state. It will be put in transition if not already there. * Update a region state. It will be put in transition if not already there.
*/ */
private RegionState updateRegionState(final HRegionInfo hri, private RegionState updateRegionState(final HRegionInfo hri,
final State state, final ServerName serverName, long openSeqNum) { final RegionState.State state, final ServerName serverName, long openSeqNum) {
if (state == State.FAILED_CLOSE || state == State.FAILED_OPEN) { if (state == RegionState.State.FAILED_CLOSE || state == RegionState.State.FAILED_OPEN) {
LOG.warn("Failed to open/close " + hri.getShortNameToLog() LOG.warn("Failed to open/close " + hri.getShortNameToLog()
+ " on " + serverName + ", set to " + state); + " on " + serverName + ", set to " + state);
} }

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler; import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -258,7 +259,7 @@ public class TableNamespaceManager {
} }
// Now check if the table is assigned, if not then fail fast // Now check if the table is assigned, if not then fail fast
if (isTableAssigned()) { if (isTableAssigned() && isTableEnabled()) {
try { try {
nsTable = new HTable(conf, TableName.NAMESPACE_TABLE_NAME); nsTable = new HTable(conf, TableName.NAMESPACE_TABLE_NAME);
zkNamespaceManager = new ZKNamespaceManager(masterServices.getZooKeeper()); zkNamespaceManager = new ZKNamespaceManager(masterServices.getZooKeeper());
@ -298,6 +299,12 @@ public class TableNamespaceManager {
return false; return false;
} }
private boolean isTableEnabled() throws IOException {
return masterServices.getTableStateManager().getTableState(
TableName.NAMESPACE_TABLE_NAME
).equals(TableState.State.ENABLED);
}
private boolean isTableAssigned() { private boolean isTableAssigned() {
return !masterServices.getAssignmentManager() return !masterServices.getAssignmentManager()
.getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty(); .getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty();

View File

@ -0,0 +1,217 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.Map;
import java.util.Set;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.TableState;
/**
* This is a helper class used to manage table states.
* States persisted in tableinfo and cached internally.
*/
@InterfaceAudience.Private
public class TableStateManager {
private static final Log LOG = LogFactory.getLog(TableStateManager.class);
private final TableDescriptors descriptors;
private final Map<TableName, TableState.State> tableStates = Maps.newConcurrentMap();
public TableStateManager(MasterServices master) {
this.descriptors = master.getTableDescriptors();
}
public void start() throws IOException {
Map<String, TableDescriptor> all = descriptors.getAllDescriptors();
for (TableDescriptor table : all.values()) {
TableName tableName = table.getHTableDescriptor().getTableName();
if (LOG.isDebugEnabled()) {
LOG.debug("Adding table state: " + tableName
+ ": " + table.getTableState());
}
tableStates.put(tableName, table.getTableState());
}
}
/**
* Set table state to provided.
* Caller should lock table on write.
* @param tableName table to change state for
* @param newState new state
* @throws IOException
*/
public void setTableState(TableName tableName, TableState.State newState) throws IOException {
synchronized (tableStates) {
TableDescriptor descriptor = readDescriptor(tableName);
if (descriptor == null) {
throw new TableNotFoundException(tableName);
}
if (descriptor.getTableState() != newState) {
writeDescriptor(
new TableDescriptor(descriptor.getHTableDescriptor(), newState));
}
}
}
/**
* Set table state to provided but only if table in specified states
* Caller should lock table on write.
* @param tableName table to change state for
* @param newState new state
* @param states states to check against
* @throws IOException
*/
public boolean setTableStateIfInStates(TableName tableName,
TableState.State newState,
TableState.State... states)
throws IOException {
synchronized (tableStates) {
TableDescriptor descriptor = readDescriptor(tableName);
if (descriptor == null) {
throw new TableNotFoundException(tableName);
}
if (TableState.isInStates(descriptor.getTableState(), states)) {
writeDescriptor(
new TableDescriptor(descriptor.getHTableDescriptor(), newState));
return true;
} else {
return false;
}
}
}
/**
* Set table state to provided but only if table not in specified states
* Caller should lock table on write.
* @param tableName table to change state for
* @param newState new state
* @param states states to check against
* @throws IOException
*/
public boolean setTableStateIfNotInStates(TableName tableName,
TableState.State newState,
TableState.State... states)
throws IOException {
synchronized (tableStates) {
TableDescriptor descriptor = readDescriptor(tableName);
if (descriptor == null) {
throw new TableNotFoundException(tableName);
}
if (!TableState.isInStates(descriptor.getTableState(), states)) {
writeDescriptor(
new TableDescriptor(descriptor.getHTableDescriptor(), newState));
return true;
} else {
return false;
}
}
}
public boolean isTableState(TableName tableName, TableState.State... states) {
TableState.State tableState = null;
try {
tableState = getTableState(tableName);
} catch (IOException e) {
LOG.error("Unable to get table state, probably table not exists");
return false;
}
return tableState != null && TableState.isInStates(tableState, states);
}
public void setDeletedTable(TableName tableName) throws IOException {
TableState.State remove = tableStates.remove(tableName);
if (remove == null) {
LOG.warn("Moving table " + tableName + " state to deleted but was " +
"already deleted");
}
}
public boolean isTablePresent(TableName tableName) throws IOException {
return getTableState(tableName) != null;
}
/**
* Return all tables in given states.
*
* @param states filter by states
* @return tables in given states
* @throws IOException
*/
public Set<TableName> getTablesInStates(TableState.State... states) throws IOException {
Set<TableName> rv = Sets.newHashSet();
for (Map.Entry<TableName, TableState.State> entry : tableStates.entrySet()) {
if (TableState.isInStates(entry.getValue(), states))
rv.add(entry.getKey());
}
return rv;
}
public TableState.State getTableState(TableName tableName) throws IOException {
TableState.State tableState = tableStates.get(tableName);
if (tableState == null) {
TableDescriptor descriptor = readDescriptor(tableName);
if (descriptor != null)
tableState = descriptor.getTableState();
}
return tableState;
}
/**
* Write descriptor in place, update cache of states.
* Write lock should be hold by caller.
*
* @param descriptor what to write
*/
private void writeDescriptor(TableDescriptor descriptor) throws IOException {
TableName tableName = descriptor.getHTableDescriptor().getTableName();
TableState.State state = descriptor.getTableState();
descriptors.add(descriptor);
LOG.debug("Table " + tableName + " written descriptor for state " + state);
tableStates.put(tableName, state);
LOG.debug("Table " + tableName + " updated state to " + state);
}
/**
* Read current descriptor for table, update cache of states.
*
* @param table descriptor to read
* @return descriptor
* @throws IOException
*/
private TableDescriptor readDescriptor(TableName tableName) throws IOException {
TableDescriptor descriptor = descriptors.getDescriptor(tableName);
if (descriptor == null)
tableStates.remove(tableName);
else
tableStates.put(tableName, descriptor.getTableState());
return descriptor;
}
}

View File

@ -31,14 +31,16 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.ipc.RequestContext; import org.apache.hadoop.hbase.ipc.RequestContext;
@ -49,7 +51,6 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock; import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSTableDescriptors;
@ -121,8 +122,6 @@ public class CreateTableHandler extends EventHandler {
if (MetaTableAccessor.tableExists(this.server.getShortCircuitConnection(), tableName)) { if (MetaTableAccessor.tableExists(this.server.getShortCircuitConnection(), tableName)) {
throw new TableExistsException(tableName); throw new TableExistsException(tableName);
} }
checkAndSetEnablingTable(assignmentManager, tableName);
success = true; success = true;
} finally { } finally {
if (!success) { if (!success) {
@ -132,47 +131,6 @@ public class CreateTableHandler extends EventHandler {
return this; return this;
} }
static void checkAndSetEnablingTable(final AssignmentManager assignmentManager,
final TableName tableName) throws IOException {
// If we have multiple client threads trying to create the table at the
// same time, given the async nature of the operation, the table
// could be in a state where hbase:meta table hasn't been updated yet in
// the process() function.
// Use enabling state to tell if there is already a request for the same
// table in progress. This will introduce a new zookeeper call. Given
// createTable isn't a frequent operation, that should be ok.
// TODO: now that we have table locks, re-evaluate above -- table locks are not enough.
// We could have cleared the hbase.rootdir and not zk. How can we detect this case?
// Having to clean zk AND hdfs is awkward.
try {
if (!assignmentManager.getTableStateManager().setTableStateIfNotInStates(tableName,
ZooKeeperProtos.Table.State.ENABLING,
ZooKeeperProtos.Table.State.ENABLING,
ZooKeeperProtos.Table.State.ENABLED)) {
throw new TableExistsException(tableName);
}
} catch (CoordinatedStateException e) {
throw new IOException("Unable to ensure that the table will be" +
" enabling because of a ZooKeeper issue", e);
}
}
static void removeEnablingTable(final AssignmentManager assignmentManager,
final TableName tableName) {
// Try deleting the enabling node in case of error
// If this does not happen then if the client tries to create the table
// again with the same Active master
// It will block the creation saying TableAlreadyExists.
try {
assignmentManager.getTableStateManager().checkAndRemoveTableState(tableName,
ZooKeeperProtos.Table.State.ENABLING, false);
} catch (CoordinatedStateException e) {
// Keeper exception should not happen here
LOG.error("Got a keeper exception while removing the ENABLING table znode "
+ tableName, e);
}
}
@Override @Override
public String toString() { public String toString() {
String name = "UnknownServerName"; String name = "UnknownServerName";
@ -218,9 +176,6 @@ public class CreateTableHandler extends EventHandler {
releaseTableLock(); releaseTableLock();
LOG.info("Table, " + this.hTableDescriptor.getTableName() + ", creation " + LOG.info("Table, " + this.hTableDescriptor.getTableName() + ", creation " +
(exception == null ? "successful" : "failed. " + exception)); (exception == null ? "successful" : "failed. " + exception));
if (exception != null) {
removeEnablingTable(this.assignmentManager, this.hTableDescriptor.getTableName());
}
} }
/** /**
@ -243,9 +198,12 @@ public class CreateTableHandler extends EventHandler {
FileSystem fs = fileSystemManager.getFileSystem(); FileSystem fs = fileSystemManager.getFileSystem();
// 1. Create Table Descriptor // 1. Create Table Descriptor
// using a copy of descriptor, table will be created enabling first
TableDescriptor underConstruction = new TableDescriptor(
this.hTableDescriptor, TableState.State.ENABLING);
Path tempTableDir = FSUtils.getTableDir(tempdir, tableName); Path tempTableDir = FSUtils.getTableDir(tempdir, tableName);
new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory( new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory(
tempTableDir, this.hTableDescriptor, false); tempTableDir, underConstruction, false);
Path tableDir = FSUtils.getTableDir(fileSystemManager.getRootDir(), tableName); Path tableDir = FSUtils.getTableDir(fileSystemManager.getRootDir(), tableName);
// 2. Create Regions // 2. Create Regions
@ -271,20 +229,15 @@ public class CreateTableHandler extends EventHandler {
ModifyRegionUtils.assignRegions(assignmentManager, regionInfos); ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);
} }
// 8. Set table enabled flag up in zk. // 6. Enable table
try { assignmentManager.getTableStateManager().setTableState(tableName,
assignmentManager.getTableStateManager().setTableState(tableName, TableState.State.ENABLED);
ZooKeeperProtos.Table.State.ENABLED);
} catch (CoordinatedStateException e) {
throw new IOException("Unable to ensure that " + tableName + " will be" +
" enabled because of a ZooKeeper issue", e);
}
} }
/** /**
* Create any replicas for the regions (the default replicas that was * Create any replicas for the regions (the default replicas that was
* already created is passed to the method) * already created is passed to the method)
* @param hTableDescriptor * @param hTableDescriptor descriptor to use
* @param regions default replicas * @param regions default replicas
* @return the combined list of default and non-default replicas * @return the combined list of default and non-default replicas
*/ */

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
@ -58,7 +59,7 @@ public class DeleteTableHandler extends TableEventHandler {
@Override @Override
protected void prepareWithTableLock() throws IOException { protected void prepareWithTableLock() throws IOException {
// The next call fails if no such table. // The next call fails if no such table.
hTableDescriptor = getTableDescriptor(); hTableDescriptor = getTableDescriptor().getHTableDescriptor();
} }
protected void waitRegionInTransition(final List<HRegionInfo> regions) protected void waitRegionInTransition(final List<HRegionInfo> regions)
@ -102,62 +103,66 @@ public class DeleteTableHandler extends TableEventHandler {
// 1. Wait because of region in transition // 1. Wait because of region in transition
waitRegionInTransition(regions); waitRegionInTransition(regions);
try {
// 2. Remove table from hbase:meta and HDFS // 2. Remove table from hbase:meta and HDFS
removeTableData(regions); removeTableData(regions);
} finally {
// 3. Update table descriptor cache
LOG.debug("Removing '" + tableName + "' descriptor.");
this.masterServices.getTableDescriptors().remove(tableName);
AssignmentManager am = this.masterServices.getAssignmentManager();
// 4. Clean up regions of the table in RegionStates.
LOG.debug("Removing '" + tableName + "' from region states.");
am.getRegionStates().tableDeleted(tableName);
// 5. If entry for this table in zk, and up in AssignmentManager, remove it.
LOG.debug("Marking '" + tableName + "' as deleted.");
am.getTableStateManager().setDeletedTable(tableName);
}
if (cpHost != null) { if (cpHost != null) {
cpHost.postDeleteTableHandler(this.tableName); cpHost.postDeleteTableHandler(this.tableName);
} }
} }
private void cleanupTableState() throws IOException {
// 3. Update table descriptor cache
LOG.debug("Removing '" + tableName + "' descriptor.");
this.masterServices.getTableDescriptors().remove(tableName);
AssignmentManager am = this.masterServices.getAssignmentManager();
// 4. Clean up regions of the table in RegionStates.
LOG.debug("Removing '" + tableName + "' from region states.");
am.getRegionStates().tableDeleted(tableName);
// 5. If entry for this table states, remove it.
LOG.debug("Marking '" + tableName + "' as deleted.");
am.getTableStateManager().setDeletedTable(tableName);
}
/** /**
* Removes the table from hbase:meta and archives the HDFS files. * Removes the table from hbase:meta and archives the HDFS files.
*/ */
protected void removeTableData(final List<HRegionInfo> regions) protected void removeTableData(final List<HRegionInfo> regions)
throws IOException, CoordinatedStateException { throws IOException, CoordinatedStateException {
// 1. Remove regions from META try {
LOG.debug("Deleting regions from META"); // 1. Remove regions from META
MetaTableAccessor.deleteRegions(this.server.getShortCircuitConnection(), regions); LOG.debug("Deleting regions from META");
MetaTableAccessor.deleteRegions(this.server.getShortCircuitConnection(), regions);
// ----------------------------------------------------------------------- // -----------------------------------------------------------------------
// NOTE: At this point we still have data on disk, but nothing in hbase:meta // NOTE: At this point we still have data on disk, but nothing in hbase:meta
// if the rename below fails, hbck will report an inconsistency. // if the rename below fails, hbck will report an inconsistency.
// ----------------------------------------------------------------------- // -----------------------------------------------------------------------
// 2. Move the table in /hbase/.tmp // 2. Move the table in /hbase/.tmp
MasterFileSystem mfs = this.masterServices.getMasterFileSystem(); MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
Path tempTableDir = mfs.moveTableToTemp(tableName); Path tempTableDir = mfs.moveTableToTemp(tableName);
// 3. Archive regions from FS (temp directory) // 3. Archive regions from FS (temp directory)
FileSystem fs = mfs.getFileSystem(); FileSystem fs = mfs.getFileSystem();
for (HRegionInfo hri: regions) { for (HRegionInfo hri : regions) {
LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS"); LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
HFileArchiver.archiveRegion(fs, mfs.getRootDir(), HFileArchiver.archiveRegion(fs, mfs.getRootDir(),
tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName())); tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
}
// 4. Delete table directory from FS (temp directory)
if (!fs.delete(tempTableDir, true)) {
LOG.error("Couldn't delete " + tempTableDir);
}
LOG.debug("Table '" + tableName + "' archived!");
} finally {
cleanupTableState();
} }
// 4. Delete table directory from FS (temp directory)
if (!fs.delete(tempTableDir, true)) {
LOG.error("Couldn't delete " + tempTableDir);
}
LOG.debug("Table '" + tableName + "' archived!");
} }
@Override @Override

View File

@ -25,13 +25,13 @@ import java.util.concurrent.ExecutorService;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
@ -39,11 +39,10 @@ import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.BulkAssigner; import org.apache.hadoop.hbase.master.BulkAssigner;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock; import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.htrace.Trace; import org.htrace.Trace;
/** /**
@ -91,16 +90,11 @@ public class DisableTableHandler extends EventHandler {
// DISABLED or ENABLED. // DISABLED or ENABLED.
//TODO: reevaluate this since we have table locks now //TODO: reevaluate this since we have table locks now
if (!skipTableStateCheck) { if (!skipTableStateCheck) {
try { if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates( this.tableName, TableState.State.DISABLING,
this.tableName, ZooKeeperProtos.Table.State.DISABLING, TableState.State.ENABLED)) {
ZooKeeperProtos.Table.State.ENABLED)) { LOG.info("Table " + tableName + " isn't enabled; skipping disable");
LOG.info("Table " + tableName + " isn't enabled; skipping disable"); throw new TableNotEnabledException(this.tableName);
throw new TableNotEnabledException(this.tableName);
}
} catch (CoordinatedStateException e) {
throw new IOException("Unable to ensure that the table will be" +
" disabling because of a coordination engine issue", e);
} }
} }
success = true; success = true;
@ -138,8 +132,6 @@ public class DisableTableHandler extends EventHandler {
} }
} catch (IOException e) { } catch (IOException e) {
LOG.error("Error trying to disable table " + this.tableName, e); LOG.error("Error trying to disable table " + this.tableName, e);
} catch (CoordinatedStateException e) {
LOG.error("Error trying to disable table " + this.tableName, e);
} finally { } finally {
releaseTableLock(); releaseTableLock();
} }
@ -155,10 +147,10 @@ public class DisableTableHandler extends EventHandler {
} }
} }
private void handleDisableTable() throws IOException, CoordinatedStateException { private void handleDisableTable() throws IOException {
// Set table disabling flag up in zk. // Set table disabling flag up in zk.
this.assignmentManager.getTableStateManager().setTableState(this.tableName, this.assignmentManager.getTableStateManager().setTableState(this.tableName,
ZooKeeperProtos.Table.State.DISABLING); TableState.State.DISABLING);
boolean done = false; boolean done = false;
while (true) { while (true) {
// Get list of online regions that are of this table. Regions that are // Get list of online regions that are of this table. Regions that are
@ -187,7 +179,7 @@ public class DisableTableHandler extends EventHandler {
} }
// Flip the table to disabled if success. // Flip the table to disabled if success.
if (done) this.assignmentManager.getTableStateManager().setTableState(this.tableName, if (done) this.assignmentManager.getTableStateManager().setTableState(this.tableName,
ZooKeeperProtos.Table.State.DISABLED); TableState.State.DISABLED);
LOG.info("Disabled table, " + this.tableName + ", is done=" + done); LOG.info("Disabled table, " + this.tableName + ", is done=" + done);
} }
@ -207,7 +199,7 @@ public class DisableTableHandler extends EventHandler {
RegionStates regionStates = assignmentManager.getRegionStates(); RegionStates regionStates = assignmentManager.getRegionStates();
for (HRegionInfo region: regions) { for (HRegionInfo region: regions) {
if (regionStates.isRegionInTransition(region) if (regionStates.isRegionInTransition(region)
&& !regionStates.isRegionInState(region, State.FAILED_CLOSE)) { && !regionStates.isRegionInState(region, RegionState.State.FAILED_CLOSE)) {
continue; continue;
} }
final HRegionInfo hri = region; final HRegionInfo hri = region;

View File

@ -27,7 +27,6 @@ import java.util.Map;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
@ -35,6 +34,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock; import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
/** /**
@ -95,16 +94,8 @@ public class EnableTableHandler extends EventHandler {
// retainAssignment is true only during recovery. In normal case it is false // retainAssignment is true only during recovery. In normal case it is false
if (!this.skipTableStateCheck) { if (!this.skipTableStateCheck) {
throw new TableNotFoundException(tableName); throw new TableNotFoundException(tableName);
}
try {
this.assignmentManager.getTableStateManager().checkAndRemoveTableState(tableName,
ZooKeeperProtos.Table.State.ENABLING, true);
throw new TableNotFoundException(tableName);
} catch (CoordinatedStateException e) {
// TODO : Use HBCK to clear such nodes
LOG.warn("Failed to delete the ENABLING node for the table " + tableName
+ ". The table will remain unusable. Run HBCK to manually fix the problem.");
} }
this.assignmentManager.getTableStateManager().setDeletedTable(tableName);
} }
// There could be multiple client requests trying to disable or enable // There could be multiple client requests trying to disable or enable
@ -112,16 +103,11 @@ public class EnableTableHandler extends EventHandler {
// After that, no other requests can be accepted until the table reaches // After that, no other requests can be accepted until the table reaches
// DISABLED or ENABLED. // DISABLED or ENABLED.
if (!skipTableStateCheck) { if (!skipTableStateCheck) {
try { if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates( this.tableName, TableState.State.ENABLING,
this.tableName, ZooKeeperProtos.Table.State.ENABLING, TableState.State.DISABLED)) {
ZooKeeperProtos.Table.State.DISABLED)) { LOG.info("Table " + tableName + " isn't disabled; skipping enable");
LOG.info("Table " + tableName + " isn't disabled; skipping enable"); throw new TableNotDisabledException(this.tableName);
throw new TableNotDisabledException(this.tableName);
}
} catch (CoordinatedStateException e) {
throw new IOException("Unable to ensure that the table will be" +
" enabling because of a coordination engine issue", e);
} }
} }
success = true; success = true;
@ -156,11 +142,7 @@ public class EnableTableHandler extends EventHandler {
if (cpHost != null) { if (cpHost != null) {
cpHost.postEnableTableHandler(this.tableName); cpHost.postEnableTableHandler(this.tableName);
} }
} catch (IOException e) { } catch (IOException | InterruptedException e) {
LOG.error("Error trying to enable the table " + this.tableName, e);
} catch (CoordinatedStateException e) {
LOG.error("Error trying to enable the table " + this.tableName, e);
} catch (InterruptedException e) {
LOG.error("Error trying to enable the table " + this.tableName, e); LOG.error("Error trying to enable the table " + this.tableName, e);
} finally { } finally {
releaseTableLock(); releaseTableLock();
@ -177,14 +159,13 @@ public class EnableTableHandler extends EventHandler {
} }
} }
private void handleEnableTable() throws IOException, CoordinatedStateException, private void handleEnableTable() throws IOException,
InterruptedException { InterruptedException {
// I could check table is disabling and if so, not enable but require // I could check table is disabling and if so, not enable but require
// that user first finish disabling but that might be obnoxious. // that user first finish disabling but that might be obnoxious.
// Set table enabling flag up in zk.
this.assignmentManager.getTableStateManager().setTableState(this.tableName, this.assignmentManager.getTableStateManager().setTableState(this.tableName,
ZooKeeperProtos.Table.State.ENABLING); TableState.State.ENABLING);
boolean done = false; boolean done = false;
ServerManager serverManager = ((HMaster)this.server).getServerManager(); ServerManager serverManager = ((HMaster)this.server).getServerManager();
// Get the regions of this table. We're done when all listed // Get the regions of this table. We're done when all listed
@ -236,7 +217,7 @@ public class EnableTableHandler extends EventHandler {
if (done) { if (done) {
// Flip the table to enabled. // Flip the table to enabled.
this.assignmentManager.getTableStateManager().setTableState( this.assignmentManager.getTableStateManager().setTableState(
this.tableName, ZooKeeperProtos.Table.State.ENABLED); this.tableName, TableState.State.ENABLED);
LOG.info("Table '" + this.tableName LOG.info("Table '" + this.tableName
+ "' was successfully enabled. Status: done=" + done); + "' was successfully enabled. Status: done=" + done);
} else { } else {

View File

@ -27,6 +27,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -65,8 +67,9 @@ public class ModifyTableHandler extends TableEventHandler {
// Check operation is possible on the table in its current state // Check operation is possible on the table in its current state
// Also checks whether the table exists // Also checks whether the table exists
if (masterServices.getAssignmentManager().getTableStateManager() if (masterServices.getAssignmentManager().getTableStateManager()
.isTableState(this.htd.getTableName(), ZooKeeperProtos.Table.State.ENABLED) .isTableState(this.htd.getTableName(), TableState.State.ENABLED)
&& this.htd.getRegionReplication() != getTableDescriptor().getRegionReplication()) { && this.htd.getRegionReplication() != getTableDescriptor()
.getHTableDescriptor().getRegionReplication()) {
throw new IOException("REGION_REPLICATION change is not supported for enabled tables"); throw new IOException("REGION_REPLICATION change is not supported for enabled tables");
} }
} }
@ -79,11 +82,14 @@ public class ModifyTableHandler extends TableEventHandler {
cpHost.preModifyTableHandler(this.tableName, this.htd); cpHost.preModifyTableHandler(this.tableName, this.htd);
} }
// Update descriptor // Update descriptor
HTableDescriptor oldHtd = getTableDescriptor(); HTableDescriptor oldDescriptor =
this.masterServices.getTableDescriptors().add(this.htd); this.masterServices.getTableDescriptors().get(this.tableName);
deleteFamilyFromFS(hris, oldHtd.getFamiliesKeys()); this.masterServices.getTableDescriptors().add(htd);
removeReplicaColumnsIfNeeded(this.htd.getRegionReplication(), oldHtd.getRegionReplication(), deleteFamilyFromFS(hris, oldDescriptor.getFamiliesKeys());
htd.getTableName()); removeReplicaColumnsIfNeeded(
this.htd.getRegionReplication(),
oldDescriptor.getRegionReplication(),
this.htd.getTableName());
if (cpHost != null) { if (cpHost != null) {
cpHost.postModifyTableHandler(this.tableName, this.htd); cpHost.postModifyTableHandler(this.tableName, this.htd);
} }

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
@ -39,10 +40,8 @@ import org.apache.hadoop.hbase.master.DeadServer;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
/** /**
@ -231,23 +230,23 @@ public class ServerShutdownHandler extends EventHandler {
continue; continue;
} }
LOG.info("Reassigning region with rs = " + rit); LOG.info("Reassigning region with rs = " + rit);
regionStates.updateRegionState(hri, State.OFFLINE); regionStates.updateRegionState(hri, RegionState.State.OFFLINE);
} else if (regionStates.isRegionInState( } else if (regionStates.isRegionInState(
hri, State.SPLITTING_NEW, State.MERGING_NEW)) { hri, RegionState.State.SPLITTING_NEW, RegionState.State.MERGING_NEW)) {
regionStates.updateRegionState(hri, State.OFFLINE); regionStates.updateRegionState(hri, RegionState.State.OFFLINE);
} }
toAssignRegions.add(hri); toAssignRegions.add(hri);
} else if (rit != null) { } else if (rit != null) {
if ((rit.isClosing() || rit.isFailedClose() || rit.isOffline()) if ((rit.isClosing() || rit.isFailedClose() || rit.isOffline())
&& am.getTableStateManager().isTableState(hri.getTable(), && am.getTableStateManager().isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) || TableState.State.DISABLED, TableState.State.DISABLING) ||
am.getReplicasToClose().contains(hri)) { am.getReplicasToClose().contains(hri)) {
// If the table was partially disabled and the RS went down, we should clear the RIT // If the table was partially disabled and the RS went down, we should clear the RIT
// and remove the node for the region. // and remove the node for the region.
// The rit that we use may be stale in case the table was in DISABLING state // The rit that we use may be stale in case the table was in DISABLING state
// but though we did assign we will not be clearing the znode in CLOSING state. // but though we did assign we will not be clearing the znode in CLOSING state.
// Doing this will have no harm. See HBASE-5927 // Doing this will have no harm. See HBASE-5927
regionStates.updateRegionState(hri, State.OFFLINE); regionStates.updateRegionState(hri, RegionState.State.OFFLINE);
am.offlineDisabledRegion(hri); am.offlineDisabledRegion(hri);
} else { } else {
LOG.warn("THIS SHOULD NOT HAPPEN: unexpected region in transition " LOG.warn("THIS SHOULD NOT HAPPEN: unexpected region in transition "
@ -323,7 +322,7 @@ public class ServerShutdownHandler extends EventHandler {
} }
// If table is not disabled but the region is offlined, // If table is not disabled but the region is offlined,
boolean disabled = assignmentManager.getTableStateManager().isTableState(hri.getTable(), boolean disabled = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED); TableState.State.DISABLED);
if (disabled){ if (disabled){
LOG.info("The table " + hri.getTable() LOG.info("The table " + hri.getTable()
+ " was disabled. Hence not proceeding."); + " was disabled. Hence not proceeding.");
@ -336,7 +335,7 @@ public class ServerShutdownHandler extends EventHandler {
return false; return false;
} }
boolean disabling = assignmentManager.getTableStateManager().isTableState(hri.getTable(), boolean disabling = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLING); TableState.State.DISABLING);
if (disabling) { if (disabling) {
LOG.info("The table " + hri.getTable() LOG.info("The table " + hri.getTable()
+ " is disabled. Hence not assigning region" + hri.getEncodedName()); + " is disabled. Hence not assigning region" + hri.getEncodedName());

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.List; import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
@ -50,8 +51,8 @@ public class TableAddFamilyHandler extends TableEventHandler {
@Override @Override
protected void prepareWithTableLock() throws IOException { protected void prepareWithTableLock() throws IOException {
super.prepareWithTableLock(); super.prepareWithTableLock();
HTableDescriptor htd = getTableDescriptor(); TableDescriptor htd = getTableDescriptor();
if (htd.hasFamily(familyDesc.getName())) { if (htd.getHTableDescriptor().hasFamily(familyDesc.getName())) {
throw new InvalidFamilyOperationException("Family '" + throw new InvalidFamilyOperationException("Family '" +
familyDesc.getNameAsString() + "' already exists so cannot be added"); familyDesc.getNameAsString() + "' already exists so cannot be added");
} }

View File

@ -50,7 +50,7 @@ public class TableDeleteFamilyHandler extends TableEventHandler {
@Override @Override
protected void prepareWithTableLock() throws IOException { protected void prepareWithTableLock() throws IOException {
super.prepareWithTableLock(); super.prepareWithTableLock();
HTableDescriptor htd = getTableDescriptor(); HTableDescriptor htd = getTableDescriptor().getHTableDescriptor();
this.familyName = hasColumnFamily(htd, familyName); this.familyName = hasColumnFamily(htd, familyName);
} }

View File

@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
@ -40,12 +41,12 @@ import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.BulkReOpen; import org.apache.hadoop.hbase.master.BulkReOpen;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock; import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
@ -130,7 +131,7 @@ public abstract class TableEventHandler extends EventHandler {
handleTableOperation(hris); handleTableOperation(hris);
if (eventType.isOnlineSchemaChangeSupported() && this.masterServices. if (eventType.isOnlineSchemaChangeSupported() && this.masterServices.
getAssignmentManager().getTableStateManager().isTableState( getAssignmentManager().getTableStateManager().isTableState(
tableName, ZooKeeperProtos.Table.State.ENABLED)) { tableName, TableState.State.ENABLED)) {
if (reOpenAllRegions(hris)) { if (reOpenAllRegions(hris)) {
LOG.info("Completed table operation " + eventType + " on table " + LOG.info("Completed table operation " + eventType + " on table " +
tableName); tableName);
@ -230,10 +231,10 @@ public abstract class TableEventHandler extends EventHandler {
* @throws FileNotFoundException * @throws FileNotFoundException
* @throws IOException * @throws IOException
*/ */
public HTableDescriptor getTableDescriptor() public TableDescriptor getTableDescriptor()
throws FileNotFoundException, IOException { throws FileNotFoundException, IOException {
HTableDescriptor htd = TableDescriptor htd =
this.masterServices.getTableDescriptors().get(tableName); this.masterServices.getTableDescriptors().getDescriptor(tableName);
if (htd == null) { if (htd == null) {
throw new IOException("HTableDescriptor missing for " + tableName); throw new IOException("HTableDescriptor missing for " + tableName);
} }

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.List; import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
@ -49,7 +50,7 @@ public class TableModifyFamilyHandler extends TableEventHandler {
@Override @Override
protected void prepareWithTableLock() throws IOException { protected void prepareWithTableLock() throws IOException {
super.prepareWithTableLock(); super.prepareWithTableLock();
HTableDescriptor htd = getTableDescriptor(); HTableDescriptor htd = getTableDescriptor().getHTableDescriptor();
hasColumnFamily(htd, familyDesc.getName()); hasColumnFamily(htd, familyDesc.getName());
} }

View File

@ -28,15 +28,17 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils; import org.apache.hadoop.hbase.util.ModifyRegionUtils;
@ -93,54 +95,44 @@ public class TruncateTableHandler extends DeleteTableHandler {
AssignmentManager assignmentManager = this.masterServices.getAssignmentManager(); AssignmentManager assignmentManager = this.masterServices.getAssignmentManager();
// 1. Set table znode // 1. Create Table Descriptor
CreateTableHandler.checkAndSetEnablingTable(assignmentManager, tableName); TableDescriptor underConstruction = new TableDescriptor(
try { this.hTableDescriptor, TableState.State.ENABLING);
// 1. Create Table Descriptor Path tempTableDir = FSUtils.getTableDir(tempdir, this.tableName);
new FSTableDescriptors(server.getConfiguration()) new FSTableDescriptors(server.getConfiguration())
.createTableDescriptorForTableDirectory(tempdir, this.hTableDescriptor, false); .createTableDescriptorForTableDirectory(tempTableDir, underConstruction, false);
Path tempTableDir = FSUtils.getTableDir(tempdir, this.tableName); Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), this.tableName);
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), this.tableName);
HRegionInfo[] newRegions; HRegionInfo[] newRegions;
if (this.preserveSplits) { if (this.preserveSplits) {
newRegions = regions.toArray(new HRegionInfo[regions.size()]); newRegions = regions.toArray(new HRegionInfo[regions.size()]);
LOG.info("Truncate will preserve " + newRegions.length + " regions"); LOG.info("Truncate will preserve " + newRegions.length + " regions");
} else { } else {
newRegions = new HRegionInfo[1]; newRegions = new HRegionInfo[1];
newRegions[0] = new HRegionInfo(this.tableName, null, null); newRegions[0] = new HRegionInfo(this.tableName, null, null);
LOG.info("Truncate will not preserve the regions"); LOG.info("Truncate will not preserve the regions");
}
// 2. Create Regions
List<HRegionInfo> regionInfos = ModifyRegionUtils.createRegions(
masterServices.getConfiguration(), tempdir,
this.hTableDescriptor, newRegions, null);
// 3. Move Table temp directory to the hbase root location
if (!fs.rename(tempTableDir, tableDir)) {
throw new IOException("Unable to move table from temp=" + tempTableDir +
" to hbase root=" + tableDir);
}
// 4. Add regions to META
MetaTableAccessor.addRegionsToMeta(masterServices.getShortCircuitConnection(),
regionInfos);
// 5. Trigger immediate assignment of the regions in round-robin fashion
ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);
// 6. Set table enabled flag up in zk.
try {
assignmentManager.getTableStateManager().setTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED);
} catch (CoordinatedStateException e) {
throw new IOException("Unable to ensure that " + tableName + " will be" +
" enabled because of a ZooKeeper issue", e);
}
} catch (IOException e) {
CreateTableHandler.removeEnablingTable(assignmentManager, tableName);
throw e;
} }
// 2. Create Regions
List<HRegionInfo> regionInfos = ModifyRegionUtils.createRegions(
masterServices.getConfiguration(), tempdir,
this.hTableDescriptor, newRegions, null);
// 3. Move Table temp directory to the hbase root location
if (!fs.rename(tempTableDir, tableDir)) {
throw new IOException("Unable to move table from temp=" + tempTableDir +
" to hbase root=" + tableDir);
}
// 4. Add regions to META
MetaTableAccessor.addRegionsToMeta(masterServices.getShortCircuitConnection(),
regionInfos);
// 5. Trigger immediate assignment of the regions in round-robin fashion
ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);
// 6. Set table enabled flag up in zk.
assignmentManager.getTableStateManager().setTableState(tableName,
TableState.State.ENABLED);
} }
} }

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.ipc.RequestContext; import org.apache.hadoop.hbase.ipc.RequestContext;
@ -566,14 +567,14 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
TableName snapshotTable = TableName.valueOf(snapshot.getTable()); TableName snapshotTable = TableName.valueOf(snapshot.getTable());
AssignmentManager assignmentMgr = master.getAssignmentManager(); AssignmentManager assignmentMgr = master.getAssignmentManager();
if (assignmentMgr.getTableStateManager().isTableState(snapshotTable, if (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
LOG.debug("Table enabled, starting distributed snapshot."); LOG.debug("Table enabled, starting distributed snapshot.");
snapshotEnabledTable(snapshot); snapshotEnabledTable(snapshot);
LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot)); LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
} }
// For disabled table, snapshot is created by the master // For disabled table, snapshot is created by the master
else if (assignmentMgr.getTableStateManager().isTableState(snapshotTable, else if (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
ZooKeeperProtos.Table.State.DISABLED)) { TableState.State.DISABLED)) {
LOG.debug("Table is disabled, running snapshot entirely on master."); LOG.debug("Table is disabled, running snapshot entirely on master.");
snapshotDisabledTable(snapshot); snapshotDisabledTable(snapshot);
LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot)); LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
@ -705,8 +706,8 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
// Execute the restore/clone operation // Execute the restore/clone operation
if (MetaTableAccessor.tableExists(master.getShortCircuitConnection(), tableName)) { if (MetaTableAccessor.tableExists(master.getShortCircuitConnection(), tableName)) {
if (master.getAssignmentManager().getTableStateManager().isTableState( if (master.getTableStateManager().isTableState(
TableName.valueOf(snapshot.getTable()), ZooKeeperProtos.Table.State.ENABLED)) { TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) {
throw new UnsupportedOperationException("Table '" + throw new UnsupportedOperationException("Table '" +
TableName.valueOf(snapshot.getTable()) + "' must be disabled in order to " + TableName.valueOf(snapshot.getTable()) + "' must be disabled in order to " +
"perform a restore operation" + "perform a restore operation" +

View File

@ -39,12 +39,14 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
@ -374,7 +376,7 @@ public class NamespaceUpgrade implements Tool {
HTableDescriptor newDesc = new HTableDescriptor(oldDesc); HTableDescriptor newDesc = new HTableDescriptor(oldDesc);
newDesc.setName(newTableName); newDesc.setName(newTableName);
new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory( new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory(
newTablePath, newDesc, true); newTablePath, new TableDescriptor(newDesc, TableState.State.ENABLED), true);
} }

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
@ -112,13 +113,14 @@ public class CompactionTool extends Configured implements Tool {
if (isFamilyDir(fs, path)) { if (isFamilyDir(fs, path)) {
Path regionDir = path.getParent(); Path regionDir = path.getParent();
Path tableDir = regionDir.getParent(); Path tableDir = regionDir.getParent();
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir); HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
compactStoreFiles(tableDir, htd, hri, path.getName(), compactOnce, major); compactStoreFiles(tableDir, htd.getHTableDescriptor(), hri,
path.getName(), compactOnce, major);
} else if (isRegionDir(fs, path)) { } else if (isRegionDir(fs, path)) {
Path tableDir = path.getParent(); Path tableDir = path.getParent();
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
compactRegion(tableDir, htd, path, compactOnce, major); compactRegion(tableDir, htd.getHTableDescriptor(), path, compactOnce, major);
} else if (isTableDir(fs, path)) { } else if (isTableDir(fs, path)) {
compactTable(path, compactOnce, major); compactTable(path, compactOnce, major);
} else { } else {
@ -129,9 +131,9 @@ public class CompactionTool extends Configured implements Tool {
private void compactTable(final Path tableDir, final boolean compactOnce, final boolean major) private void compactTable(final Path tableDir, final boolean compactOnce, final boolean major)
throws IOException { throws IOException {
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) { for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
compactRegion(tableDir, htd, regionDir, compactOnce, major); compactRegion(tableDir, htd.getHTableDescriptor(), regionDir, compactOnce, major);
} }
} }

View File

@ -46,6 +46,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.protobuf.ServiceException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -56,7 +59,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -66,7 +68,6 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagType;
import org.apache.hadoop.hbase.client.ConnectionUtils; import org.apache.hadoop.hbase.client.ConnectionUtils;
@ -78,6 +79,7 @@ import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination; import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.RegionOpeningException; import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@ -92,7 +94,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Mut
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId;
@ -112,10 +113,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.io.MultipleIOException; import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.protobuf.ServiceException;
/** /**
* This class is responsible for splitting up a bunch of regionserver commit log * This class is responsible for splitting up a bunch of regionserver commit log
* files that are no longer being written to, into new files, one per region for * files that are no longer being written to, into new files, one per region for
@ -286,12 +283,13 @@ public class HLogSplitter {
return true; return true;
} }
if(csm != null) { if(csm != null) {
try { HConnection scc = csm.getServer().getShortCircuitConnection();
TableStateManager tsm = csm.getTableStateManager(); TableName[] tables = scc.listTableNames();
disablingOrDisabledTables = tsm.getTablesInStates( for (TableName table : tables) {
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING); if (scc.getTableState(table)
} catch (CoordinatedStateException e) { .inStates(TableState.State.DISABLED, TableState.State.DISABLING)) {
throw new IOException("Can't get disabling/disabled tables", e); disablingOrDisabledTables.add(table);
}
} }
} }
int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3); int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3);

View File

@ -89,6 +89,7 @@ public class WALCellCodec implements Codec {
* Fully prepares the codec for use. * Fully prepares the codec for use.
* @param conf {@link Configuration} to read for the user-specified codec. If none is specified, * @param conf {@link Configuration} to read for the user-specified codec. If none is specified,
* uses a {@link WALCellCodec}. * uses a {@link WALCellCodec}.
* @param cellCodecClsName name of codec
* @param compression compression the codec should use * @param compression compression the codec should use
* @return a {@link WALCellCodec} ready for use. * @return a {@link WALCellCodec} ready for use.
* @throws UnsupportedOperationException if the codec cannot be instantiated * @throws UnsupportedOperationException if the codec cannot be instantiated

View File

@ -38,6 +38,8 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; 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.SnapshotDataManifest;
@ -259,7 +261,8 @@ public class SnapshotManifest {
private void load() throws IOException { private void load() throws IOException {
switch (getSnapshotFormat(desc)) { switch (getSnapshotFormat(desc)) {
case SnapshotManifestV1.DESCRIPTOR_VERSION: { case SnapshotManifestV1.DESCRIPTOR_VERSION: {
this.htd = FSTableDescriptors.getTableDescriptorFromFs(fs, workingDir); this.htd = FSTableDescriptors.getTableDescriptorFromFs(fs, workingDir)
.getHTableDescriptor();
ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader"); ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
try { try {
this.regionManifests = this.regionManifests =
@ -353,7 +356,8 @@ public class SnapshotManifest {
LOG.info("Using old Snapshot Format"); LOG.info("Using old Snapshot Format");
// write a copy of descriptor to the snapshot directory // write a copy of descriptor to the snapshot directory
new FSTableDescriptors(fs, rootDir) new FSTableDescriptors(fs, rootDir)
.createTableDescriptorForTableDirectory(workingDir, htd, false); .createTableDescriptorForTableDirectory(workingDir, new TableDescriptor(
htd, TableState.State.ENABLED), false);
} else { } else {
LOG.debug("Convert to Single Snapshot Manifest"); LOG.debug("Convert to Single Snapshot Manifest");
convertToV2SingleManifest(); convertToV2SingleManifest();

View File

@ -17,6 +17,7 @@
*/ */
package org.apache.hadoop.hbase.util; package org.apache.hadoop.hbase.util;
import javax.annotation.Nullable;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.Comparator; import java.util.Comparator;
@ -38,7 +39,9 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
@ -92,11 +95,11 @@ public class FSTableDescriptors implements TableDescriptors {
* Data structure to hold modification time and table descriptor. * Data structure to hold modification time and table descriptor.
*/ */
private static class TableDescriptorAndModtime { private static class TableDescriptorAndModtime {
private final HTableDescriptor htd; private final TableDescriptor td;
private final long modtime; private final long modtime;
TableDescriptorAndModtime(final long modtime, final HTableDescriptor htd) { TableDescriptorAndModtime(final long modtime, final TableDescriptor td) {
this.htd = htd; this.td = td;
this.modtime = modtime; this.modtime = modtime;
} }
@ -104,8 +107,16 @@ public class FSTableDescriptors implements TableDescriptors {
return this.modtime; return this.modtime;
} }
HTableDescriptor getTableDescriptor() { TableDescriptor getTableDescriptor() {
return this.htd; return this.td;
}
HTableDescriptor getHTableDescriptor() {
return this.td.getHTableDescriptor();
}
TableState.State getTableState() {
return this.td.getTableState();
} }
} }
@ -141,12 +152,13 @@ public class FSTableDescriptors implements TableDescriptors {
* to see if a newer file has been created since the cached one was read. * to see if a newer file has been created since the cached one was read.
*/ */
@Override @Override
public HTableDescriptor get(final TableName tablename) @Nullable
public TableDescriptor getDescriptor(final TableName tablename)
throws IOException { throws IOException {
invocations++; invocations++;
if (HTableDescriptor.META_TABLEDESC.getTableName().equals(tablename)) { if (HTableDescriptor.META_TABLEDESC.getTableName().equals(tablename)) {
cachehits++; cachehits++;
return HTableDescriptor.META_TABLEDESC; return new TableDescriptor(HTableDescriptor.META_TABLEDESC, TableState.State.ENABLED);
} }
// hbase:meta is already handled. If some one tries to get the descriptor for // hbase:meta is already handled. If some one tries to get the descriptor for
// .logs, .oldlogs or .corrupt throw an exception. // .logs, .oldlogs or .corrupt throw an exception.
@ -182,31 +194,62 @@ public class FSTableDescriptors implements TableDescriptors {
return tdmt == null ? null : tdmt.getTableDescriptor(); return tdmt == null ? null : tdmt.getTableDescriptor();
} }
/**
* Get the current table descriptor for the given table, or null if none exists.
*
* Uses a local cache of the descriptor but still checks the filesystem on each call
* to see if a newer file has been created since the cached one was read.
*/
@Override
public HTableDescriptor get(TableName tableName) throws IOException {
if (HTableDescriptor.META_TABLEDESC.getTableName().equals(tableName)) {
cachehits++;
return HTableDescriptor.META_TABLEDESC;
}
TableDescriptor descriptor = getDescriptor(tableName);
return descriptor == null ? null : descriptor.getHTableDescriptor();
}
/** /**
* Returns a map from table name to table descriptor for all tables. * Returns a map from table name to table descriptor for all tables.
*/ */
@Override @Override
public Map<String, HTableDescriptor> getAll() public Map<String, TableDescriptor> getAllDescriptors()
throws IOException { throws IOException {
Map<String, HTableDescriptor> htds = new TreeMap<String, HTableDescriptor>(); Map<String, TableDescriptor> tds = new TreeMap<String, TableDescriptor>();
List<Path> tableDirs = FSUtils.getTableDirs(fs, rootdir); List<Path> tableDirs = FSUtils.getTableDirs(fs, rootdir);
for (Path d: tableDirs) { for (Path d: tableDirs) {
HTableDescriptor htd = null; TableDescriptor htd = null;
try { try {
htd = get(FSUtils.getTableName(d)); htd = getDescriptor(FSUtils.getTableName(d));
} catch (FileNotFoundException fnfe) { } catch (FileNotFoundException fnfe) {
// inability of retrieving one HTD shouldn't stop getting the remaining // inability of retrieving one HTD shouldn't stop getting the remaining
LOG.warn("Trouble retrieving htd", fnfe); LOG.warn("Trouble retrieving htd", fnfe);
} }
if (htd == null) continue; if (htd == null) continue;
htds.put(htd.getTableName().getNameAsString(), htd); tds.put(htd.getHTableDescriptor().getTableName().getNameAsString(), htd);
}
return tds;
}
/**
* 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<String, HTableDescriptor>();
Map<String, TableDescriptor> allDescriptors = getAllDescriptors();
for (Map.Entry<String, TableDescriptor> entry : allDescriptors
.entrySet()) {
htds.put(entry.getKey(), entry.getValue().getHTableDescriptor());
} }
return htds; return htds;
} }
/* (non-Javadoc) /**
* @see org.apache.hadoop.hbase.TableDescriptors#getTableDescriptors(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path) * Find descriptors by namespace.
*/ * @see #get(org.apache.hadoop.hbase.TableName)
*/
@Override @Override
public Map<String, HTableDescriptor> getByNamespace(String name) public Map<String, HTableDescriptor> getByNamespace(String name)
throws IOException { throws IOException {
@ -227,6 +270,27 @@ public class FSTableDescriptors implements TableDescriptors {
return htds; return htds;
} }
/**
* Adds (or updates) the table descriptor to the FileSystem
* and updates the local cache with it.
*/
@Override
public void add(TableDescriptor htd) throws IOException {
if (fsreadonly) {
throw new NotImplementedException("Cannot add a table descriptor - in read only mode");
}
TableName tableName = htd.getHTableDescriptor().getTableName();
if (TableName.META_TABLE_NAME.equals(tableName)) {
throw new NotImplementedException();
}
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tableName.getNameAsString())) {
throw new NotImplementedException(
"Cannot add a table descriptor for a reserved subdirectory name: "
+ htd.getHTableDescriptor().getNameAsString());
}
updateTableDescriptor(htd);
}
/** /**
* Adds (or updates) the table descriptor to the FileSystem * Adds (or updates) the table descriptor to the FileSystem
* and updates the local cache with it. * and updates the local cache with it.
@ -236,16 +300,21 @@ public class FSTableDescriptors implements TableDescriptors {
if (fsreadonly) { if (fsreadonly) {
throw new NotImplementedException("Cannot add a table descriptor - in read only mode"); throw new NotImplementedException("Cannot add a table descriptor - in read only mode");
} }
if (TableName.META_TABLE_NAME.equals(htd.getTableName())) { TableName tableName = htd.getTableName();
if (TableName.META_TABLE_NAME.equals(tableName)) {
throw new NotImplementedException(); throw new NotImplementedException();
} }
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(htd.getTableName().getNameAsString())) { if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tableName.getNameAsString())) {
throw new NotImplementedException( throw new NotImplementedException(
"Cannot add a table descriptor for a reserved subdirectory name: " + htd.getNameAsString()); "Cannot add a table descriptor for a reserved subdirectory name: "
+ htd.getNameAsString());
} }
updateTableDescriptor(htd); TableDescriptor descriptor = getDescriptor(htd.getTableName());
long modtime = getTableInfoModtime(htd.getTableName()); if (descriptor == null)
this.cache.put(htd.getTableName(), new TableDescriptorAndModtime(modtime, htd)); descriptor = new TableDescriptor(htd);
else
descriptor.setHTableDescriptor(htd);
updateTableDescriptor(descriptor);
} }
/** /**
@ -266,7 +335,7 @@ public class FSTableDescriptors implements TableDescriptors {
} }
} }
TableDescriptorAndModtime tdm = this.cache.remove(tablename); TableDescriptorAndModtime tdm = this.cache.remove(tablename);
return tdm == null ? null : tdm.getTableDescriptor(); return tdm == null ? null : tdm.getHTableDescriptor();
} }
/** /**
@ -463,7 +532,7 @@ public class FSTableDescriptors implements TableDescriptors {
* if it exists, bypassing the local cache. * if it exists, bypassing the local cache.
* Returns null if it's not found. * 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 hbaseRootDir, TableName tableName) throws IOException {
Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName); Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
return getTableDescriptorFromFs(fs, tableDir); return getTableDescriptorFromFs(fs, tableDir);
@ -474,7 +543,7 @@ public class FSTableDescriptors implements TableDescriptors {
* directly from the file system if it exists. * directly from the file system if it exists.
* @throws TableInfoMissingException if there is no descriptor * @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 { throws IOException {
FileStatus status = getTableInfoPath(fs, tableDir, false); FileStatus status = getTableInfoPath(fs, tableDir, false);
if (status == null) { if (status == null) {
@ -509,11 +578,11 @@ public class FSTableDescriptors implements TableDescriptors {
if (status == null) { if (status == null) {
return null; return null;
} }
HTableDescriptor htd = readTableDescriptor(fs, status, !fsreadonly); TableDescriptor td = readTableDescriptor(fs, status, !fsreadonly);
return new TableDescriptorAndModtime(status.getModificationTime(), htd); return new TableDescriptorAndModtime(status.getModificationTime(), td);
} }
private static HTableDescriptor readTableDescriptor(FileSystem fs, FileStatus status, private static TableDescriptor readTableDescriptor(FileSystem fs, FileStatus status,
boolean rewritePb) throws IOException { boolean rewritePb) throws IOException {
int len = Ints.checkedCast(status.getLen()); int len = Ints.checkedCast(status.getLen());
byte [] content = new byte[len]; byte [] content = new byte[len];
@ -523,9 +592,9 @@ public class FSTableDescriptors implements TableDescriptors {
} finally { } finally {
fsDataInputStream.close(); fsDataInputStream.close();
} }
HTableDescriptor htd = null; TableDescriptor td = null;
try { try {
htd = HTableDescriptor.parseFrom(content); td = TableDescriptor.parseFrom(content);
} catch (DeserializationException e) { } catch (DeserializationException e) {
throw new IOException("content=" + Bytes.toShort(content), e); throw new IOException("content=" + Bytes.toShort(content), e);
} }
@ -533,25 +602,28 @@ public class FSTableDescriptors implements TableDescriptors {
// Convert the file over to be pb before leaving here. // Convert the file over to be pb before leaving here.
Path tableInfoDir = status.getPath().getParent(); Path tableInfoDir = status.getPath().getParent();
Path tableDir = tableInfoDir.getParent(); Path tableDir = tableInfoDir.getParent();
writeTableDescriptor(fs, htd, tableDir, status); writeTableDescriptor(fs, td, tableDir, status);
} }
return htd; return td;
} }
/** /**
* Update table descriptor on the file system * Update table descriptor on the file system
* @throws IOException Thrown if failed update. * @throws IOException Thrown if failed update.
* @throws NotImplementedException if in read only mode * @throws NotImplementedException if in read only mode
*/ */
@VisibleForTesting Path updateTableDescriptor(HTableDescriptor htd) @VisibleForTesting Path updateTableDescriptor(TableDescriptor td)
throws IOException { throws IOException {
if (fsreadonly) { if (fsreadonly) {
throw new NotImplementedException("Cannot update a table descriptor - in read only mode"); throw new NotImplementedException("Cannot update a table descriptor - in read only mode");
} }
Path tableDir = getTableDir(htd.getTableName()); TableName tableName = td.getHTableDescriptor().getTableName();
Path p = writeTableDescriptor(fs, htd, tableDir, getTableInfoPath(tableDir)); Path tableDir = getTableDir(tableName);
Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(tableDir));
if (p == null) throw new IOException("Failed update"); if (p == null) throw new IOException("Failed update");
LOG.info("Updated tableinfo=" + p); LOG.info("Updated tableinfo=" + p);
long modtime = getTableInfoModtime(tableName);
this.cache.put(tableName, new TableDescriptorAndModtime(modtime, td));
return p; return p;
} }
@ -601,7 +673,7 @@ public class FSTableDescriptors implements TableDescriptors {
* @return Descriptor file or null if we failed write. * @return Descriptor file or null if we failed write.
*/ */
private static Path writeTableDescriptor(final FileSystem fs, private static Path writeTableDescriptor(final FileSystem fs,
final HTableDescriptor htd, final Path tableDir, final TableDescriptor htd, final Path tableDir,
final FileStatus currentDescriptorFile) final FileStatus currentDescriptorFile)
throws IOException { throws IOException {
// Get temporary dir into which we'll first write a file to avoid half-written file phenomenon. // Get temporary dir into which we'll first write a file to avoid half-written file phenomenon.
@ -632,7 +704,7 @@ public class FSTableDescriptors implements TableDescriptors {
} }
tableInfoDirPath = new Path(tableInfoDir, filename); tableInfoDirPath = new Path(tableInfoDir, filename);
try { try {
writeHTD(fs, tempPath, htd); writeTD(fs, tempPath, htd);
fs.mkdirs(tableInfoDirPath.getParent()); fs.mkdirs(tableInfoDirPath.getParent());
if (!fs.rename(tempPath, tableInfoDirPath)) { if (!fs.rename(tempPath, tableInfoDirPath)) {
throw new IOException("Failed rename of " + tempPath + " to " + tableInfoDirPath); throw new IOException("Failed rename of " + tempPath + " to " + tableInfoDirPath);
@ -656,7 +728,7 @@ public class FSTableDescriptors implements TableDescriptors {
return tableInfoDirPath; return tableInfoDirPath;
} }
private static void writeHTD(final FileSystem fs, final Path p, final HTableDescriptor htd) private static void writeTD(final FileSystem fs, final Path p, final TableDescriptor htd)
throws IOException { throws IOException {
FSDataOutputStream out = fs.create(p, false); FSDataOutputStream out = fs.create(p, false);
try { try {
@ -673,10 +745,19 @@ public class FSTableDescriptors implements TableDescriptors {
* Used by tests. * Used by tests.
* @return True if we successfully created file. * @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); return createTableDescriptor(htd, false);
} }
/**
* Create new HTableDescriptor 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 {
return createTableDescriptor(new TableDescriptor(htd), false);
}
/** /**
* Create new HTableDescriptor in HDFS. Happens when we are creating table. If * Create new HTableDescriptor in HDFS. Happens when we are creating table. If
* forceCreation is true then even if previous table descriptor is present it * forceCreation is true then even if previous table descriptor is present it
@ -684,12 +765,21 @@ public class FSTableDescriptors implements TableDescriptors {
* *
* @return True if we successfully created file. * @return True if we successfully created file.
*/ */
public boolean createTableDescriptor(HTableDescriptor htd, boolean forceCreation) public boolean createTableDescriptor(TableDescriptor htd, boolean forceCreation)
throws IOException { throws IOException {
Path tableDir = getTableDir(htd.getTableName()); Path tableDir = getTableDir(htd.getHTableDescriptor().getTableName());
return createTableDescriptorForTableDirectory(tableDir, htd, forceCreation); return createTableDescriptorForTableDirectory(tableDir, htd, forceCreation);
} }
/**
* Create tables descriptor for given HTableDescriptor. Default TableDescriptor state
* will be used (typically ENABLED).
*/
public boolean createTableDescriptor(HTableDescriptor htd, boolean forceCreation)
throws IOException {
return createTableDescriptor(new TableDescriptor(htd), forceCreation);
}
/** /**
* Create a new HTableDescriptor in HDFS in the specified table directory. Happens when we create * Create a new HTableDescriptor in HDFS in the specified table directory. Happens when we create
* a new table or snapshot a table. * a new table or snapshot a table.
@ -702,7 +792,7 @@ public class FSTableDescriptors implements TableDescriptors {
* @throws IOException if a filesystem error occurs * @throws IOException if a filesystem error occurs
*/ */
public boolean createTableDescriptorForTableDirectory(Path tableDir, public boolean createTableDescriptorForTableDirectory(Path tableDir,
HTableDescriptor htd, boolean forceCreation) throws IOException { TableDescriptor htd, boolean forceCreation) throws IOException {
if (fsreadonly) { if (fsreadonly) {
throw new NotImplementedException("Cannot create a table descriptor - in read only mode"); throw new NotImplementedException("Cannot create a table descriptor - in read only mode");
} }

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.util;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.io.StringWriter; import java.io.StringWriter;
import java.net.URI; import java.net.URI;
@ -70,6 +69,7 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
@ -107,7 +108,6 @@ import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler;
import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl; import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl;
import org.apache.hadoop.hbase.util.hbck.TableLockChecker; import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
@ -953,9 +953,9 @@ public class HBaseFsck extends Configured {
modTInfo = new TableInfo(tableName); modTInfo = new TableInfo(tableName);
tablesInfo.put(tableName, modTInfo); tablesInfo.put(tableName, modTInfo);
try { try {
HTableDescriptor htd = TableDescriptor htd =
FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName); FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName);
modTInfo.htds.add(htd); modTInfo.htds.add(htd.getHTableDescriptor());
} catch (IOException ioe) { } catch (IOException ioe) {
if (!orphanTableDirs.containsKey(tableName)) { if (!orphanTableDirs.containsKey(tableName)) {
LOG.warn("Unable to read .tableinfo from " + hbaseRoot, ioe); LOG.warn("Unable to read .tableinfo from " + hbaseRoot, ioe);
@ -1009,7 +1009,7 @@ public class HBaseFsck extends Configured {
for (String columnfamimly : columns) { for (String columnfamimly : columns) {
htd.addFamily(new HColumnDescriptor(columnfamimly)); htd.addFamily(new HColumnDescriptor(columnfamimly));
} }
fstd.createTableDescriptor(htd, true); fstd.createTableDescriptor(new TableDescriptor(htd, TableState.State.ENABLED), true);
return true; return true;
} }
@ -1057,7 +1057,7 @@ public class HBaseFsck extends Configured {
if (tableName.equals(htds[j].getTableName())) { if (tableName.equals(htds[j].getTableName())) {
HTableDescriptor htd = htds[j]; HTableDescriptor htd = htds[j];
LOG.info("fixing orphan table: " + tableName + " from cache"); LOG.info("fixing orphan table: " + tableName + " from cache");
fstd.createTableDescriptor(htd, true); fstd.createTableDescriptor(new TableDescriptor(htd, TableState.State.ENABLED), true);
j++; j++;
iter.remove(); iter.remove();
} }
@ -1382,22 +1382,16 @@ public class HBaseFsck extends Configured {
* @throws IOException * @throws IOException
*/ */
private void loadDisabledTables() private void loadDisabledTables()
throws ZooKeeperConnectionException, IOException { throws IOException {
HConnectionManager.execute(new HConnectable<Void>(getConf()) { HConnectionManager.execute(new HConnectable<Void>(getConf()) {
@Override @Override
public Void connect(HConnection connection) throws IOException { public Void connect(HConnection connection) throws IOException {
ZooKeeperWatcher zkw = createZooKeeperWatcher(); TableName[] tables = connection.listTableNames();
try { for (TableName table : tables) {
for (TableName tableName : if (connection.getTableState(table)
ZKTableStateClientSideReader.getDisabledOrDisablingTables(zkw)) { .inStates(TableState.State.DISABLED, TableState.State.DISABLING)) {
disabledTables.add(tableName); disabledTables.add(table);
} }
} catch (KeeperException ke) {
throw new IOException(ke);
} catch (InterruptedException e) {
throw new InterruptedIOException();
} finally {
zkw.close();
} }
return null; return null;
} }

View File

@ -156,7 +156,8 @@ class HMerge {
this.rootDir = FSUtils.getRootDir(conf); this.rootDir = FSUtils.getRootDir(conf);
Path tabledir = FSUtils.getTableDir(this.rootDir, tableName); Path tabledir = FSUtils.getTableDir(this.rootDir, tableName);
this.htd = FSTableDescriptors.getTableDescriptorFromFs(this.fs, tabledir); this.htd = FSTableDescriptors.getTableDescriptorFromFs(this.fs, tabledir)
.getHTableDescriptor();
String logname = "merge_" + System.currentTimeMillis() + HConstants.HREGION_LOGDIR_NAME; String logname = "merge_" + System.currentTimeMillis() + HConstants.HREGION_LOGDIR_NAME;
this.hlog = HLogFactory.createHLog(fs, tabledir, logname, conf); this.hlog = HLogFactory.createHLog(fs, tabledir, logname, conf);

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured; import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -153,9 +154,9 @@ public class Merge extends Configured implements Tool {
if (info2 == null) { if (info2 == null) {
throw new NullPointerException("info2 is null using key " + meta); throw new NullPointerException("info2 is null using key " + meta);
} }
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(FileSystem.get(getConf()), TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(FileSystem.get(getConf()),
this.rootdir, this.tableName); this.rootdir, this.tableName);
HRegion merged = merge(htd, meta, info1, info2); HRegion merged = merge(htd.getHTableDescriptor(), meta, info1, info2);
LOG.info("Adding " + merged.getRegionInfo() + " to " + LOG.info("Adding " + merged.getRegionInfo() + " to " +
meta.getRegionInfo()); meta.getRegionInfo());

View File

@ -18,8 +18,11 @@
package org.apache.hadoop.hbase.util; package org.apache.hadoop.hbase.util;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -27,8 +30,12 @@ import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ReplicationPeer; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ReplicationPeer;
import org.apache.hadoop.hbase.replication.ReplicationStateZKBase; import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
@ -151,8 +158,9 @@ public class ZKDataMigrator extends Configured implements Tool {
} }
byte[] data = ZKUtil.getData(zkw, znode); byte[] data = ZKUtil.getData(zkw, znode);
if (ProtobufUtil.isPBMagicPrefix(data)) continue; if (ProtobufUtil.isPBMagicPrefix(data)) continue;
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder(); ZooKeeperProtos.DeprecatedTableState.Builder builder =
builder.setState(ZooKeeperProtos.Table.State.valueOf(Bytes.toString(data))); ZooKeeperProtos.DeprecatedTableState.newBuilder();
builder.setState(ZooKeeperProtos.DeprecatedTableState.State.valueOf(Bytes.toString(data)));
data = ProtobufUtil.prependPBMagic(builder.build().toByteArray()); data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
ZKUtil.setData(zkw, znode, data); ZKUtil.setData(zkw, znode, data);
} }
@ -247,6 +255,77 @@ public class ZKDataMigrator extends Configured implements Tool {
} }
} }
/**
* Method for table states migration.
* Reading state from zk, applying them to internal state
* and delete.
* Used by master to clean migration from zk based states to
* table descriptor based states.
*/
@Deprecated
public static Map<TableName, TableState.State> queryForTableStates(ZooKeeperWatcher zkw)
throws KeeperException, InterruptedException {
Map<TableName, TableState.State> rv = new HashMap<>();
List<String> children = ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
if (children == null)
return rv;
for (String child: children) {
TableName tableName = TableName.valueOf(child);
ZooKeeperProtos.DeprecatedTableState.State state = getTableState(zkw, tableName);
TableState.State newState = TableState.State.ENABLED;
if (state != null) {
switch (state) {
case ENABLED:
newState = TableState.State.ENABLED;
break;
case DISABLED:
newState = TableState.State.DISABLED;
break;
case DISABLING:
newState = TableState.State.DISABLING;
break;
case ENABLING:
newState = TableState.State.ENABLING;
break;
default:
}
}
rv.put(tableName, newState);
}
return rv;
}
/**
* Gets table state from ZK.
* @param zkw ZooKeeperWatcher instance to use
* @param tableName table we're checking
* @return Null or {@link ZooKeeperProtos.DeprecatedTableState.State} found in znode.
* @throws KeeperException
*/
@Deprecated
private static ZooKeeperProtos.DeprecatedTableState.State getTableState(
final ZooKeeperWatcher zkw, final TableName tableName)
throws KeeperException, InterruptedException {
String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
byte [] data = ZKUtil.getData(zkw, znode);
if (data == null || data.length <= 0) return null;
try {
ProtobufUtil.expectPBMagicPrefix(data);
ZooKeeperProtos.DeprecatedTableState.Builder builder =
ZooKeeperProtos.DeprecatedTableState.newBuilder();
int magicLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.DeprecatedTableState t = builder.mergeFrom(data,
magicLen, data.length - magicLen).build();
return t.getState();
} catch (InvalidProtocolBufferException e) {
KeeperException ke = new KeeperException.DataInconsistencyException();
ke.initCause(e);
throw ke;
} catch (DeserializationException e) {
throw ZKUtil.convert(e);
}
}
public static void main(String args[]) throws Exception { public static void main(String args[]) throws Exception {
System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZKDataMigrator(), args)); System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZKDataMigrator(), args));
} }

View File

@ -1,330 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.zookeeper;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.zookeeper.KeeperException;
import java.io.InterruptedIOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* Implementation of TableStateManager which reads, caches and sets state
* up in ZooKeeper. If multiple read/write clients, will make for confusion.
* Code running on client side without consensus context should use
* {@link ZKTableStateClientSideReader} instead.
*
* <p>To save on trips to the zookeeper ensemble, internally we cache table
* state.
*/
@InterfaceAudience.Private
public class ZKTableStateManager implements TableStateManager {
// A znode will exist under the table directory if it is in any of the
// following states: {@link TableState#ENABLING} , {@link TableState#DISABLING},
// or {@link TableState#DISABLED}. If {@link TableState#ENABLED}, there will
// be no entry for a table in zk. Thats how it currently works.
private static final Log LOG = LogFactory.getLog(ZKTableStateManager.class);
private final ZooKeeperWatcher watcher;
/**
* Cache of what we found in zookeeper so we don't have to go to zk ensemble
* for every query. Synchronize access rather than use concurrent Map because
* synchronization needs to span query of zk.
*/
private final Map<TableName, ZooKeeperProtos.Table.State> cache =
new HashMap<TableName, ZooKeeperProtos.Table.State>();
public ZKTableStateManager(final ZooKeeperWatcher zkw) throws KeeperException,
InterruptedException {
super();
this.watcher = zkw;
populateTableStates();
}
/**
* Gets a list of all the tables set as disabled in zookeeper.
* @throws KeeperException, InterruptedException
*/
private void populateTableStates() throws KeeperException, InterruptedException {
synchronized (this.cache) {
List<String> children = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode);
if (children == null) return;
for (String child: children) {
TableName tableName = TableName.valueOf(child);
ZooKeeperProtos.Table.State state = getTableState(this.watcher, tableName);
if (state != null) this.cache.put(tableName, state);
}
}
}
/**
* Sets table state in ZK. Sets no watches.
*
* {@inheritDoc}
*/
@Override
public void setTableState(TableName tableName, ZooKeeperProtos.Table.State state)
throws CoordinatedStateException {
synchronized (this.cache) {
LOG.warn("Moving table " + tableName + " state from " + this.cache.get(tableName)
+ " to " + state);
try {
setTableStateInZK(tableName, state);
} catch (KeeperException e) {
throw new CoordinatedStateException(e);
}
}
}
/**
* Checks and sets table state in ZK. Sets no watches.
* {@inheritDoc}
*/
@Override
public boolean setTableStateIfInStates(TableName tableName,
ZooKeeperProtos.Table.State newState,
ZooKeeperProtos.Table.State... states)
throws CoordinatedStateException {
synchronized (this.cache) {
// Transition ENABLED->DISABLING has to be performed with a hack, because
// we treat empty state as enabled in this case because 0.92- clusters.
if (
(newState == ZooKeeperProtos.Table.State.DISABLING) &&
this.cache.get(tableName) != null && !isTableState(tableName, states) ||
(newState != ZooKeeperProtos.Table.State.DISABLING &&
!isTableState(tableName, states) )) {
return false;
}
try {
setTableStateInZK(tableName, newState);
} catch (KeeperException e) {
throw new CoordinatedStateException(e);
}
return true;
}
}
/**
* Checks and sets table state in ZK. Sets no watches.
* {@inheritDoc}
*/
@Override
public boolean setTableStateIfNotInStates(TableName tableName,
ZooKeeperProtos.Table.State newState,
ZooKeeperProtos.Table.State... states)
throws CoordinatedStateException {
synchronized (this.cache) {
if (isTableState(tableName, states)) {
return false;
}
try {
setTableStateInZK(tableName, newState);
} catch (KeeperException e) {
throw new CoordinatedStateException(e);
}
return true;
}
}
private void setTableStateInZK(final TableName tableName,
final ZooKeeperProtos.Table.State state)
throws KeeperException {
String znode = ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString());
if (ZKUtil.checkExists(this.watcher, znode) == -1) {
ZKUtil.createAndFailSilent(this.watcher, znode);
}
synchronized (this.cache) {
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
builder.setState(state);
byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
ZKUtil.setData(this.watcher, znode, data);
this.cache.put(tableName, state);
}
}
/**
* Checks if table is marked in specified state in ZK.
*
* {@inheritDoc}
*/
@Override
public boolean isTableState(final TableName tableName,
final ZooKeeperProtos.Table.State... states) {
synchronized (this.cache) {
ZooKeeperProtos.Table.State currentState = this.cache.get(tableName);
return isTableInState(Arrays.asList(states), currentState);
}
}
/**
* Deletes the table in zookeeper. Fails silently if the
* table is not currently disabled in zookeeper. Sets no watches.
*
* {@inheritDoc}
*/
@Override
public void setDeletedTable(final TableName tableName)
throws CoordinatedStateException {
synchronized (this.cache) {
if (this.cache.remove(tableName) == null) {
LOG.warn("Moving table " + tableName + " state to deleted but was " +
"already deleted");
}
try {
ZKUtil.deleteNodeFailSilent(this.watcher,
ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
} catch (KeeperException e) {
throw new CoordinatedStateException(e);
}
}
}
/**
* check if table is present.
*
* @param tableName table we're working on
* @return true if the table is present
*/
@Override
public boolean isTablePresent(final TableName tableName) {
synchronized (this.cache) {
ZooKeeperProtos.Table.State state = this.cache.get(tableName);
return !(state == null);
}
}
/**
* Gets a list of all the tables set as disabling in zookeeper.
* @return Set of disabling tables, empty Set if none
* @throws CoordinatedStateException if error happened in underlying coordination engine
*/
@Override
public Set<TableName> getTablesInStates(ZooKeeperProtos.Table.State... states)
throws InterruptedIOException, CoordinatedStateException {
try {
return getAllTables(states);
} catch (KeeperException e) {
throw new CoordinatedStateException(e);
}
}
/**
* {@inheritDoc}
*/
@Override
public void checkAndRemoveTableState(TableName tableName, ZooKeeperProtos.Table.State states,
boolean deletePermanentState)
throws CoordinatedStateException {
synchronized (this.cache) {
if (isTableState(tableName, states)) {
this.cache.remove(tableName);
if (deletePermanentState) {
try {
ZKUtil.deleteNodeFailSilent(this.watcher,
ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
} catch (KeeperException e) {
throw new CoordinatedStateException(e);
}
}
}
}
}
/**
* Gets a list of all the tables of specified states in zookeeper.
* @return Set of tables of specified states, empty Set if none
* @throws KeeperException
*/
Set<TableName> getAllTables(final ZooKeeperProtos.Table.State... states)
throws KeeperException, InterruptedIOException {
Set<TableName> allTables = new HashSet<TableName>();
List<String> children =
ZKUtil.listChildrenNoWatch(watcher, watcher.tableZNode);
if(children == null) return allTables;
for (String child: children) {
TableName tableName = TableName.valueOf(child);
ZooKeeperProtos.Table.State state;
try {
state = getTableState(watcher, tableName);
} catch (InterruptedException e) {
throw new InterruptedIOException();
}
for (ZooKeeperProtos.Table.State expectedState: states) {
if (state == expectedState) {
allTables.add(tableName);
break;
}
}
}
return allTables;
}
/**
* Gets table state from ZK.
* @param zkw ZooKeeperWatcher instance to use
* @param tableName table we're checking
* @return Null or {@link ZooKeeperProtos.Table.State} found in znode.
* @throws KeeperException
*/
private ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,
final TableName tableName)
throws KeeperException, InterruptedException {
String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
byte [] data = ZKUtil.getData(zkw, znode);
if (data == null || data.length <= 0) return null;
try {
ProtobufUtil.expectPBMagicPrefix(data);
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
int magicLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build();
return t.getState();
} catch (InvalidProtocolBufferException e) {
KeeperException ke = new KeeperException.DataInconsistencyException();
ke.initCause(e);
throw ke;
} catch (DeserializationException e) {
throw ZKUtil.convert(e);
}
}
/**
* @return true if current state isn't null and is contained
* in the list of expected states.
*/
private boolean isTableInState(final List<ZooKeeperProtos.Table.State> expectedStates,
final ZooKeeperProtos.Table.State currentState) {
return currentState != null && expectedStates.contains(currentState);
}
}

View File

@ -2817,6 +2817,48 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
} }
} }
/**
* Waits for a table to be 'disabled'. Disabled means that table is set as 'disabled'
* Will timeout after default period (30 seconds)
* @param table Table to wait on.
* @throws InterruptedException
* @throws IOException
*/
public void waitTableDisabled(byte[] table)
throws InterruptedException, IOException {
waitTableDisabled(getHBaseAdmin(), table, 30000);
}
public void waitTableDisabled(Admin admin, byte[] table)
throws InterruptedException, IOException {
waitTableDisabled(admin, table, 30000);
}
/**
* Waits for a table to be 'disabled'. Disabled means that table is set as 'disabled'
* @see #waitTableAvailable(byte[])
* @param table Table to wait on.
* @param timeoutMillis Time to wait on it being marked disabled.
* @throws InterruptedException
* @throws IOException
*/
public void waitTableDisabled(byte[] table, long timeoutMillis)
throws InterruptedException, IOException {
waitTableDisabled(getHBaseAdmin(), table, timeoutMillis);
}
public void waitTableDisabled(Admin admin, byte[] table, long timeoutMillis)
throws InterruptedException, IOException {
TableName tableName = TableName.valueOf(table);
long startWait = System.currentTimeMillis();
while (!admin.isTableDisabled(tableName)) {
assertTrue("Timed out waiting for table to become disabled " +
Bytes.toStringBinary(table),
System.currentTimeMillis() - startWait < timeoutMillis);
Thread.sleep(200);
}
}
/** /**
* Make sure that at least the specified number of region servers * Make sure that at least the specified number of region servers
* are running * are running

View File

@ -148,8 +148,8 @@ public class TestHColumnDescriptorDefaultVersions {
// Verify descriptor from HDFS // Verify descriptor from HDFS
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem(); MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName); Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
htd = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir); TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
hcds = htd.getColumnFamilies(); hcds = td.getHTableDescriptor().getColumnFamilies();
verifyHColumnDescriptor(expected, hcds, tableName, families); verifyHColumnDescriptor(expected, hcds, tableName, families);
} }

View File

@ -0,0 +1,57 @@
/**
* Copyright The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
/**
* Test setting values in the descriptor
*/
@Category(SmallTests.class)
public class TestTableDescriptor {
final static Log LOG = LogFactory.getLog(TestTableDescriptor.class);
@Test
public void testPb() throws DeserializationException, IOException {
HTableDescriptor htd = new HTableDescriptor(HTableDescriptor.META_TABLEDESC);
final int v = 123;
htd.setMaxFileSize(v);
htd.setDurability(Durability.ASYNC_WAL);
htd.setReadOnly(true);
htd.setRegionReplication(2);
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
byte[] bytes = td.toByteArray();
TableDescriptor deserializedTd = TableDescriptor.parseFrom(bytes);
assertEquals(td, deserializedTd);
assertEquals(td.getHTableDescriptor(), deserializedTd.getHTableDescriptor());
assertEquals(td.getTableState(), deserializedTd.getTableState());
}
}

View File

@ -58,7 +58,6 @@ import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtilsForTests; import org.apache.hadoop.hbase.regionserver.wal.HLogUtilsForTests;
@ -66,7 +65,6 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -258,7 +256,7 @@ public class TestAdmin {
this.admin.disableTable(ht.getName()); this.admin.disableTable(ht.getName());
assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster() assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
.getMaster().getAssignmentManager().getTableStateManager().isTableState( .getMaster().getAssignmentManager().getTableStateManager().isTableState(
ht.getName(), ZooKeeperProtos.Table.State.DISABLED)); ht.getName(), TableState.State.DISABLED));
// Test that table is disabled // Test that table is disabled
get = new Get(row); get = new Get(row);
@ -273,7 +271,7 @@ public class TestAdmin {
this.admin.enableTable(table); this.admin.enableTable(table);
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster() assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
.getMaster().getAssignmentManager().getTableStateManager().isTableState( .getMaster().getAssignmentManager().getTableStateManager().isTableState(
ht.getName(), ZooKeeperProtos.Table.State.ENABLED)); ht.getName(), TableState.State.ENABLED));
// Test that table is enabled // Test that table is enabled
try { try {
@ -346,7 +344,7 @@ public class TestAdmin {
assertEquals(numTables + 1, tables.length); assertEquals(numTables + 1, tables.length);
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster() assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
.getMaster().getAssignmentManager().getTableStateManager().isTableState( .getMaster().getAssignmentManager().getTableStateManager().isTableState(
TableName.valueOf("testCreateTable"), ZooKeeperProtos.Table.State.ENABLED)); TableName.valueOf("testCreateTable"), TableState.State.ENABLED));
} }
@Test (timeout=300000) @Test (timeout=300000)
@ -1128,8 +1126,7 @@ public class TestAdmin {
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL); ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
TableName tableName = TableName.valueOf("testMasterAdmin"); TableName tableName = TableName.valueOf("testMasterAdmin");
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close(); TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
while (!ZKTableStateClientSideReader.isEnabledTable(zkw, while (!this.admin.isTableEnabled(TableName.valueOf("testMasterAdmin"))) {
TableName.valueOf("testMasterAdmin"))) {
Thread.sleep(10); Thread.sleep(10);
} }
this.admin.disableTable(tableName); this.admin.disableTable(tableName);

View File

@ -53,15 +53,14 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.coprocessor.RegionObserver;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer; import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -127,7 +126,8 @@ public class TestAssignmentManagerOnCluster {
} }
RegionState metaState = RegionState metaState =
MetaTableLocator.getMetaRegionState(master.getZooKeeper()); MetaTableLocator.getMetaRegionState(master.getZooKeeper());
assertEquals("Meta should be not in transition", metaState.getState(), State.OPEN); assertEquals("Meta should be not in transition",
metaState.getState(), RegionState.State.OPEN);
assertNotEquals("Meta should be moved off master", assertNotEquals("Meta should be moved off master",
metaState.getServerName(), master.getServerName()); metaState.getServerName(), master.getServerName());
assertEquals("Meta should be on the meta server", assertEquals("Meta should be on the meta server",
@ -153,7 +153,8 @@ public class TestAssignmentManagerOnCluster {
regionStates.isRegionOnline(HRegionInfo.FIRST_META_REGIONINFO)); regionStates.isRegionOnline(HRegionInfo.FIRST_META_REGIONINFO));
// Now, make sure meta is registered in zk // Now, make sure meta is registered in zk
metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper()); metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper());
assertEquals("Meta should be not in transition", metaState.getState(), State.OPEN); assertEquals("Meta should be not in transition",
metaState.getState(), RegionState.State.OPEN);
assertEquals("Meta should be assigned", metaState.getServerName(), assertEquals("Meta should be assigned", metaState.getServerName(),
regionStates.getRegionServerOfRegion(HRegionInfo.FIRST_META_REGIONINFO)); regionStates.getRegionServerOfRegion(HRegionInfo.FIRST_META_REGIONINFO));
assertNotEquals("Meta should be assigned on a different server", assertNotEquals("Meta should be assigned on a different server",
@ -209,7 +210,8 @@ public class TestAssignmentManagerOnCluster {
String table = "testAssignRegionOnRestartedServer"; String table = "testAssignRegionOnRestartedServer";
TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 20); TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 20);
TEST_UTIL.getMiniHBaseCluster().stopMaster(0); TEST_UTIL.getMiniHBaseCluster().stopMaster(0);
TEST_UTIL.getMiniHBaseCluster().startMaster(); //restart the master so that conf take into affect //restart the master so that conf take into affect
TEST_UTIL.getMiniHBaseCluster().startMaster();
ServerName deadServer = null; ServerName deadServer = null;
HMaster master = null; HMaster master = null;
@ -619,9 +621,9 @@ public class TestAssignmentManagerOnCluster {
} }
} }
am.regionOffline(hri); am.regionOffline(hri);
am.getRegionStates().updateRegionState(hri, State.PENDING_OPEN, destServerName); am.getRegionStates().updateRegionState(hri, RegionState.State.PENDING_OPEN, destServerName);
am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLING); am.getTableStateManager().setTableState(table, TableState.State.DISABLING);
List<HRegionInfo> toAssignRegions = am.processServerShutdown(destServerName); List<HRegionInfo> toAssignRegions = am.processServerShutdown(destServerName);
assertTrue("Regions to be assigned should be empty.", toAssignRegions.isEmpty()); assertTrue("Regions to be assigned should be empty.", toAssignRegions.isEmpty());
assertTrue("Regions to be assigned should be empty.", am.getRegionStates() assertTrue("Regions to be assigned should be empty.", am.getRegionStates()
@ -630,7 +632,7 @@ public class TestAssignmentManagerOnCluster {
if (hri != null && serverName != null) { if (hri != null && serverName != null) {
am.regionOnline(hri, serverName); am.regionOnline(hri, serverName);
} }
am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLED); am.getTableStateManager().setTableState(table, TableState.State.DISABLED);
TEST_UTIL.deleteTable(table); TEST_UTIL.deleteTable(table);
} }
} }

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
@ -55,6 +56,7 @@ import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator; import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
@ -299,13 +301,18 @@ public class TestCatalogJanitor {
return new TableDescriptors() { return new TableDescriptors() {
@Override @Override
public HTableDescriptor remove(TableName tablename) throws IOException { public HTableDescriptor remove(TableName tablename) throws IOException {
// TODO Auto-generated method stub // noop
return null; return null;
} }
@Override @Override
public Map<String, HTableDescriptor> getAll() throws IOException { public Map<String, HTableDescriptor> getAll() throws IOException {
// TODO Auto-generated method stub // noop
return null;
}
@Override public Map<String, TableDescriptor> getAllDescriptors() throws IOException {
// noop
return null; return null;
} }
@ -315,6 +322,12 @@ public class TestCatalogJanitor {
return createHTableDescriptor(); return createHTableDescriptor();
} }
@Override
public TableDescriptor getDescriptor(TableName tablename)
throws IOException {
return createTableDescriptor();
}
@Override @Override
public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException { public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException {
return null; return null;
@ -322,8 +335,12 @@ public class TestCatalogJanitor {
@Override @Override
public void add(HTableDescriptor htd) throws IOException { public void add(HTableDescriptor htd) throws IOException {
// TODO Auto-generated method stub // noop
}
@Override
public void add(TableDescriptor htd) throws IOException {
// noop
} }
}; };
} }
@ -407,6 +424,11 @@ public class TestCatalogJanitor {
return null; return null;
} }
@Override
public TableStateManager getTableStateManager() {
return null;
}
@Override @Override
public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b, public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
boolean forcible) throws IOException { boolean forcible) throws IOException {
@ -978,6 +1000,11 @@ public class TestCatalogJanitor {
return htd; return htd;
} }
private TableDescriptor createTableDescriptor() {
TableDescriptor htd = new TableDescriptor(createHTableDescriptor(), TableState.State.ENABLED);
return htd;
}
private MultiResponse buildMultiResponse(MultiRequest req) { private MultiResponse buildMultiResponse(MultiRequest req) {
MultiResponse.Builder builder = MultiResponse.newBuilder(); MultiResponse.Builder builder = MultiResponse.newBuilder();
RegionActionResult.Builder regionActionResultBuilder = RegionActionResult.Builder regionActionResultBuilder =

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -82,7 +82,7 @@ public class TestMaster {
HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME); HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME);
assertTrue(m.assignmentManager.getTableStateManager().isTableState(TABLENAME, assertTrue(m.assignmentManager.getTableStateManager().isTableState(TABLENAME,
ZooKeeperProtos.Table.State.ENABLED)); TableState.State.ENABLED));
TEST_UTIL.loadTable(ht, FAMILYNAME, false); TEST_UTIL.loadTable(ht, FAMILYNAME, false);
ht.close(); ht.close();

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -95,8 +95,8 @@ public class TestMasterRestartAfterDisablingTable {
assertTrue("The table should not be in enabled state", cluster.getMaster() assertTrue("The table should not be in enabled state", cluster.getMaster()
.getAssignmentManager().getTableStateManager().isTableState( .getAssignmentManager().getTableStateManager().isTableState(
TableName.valueOf("tableRestart"), ZooKeeperProtos.Table.State.DISABLED, TableName.valueOf("tableRestart"), TableState.State.DISABLED,
ZooKeeperProtos.Table.State.DISABLING)); TableState.State.DISABLING));
log("Enabling table\n"); log("Enabling table\n");
// Need a new Admin, the previous one is on the old master // Need a new Admin, the previous one is on the old master
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
@ -111,7 +111,7 @@ public class TestMasterRestartAfterDisablingTable {
6, regions.size()); 6, regions.size());
assertTrue("The table should be in enabled state", cluster.getMaster() assertTrue("The table should be in enabled state", cluster.getMaster()
.getAssignmentManager().getTableStateManager() .getAssignmentManager().getTableStateManager()
.isTableState(TableName.valueOf("tableRestart"), ZooKeeperProtos.Table.State.ENABLED)); .isTableState(TableName.valueOf("tableRestart"), TableState.State.ENABLED));
ht.close(); ht.close();
TEST_UTIL.shutdownMiniCluster(); TEST_UTIL.shutdownMiniCluster();
} }

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver; import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@ -393,12 +394,14 @@ public class TestTableLockManager {
alterThread.start(); alterThread.start();
splitThread.start(); splitThread.start();
TEST_UTIL.waitTableEnabled(tableName.toBytes());
while (true) { while (true) {
List<HRegionInfo> regions = admin.getTableRegions(tableName); List<HRegionInfo> regions = admin.getTableRegions(tableName);
LOG.info(String.format("Table #regions: %d regions: %s:", regions.size(), regions)); LOG.info(String.format("Table #regions: %d regions: %s:", regions.size(), regions));
assertEquals(admin.getTableDescriptor(tableName), desc); assertEquals(admin.getTableDescriptor(tableName), desc);
for (HRegion region : TEST_UTIL.getMiniHBaseCluster().getRegions(tableName)) { for (HRegion region : TEST_UTIL.getMiniHBaseCluster().getRegions(tableName)) {
assertEquals(desc, region.getTableDesc()); HTableDescriptor regionTableDesc = region.getTableDesc();
assertEquals(desc, regionTableDesc);
} }
if (regions.size() >= 5) { if (regions.size() >= 5) {
break; break;

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
@ -154,8 +155,9 @@ public class TestTableDescriptorModification {
// Verify descriptor from HDFS // Verify descriptor from HDFS
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem(); MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName); Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
htd = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir); TableDescriptor td =
verifyTableDescriptor(htd, tableName, families); FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
verifyTableDescriptor(td.getHTableDescriptor(), tableName, families);
} }
private void verifyTableDescriptor(final HTableDescriptor htd, private void verifyTableDescriptor(final HTableDescriptor htd,

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
@ -479,7 +480,8 @@ public class SnapshotTestingUtils {
this.tableRegions = tableRegions; this.tableRegions = tableRegions;
this.snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir); this.snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir);
new FSTableDescriptors(conf) new FSTableDescriptors(conf)
.createTableDescriptorForTableDirectory(snapshotDir, htd, false); .createTableDescriptorForTableDirectory(snapshotDir,
new TableDescriptor(htd), false);
} }
public HTableDescriptor getTableDescriptor() { public HTableDescriptor getTableDescriptor() {
@ -574,7 +576,8 @@ public class SnapshotTestingUtils {
private RegionData[] createTable(final HTableDescriptor htd, final int nregions) private RegionData[] createTable(final HTableDescriptor htd, final int nregions)
throws IOException { throws IOException {
Path tableDir = FSUtils.getTableDir(rootDir, htd.getTableName()); Path tableDir = FSUtils.getTableDir(rootDir, htd.getTableName());
new FSTableDescriptors(conf).createTableDescriptorForTableDirectory(tableDir, htd, false); new FSTableDescriptors(conf).createTableDescriptorForTableDirectory(tableDir,
new TableDescriptor(htd), false);
assertTrue(nregions % 2 == 0); assertTrue(nregions % 2 == 0);
RegionData[] regions = new RegionData[nregions]; RegionData[] regions = new RegionData[nregions];

View File

@ -28,12 +28,13 @@ import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Comparator; import java.util.Comparator;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
@ -69,14 +70,15 @@ public class TestFSTableDescriptors {
public void testCreateAndUpdate() throws IOException { public void testCreateAndUpdate() throws IOException {
Path testdir = UTIL.getDataTestDir("testCreateAndUpdate"); Path testdir = UTIL.getDataTestDir("testCreateAndUpdate");
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCreate")); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCreate"));
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FileSystem fs = FileSystem.get(UTIL.getConfiguration());
FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir); FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
assertTrue(fstd.createTableDescriptor(htd)); assertTrue(fstd.createTableDescriptor(td));
assertFalse(fstd.createTableDescriptor(htd)); assertFalse(fstd.createTableDescriptor(td));
FileStatus [] statuses = fs.listStatus(testdir); FileStatus [] statuses = fs.listStatus(testdir);
assertTrue("statuses.length="+statuses.length, statuses.length == 1); assertTrue("statuses.length="+statuses.length, statuses.length == 1);
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
fstd.updateTableDescriptor(htd); fstd.updateTableDescriptor(td);
} }
statuses = fs.listStatus(testdir); statuses = fs.listStatus(testdir);
assertTrue(statuses.length == 1); assertTrue(statuses.length == 1);
@ -90,20 +92,29 @@ public class TestFSTableDescriptors {
Path testdir = UTIL.getDataTestDir("testSequenceidAdvancesOnTableInfo"); Path testdir = UTIL.getDataTestDir("testSequenceidAdvancesOnTableInfo");
HTableDescriptor htd = new HTableDescriptor( HTableDescriptor htd = new HTableDescriptor(
TableName.valueOf("testSequenceidAdvancesOnTableInfo")); TableName.valueOf("testSequenceidAdvancesOnTableInfo"));
TableDescriptor td = new TableDescriptor(htd);
FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FileSystem fs = FileSystem.get(UTIL.getConfiguration());
FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir); FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
Path p0 = fstd.updateTableDescriptor(htd); Path p0 = fstd.updateTableDescriptor(td);
int i0 = FSTableDescriptors.getTableInfoSequenceId(p0); int i0 = FSTableDescriptors.getTableInfoSequenceId(p0);
Path p1 = fstd.updateTableDescriptor(htd); Path p1 = fstd.updateTableDescriptor(td);
// Assert we cleaned up the old file. // Assert we cleaned up the old file.
assertTrue(!fs.exists(p0)); assertTrue(!fs.exists(p0));
int i1 = FSTableDescriptors.getTableInfoSequenceId(p1); int i1 = FSTableDescriptors.getTableInfoSequenceId(p1);
assertTrue(i1 == i0 + 1); assertTrue(i1 == i0 + 1);
Path p2 = fstd.updateTableDescriptor(htd); Path p2 = fstd.updateTableDescriptor(td);
// Assert we cleaned up the old file. // Assert we cleaned up the old file.
assertTrue(!fs.exists(p1)); assertTrue(!fs.exists(p1));
int i2 = FSTableDescriptors.getTableInfoSequenceId(p2); int i2 = FSTableDescriptors.getTableInfoSequenceId(p2);
assertTrue(i2 == i1 + 1); assertTrue(i2 == i1 + 1);
td = new TableDescriptor(htd, TableState.State.DISABLED);
Path p3 = fstd.updateTableDescriptor(td);
// Assert we cleaned up the old file.
assertTrue(!fs.exists(p2));
int i3 = FSTableDescriptors.getTableInfoSequenceId(p3);
assertTrue(i3 == i2 + 1);
TableDescriptor descriptor = fstd.getDescriptor(htd.getTableName());
assertEquals(descriptor, td);
} }
@Test @Test
@ -155,12 +166,13 @@ public class TestFSTableDescriptors {
final String name = "testReadingHTDFromFS"; final String name = "testReadingHTDFromFS";
FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FileSystem fs = FileSystem.get(UTIL.getConfiguration());
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
Path rootdir = UTIL.getDataTestDir(name); Path rootdir = UTIL.getDataTestDir(name);
FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir); FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir);
fstd.createTableDescriptor(htd); fstd.createTableDescriptor(td);
HTableDescriptor htd2 = TableDescriptor td2 =
FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName()); FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName());
assertTrue(htd.equals(htd2)); assertTrue(td.equals(td2));
} }
@Test public void testHTableDescriptors() @Test public void testHTableDescriptors()
@ -180,7 +192,8 @@ public class TestFSTableDescriptors {
final int count = 10; final int count = 10;
// Write out table infos. // Write out table infos.
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
HTableDescriptor htd = new HTableDescriptor(name + i); TableDescriptor htd = new TableDescriptor(new HTableDescriptor(name + i),
TableState.State.ENABLED);
htds.createTableDescriptor(htd); htds.createTableDescriptor(htd);
} }
@ -194,7 +207,7 @@ public class TestFSTableDescriptors {
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
htd.addFamily(new HColumnDescriptor("" + i)); htd.addFamily(new HColumnDescriptor("" + i));
htds.updateTableDescriptor(htd); htds.updateTableDescriptor(new TableDescriptor(htd));
} }
// Wait a while so mod time we write is for sure different. // Wait a while so mod time we write is for sure different.
Thread.sleep(100); Thread.sleep(100);
@ -277,18 +290,19 @@ public class TestFSTableDescriptors {
Path testdir = UTIL.getDataTestDir("testCreateTableDescriptorUpdatesIfThereExistsAlready"); Path testdir = UTIL.getDataTestDir("testCreateTableDescriptorUpdatesIfThereExistsAlready");
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf( HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(
"testCreateTableDescriptorUpdatesIfThereExistsAlready")); "testCreateTableDescriptorUpdatesIfThereExistsAlready"));
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FileSystem fs = FileSystem.get(UTIL.getConfiguration());
FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir); FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
assertTrue(fstd.createTableDescriptor(htd)); assertTrue(fstd.createTableDescriptor(td));
assertFalse(fstd.createTableDescriptor(htd)); assertFalse(fstd.createTableDescriptor(td));
htd.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue")); htd.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue"));
assertTrue(fstd.createTableDescriptor(htd)); //this will re-create assertTrue(fstd.createTableDescriptor(td)); //this will re-create
Path tableDir = fstd.getTableDir(htd.getTableName()); Path tableDir = fstd.getTableDir(htd.getTableName());
Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR); Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR);
FileStatus[] statuses = fs.listStatus(tmpTableDir); FileStatus[] statuses = fs.listStatus(tmpTableDir);
assertTrue(statuses.length == 0); assertTrue(statuses.length == 0);
assertEquals(htd, FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir)); assertEquals(td, FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir));
} }
} }

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -146,7 +147,8 @@ public class TestMergeTool extends HBaseTestCase {
try { try {
// Create meta region // Create meta region
createMetaRegion(); createMetaRegion();
new FSTableDescriptors(this.fs, this.testDir).createTableDescriptor(this.desc); new FSTableDescriptors(this.fs, this.testDir).createTableDescriptor(
new TableDescriptor(this.desc));
/* /*
* Create the regions we will merge * Create the regions we will merge
*/ */

View File

@ -1,115 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.zookeeper;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table;
@Category({MiscTests.class, MediumTests.class})
public class TestZKTableStateManager {
private static final Log LOG = LogFactory.getLog(TestZKTableStateManager.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniZKCluster();
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniZKCluster();
}
@Test
public void testTableStates()
throws CoordinatedStateException, IOException, KeeperException, InterruptedException {
final TableName name =
TableName.valueOf("testDisabled");
Abortable abortable = new Abortable() {
@Override
public void abort(String why, Throwable e) {
LOG.info(why, e);
}
@Override
public boolean isAborted() {
return false;
}
};
ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
name.getNameAsString(), abortable, true);
TableStateManager zkt = new ZKTableStateManager(zkw);
assertFalse(zkt.isTableState(name, Table.State.ENABLED));
assertFalse(zkt.isTableState(name, Table.State.DISABLING));
assertFalse(zkt.isTableState(name, Table.State.DISABLED));
assertFalse(zkt.isTableState(name, Table.State.ENABLING));
assertFalse(zkt.isTableState(name, Table.State.DISABLED, Table.State.DISABLING));
assertFalse(zkt.isTableState(name, Table.State.DISABLED, Table.State.ENABLING));
assertFalse(zkt.isTablePresent(name));
zkt.setTableState(name, Table.State.DISABLING);
assertTrue(zkt.isTableState(name, Table.State.DISABLING));
assertTrue(zkt.isTableState(name, Table.State.DISABLED, Table.State.DISABLING));
assertFalse(zkt.getTablesInStates(Table.State.DISABLED).contains(name));
assertTrue(zkt.isTablePresent(name));
zkt.setTableState(name, Table.State.DISABLED);
assertTrue(zkt.isTableState(name, Table.State.DISABLED));
assertTrue(zkt.isTableState(name, Table.State.DISABLED, Table.State.DISABLING));
assertFalse(zkt.isTableState(name, Table.State.DISABLING));
assertTrue(zkt.getTablesInStates(Table.State.DISABLED).contains(name));
assertTrue(zkt.isTablePresent(name));
zkt.setTableState(name, Table.State.ENABLING);
assertTrue(zkt.isTableState(name, Table.State.ENABLING));
assertTrue(zkt.isTableState(name, Table.State.DISABLED, Table.State.ENABLING));
assertFalse(zkt.isTableState(name, Table.State.DISABLED));
assertFalse(zkt.getTablesInStates(Table.State.DISABLED).contains(name));
assertTrue(zkt.isTablePresent(name));
zkt.setTableState(name, Table.State.ENABLED);
assertTrue(zkt.isTableState(name, Table.State.ENABLED));
assertFalse(zkt.isTableState(name, Table.State.ENABLING));
assertTrue(zkt.isTablePresent(name));
zkt.setDeletedTable(name);
assertFalse(zkt.isTableState(name, Table.State.ENABLED));
assertFalse(zkt.isTableState(name, Table.State.DISABLING));
assertFalse(zkt.isTableState(name, Table.State.DISABLED));
assertFalse(zkt.isTableState(name, Table.State.ENABLING));
assertFalse(zkt.isTableState(name, Table.State.DISABLED, Table.State.DISABLING));
assertFalse(zkt.isTableState(name, Table.State.DISABLED, Table.State.ENABLING));
assertFalse(zkt.isTablePresent(name));
}
}