HBASE-7767 Get rid of ZKTable, and table enable/disable state in ZK (Andrey Stepachev)
Signed-off-by: Andrew Purtell <apurtell@apache.org>
(cherry picked from commit 3cc5d19039
)
This commit is contained in:
parent
18200b09e2
commit
cb756629b0
|
@ -184,6 +184,11 @@ abstract class ConnectionAdapter implements ClusterConnection {
|
|||
return wrappedConnection.isTableAvailable(tableName, splitKeys);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableState getTableState(TableName tableName) throws IOException {
|
||||
return wrappedConnection.getTableState(tableName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HTableDescriptor[] listTables() throws IOException {
|
||||
return wrappedConnection.listTables();
|
||||
|
|
|
@ -124,6 +124,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescripto
|
|||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
|
||||
|
@ -1006,7 +1008,7 @@ class ConnectionManager {
|
|||
|
||||
@Override
|
||||
public boolean isTableEnabled(TableName tableName) throws IOException {
|
||||
return this.registry.isTableOnlineState(tableName, true);
|
||||
return getTableState(tableName).inStates(TableState.State.ENABLED);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1016,7 +1018,7 @@ class ConnectionManager {
|
|||
|
||||
@Override
|
||||
public boolean isTableDisabled(TableName tableName) throws IOException {
|
||||
return this.registry.isTableOnlineState(tableName, false);
|
||||
return getTableState(tableName).inStates(TableState.State.DISABLED);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2173,6 +2175,13 @@ class ConnectionManager {
|
|||
return stub.listTableNamesByNamespace(controller, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetTableStateResponse getTableState(
|
||||
RpcController controller, GetTableStateRequest request)
|
||||
throws ServiceException {
|
||||
return stub.getTableState(controller, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
release(this.mss);
|
||||
|
@ -2800,6 +2809,19 @@ class ConnectionManager {
|
|||
public RpcControllerFactory getRpcControllerFactory() {
|
||||
return this.rpcControllerFactory;
|
||||
}
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -212,6 +212,13 @@ public interface HConnection extends Connection {
|
|||
@Deprecated
|
||||
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
|
||||
* @return true if all regions of the table are available, false otherwise
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
||||
/**
|
||||
* Cluster registry.
|
||||
|
@ -46,12 +45,6 @@ interface Registry {
|
|||
*/
|
||||
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
|
||||
* @throws IOException
|
||||
|
|
|
@ -0,0 +1,205 @@
|
|||
/**
|
||||
* 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.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
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;
|
||||
}
|
||||
}
|
|
@ -18,7 +18,6 @@
|
|||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -27,10 +26,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -116,24 +113,6 @@ class ZooKeeperRegistry implements Registry {
|
|||
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
|
||||
public int getCurrentNrHRS() throws IOException {
|
||||
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
|
||||
|
|
|
@ -22,6 +22,8 @@ import java.util.List;
|
|||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -95,6 +97,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusR
|
|||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
|
||||
|
@ -117,7 +120,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEn
|
|||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -1411,6 +1413,18 @@ public final class RequestConverter {
|
|||
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
|
||||
*
|
||||
|
|
|
@ -1,205 +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 org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.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.io.IOException;
|
||||
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, TableNotFoundException {
|
||||
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, TableNotFoundException {
|
||||
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, TableNotFoundException {
|
||||
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, TableNotFoundException {
|
||||
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, TableNotFoundException {
|
||||
return
|
||||
getTablesInStates(
|
||||
zkw,
|
||||
ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as enabling in zookeeper.
|
||||
* @param zkw ZooKeeperWatcher instance to use
|
||||
* @return Set of enabling tables, empty Set if none
|
||||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public static Set<TableName> getEnablingTables(ZooKeeperWatcher zkw)
|
||||
throws KeeperException, InterruptedException, TableNotFoundException {
|
||||
return getTablesInStates(zkw, ZooKeeperProtos.Table.State.ENABLING);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of tables that are set as one of the passing in states in zookeeper.
|
||||
* @param zkw ZooKeeperWatcher instance to use
|
||||
* @param states the list of states that a table could be in
|
||||
* @return Set of tables in one of the states, empty Set if none
|
||||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private static Set<TableName> getTablesInStates(
|
||||
ZooKeeperWatcher zkw,
|
||||
ZooKeeperProtos.Table.State... states)
|
||||
throws KeeperException, InterruptedException, TableNotFoundException {
|
||||
Set<TableName> tableNameSet = new HashSet<TableName>();
|
||||
List<String> children = ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
|
||||
TableName tableName;
|
||||
ZooKeeperProtos.Table.State tableState;
|
||||
for (String child: children) {
|
||||
tableName = TableName.valueOf(child);
|
||||
tableState = getTableState(zkw, tableName);
|
||||
for (ZooKeeperProtos.Table.State state : states) {
|
||||
if (tableState == state) {
|
||||
tableNameSet.add(tableName);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return tableNameSet;
|
||||
}
|
||||
|
||||
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 {@link ZooKeeperProtos.Table.State} found in znode.
|
||||
* @throws KeeperException
|
||||
* @throws TableNotFoundException if tableName doesn't exist
|
||||
*/
|
||||
static ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,
|
||||
final TableName tableName)
|
||||
throws KeeperException, InterruptedException, TableNotFoundException {
|
||||
String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
|
||||
byte [] data = ZKUtil.getData(zkw, znode);
|
||||
if (data == null || data.length <= 0) {
|
||||
throw new TableNotFoundException(tableName);
|
||||
}
|
||||
try {
|
||||
ProtobufUtil.expectPBMagicPrefix(data);
|
||||
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
|
||||
int magicLen = ProtobufUtil.lengthOfPBMagic();
|
||||
ProtobufUtil.mergeFrom(builder, data, magicLen, data.length - magicLen);
|
||||
return builder.getState();
|
||||
} catch (IOException e) {
|
||||
KeeperException ke = new KeeperException.DataInconsistencyException();
|
||||
ke.initCause(e);
|
||||
throw ke;
|
||||
} catch (DeserializationException e) {
|
||||
throw ZKUtil.convert(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -138,6 +138,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
// znode used for region transitioning and assignment
|
||||
public String assignmentZNode;
|
||||
// znode used for table disabling/enabling
|
||||
@Deprecated
|
||||
public String tableZNode;
|
||||
// znode containing the unique cluster ID
|
||||
public String clusterIdZNode;
|
||||
|
|
|
@ -474,11 +474,6 @@ public class TestAsyncProcess {
|
|||
return "testClusterId";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isTableOnlineState(TableName tableName, boolean enabled) throws IOException {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentNrHRS() throws IOException {
|
||||
return 1;
|
||||
|
|
|
@ -129,12 +129,6 @@ public class TestClientNoCluster extends Configured implements Tool {
|
|||
return HConstants.CLUSTER_ID_DEFAULT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isTableOnlineState(TableName tableName, boolean enabled)
|
||||
throws IOException {
|
||||
return enabled;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentNrHRS() throws IOException {
|
||||
return 1;
|
||||
|
|
|
@ -1,52 +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 static org.junit.Assert.fail;
|
||||
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
@Category({SmallTests.class})
|
||||
public class TestZKTableStateClientSideReader {
|
||||
|
||||
@Test
|
||||
public void test() throws Exception {
|
||||
ZooKeeperWatcher zkw = Mockito.mock(ZooKeeperWatcher.class);
|
||||
RecoverableZooKeeper rzk = Mockito.mock(RecoverableZooKeeper.class);
|
||||
Mockito.doReturn(rzk).when(zkw).getRecoverableZooKeeper();
|
||||
Mockito.doReturn(null).when(rzk).getData(Mockito.anyString(),
|
||||
Mockito.any(Watcher.class), Mockito.any(Stat.class));
|
||||
TableName table = TableName.valueOf("table-not-exists");
|
||||
try {
|
||||
ZKTableStateClientSideReader.getTableState(zkw, table);
|
||||
fail("Shouldn't reach here");
|
||||
} catch(TableNotFoundException e) {
|
||||
// Expected Table not found exception
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -4419,12 +4419,12 @@ public final class ZooKeeperProtos {
|
|||
// @@protoc_insertion_point(class_scope:hbase.pb.SplitLogTask)
|
||||
}
|
||||
|
||||
public interface TableOrBuilder
|
||||
public interface DeprecatedTableStateOrBuilder
|
||||
extends com.google.protobuf.MessageOrBuilder {
|
||||
|
||||
// required .hbase.pb.Table.State state = 1 [default = ENABLED];
|
||||
// required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];
|
||||
/**
|
||||
* <code>required .hbase.pb.Table.State state = 1 [default = ENABLED];</code>
|
||||
* <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
|
||||
*
|
||||
* <pre>
|
||||
* This is the table's state. If no znode for a table,
|
||||
|
@ -4434,7 +4434,7 @@ public final class ZooKeeperProtos {
|
|||
*/
|
||||
boolean hasState();
|
||||
/**
|
||||
* <code>required .hbase.pb.Table.State state = 1 [default = ENABLED];</code>
|
||||
* <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
|
||||
*
|
||||
* <pre>
|
||||
* This is the table's state. If no znode for a table,
|
||||
|
@ -4442,32 +4442,33 @@ public final class ZooKeeperProtos {
|
|||
* for more.
|
||||
* </pre>
|
||||
*/
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState();
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State getState();
|
||||
}
|
||||
/**
|
||||
* Protobuf type {@code hbase.pb.Table}
|
||||
* Protobuf type {@code hbase.pb.DeprecatedTableState}
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The znode that holds state of table.
|
||||
* Deprected, table state is stored in table descriptor on HDFS.
|
||||
* </pre>
|
||||
*/
|
||||
public static final class Table extends
|
||||
public static final class DeprecatedTableState extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
implements TableOrBuilder {
|
||||
// Use Table.newBuilder() to construct.
|
||||
private Table(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
|
||||
implements DeprecatedTableStateOrBuilder {
|
||||
// Use DeprecatedTableState.newBuilder() to construct.
|
||||
private DeprecatedTableState(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
|
||||
super(builder);
|
||||
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;
|
||||
public static Table getDefaultInstance() {
|
||||
private static final DeprecatedTableState defaultInstance;
|
||||
public static DeprecatedTableState getDefaultInstance() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public Table getDefaultInstanceForType() {
|
||||
public DeprecatedTableState getDefaultInstanceForType() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
|
@ -4477,7 +4478,7 @@ public final class ZooKeeperProtos {
|
|||
getUnknownFields() {
|
||||
return this.unknownFields;
|
||||
}
|
||||
private Table(
|
||||
private DeprecatedTableState(
|
||||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
|
@ -4502,7 +4503,7 @@ public final class ZooKeeperProtos {
|
|||
}
|
||||
case 8: {
|
||||
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) {
|
||||
unknownFields.mergeVarintField(1, rawValue);
|
||||
} else {
|
||||
|
@ -4525,33 +4526,33 @@ public final class ZooKeeperProtos {
|
|||
}
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Table_descriptor;
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Table_fieldAccessorTable
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable
|
||||
.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 =
|
||||
new com.google.protobuf.AbstractParser<Table>() {
|
||||
public Table parsePartialFrom(
|
||||
public static com.google.protobuf.Parser<DeprecatedTableState> PARSER =
|
||||
new com.google.protobuf.AbstractParser<DeprecatedTableState>() {
|
||||
public DeprecatedTableState parsePartialFrom(
|
||||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return new Table(input, extensionRegistry);
|
||||
return new DeprecatedTableState(input, extensionRegistry);
|
||||
}
|
||||
};
|
||||
|
||||
@java.lang.Override
|
||||
public com.google.protobuf.Parser<Table> getParserForType() {
|
||||
public com.google.protobuf.Parser<DeprecatedTableState> getParserForType() {
|
||||
return PARSER;
|
||||
}
|
||||
|
||||
/**
|
||||
* Protobuf enum {@code hbase.pb.Table.State}
|
||||
* Protobuf enum {@code hbase.pb.DeprecatedTableState.State}
|
||||
*
|
||||
* <pre>
|
||||
* Table's current state
|
||||
|
@ -4629,7 +4630,7 @@ public final class ZooKeeperProtos {
|
|||
}
|
||||
public static final com.google.protobuf.Descriptors.EnumDescriptor
|
||||
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();
|
||||
|
@ -4651,15 +4652,15 @@ public final class ZooKeeperProtos {
|
|||
this.value = value;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(enum_scope:hbase.pb.Table.State)
|
||||
// @@protoc_insertion_point(enum_scope:hbase.pb.DeprecatedTableState.State)
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
// required .hbase.pb.Table.State state = 1 [default = ENABLED];
|
||||
// required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];
|
||||
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 .hbase.pb.Table.State state = 1 [default = ENABLED];</code>
|
||||
* <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
|
||||
*
|
||||
* <pre>
|
||||
* This is the table's state. If no znode for a table,
|
||||
|
@ -4671,7 +4672,7 @@ public final class ZooKeeperProtos {
|
|||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
/**
|
||||
* <code>required .hbase.pb.Table.State state = 1 [default = ENABLED];</code>
|
||||
* <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
|
||||
*
|
||||
* <pre>
|
||||
* This is the table's state. If no znode for a table,
|
||||
|
@ -4679,12 +4680,12 @@ public final class ZooKeeperProtos {
|
|||
* for more.
|
||||
* </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_;
|
||||
}
|
||||
|
||||
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;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -4735,10 +4736,10 @@ public final class ZooKeeperProtos {
|
|||
if (obj == this) {
|
||||
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);
|
||||
}
|
||||
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;
|
||||
result = result && (hasState() == other.hasState());
|
||||
|
@ -4768,53 +4769,53 @@ public final class ZooKeeperProtos {
|
|||
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)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
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.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
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 {
|
||||
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,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
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 {
|
||||
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,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
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 {
|
||||
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,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
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)
|
||||
throws java.io.IOException {
|
||||
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.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
|
@ -4823,7 +4824,7 @@ public final class ZooKeeperProtos {
|
|||
|
||||
public static Builder newBuilder() { return Builder.create(); }
|
||||
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);
|
||||
}
|
||||
public Builder toBuilder() { return newBuilder(this); }
|
||||
|
@ -4835,29 +4836,30 @@ public final class ZooKeeperProtos {
|
|||
return builder;
|
||||
}
|
||||
/**
|
||||
* Protobuf type {@code hbase.pb.Table}
|
||||
* Protobuf type {@code hbase.pb.DeprecatedTableState}
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The znode that holds state of table.
|
||||
* Deprected, table state is stored in table descriptor on HDFS.
|
||||
* </pre>
|
||||
*/
|
||||
public static final class Builder extends
|
||||
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
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Table_descriptor;
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Table_fieldAccessorTable
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable
|
||||
.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() {
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
|
@ -4877,7 +4879,7 @@ public final class ZooKeeperProtos {
|
|||
|
||||
public Builder 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);
|
||||
return this;
|
||||
}
|
||||
|
@ -4888,23 +4890,23 @@ public final class ZooKeeperProtos {
|
|||
|
||||
public com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptorForType() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Table_descriptor;
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_descriptor;
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table getDefaultInstanceForType() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance();
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState getDefaultInstanceForType() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.getDefaultInstance();
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table build() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table result = buildPartial();
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState build() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table buildPartial() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table(this);
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState buildPartial() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState(this);
|
||||
int from_bitField0_ = bitField0_;
|
||||
int to_bitField0_ = 0;
|
||||
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
|
@ -4917,16 +4919,16 @@ public final class ZooKeeperProtos {
|
|||
}
|
||||
|
||||
public Builder mergeFrom(com.google.protobuf.Message other) {
|
||||
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table) {
|
||||
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table)other);
|
||||
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState) {
|
||||
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState)other);
|
||||
} else {
|
||||
super.mergeFrom(other);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table other) {
|
||||
if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance()) return this;
|
||||
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState other) {
|
||||
if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.getDefaultInstance()) return this;
|
||||
if (other.hasState()) {
|
||||
setState(other.getState());
|
||||
}
|
||||
|
@ -4946,11 +4948,11 @@ public final class ZooKeeperProtos {
|
|||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
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 {
|
||||
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
|
||||
} 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;
|
||||
} finally {
|
||||
if (parsedMessage != null) {
|
||||
|
@ -4961,10 +4963,10 @@ public final class ZooKeeperProtos {
|
|||
}
|
||||
private int bitField0_;
|
||||
|
||||
// required .hbase.pb.Table.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;
|
||||
// required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];
|
||||
private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.ENABLED;
|
||||
/**
|
||||
* <code>required .hbase.pb.Table.State state = 1 [default = ENABLED];</code>
|
||||
* <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
|
||||
*
|
||||
* <pre>
|
||||
* This is the table's state. If no znode for a table,
|
||||
|
@ -4976,7 +4978,7 @@ public final class ZooKeeperProtos {
|
|||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
/**
|
||||
* <code>required .hbase.pb.Table.State state = 1 [default = ENABLED];</code>
|
||||
* <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
|
||||
*
|
||||
* <pre>
|
||||
* This is the table's state. If no znode for a table,
|
||||
|
@ -4984,11 +4986,11 @@ public final class ZooKeeperProtos {
|
|||
* for more.
|
||||
* </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_;
|
||||
}
|
||||
/**
|
||||
* <code>required .hbase.pb.Table.State state = 1 [default = ENABLED];</code>
|
||||
* <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
|
||||
*
|
||||
* <pre>
|
||||
* This is the table's state. If no znode for a table,
|
||||
|
@ -4996,7 +4998,7 @@ public final class ZooKeeperProtos {
|
|||
* for more.
|
||||
* </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) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
|
@ -5006,7 +5008,7 @@ public final class ZooKeeperProtos {
|
|||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>required .hbase.pb.Table.State state = 1 [default = ENABLED];</code>
|
||||
* <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
|
||||
*
|
||||
* <pre>
|
||||
* This is the table's state. If no znode for a table,
|
||||
|
@ -5016,20 +5018,20 @@ public final class ZooKeeperProtos {
|
|||
*/
|
||||
public Builder clearState() {
|
||||
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();
|
||||
return this;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:hbase.pb.Table)
|
||||
// @@protoc_insertion_point(builder_scope:hbase.pb.DeprecatedTableState)
|
||||
}
|
||||
|
||||
static {
|
||||
defaultInstance = new Table(true);
|
||||
defaultInstance = new DeprecatedTableState(true);
|
||||
defaultInstance.initFields();
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(class_scope:hbase.pb.Table)
|
||||
// @@protoc_insertion_point(class_scope:hbase.pb.DeprecatedTableState)
|
||||
}
|
||||
|
||||
public interface TableCFOrBuilder
|
||||
|
@ -10934,10 +10936,10 @@ public final class ZooKeeperProtos {
|
|||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_hbase_pb_SplitLogTask_fieldAccessorTable;
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_hbase_pb_Table_descriptor;
|
||||
internal_static_hbase_pb_DeprecatedTableState_descriptor;
|
||||
private static
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_hbase_pb_Table_fieldAccessorTable;
|
||||
internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable;
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_hbase_pb_TableCF_descriptor;
|
||||
private static
|
||||
|
@ -11001,28 +11003,29 @@ public final class ZooKeeperProtos {
|
|||
"\022\016\n\nUNASSIGNED\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\007" +
|
||||
"UNKNOWN\020\000\022\021\n\rLOG_SPLITTING\020\001\022\016\n\nLOG_REPL" +
|
||||
"AY\020\002\"w\n\005Table\022-\n\005state\030\001 \002(\0162\025.hbase.pb.",
|
||||
"Table.State:\007ENABLED\"?\n\005State\022\013\n\007ENABLED" +
|
||||
"\020\000\022\014\n\010DISABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABL" +
|
||||
"ING\020\003\"D\n\007TableCF\022\'\n\ntable_name\030\001 \001(\0132\023.h" +
|
||||
"base.pb.TableName\022\020\n\010families\030\002 \003(\014\"\330\001\n\017" +
|
||||
"ReplicationPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027r" +
|
||||
"eplicationEndpointImpl\030\002 \001(\t\022&\n\004data\030\003 \003" +
|
||||
"(\0132\030.hbase.pb.BytesBytesPair\022/\n\rconfigur" +
|
||||
"ation\030\004 \003(\0132\030.hbase.pb.NameStringPair\022$\n" +
|
||||
"\ttable_cfs\030\005 \003(\0132\021.hbase.pb.TableCF\022\021\n\tb" +
|
||||
"andwidth\030\006 \001(\003\"g\n\020ReplicationState\022/\n\005st",
|
||||
"ate\030\001 \002(\0162 .hbase.pb.ReplicationState.St" +
|
||||
"ate\"\"\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001\"" +
|
||||
"+\n\027ReplicationHLogPosition\022\020\n\010position\030\001" +
|
||||
" \002(\003\"%\n\017ReplicationLock\022\022\n\nlock_owner\030\001 " +
|
||||
"\002(\t\"\252\001\n\tTableLock\022\'\n\ntable_name\030\001 \001(\0132\023." +
|
||||
"hbase.pb.TableName\022(\n\nlock_owner\030\002 \001(\0132\024" +
|
||||
".hbase.pb.ServerName\022\021\n\tthread_id\030\003 \001(\003\022" +
|
||||
"\021\n\tis_shared\030\004 \001(\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013c" +
|
||||
"reate_time\030\006 \001(\003\"\036\n\013SwitchState\022\017\n\007enabl" +
|
||||
"ed\030\001 \001(\010BE\n*org.apache.hadoop.hbase.prot",
|
||||
"obuf.generatedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
|
||||
"AY\020\002\"\225\001\n\024DeprecatedTableState\022<\n\005state\030\001",
|
||||
" \002(\0162$.hbase.pb.DeprecatedTableState.Sta" +
|
||||
"te:\007ENABLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DIS" +
|
||||
"ABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"D\n\007" +
|
||||
"TableCF\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.T" +
|
||||
"ableName\022\020\n\010families\030\002 \003(\014\"\330\001\n\017Replicati" +
|
||||
"onPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027replicatio" +
|
||||
"nEndpointImpl\030\002 \001(\t\022&\n\004data\030\003 \003(\0132\030.hbas" +
|
||||
"e.pb.BytesBytesPair\022/\n\rconfiguration\030\004 \003" +
|
||||
"(\0132\030.hbase.pb.NameStringPair\022$\n\ttable_cf" +
|
||||
"s\030\005 \003(\0132\021.hbase.pb.TableCF\022\021\n\tbandwidth\030",
|
||||
"\006 \001(\003\"g\n\020ReplicationState\022/\n\005state\030\001 \002(\016" +
|
||||
"2 .hbase.pb.ReplicationState.State\"\"\n\005St" +
|
||||
"ate\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001\"+\n\027Replic" +
|
||||
"ationHLogPosition\022\020\n\010position\030\001 \002(\003\"%\n\017R" +
|
||||
"eplicationLock\022\022\n\nlock_owner\030\001 \002(\t\"\252\001\n\tT" +
|
||||
"ableLock\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb." +
|
||||
"TableName\022(\n\nlock_owner\030\002 \001(\0132\024.hbase.pb" +
|
||||
".ServerName\022\021\n\tthread_id\030\003 \001(\003\022\021\n\tis_sha" +
|
||||
"red\030\004 \001(\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013create_tim" +
|
||||
"e\030\006 \001(\003\"\036\n\013SwitchState\022\017\n\007enabled\030\001 \001(\010B",
|
||||
"E\n*org.apache.hadoop.hbase.protobuf.gene" +
|
||||
"ratedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -11059,11 +11062,11 @@ public final class ZooKeeperProtos {
|
|||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_hbase_pb_SplitLogTask_descriptor,
|
||||
new java.lang.String[] { "State", "ServerName", "Mode", });
|
||||
internal_static_hbase_pb_Table_descriptor =
|
||||
internal_static_hbase_pb_DeprecatedTableState_descriptor =
|
||||
getDescriptor().getMessageTypes().get(5);
|
||||
internal_static_hbase_pb_Table_fieldAccessorTable = new
|
||||
internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_hbase_pb_Table_descriptor,
|
||||
internal_static_hbase_pb_DeprecatedTableState_descriptor,
|
||||
new java.lang.String[] { "State", });
|
||||
internal_static_hbase_pb_TableCF_descriptor =
|
||||
getDescriptor().getMessageTypes().get(6);
|
||||
|
|
|
@ -39,6 +39,27 @@ message TableSchema {
|
|||
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
|
||||
* Inspired by the rest ColumSchemaMessage
|
||||
|
|
|
@ -456,6 +456,14 @@ message GetTableNamesResponse {
|
|||
repeated TableName table_names = 1;
|
||||
}
|
||||
|
||||
message GetTableStateRequest {
|
||||
required TableName table_name = 1;
|
||||
}
|
||||
|
||||
message GetTableStateResponse {
|
||||
required TableState table_state = 1;
|
||||
}
|
||||
|
||||
message GetClusterStatusRequest {
|
||||
}
|
||||
|
||||
|
@ -901,7 +909,9 @@ service MasterService {
|
|||
rpc IsSnapshotCleanupEnabled (IsSnapshotCleanupEnabledRequest)
|
||||
returns (IsSnapshotCleanupEnabledResponse);
|
||||
|
||||
|
||||
/** returns table state */
|
||||
rpc GetTableState(GetTableStateRequest)
|
||||
returns(GetTableStateResponse);
|
||||
}
|
||||
|
||||
/** Request and response to get the clusterID for this cluster */
|
||||
|
|
|
@ -105,8 +105,9 @@ message SplitLogTask {
|
|||
|
||||
/**
|
||||
* 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
|
||||
enum State {
|
||||
ENABLED = 0;
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.LoadBalancer;
|
||||
|
@ -54,7 +55,6 @@ import org.apache.hadoop.hbase.master.RegionState;
|
|||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.net.Address;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
|
||||
/**
|
||||
* Service to support Region Server Grouping (HBase-6721)
|
||||
|
@ -269,8 +269,8 @@ public class RSGroupAdminServer implements RSGroupAdmin {
|
|||
}
|
||||
for(TableName table: tables) {
|
||||
if (master.getAssignmentManager().getTableStateManager().isTableState(table,
|
||||
ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED,
|
||||
TableState.State.DISABLING)) {
|
||||
LOG.debug("Skipping move regions because the table" + table + " is disabled.");
|
||||
continue;
|
||||
}
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
|
@ -61,11 +60,13 @@ import org.apache.hadoop.hbase.client.Mutation;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||
import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.ServerListener;
|
||||
import org.apache.hadoop.hbase.master.TableStateManager;
|
||||
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
|
||||
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
|
||||
import org.apache.hadoop.hbase.net.Address;
|
||||
|
@ -74,7 +75,6 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -646,7 +646,7 @@ public class RSGroupInfoManagerImpl implements RSGroupInfoManager, ServerListene
|
|||
if (sn == null) {
|
||||
found.set(false);
|
||||
} else if (tsm.isTableState(RSGROUP_TABLE_NAME,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
TableState.State.ENABLED)) {
|
||||
try {
|
||||
ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn);
|
||||
ClientProtos.GetRequest request =
|
||||
|
@ -670,7 +670,7 @@ public class RSGroupInfoManagerImpl implements RSGroupInfoManager, ServerListene
|
|||
if (sn == null) {
|
||||
nsFound.set(false);
|
||||
} else if (tsm.isTableState(TableName.NAMESPACE_TABLE_NAME,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
TableState.State.ENABLED)) {
|
||||
try {
|
||||
ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn);
|
||||
ClientProtos.GetRequest request =
|
||||
|
|
|
@ -55,12 +55,4 @@ public interface CoordinatedStateManager {
|
|||
* @return instance of Server coordinated state manager runs within
|
||||
*/
|
||||
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;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,161 @@
|
|||
/**
|
||||
* 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.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
/**
|
||||
* 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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -36,6 +36,14 @@ public interface TableDescriptors {
|
|||
HTableDescriptor get(final TableName tableName)
|
||||
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.
|
||||
* @return Map of all descriptors.
|
||||
|
@ -53,6 +61,15 @@ public interface TableDescriptors {
|
|||
Map<String, HTableDescriptor> getAll()
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Get Map of all TableDescriptors. Populates the descriptor cache as a
|
||||
* side effect.
|
||||
* @return Map of all descriptors.
|
||||
* @throws IOException
|
||||
*/
|
||||
Map<String, TableDescriptor> getAllDescriptors()
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Add or update descriptor
|
||||
* @param htd Descriptor to set into TableDescriptors
|
||||
|
@ -61,6 +78,14 @@ public interface TableDescriptors {
|
|||
void add(final HTableDescriptor htd)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Add or update descriptor
|
||||
* @param htd Descriptor to set into TableDescriptors
|
||||
* @throws IOException
|
||||
*/
|
||||
void add(final TableDescriptor htd)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* @param tablename
|
||||
* @return Instance of table descriptor or null if none found.
|
||||
|
|
|
@ -1,121 +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 java.io.InterruptedIOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
|
||||
/**
|
||||
* 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);
|
||||
|
||||
/**
|
||||
* @return true if the table is in any one of the listed states, false otherwise.
|
||||
*/
|
||||
boolean isTableState(TableName tableName, boolean checkSource,
|
||||
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;
|
||||
}
|
|
@ -18,10 +18,8 @@
|
|||
package org.apache.hadoop.hbase.coordination;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
|
||||
/**
|
||||
* Base class for {@link org.apache.hadoop.hbase.CoordinatedStateManager} implementations.
|
||||
|
@ -49,9 +47,6 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public abstract TableStateManager getTableStateManager() throws InterruptedException,
|
||||
CoordinatedStateException;
|
||||
/**
|
||||
* Method to retrieve coordination for split log worker
|
||||
*/
|
||||
|
|
|
@ -20,13 +20,9 @@ package org.apache.hadoop.hbase.coordination;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* ZooKeeper-based implementation of {@link org.apache.hadoop.hbase.CoordinatedStateManager}.
|
||||
|
@ -60,16 +56,6 @@ public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
|
|||
return server;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableStateManager getTableStateManager() throws InterruptedException,
|
||||
CoordinatedStateException {
|
||||
try {
|
||||
return new ZKTableStateManager(server.getZooKeeper());
|
||||
} catch (KeeperException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SplitLogWorkerCoordination getSplitLogWorkerCoordination() {
|
||||
return splitLogWorkerCoordination;
|
||||
|
|
|
@ -23,11 +23,11 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
|
@ -309,7 +309,7 @@ public class ZkOpenRegionCoordination implements OpenRegionCoordination {
|
|||
}
|
||||
if (!openedNodeDeleted) {
|
||||
if (assignmentManager.getTableStateManager().isTableState(regionInfo.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
debugLog(regionInfo, "Opened region "
|
||||
+ regionInfo.getShortNameToLog() + " but "
|
||||
+ "this table is disabled, triggering close of region");
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -64,7 +63,6 @@ import org.apache.hadoop.hbase.RegionTransition;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Admin.MasterSwitchType;
|
||||
|
@ -77,6 +75,7 @@ import org.apache.hadoop.hbase.coordination.SplitTransactionCoordination.SplitTr
|
|||
import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
|
||||
import org.apache.hadoop.hbase.coordination.ZkRegionMergeCoordination;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
|
@ -92,12 +91,12 @@ import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
|
|||
import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
|
||||
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.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
||||
import org.apache.hadoop.hbase.util.ConfigUtil;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.KeyLocker;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -286,14 +285,11 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @param service Executor service
|
||||
* @param metricsMaster metrics manager
|
||||
* @param tableLockManager TableLock manager
|
||||
* @throws KeeperException
|
||||
* @throws IOException
|
||||
*/
|
||||
public AssignmentManager(MasterServices server, ServerManager serverManager,
|
||||
final LoadBalancer balancer,
|
||||
final ExecutorService service, MetricsMaster metricsMaster,
|
||||
final TableLockManager tableLockManager) throws KeeperException,
|
||||
IOException, CoordinatedStateException {
|
||||
final TableLockManager tableLockManager, final TableStateManager tableStateManager) {
|
||||
super(server.getZooKeeper());
|
||||
this.server = server;
|
||||
this.serverManager = serverManager;
|
||||
|
@ -306,15 +302,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
this.shouldAssignRegionsWithFavoredNodes = conf.getClass(
|
||||
HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals(
|
||||
FavoredNodeLoadBalancer.class);
|
||||
try {
|
||||
if (server.getCoordinatedStateManager() != null) {
|
||||
this.tableStateManager = server.getCoordinatedStateManager().getTableStateManager();
|
||||
} else {
|
||||
this.tableStateManager = null;
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
|
||||
this.tableStateManager = tableStateManager;
|
||||
|
||||
// This is the max attempts, not retries, so it should be at least 1.
|
||||
this.maximumAttempts = Math.max(1,
|
||||
this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10));
|
||||
|
@ -392,7 +382,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return Instance of ZKTableStateManager.
|
||||
* @return Instance of TableStateManager.
|
||||
*/
|
||||
public TableStateManager getTableStateManager() {
|
||||
// These are 'expensive' to make involving trip to zk ensemble so allow
|
||||
|
@ -516,10 +506,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
* @throws CoordinatedStateException
|
||||
*/
|
||||
void joinCluster() throws IOException,
|
||||
KeeperException, InterruptedException, CoordinatedStateException {
|
||||
KeeperException, CoordinatedStateException {
|
||||
long startTime = System.currentTimeMillis();
|
||||
// Concurrency note: In the below the accesses on regionsInTransition are
|
||||
// outside of a synchronization block where usually all accesses to RIT are
|
||||
|
@ -560,7 +549,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws InterruptedException
|
||||
*/
|
||||
boolean processDeadServersAndRegionsInTransition(final Set<ServerName> deadServers)
|
||||
throws KeeperException, IOException, InterruptedException, CoordinatedStateException {
|
||||
throws KeeperException, IOException {
|
||||
List<String> nodes = ZKUtil.listChildrenNoWatch(watcher, watcher.assignmentZNode);
|
||||
|
||||
if (useZKForAssignment && nodes == null) {
|
||||
|
@ -568,7 +557,6 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
server.abort(errorMessage, new IOException(errorMessage));
|
||||
return true; // Doesn't matter in this case
|
||||
}
|
||||
|
||||
boolean failover = !serverManager.getDeadServers().isEmpty();
|
||||
if (failover) {
|
||||
// This may not be a failover actually, especially if meta is on this master.
|
||||
|
@ -689,7 +677,11 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
if (!failover) {
|
||||
// Fresh cluster startup.
|
||||
LOG.info("Clean cluster startup. Assigning user regions");
|
||||
assignAllUserRegions(allRegions);
|
||||
try {
|
||||
assignAllUserRegions(allRegions);
|
||||
} catch (InterruptedException ie) {
|
||||
ExceptionUtil.rethrowIfInterrupt(ie);
|
||||
}
|
||||
}
|
||||
// unassign replicas of the split parents and the merged regions
|
||||
// the daughter replicas are opened in assignAllUserRegions if it was
|
||||
|
@ -707,11 +699,10 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* locations are returned.
|
||||
*/
|
||||
private Map<HRegionInfo, ServerName> getUserRegionsToAssign()
|
||||
throws InterruptedIOException, CoordinatedStateException {
|
||||
throws IOException {
|
||||
Set<TableName> disabledOrDisablingOrEnabling =
|
||||
tableStateManager.getTablesInStates(ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING, ZooKeeperProtos.Table.State.ENABLING);
|
||||
|
||||
tableStateManager.getTablesInStates(TableState.State.DISABLED,
|
||||
TableState.State.DISABLING, TableState.State.ENABLING);
|
||||
// Clean re/start, mark all user regions closed before reassignment
|
||||
return regionStates.closeAllUserRegions(disabledOrDisablingOrEnabling);
|
||||
}
|
||||
|
@ -739,7 +730,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
try {
|
||||
// Assign the regions
|
||||
assignAllUserRegions(getUserRegionsToAssign());
|
||||
} catch (CoordinatedStateException | IOException | InterruptedException e) {
|
||||
} catch (IOException | InterruptedException e) {
|
||||
LOG.error("Exception occured while assigning user regions.", e);
|
||||
}
|
||||
};
|
||||
|
@ -1482,7 +1473,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
LOG.debug("Znode " + regionNameStr + " deleted, state: " + rs);
|
||||
|
||||
boolean disabled = getTableStateManager().isTableState(regionInfo.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING);
|
||||
TableState.State.DISABLED, TableState.State.DISABLING);
|
||||
|
||||
ServerName serverName = rs.getServerName();
|
||||
if (serverManager.isServerOnline(serverName)) {
|
||||
|
@ -2269,7 +2260,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// will not be in ENABLING or ENABLED state.
|
||||
TableName tableName = region.getTable();
|
||||
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.");
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
|
@ -2495,8 +2486,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
|
||||
if (this.tableStateManager.isTableState(region.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING) || replicasToClose.contains(region)) {
|
||||
TableState.State.DISABLED,
|
||||
TableState.State.DISABLING) || replicasToClose.contains(region)) {
|
||||
LOG.info("Table " + region.getTable() + " is disabled or disabling;"
|
||||
+ " skipping assign of " + region.getRegionNameAsString());
|
||||
offlineDisabledRegion(region);
|
||||
|
@ -3127,7 +3118,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
for (HRegionInfo hri : regionsFromMetaScan) {
|
||||
TableName tableName = hri.getTable();
|
||||
if (!tableStateManager.isTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
TableState.State.ENABLED)) {
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
}
|
||||
|
@ -3194,14 +3185,14 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
*/
|
||||
Set<ServerName> rebuildUserRegions() throws
|
||||
IOException, KeeperException, CoordinatedStateException {
|
||||
IOException, KeeperException {
|
||||
Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates(
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.ENABLING);
|
||||
TableState.State.DISABLED, TableState.State.ENABLING);
|
||||
|
||||
Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
|
||||
ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING,
|
||||
ZooKeeperProtos.Table.State.ENABLING);
|
||||
TableState.State.DISABLED,
|
||||
TableState.State.DISABLING,
|
||||
TableState.State.ENABLING);
|
||||
|
||||
// Region assignment from META
|
||||
List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getConnection());
|
||||
|
@ -3253,7 +3244,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
ServerName lastHost = hrl.getServerName();
|
||||
ServerName regionLocation = RegionStateStore.getRegionServer(result, replicaId);
|
||||
if (tableStateManager.isTableState(regionInfo.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED)) {
|
||||
TableState.State.DISABLED)) {
|
||||
// force region to forget it hosts for disabled/disabling tables.
|
||||
// see HBASE-13326
|
||||
lastHost = null;
|
||||
|
@ -3283,7 +3274,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// this will be used in rolling restarts
|
||||
if (!disabledOrDisablingOrEnabling.contains(tableName)
|
||||
&& !getTableStateManager().isTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
TableState.State.ENABLED)) {
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
}
|
||||
|
@ -3300,9 +3291,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
*/
|
||||
private void recoverTableInDisablingState()
|
||||
throws KeeperException, IOException, CoordinatedStateException {
|
||||
throws KeeperException, IOException {
|
||||
Set<TableName> disablingTables =
|
||||
tableStateManager.getTablesInStates(ZooKeeperProtos.Table.State.DISABLING);
|
||||
tableStateManager.getTablesInStates(TableState.State.DISABLING);
|
||||
if (disablingTables.size() != 0) {
|
||||
for (TableName tableName : disablingTables) {
|
||||
// Recover by calling DisableTableHandler
|
||||
|
@ -3324,9 +3315,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
*/
|
||||
private void recoverTableInEnablingState()
|
||||
throws KeeperException, IOException, CoordinatedStateException {
|
||||
throws KeeperException, IOException {
|
||||
Set<TableName> enablingTables = tableStateManager.
|
||||
getTablesInStates(ZooKeeperProtos.Table.State.ENABLING);
|
||||
getTablesInStates(TableState.State.ENABLING);
|
||||
if (enablingTables.size() != 0) {
|
||||
for (TableName tableName : enablingTables) {
|
||||
// Recover by calling EnableTableHandler
|
||||
|
@ -3398,9 +3389,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
LOG.info("Server " + serverName + " isn't online. SSH will handle this");
|
||||
continue;
|
||||
}
|
||||
RegionState.State state = regionState.getState();
|
||||
HRegionInfo regionInfo = regionState.getRegion();
|
||||
State state = regionState.getState();
|
||||
|
||||
LOG.info("Processing " + regionState);
|
||||
switch (state) {
|
||||
case CLOSED:
|
||||
invokeAssign(regionInfo);
|
||||
|
@ -3790,7 +3781,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
server.abort("Unexpected ZK exception deleting node " + hri, ke);
|
||||
}
|
||||
if (tableStateManager.isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
regionStates.regionOffline(hri);
|
||||
it.remove();
|
||||
continue;
|
||||
|
@ -3813,7 +3804,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
HRegionInfo hri = plan.getRegionInfo();
|
||||
TableName tableName = hri.getTable();
|
||||
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 "
|
||||
+ tableName);
|
||||
return;
|
||||
|
@ -3861,8 +3852,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
protected void setEnabledTable(TableName tableName) {
|
||||
try {
|
||||
this.tableStateManager.setTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED);
|
||||
} catch (CoordinatedStateException e) {
|
||||
TableState.State.ENABLED);
|
||||
} catch (IOException e) {
|
||||
// here we can abort as it is the start up flow
|
||||
String errorMsg = "Unable to ensure that the table " + tableName
|
||||
+ " will be" + " enabled because of a ZooKeeper issue";
|
||||
|
@ -3967,8 +3958,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// 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)
|
||||
if (getTableStateManager().isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
|
||||
replicasToClose.contains(hri)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING) ||
|
||||
replicasToClose.contains(hri)) {
|
||||
offlineDisabledRegion(hri);
|
||||
return;
|
||||
}
|
||||
|
@ -3996,15 +3987,14 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// reset the count, if any
|
||||
failedOpenTracker.remove(hri.getEncodedName());
|
||||
if (getTableStateManager().isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
invokeUnAssign(hri);
|
||||
}
|
||||
}
|
||||
|
||||
private void onRegionClosed(final HRegionInfo hri) {
|
||||
if (getTableStateManager().isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
|
||||
replicasToClose.contains(hri)) {
|
||||
if (getTableStateManager().isTableState(hri.getTable(), TableState.State.DISABLED,
|
||||
TableState.State.DISABLING) || replicasToClose.contains(hri)) {
|
||||
offlineDisabledRegion(hri);
|
||||
return;
|
||||
}
|
||||
|
@ -4050,7 +4040,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
if (getTableStateManager().isTableState(p.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
invokeUnAssign(p);
|
||||
}
|
||||
return null;
|
||||
|
@ -4076,7 +4066,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
// User could disable the table before master knows the new region.
|
||||
if (getTableStateManager().isTableState(p.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
invokeUnAssign(a);
|
||||
invokeUnAssign(b);
|
||||
} else {
|
||||
|
@ -4130,7 +4120,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
// User could disable the table before master knows the new region.
|
||||
if (getTableStateManager().isTableState(p.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
invokeUnAssign(p);
|
||||
} else {
|
||||
Callable<Object> mergeReplicasCallable = new Callable<Object>() {
|
||||
|
@ -4170,7 +4160,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
if (getTableStateManager().isTableState(p.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
invokeUnAssign(a);
|
||||
invokeUnAssign(b);
|
||||
}
|
||||
|
@ -4291,7 +4281,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
// User could disable the table before master knows the new region.
|
||||
if (tableStateManager.isTableState(p.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
unassign(p);
|
||||
}
|
||||
}
|
||||
|
@ -4421,7 +4411,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
// User could disable the table before master knows the new region.
|
||||
if (tableStateManager.isTableState(p.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
unassign(hri_a);
|
||||
unassign(hri_b);
|
||||
}
|
||||
|
@ -4692,7 +4682,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
errorMsg = hri.getShortNameToLog()
|
||||
+ " is not pending close on " + serverName;
|
||||
} else {
|
||||
onRegionClosed(hri);
|
||||
onRegionClosed(hri);
|
||||
}
|
||||
break;
|
||||
|
||||
|
|
|
@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
|||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
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.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorType;
|
||||
|
@ -139,7 +140,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
|||
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
|
||||
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
|
||||
|
@ -167,6 +167,7 @@ import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
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.LoadBalancerTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
|
||||
|
@ -385,6 +386,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
private long splitPlanCount;
|
||||
private long mergePlanCount;
|
||||
|
||||
// handle table states
|
||||
private TableStateManager tableStateManager;
|
||||
|
||||
/** flag used in test cases in order to simulate RS failures during master initialization */
|
||||
private volatile boolean initializationBeforeMetaAssignment = false;
|
||||
|
||||
|
@ -694,9 +698,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
|
||||
this.assignmentManager = new AssignmentManager(this, serverManager,
|
||||
this.balancer, this.service, this.metricsMaster,
|
||||
this.tableLockManager);
|
||||
this.tableLockManager, tableStateManager);
|
||||
zooKeeper.registerListenerFirst(assignmentManager);
|
||||
|
||||
this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
|
||||
this.serverManager);
|
||||
this.regionServerTracker.start();
|
||||
|
@ -728,6 +731,14 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
this.mpmHost.register(new MasterFlushTableProcedureManager());
|
||||
this.mpmHost.loadProcedures(conf);
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -792,6 +803,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
// Invalidate all write locks held previously
|
||||
this.tableLockManager.reapWriteLocks();
|
||||
|
||||
this.tableStateManager = new TableStateManager(this);
|
||||
this.tableStateManager.start();
|
||||
|
||||
status.setStatus("Initializing ZK system trackers");
|
||||
initializeZKBasedSystemTrackers();
|
||||
|
||||
|
@ -1186,8 +1200,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
}
|
||||
|
||||
private void enableMeta(TableName metaTableName) {
|
||||
if (!this.assignmentManager.getTableStateManager().isTableState(metaTableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
if (!this.tableStateManager.isTableState(metaTableName,
|
||||
TableState.State.ENABLED)) {
|
||||
this.assignmentManager.setEnabledTable(metaTableName);
|
||||
}
|
||||
}
|
||||
|
@ -1231,6 +1245,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
return tableNamespaceManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableStateManager getTableStateManager() {
|
||||
return tableStateManager;
|
||||
}
|
||||
|
||||
/*
|
||||
* 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
|
||||
|
@ -1663,7 +1682,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
// Don't run the normalizer concurrently
|
||||
List<TableName> allEnabledTables = new ArrayList<>(
|
||||
this.assignmentManager.getTableStateManager().getTablesInStates(
|
||||
ZooKeeperProtos.Table.State.ENABLED));
|
||||
TableState.State.ENABLED));
|
||||
|
||||
Collections.shuffle(allEnabledTables);
|
||||
|
||||
|
@ -2508,7 +2527,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
throw new TableNotFoundException(tableName);
|
||||
}
|
||||
if (!getAssignmentManager().getTableStateManager().
|
||||
isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED)) {
|
||||
isTableState(tableName, TableState.State.DISABLED)) {
|
||||
throw new TableNotDisabledException(tableName);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -547,7 +547,6 @@ public class MasterFileSystem {
|
|||
fsd.createTableDescriptor(
|
||||
new HTableDescriptor(fsd.get(TableName.META_TABLE_NAME)));
|
||||
}
|
||||
|
||||
return rd;
|
||||
}
|
||||
|
||||
|
|
|
@ -40,10 +40,12 @@ import org.apache.hadoop.hbase.ProcedureInfo;
|
|||
import org.apache.hadoop.hbase.ServerLoad;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
|
||||
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
|
||||
import org.apache.hadoop.hbase.ipc.PriorityFunction;
|
||||
|
@ -952,13 +954,11 @@ public class MasterRpcServices extends RSRpcServices
|
|||
public GetTableNamesResponse getTableNames(RpcController controller,
|
||||
GetTableNamesRequest req) throws ServiceException {
|
||||
try {
|
||||
master.checkInitialized();
|
||||
|
||||
master.checkServiceStarted();
|
||||
final String regex = req.hasRegex() ? req.getRegex() : null;
|
||||
final String namespace = req.hasNamespace() ? req.getNamespace() : null;
|
||||
List<TableName> tableNames = master.listTableNames(namespace, regex,
|
||||
req.getIncludeSysTables());
|
||||
|
||||
GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder();
|
||||
if (tableNames != null && tableNames.size() > 0) {
|
||||
// Add the table names to the response
|
||||
|
@ -972,6 +972,26 @@ 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
|
||||
public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c,
|
||||
IsCatalogJanitorEnabledRequest req) throws ServiceException {
|
||||
|
|
|
@ -83,6 +83,11 @@ public interface MasterServices extends Server {
|
|||
*/
|
||||
TableLockManager getTableLockManager();
|
||||
|
||||
/**
|
||||
* @return Master's instance of {@link TableStateManager}
|
||||
*/
|
||||
TableStateManager getTableStateManager();
|
||||
|
||||
/**
|
||||
* @return Master's instance of {@link MasterCoprocessorHost}
|
||||
*/
|
||||
|
|
|
@ -31,6 +31,8 @@ import java.util.SortedSet;
|
|||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -42,14 +44,13 @@ import org.apache.hadoop.hbase.RegionTransition;
|
|||
import org.apache.hadoop.hbase.ServerLoad;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.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.ConfigUtil;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
@ -59,9 +60,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
|||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
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.
|
||||
* In normal scenario, it should match the meta table and the true region states.
|
||||
|
@ -720,7 +718,7 @@ public class RegionStates {
|
|||
if (oldServerName != null && serverHoldings.containsKey(oldServerName)) {
|
||||
if (newState == State.MERGED || newState == State.SPLIT
|
||||
|| 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.
|
||||
// Otherwise, offline it from this server only when it is online on a different server.
|
||||
LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
|
||||
|
@ -1295,8 +1293,8 @@ public class RegionStates {
|
|||
* Update a region state. It will be put in transition if not already there.
|
||||
*/
|
||||
private RegionState updateRegionState(final HRegionInfo hri,
|
||||
final State state, final ServerName serverName, long openSeqNum) {
|
||||
if (state == State.FAILED_CLOSE || state == State.FAILED_OPEN) {
|
||||
final RegionState.State state, final ServerName serverName, long openSeqNum) {
|
||||
if (state == RegionState.State.FAILED_CLOSE || state == RegionState.State.FAILED_OPEN) {
|
||||
LOG.warn("Failed to open/close " + hri.getShortNameToLog()
|
||||
+ " on " + serverName + ", set to " + state);
|
||||
}
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||
import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -228,7 +229,7 @@ public class TableNamespaceManager {
|
|||
}
|
||||
|
||||
// Now check if the table is assigned, if not then fail fast
|
||||
if (isTableAssigned()) {
|
||||
if (isTableAssigned() && isTableEnabled()) {
|
||||
try {
|
||||
boolean initGoodSofar = true;
|
||||
nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
|
||||
|
@ -297,6 +298,12 @@ public class TableNamespaceManager {
|
|||
return false;
|
||||
}
|
||||
|
||||
private boolean isTableEnabled() throws IOException {
|
||||
return masterServices.getTableStateManager().getTableState(
|
||||
TableName.NAMESPACE_TABLE_NAME
|
||||
).equals(TableState.State.ENABLED);
|
||||
}
|
||||
|
||||
private boolean isTableAssigned() {
|
||||
return !masterServices.getAssignmentManager().getRegionStates().
|
||||
getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty();
|
||||
|
|
|
@ -0,0 +1,219 @@
|
|||
/**
|
||||
* 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.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.classification.InterfaceAudience;
|
||||
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;
|
||||
}
|
||||
}
|
|
@ -23,11 +23,10 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
|
||||
/**
|
||||
* Handles CLOSED region event on Master.
|
||||
|
@ -93,7 +92,7 @@ public class ClosedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName());
|
||||
// Check if this table is being disabled or not
|
||||
if (this.assignmentManager.getTableStateManager().isTableState(this.regionInfo.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
|
||||
TableState.State.DISABLED, TableState.State.DISABLING) ||
|
||||
assignmentManager.getReplicasToClose().contains(regionInfo)) {
|
||||
assignmentManager.offlineDisabledRegion(regionInfo);
|
||||
return;
|
||||
|
|
|
@ -30,14 +30,16 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
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.EventType;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
|
@ -48,7 +50,6 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
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.UserProvider;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
|
@ -119,13 +120,6 @@ public class CreateTableHandler extends EventHandler {
|
|||
if (MetaTableAccessor.tableExists(this.server.getConnection(), tableName)) {
|
||||
throw new TableExistsException(tableName);
|
||||
}
|
||||
|
||||
// During master initialization, the ZK state could be inconsistent from failed DDL
|
||||
// in the past. If we fail here, it would prevent master to start. We should force
|
||||
// setting the system table state regardless the table state.
|
||||
boolean skipTableStateCheck =
|
||||
!((HMaster) this.server).isInitialized() && tableName.isSystemTable();
|
||||
checkAndSetEnablingTable(assignmentManager, tableName, skipTableStateCheck);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -135,52 +129,6 @@ public class CreateTableHandler extends EventHandler {
|
|||
return this;
|
||||
}
|
||||
|
||||
static void checkAndSetEnablingTable(final AssignmentManager assignmentManager,
|
||||
final TableName tableName, boolean skipTableStateCheck) 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 (skipTableStateCheck) {
|
||||
assignmentManager.getTableStateManager().setTableState(
|
||||
tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLING);
|
||||
} else 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
|
||||
public String toString() {
|
||||
String name = "UnknownServerName";
|
||||
|
@ -228,9 +176,6 @@ public class CreateTableHandler extends EventHandler {
|
|||
releaseTableLock();
|
||||
LOG.info("Table, " + this.hTableDescriptor.getTableName() + ", creation " +
|
||||
(exception == null ? "successful" : "failed. " + exception));
|
||||
if (exception != null) {
|
||||
removeEnablingTable(this.assignmentManager, this.hTableDescriptor.getTableName());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -253,9 +198,12 @@ public class CreateTableHandler extends EventHandler {
|
|||
FileSystem fs = fileSystemManager.getFileSystem();
|
||||
|
||||
// 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);
|
||||
new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory(
|
||||
tempTableDir, this.hTableDescriptor, false);
|
||||
tempTableDir, underConstruction, false);
|
||||
Path tableDir = FSUtils.getTableDir(fileSystemManager.getRootDir(), tableName);
|
||||
|
||||
// 2. Create Regions
|
||||
|
@ -280,24 +228,18 @@ public class CreateTableHandler extends EventHandler {
|
|||
// 7. Trigger immediate assignment of the regions in round-robin fashion
|
||||
ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);
|
||||
}
|
||||
|
||||
// 8. 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);
|
||||
}
|
||||
|
||||
// 8. Update the tabledescriptor cache.
|
||||
((HMaster) this.server).getTableDescriptors().get(tableName);
|
||||
|
||||
// 9. Enable table
|
||||
assignmentManager.getTableStateManager().setTableState(tableName,
|
||||
TableState.State.ENABLED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create any replicas for the regions (the default replicas that was
|
||||
* already created is passed to the method)
|
||||
* @param hTableDescriptor
|
||||
* @param hTableDescriptor descriptor to use
|
||||
* @param regions default replicas
|
||||
* @return the combined list of default and non-default replicas
|
||||
*/
|
||||
|
|
|
@ -25,13 +25,13 @@ import java.util.concurrent.ExecutorService;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
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.executor.EventHandler;
|
||||
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.HMaster;
|
||||
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.TableLockManager;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.htrace.Trace;
|
||||
|
||||
/**
|
||||
|
@ -91,16 +90,11 @@ public class DisableTableHandler extends EventHandler {
|
|||
// DISABLED or ENABLED.
|
||||
//TODO: reevaluate this since we have table locks now
|
||||
if (!skipTableStateCheck) {
|
||||
try {
|
||||
if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
|
||||
this.tableName, ZooKeeperProtos.Table.State.DISABLING,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
LOG.info("Table " + tableName + " isn't enabled; skipping disable");
|
||||
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);
|
||||
if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
|
||||
this.tableName, TableState.State.DISABLING,
|
||||
TableState.State.ENABLED)) {
|
||||
LOG.info("Table " + tableName + " isn't enabled; skipping disable");
|
||||
throw new TableNotEnabledException(this.tableName);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
|
@ -139,8 +133,6 @@ public class DisableTableHandler extends EventHandler {
|
|||
}
|
||||
} catch (IOException 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 {
|
||||
releaseTableLock();
|
||||
}
|
||||
|
@ -156,10 +148,10 @@ public class DisableTableHandler extends EventHandler {
|
|||
}
|
||||
}
|
||||
|
||||
private void handleDisableTable() throws IOException, CoordinatedStateException {
|
||||
private void handleDisableTable() throws IOException {
|
||||
// Set table disabling flag up in zk.
|
||||
this.assignmentManager.getTableStateManager().setTableState(this.tableName,
|
||||
ZooKeeperProtos.Table.State.DISABLING);
|
||||
TableState.State.DISABLING);
|
||||
boolean done = false;
|
||||
while (true) {
|
||||
// Get list of online regions that are of this table. Regions that are
|
||||
|
@ -188,7 +180,7 @@ public class DisableTableHandler extends EventHandler {
|
|||
}
|
||||
// Flip the table to disabled if success.
|
||||
if (done) this.assignmentManager.getTableStateManager().setTableState(this.tableName,
|
||||
ZooKeeperProtos.Table.State.DISABLED);
|
||||
TableState.State.DISABLED);
|
||||
LOG.info("Disabled table, " + this.tableName + ", is done=" + done);
|
||||
}
|
||||
|
||||
|
@ -208,7 +200,7 @@ public class DisableTableHandler extends EventHandler {
|
|||
RegionStates regionStates = assignmentManager.getRegionStates();
|
||||
for (HRegionInfo region: regions) {
|
||||
if (regionStates.isRegionInTransition(region)
|
||||
&& !regionStates.isRegionInState(region, State.FAILED_CLOSE)) {
|
||||
&& !regionStates.isRegionInState(region, RegionState.State.FAILED_CLOSE)) {
|
||||
continue;
|
||||
}
|
||||
final HRegionInfo hri = region;
|
||||
|
|
|
@ -26,15 +26,15 @@ import java.util.Map;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
|
@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.master.RegionStates;
|
|||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.master.TableStateManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
|
||||
|
@ -97,14 +97,9 @@ public class EnableTableHandler extends EventHandler {
|
|||
if (!this.skipTableStateCheck) {
|
||||
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.");
|
||||
TableStateManager tsm = assignmentManager.getTableStateManager();
|
||||
if (tsm.isTableState(tableName, TableState.State.ENABLING)) {
|
||||
tsm.setDeletedTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -113,16 +108,11 @@ public class EnableTableHandler extends EventHandler {
|
|||
// After that, no other requests can be accepted until the table reaches
|
||||
// DISABLED or ENABLED.
|
||||
if (!skipTableStateCheck) {
|
||||
try {
|
||||
if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
|
||||
this.tableName, ZooKeeperProtos.Table.State.ENABLING,
|
||||
ZooKeeperProtos.Table.State.DISABLED)) {
|
||||
LOG.info("Table " + tableName + " isn't disabled; skipping enable");
|
||||
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);
|
||||
if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
|
||||
this.tableName, TableState.State.ENABLING,
|
||||
TableState.State.DISABLED)) {
|
||||
LOG.info("Table " + tableName + " isn't disabled; skipping enable");
|
||||
throw new TableNotDisabledException(this.tableName);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
|
@ -158,11 +148,7 @@ public class EnableTableHandler extends EventHandler {
|
|||
if (cpHost != null) {
|
||||
cpHost.postEnableTableHandler(this.tableName, null);
|
||||
}
|
||||
} catch (IOException 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) {
|
||||
} catch (IOException | InterruptedException e) {
|
||||
LOG.error("Error trying to enable the table " + this.tableName, e);
|
||||
} finally {
|
||||
releaseTableLock();
|
||||
|
@ -179,14 +165,13 @@ public class EnableTableHandler extends EventHandler {
|
|||
}
|
||||
}
|
||||
|
||||
private void handleEnableTable() throws IOException, CoordinatedStateException,
|
||||
private void handleEnableTable() throws IOException,
|
||||
InterruptedException {
|
||||
// I could check table is disabling and if so, not enable but require
|
||||
// that user first finish disabling but that might be obnoxious.
|
||||
|
||||
// Set table enabling flag up in zk.
|
||||
this.assignmentManager.getTableStateManager().setTableState(this.tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLING);
|
||||
TableState.State.ENABLING);
|
||||
boolean done = false;
|
||||
ServerManager serverManager = ((HMaster)this.server).getServerManager();
|
||||
// Get the regions of this table. We're done when all listed
|
||||
|
@ -251,7 +236,7 @@ public class EnableTableHandler extends EventHandler {
|
|||
if (done) {
|
||||
// Flip the table to enabled.
|
||||
this.assignmentManager.getTableStateManager().setTableState(
|
||||
this.tableName, ZooKeeperProtos.Table.State.ENABLED);
|
||||
this.tableName, TableState.State.ENABLED);
|
||||
LOG.info("Table '" + this.tableName
|
||||
+ "' was successfully enabled. Status: done=" + done);
|
||||
} else {
|
||||
|
|
|
@ -29,6 +29,8 @@ import java.util.TreeMap;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
@ -36,16 +38,15 @@ import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
|||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.BulkReOpen;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
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.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -137,7 +138,7 @@ public abstract class TableEventHandler extends EventHandler {
|
|||
handleTableOperation(hris);
|
||||
if (eventType.isOnlineSchemaChangeSupported() && this.masterServices.
|
||||
getAssignmentManager().getTableStateManager().isTableState(
|
||||
tableName, ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
tableName, TableState.State.ENABLED)) {
|
||||
if (reOpenAllRegions(hris)) {
|
||||
LOG.info("Completed table operation " + eventType + " on table " +
|
||||
tableName);
|
||||
|
@ -236,10 +237,10 @@ public abstract class TableEventHandler extends EventHandler {
|
|||
* @throws FileNotFoundException
|
||||
* @throws IOException
|
||||
*/
|
||||
public HTableDescriptor getTableDescriptor()
|
||||
public TableDescriptor getTableDescriptor()
|
||||
throws FileNotFoundException, IOException {
|
||||
HTableDescriptor htd =
|
||||
this.masterServices.getTableDescriptors().get(tableName);
|
||||
TableDescriptor htd =
|
||||
this.masterServices.getTableDescriptors().getDescriptor(tableName);
|
||||
if (htd == null) {
|
||||
throw new IOException("HTableDescriptor missing for " + tableName);
|
||||
}
|
||||
|
|
|
@ -32,12 +32,12 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
|
||||
/**
|
||||
|
@ -336,7 +336,7 @@ public class AddColumnFamilyProcedure
|
|||
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
|
||||
// This operation only run when the table is enabled.
|
||||
if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
|
||||
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
.isTableState(getTableName(), TableState.State.ENABLED)) {
|
||||
return;
|
||||
}
|
||||
|
||||
|
|
|
@ -33,20 +33,21 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.exceptions.HBaseException;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.TableStateManager;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
@ -299,8 +300,8 @@ public class CreateTableProcedure
|
|||
!(env.getMasterServices().isInitialized()) && tableName.isSystemTable();
|
||||
if (!skipTableStateCheck) {
|
||||
TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager();
|
||||
if (tsm.isTableState(tableName, true, ZooKeeperProtos.Table.State.ENABLING,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
if (tsm.isTableState(tableName, TableState.State.ENABLING,
|
||||
TableState.State.ENABLED)) {
|
||||
LOG.warn("The table " + tableName + " does not exist in meta but has a znode. " +
|
||||
"run hbck to fix inconsistencies.");
|
||||
setFailure("master-create-table", new TableExistsException(getTableName()));
|
||||
|
@ -375,7 +376,7 @@ public class CreateTableProcedure
|
|||
// using a copy of descriptor, table will be created enabling first
|
||||
final Path tempTableDir = FSUtils.getTableDir(tempdir, hTableDescriptor.getTableName());
|
||||
new FSTableDescriptors(env.getMasterConfiguration()).createTableDescriptorForTableDirectory(
|
||||
tempTableDir, hTableDescriptor, false);
|
||||
tempTableDir, new TableDescriptor(hTableDescriptor), false);
|
||||
|
||||
// 2. Create Regions
|
||||
newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir,
|
||||
|
@ -448,14 +449,14 @@ public class CreateTableProcedure
|
|||
|
||||
// Mark the table as Enabling
|
||||
assignmentManager.getTableStateManager().setTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLING);
|
||||
TableState.State.ENABLING);
|
||||
|
||||
// Trigger immediate assignment of the regions in round-robin fashion
|
||||
ModifyRegionUtils.assignRegions(assignmentManager, regions);
|
||||
|
||||
// Enable table
|
||||
assignmentManager.getTableStateManager()
|
||||
.setTableState(tableName, ZooKeeperProtos.Table.State.ENABLED);
|
||||
.setTableState(tableName, TableState.State.ENABLED);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -31,12 +31,12 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -357,7 +357,7 @@ public class DeleteColumnFamilyProcedure
|
|||
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
|
||||
// This operation only run when the table is enabled.
|
||||
if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
|
||||
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
.isTableState(getTableName(), TableState.State.ENABLED)) {
|
||||
return;
|
||||
}
|
||||
|
||||
|
|
|
@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||
import org.apache.hadoop.hbase.exceptions.HBaseException;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
|
@ -41,11 +41,11 @@ import org.apache.hadoop.hbase.master.BulkAssigner;
|
|||
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.TableStateManager;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.htrace.Trace;
|
||||
|
@ -286,8 +286,8 @@ public class DisableTableProcedure
|
|||
// this issue.
|
||||
TableStateManager tsm =
|
||||
env.getMasterServices().getAssignmentManager().getTableStateManager();
|
||||
if (!tsm.setTableStateIfInStates(tableName, ZooKeeperProtos.Table.State.DISABLING,
|
||||
ZooKeeperProtos.Table.State.DISABLING, ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
if (!tsm.setTableStateIfInStates(tableName, TableState.State.DISABLING,
|
||||
TableState.State.DISABLING, TableState.State.ENABLED)) {
|
||||
LOG.info("Table " + tableName + " isn't enabled; skipping disable");
|
||||
setFailure("master-disable-table", new TableNotEnabledException(tableName));
|
||||
canTableBeDisabled = false;
|
||||
|
@ -311,7 +311,7 @@ public class DisableTableProcedure
|
|||
try {
|
||||
// If the state was changed, undo it.
|
||||
if (env.getMasterServices().getAssignmentManager().getTableStateManager().isTableState(
|
||||
tableName, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
tableName, TableState.State.DISABLING)) {
|
||||
EnableTableProcedure.setTableStateToEnabled(env, tableName);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
|
@ -344,7 +344,7 @@ public class DisableTableProcedure
|
|||
// Set table disabling flag up in zk.
|
||||
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
|
||||
tableName,
|
||||
ZooKeeperProtos.Table.State.DISABLING);
|
||||
TableState.State.DISABLING);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -435,7 +435,7 @@ public class DisableTableProcedure
|
|||
// Flip the table to disabled
|
||||
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
|
||||
tableName,
|
||||
ZooKeeperProtos.Table.State.DISABLED);
|
||||
TableState.State.DISABLED);
|
||||
LOG.info("Disabled table, " + tableName + ", is completed.");
|
||||
}
|
||||
|
||||
|
|
|
@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.exceptions.HBaseException;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.BulkAssigner;
|
||||
|
@ -45,11 +45,11 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionStates;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.TableStateManager;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
|
@ -307,7 +307,7 @@ public class EnableTableProcedure
|
|||
// was implemented. With table lock, there is no need to set the state here (it will
|
||||
// set the state later on). A quick state check should be enough for us to move forward.
|
||||
TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager();
|
||||
if (!tsm.isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED)) {
|
||||
if (!tsm.isTableState(tableName, TableState.State.DISABLED)) {
|
||||
LOG.info("Table " + tableName + " isn't disabled; skipping enable");
|
||||
setFailure("master-enable-table", new TableNotDisabledException(this.tableName));
|
||||
canTableBeEnabled = false;
|
||||
|
@ -344,8 +344,7 @@ public class EnableTableProcedure
|
|||
// Set table disabling flag up in zk.
|
||||
LOG.info("Attempting to enable the table " + tableName);
|
||||
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
|
||||
tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLING);
|
||||
tableName, TableState.State.ENABLING);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -490,8 +489,7 @@ public class EnableTableProcedure
|
|||
final TableName tableName) throws HBaseException, IOException {
|
||||
// Flip the table to Enabled
|
||||
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
|
||||
tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED);
|
||||
tableName, TableState.State.ENABLED);
|
||||
LOG.info("Table '" + tableName + "' was successfully enabled.");
|
||||
}
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.TableNotFoundException;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.BulkReOpen;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
|
@ -78,7 +78,7 @@ public final class MasterDDLOperationHelper {
|
|||
|
||||
// We only execute this procedure with table online if online schema change config is set.
|
||||
if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
|
||||
.isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED)
|
||||
.isTableState(tableName, TableState.State.DISABLED)
|
||||
&& !MasterDDLOperationHelper.isOnlineSchemaChangeAllowed(env)) {
|
||||
throw new TableNotDisabledException(tableName);
|
||||
}
|
||||
|
|
|
@ -32,12 +32,12 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
|
||||
/**
|
||||
|
@ -316,7 +316,7 @@ public class ModifyColumnFamilyProcedure
|
|||
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
|
||||
// This operation only run when the table is enabled.
|
||||
if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
|
||||
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
.isTableState(getTableName(), TableState.State.ENABLED)) {
|
||||
return;
|
||||
}
|
||||
|
||||
|
|
|
@ -42,11 +42,11 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
||||
|
||||
|
@ -294,7 +294,7 @@ public class ModifyTableProcedure
|
|||
env.getMasterServices().getTableDescriptors().get(getTableName());
|
||||
|
||||
if (env.getMasterServices().getAssignmentManager().getTableStateManager()
|
||||
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
.isTableState(getTableName(), TableState.State.ENABLED)) {
|
||||
// We only execute this procedure with table online if online schema change config is set.
|
||||
if (!MasterDDLOperationHelper.isOnlineSchemaChangeAllowed(env)) {
|
||||
throw new TableNotDisabledException(getTableName());
|
||||
|
@ -432,7 +432,7 @@ public class ModifyTableProcedure
|
|||
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
|
||||
// This operation only run when the table is enabled.
|
||||
if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
|
||||
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
.isTableState(getTableName(), TableState.State.ENABLED)) {
|
||||
return;
|
||||
}
|
||||
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
|
@ -48,7 +49,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
|
@ -526,7 +526,7 @@ implements ServerProcedureInterface {
|
|||
} else if (rit != null) {
|
||||
if ((rit.isPendingCloseOrClosing() || rit.isOffline())
|
||||
&& am.getTableStateManager().isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
|
||||
TableState.State.DISABLED, TableState.State.DISABLING) ||
|
||||
am.getReplicasToClose().contains(hri)) {
|
||||
// If the table was partially disabled and the RS went down, we should clear the
|
||||
// RIT and remove the node for the region.
|
||||
|
@ -713,7 +713,7 @@ implements ServerProcedureInterface {
|
|||
}
|
||||
// If table is not disabled but the region is offlined,
|
||||
boolean disabled = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED);
|
||||
TableState.State.DISABLED);
|
||||
if (disabled){
|
||||
LOG.info("The table " + hri.getTable() + " was disabled. Hence not proceeding.");
|
||||
return false;
|
||||
|
@ -725,7 +725,7 @@ implements ServerProcedureInterface {
|
|||
return false;
|
||||
}
|
||||
boolean disabling = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLING);
|
||||
TableState.State.DISABLING);
|
||||
if (disabling) {
|
||||
LOG.info("The table " + hri.getTable() + " is disabled. Hence not assigning region" +
|
||||
hri.getEncodedName());
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.Stoppable;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
|
@ -72,7 +73,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
|
||||
import org.apache.hadoop.hbase.security.AccessDeniedException;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
|
@ -622,7 +622,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
TableName snapshotTable = TableName.valueOf(snapshot.getTable());
|
||||
AssignmentManager assignmentMgr = master.getAssignmentManager();
|
||||
if (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
TableState.State.ENABLED)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Table enabled, starting distributed snapshot for "
|
||||
+ ClientSnapshotDescriptionUtils.toString(snapshot));
|
||||
|
@ -634,7 +634,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
}
|
||||
// For disabled table, snapshot is created by the master
|
||||
else if (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
|
||||
ZooKeeperProtos.Table.State.DISABLED)) {
|
||||
TableState.State.DISABLED)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Table is disabled, running snapshot entirely on master "
|
||||
+ ClientSnapshotDescriptionUtils.toString(snapshot));
|
||||
|
@ -801,7 +801,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
// Execute the restore/clone operation
|
||||
if (MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
|
||||
if (master.getAssignmentManager().getTableStateManager().isTableState(
|
||||
TableName.valueOf(snapshot.getTable()), ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) {
|
||||
throw new UnsupportedOperationException("Table '" +
|
||||
TableName.valueOf(snapshot.getTable()) + "' must be disabled in order to " +
|
||||
"perform a restore operation" +
|
||||
|
|
|
@ -39,12 +39,14 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
|
@ -373,7 +375,7 @@ public class NamespaceUpgrade implements Tool {
|
|||
HTableDescriptor newDesc = new HTableDescriptor(oldDesc);
|
||||
newDesc.setName(newTableName);
|
||||
new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory(
|
||||
newTablePath, newDesc, true);
|
||||
newTablePath, new TableDescriptor(newDesc, TableState.State.ENABLED), true);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
@ -111,13 +112,14 @@ public class CompactionTool extends Configured implements Tool {
|
|||
if (isFamilyDir(fs, path)) {
|
||||
Path regionDir = path.getParent();
|
||||
Path tableDir = regionDir.getParent();
|
||||
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
|
||||
compactStoreFiles(tableDir, htd, hri, path.getName(), compactOnce, major);
|
||||
compactStoreFiles(tableDir, htd.getHTableDescriptor(), hri,
|
||||
path.getName(), compactOnce, major);
|
||||
} else if (isRegionDir(fs, path)) {
|
||||
Path tableDir = path.getParent();
|
||||
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
compactRegion(tableDir, htd, path, compactOnce, major);
|
||||
TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
compactRegion(tableDir, htd.getHTableDescriptor(), path, compactOnce, major);
|
||||
} else if (isTableDir(fs, path)) {
|
||||
compactTable(path, compactOnce, major);
|
||||
} else {
|
||||
|
@ -128,9 +130,9 @@ public class CompactionTool extends Configured implements Tool {
|
|||
|
||||
private void compactTable(final Path tableDir, final boolean compactOnce, final boolean major)
|
||||
throws IOException {
|
||||
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
|
||||
for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
|
||||
compactRegion(tableDir, htd, regionDir, compactOnce, major);
|
||||
compactRegion(tableDir, htd.getHTableDescriptor(), regionDir, compactOnce, major);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -90,6 +90,7 @@ public class WALCellCodec implements Codec {
|
|||
* Fully prepares the codec for use.
|
||||
* @param conf {@link Configuration} to read for the user-specified codec. If none is specified,
|
||||
* uses a {@link WALCellCodec}.
|
||||
* @param cellCodecClsName name of codec
|
||||
* @param compression compression the codec should use
|
||||
* @return a {@link WALCellCodec} ready for use.
|
||||
* @throws UnsupportedOperationException if the codec cannot be instantiated
|
||||
|
|
|
@ -43,7 +43,9 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -305,7 +307,8 @@ public final class SnapshotManifest {
|
|||
private void load() throws IOException {
|
||||
switch (getSnapshotFormat(desc)) {
|
||||
case SnapshotManifestV1.DESCRIPTOR_VERSION: {
|
||||
this.htd = FSTableDescriptors.getTableDescriptorFromFs(workingDirFs, workingDir);
|
||||
this.htd = FSTableDescriptors.getTableDescriptorFromFs(workingDirFs, workingDir)
|
||||
.getHTableDescriptor();
|
||||
ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
|
||||
try {
|
||||
this.regionManifests =
|
||||
|
@ -410,7 +413,8 @@ public final class SnapshotManifest {
|
|||
LOG.info("Using old Snapshot Format");
|
||||
// write a copy of descriptor to the snapshot directory
|
||||
new FSTableDescriptors(conf, workingDirFs, rootDir)
|
||||
.createTableDescriptorForTableDirectory(workingDir, htd, false);
|
||||
.createTableDescriptorForTableDirectory(workingDir, new TableDescriptor(
|
||||
htd, TableState.State.ENABLED), false);
|
||||
} else {
|
||||
LOG.debug("Convert to Single Snapshot Manifest for " + this.desc.getName());
|
||||
convertToV2SingleManifest();
|
||||
|
|
|
@ -38,7 +38,9 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
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.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
@ -88,15 +90,10 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
// This cache does not age out the old stuff. Thinking is that the amount
|
||||
// of data we keep up in here is so small, no need to do occasional purge.
|
||||
// TODO.
|
||||
private final Map<TableName, HTableDescriptor> cache =
|
||||
new ConcurrentHashMap<TableName, HTableDescriptor>();
|
||||
private final Map<TableName, TableDescriptor> cache =
|
||||
new ConcurrentHashMap<TableName, TableDescriptor>();
|
||||
|
||||
/**
|
||||
* Table descriptor for <code>hbase:meta</code> catalog table
|
||||
*/
|
||||
private final HTableDescriptor metaTableDescriptor;
|
||||
|
||||
/**
|
||||
* Construct a FSTableDescriptors instance using the hbase root dir of the given
|
||||
* conf and the filesystem where that root dir lives.
|
||||
* This instance can do write operations (is not read only).
|
||||
|
@ -121,7 +118,6 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
this.rootdir = rootdir;
|
||||
this.fsreadonly = fsreadonly;
|
||||
this.usecache = usecache;
|
||||
this.metaTableDescriptor = HTableDescriptor.metaTableDescriptor(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -148,12 +144,12 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* to see if a newer file has been created since the cached one was read.
|
||||
*/
|
||||
@Override
|
||||
public HTableDescriptor get(final TableName tablename)
|
||||
public TableDescriptor getDescriptor(final TableName tablename)
|
||||
throws IOException {
|
||||
invocations++;
|
||||
if (TableName.META_TABLE_NAME.equals(tablename)) {
|
||||
cachehits++;
|
||||
return metaTableDescriptor;
|
||||
return new TableDescriptor(HTableDescriptor.META_TABLEDESC, TableState.State.ENABLED);
|
||||
}
|
||||
// hbase:meta is already handled. If some one tries to get the descriptor for
|
||||
// .logs, .oldlogs or .corrupt throw an exception.
|
||||
|
@ -163,73 +159,100 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
|
||||
if (usecache) {
|
||||
// Look in cache of descriptors.
|
||||
HTableDescriptor cachedtdm = this.cache.get(tablename);
|
||||
TableDescriptor cachedtdm = this.cache.get(tablename);
|
||||
if (cachedtdm != null) {
|
||||
cachehits++;
|
||||
return cachedtdm;
|
||||
}
|
||||
}
|
||||
HTableDescriptor tdmt = null;
|
||||
TableDescriptor tdmt = null;
|
||||
try {
|
||||
tdmt = getTableDescriptorFromFs(fs, rootdir, tablename, !fsreadonly);
|
||||
} catch (NullPointerException e) {
|
||||
LOG.debug("Exception during readTableDecriptor. Current table name = "
|
||||
+ tablename, e);
|
||||
tdmt = getTableDescriptorFromFs(fs, rootdir, tablename);
|
||||
} catch (TableInfoMissingException e) {
|
||||
// ignore. This is regular operation
|
||||
} catch (IOException ioe) {
|
||||
} catch (NullPointerException | IOException e) {
|
||||
LOG.debug("Exception during readTableDecriptor. Current table name = "
|
||||
+ tablename, ioe);
|
||||
+ tablename, e);
|
||||
}
|
||||
// last HTD written wins
|
||||
if (usecache && tdmt != null) {
|
||||
this.cache.put(tablename, tdmt);
|
||||
}
|
||||
|
||||
return tdmt;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
@Override
|
||||
public Map<String, HTableDescriptor> getAll()
|
||||
public Map<String, TableDescriptor> getAllDescriptors()
|
||||
throws IOException {
|
||||
Map<String, HTableDescriptor> htds = new TreeMap<String, HTableDescriptor>();
|
||||
Map<String, TableDescriptor> tds = new TreeMap<String, TableDescriptor>();
|
||||
|
||||
if (fsvisited && usecache) {
|
||||
for (Map.Entry<TableName, HTableDescriptor> entry: this.cache.entrySet()) {
|
||||
htds.put(entry.getKey().toString(), entry.getValue());
|
||||
for (Map.Entry<TableName, TableDescriptor> entry: this.cache.entrySet()) {
|
||||
tds.put(entry.getKey().toString(), entry.getValue());
|
||||
}
|
||||
// add hbase:meta to the response
|
||||
htds.put(HTableDescriptor.META_TABLEDESC.getTableName().getNameAsString(),
|
||||
HTableDescriptor.META_TABLEDESC);
|
||||
tds.put(HTableDescriptor.META_TABLEDESC.getTableName().getNameAsString(),
|
||||
new TableDescriptor(HTableDescriptor.META_TABLEDESC, TableState.State.ENABLED));
|
||||
} else {
|
||||
LOG.debug("Fetching table descriptors from the filesystem.");
|
||||
boolean allvisited = true;
|
||||
for (Path d : FSUtils.getTableDirs(fs, rootdir)) {
|
||||
HTableDescriptor htd = null;
|
||||
TableDescriptor td = null;
|
||||
try {
|
||||
htd = get(FSUtils.getTableName(d));
|
||||
td = getDescriptor(FSUtils.getTableName(d));
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
// inability of retrieving one HTD shouldn't stop getting the remaining
|
||||
LOG.warn("Trouble retrieving htd", fnfe);
|
||||
}
|
||||
if (htd == null) {
|
||||
if (td == null) {
|
||||
allvisited = false;
|
||||
continue;
|
||||
} else {
|
||||
htds.put(htd.getTableName().getNameAsString(), htd);
|
||||
tds.put(td.getHTableDescriptor().getTableName().getNameAsString(), td);
|
||||
}
|
||||
fsvisited = allvisited;
|
||||
}
|
||||
}
|
||||
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;
|
||||
}
|
||||
|
||||
/* (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
|
||||
public Map<String, HTableDescriptor> getByNamespace(String name)
|
||||
throws IOException {
|
||||
|
@ -250,6 +273,27 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
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
|
||||
* and updates the local cache with it.
|
||||
|
@ -259,14 +303,23 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
if (fsreadonly) {
|
||||
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();
|
||||
}
|
||||
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(htd.getTableName().getNameAsString())) {
|
||||
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tableName.getNameAsString())) {
|
||||
throw new NotImplementedException(
|
||||
"Cannot add a table descriptor for a reserved subdirectory name: " + htd.getNameAsString());
|
||||
"Cannot add a table descriptor for a reserved subdirectory name: "
|
||||
+ htd.getNameAsString());
|
||||
}
|
||||
updateTableDescriptor(htd);
|
||||
TableDescriptor descriptor = getDescriptor(htd.getTableName());
|
||||
if (descriptor == null) {
|
||||
descriptor = new TableDescriptor(htd);
|
||||
}
|
||||
else {
|
||||
descriptor.setHTableDescriptor(htd);
|
||||
}
|
||||
updateTableDescriptor(descriptor);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -286,11 +339,11 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
throw new IOException("Failed delete of " + tabledir.toString());
|
||||
}
|
||||
}
|
||||
HTableDescriptor descriptor = this.cache.remove(tablename);
|
||||
TableDescriptor descriptor = this.cache.remove(tablename);
|
||||
if (descriptor == null) {
|
||||
return null;
|
||||
} else {
|
||||
return descriptor;
|
||||
return descriptor.getHTableDescriptor();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -474,8 +527,8 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* if it exists, bypassing the local cache.
|
||||
* Returns null if it's not found.
|
||||
*/
|
||||
public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs,
|
||||
Path hbaseRootDir, TableName tableName) throws IOException {
|
||||
public static TableDescriptor getTableDescriptorFromFs(FileSystem fs,
|
||||
Path hbaseRootDir, TableName tableName) throws IOException {
|
||||
Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
|
||||
return getTableDescriptorFromFs(fs, tableDir);
|
||||
}
|
||||
|
@ -485,37 +538,16 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* directly from the file system if it exists.
|
||||
* @throws TableInfoMissingException if there is no descriptor
|
||||
*/
|
||||
public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs,
|
||||
Path hbaseRootDir, TableName tableName, boolean rewritePb) throws IOException {
|
||||
Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
|
||||
return getTableDescriptorFromFs(fs, tableDir, rewritePb);
|
||||
}
|
||||
/**
|
||||
* Returns the latest table descriptor for the table located at the given directory
|
||||
* directly from the file system if it exists.
|
||||
* @throws TableInfoMissingException if there is no descriptor
|
||||
*/
|
||||
public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs, Path tableDir)
|
||||
public static TableDescriptor getTableDescriptorFromFs(FileSystem fs, Path tableDir)
|
||||
throws IOException {
|
||||
return getTableDescriptorFromFs(fs, tableDir, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the latest table descriptor for the table located at the given directory
|
||||
* directly from the file system if it exists.
|
||||
* @throws TableInfoMissingException if there is no descriptor
|
||||
*/
|
||||
public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs, Path tableDir,
|
||||
boolean rewritePb)
|
||||
throws IOException {
|
||||
FileStatus status = getTableInfoPath(fs, tableDir, false);
|
||||
if (status == null) {
|
||||
throw new TableInfoMissingException("No table descriptor file under " + tableDir);
|
||||
}
|
||||
return readTableDescriptor(fs, status, rewritePb);
|
||||
return readTableDescriptor(fs, status, false);
|
||||
}
|
||||
|
||||
private static HTableDescriptor readTableDescriptor(FileSystem fs, FileStatus status,
|
||||
private static TableDescriptor readTableDescriptor(FileSystem fs, FileStatus status,
|
||||
boolean rewritePb) throws IOException {
|
||||
int len = Ints.checkedCast(status.getLen());
|
||||
byte [] content = new byte[len];
|
||||
|
@ -525,30 +557,32 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
} finally {
|
||||
fsDataInputStream.close();
|
||||
}
|
||||
HTableDescriptor htd = null;
|
||||
TableDescriptor td = null;
|
||||
try {
|
||||
htd = HTableDescriptor.parseFrom(content);
|
||||
td = TableDescriptor.parseFrom(content);
|
||||
} catch (DeserializationException e) {
|
||||
// we have old HTableDescriptor here
|
||||
try {
|
||||
HTableDescriptor ohtd = HTableDescriptor.parseFrom(content);
|
||||
LOG.warn("Found old table descriptor, converting to new format for table " +
|
||||
ohtd.getTableName());
|
||||
htd = new HTableDescriptor(ohtd);
|
||||
if (rewritePb) rewriteTableDescriptor(fs, status, htd);
|
||||
td = new TableDescriptor(ohtd);
|
||||
if (rewritePb) {
|
||||
rewriteTableDescriptor(fs, status, td);
|
||||
}
|
||||
} catch (DeserializationException e1) {
|
||||
throw new IOException("content=" + Bytes.toShort(content), e1);
|
||||
}
|
||||
}
|
||||
if (rewritePb && !ProtobufUtil.isPBMagicPrefix(content)) {
|
||||
// Convert the file over to be pb before leaving here.
|
||||
rewriteTableDescriptor(fs, status, htd);
|
||||
rewriteTableDescriptor(fs, status, td);
|
||||
}
|
||||
return htd;
|
||||
return td;
|
||||
}
|
||||
|
||||
private static void rewriteTableDescriptor(final FileSystem fs, final FileStatus status,
|
||||
final HTableDescriptor td)
|
||||
final TableDescriptor td)
|
||||
throws IOException {
|
||||
Path tableInfoDir = status.getPath().getParent();
|
||||
Path tableDir = tableInfoDir.getParent();
|
||||
|
@ -560,17 +594,18 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* @throws IOException Thrown if failed update.
|
||||
* @throws NotImplementedException if in read only mode
|
||||
*/
|
||||
@VisibleForTesting Path updateTableDescriptor(HTableDescriptor htd)
|
||||
@VisibleForTesting Path updateTableDescriptor(TableDescriptor td)
|
||||
throws IOException {
|
||||
if (fsreadonly) {
|
||||
throw new NotImplementedException("Cannot update a table descriptor - in read only mode");
|
||||
}
|
||||
Path tableDir = getTableDir(htd.getTableName());
|
||||
Path p = writeTableDescriptor(fs, htd, tableDir, getTableInfoPath(tableDir));
|
||||
TableName tableName = td.getHTableDescriptor().getTableName();
|
||||
Path tableDir = getTableDir(tableName);
|
||||
Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(tableDir));
|
||||
if (p == null) throw new IOException("Failed update");
|
||||
LOG.info("Updated tableinfo=" + p);
|
||||
if (usecache) {
|
||||
this.cache.put(htd.getTableName(), htd);
|
||||
this.cache.put(td.getHTableDescriptor().getTableName(), td);
|
||||
}
|
||||
return p;
|
||||
}
|
||||
|
@ -621,9 +656,8 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* @return Descriptor file or null if we failed write.
|
||||
*/
|
||||
private static Path writeTableDescriptor(final FileSystem fs,
|
||||
final HTableDescriptor htd, final Path tableDir,
|
||||
final FileStatus currentDescriptorFile)
|
||||
throws IOException {
|
||||
final TableDescriptor htd, final Path tableDir,
|
||||
final FileStatus currentDescriptorFile) throws IOException {
|
||||
// Get temporary dir into which we'll first write a file to avoid half-written file phenomenon.
|
||||
// This directory is never removed to avoid removing it out from under a concurrent writer.
|
||||
Path tmpTableDir = new Path(tableDir, TMP_DIR);
|
||||
|
@ -652,7 +686,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
}
|
||||
tableInfoDirPath = new Path(tableInfoDir, filename);
|
||||
try {
|
||||
writeHTD(fs, tempPath, htd);
|
||||
writeTD(fs, tempPath, htd);
|
||||
fs.mkdirs(tableInfoDirPath.getParent());
|
||||
if (!fs.rename(tempPath, tableInfoDirPath)) {
|
||||
throw new IOException("Failed rename of " + tempPath + " to " + tableInfoDirPath);
|
||||
|
@ -676,7 +710,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
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 {
|
||||
FSDataOutputStream out = fs.create(p, false);
|
||||
try {
|
||||
|
@ -693,10 +727,19 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* Used by tests.
|
||||
* @return True if we successfully created file.
|
||||
*/
|
||||
public boolean createTableDescriptor(HTableDescriptor htd) throws IOException {
|
||||
public boolean createTableDescriptor(TableDescriptor htd) throws IOException {
|
||||
return createTableDescriptor(htd, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create new HTableDescriptor in HDFS. Happens when we are creating table.
|
||||
* 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
|
||||
* forceCreation is true then even if previous table descriptor is present it
|
||||
|
@ -704,9 +747,9 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
*
|
||||
* @return True if we successfully created file.
|
||||
*/
|
||||
public boolean createTableDescriptor(HTableDescriptor htd, boolean forceCreation)
|
||||
public boolean createTableDescriptor(TableDescriptor htd, boolean forceCreation)
|
||||
throws IOException {
|
||||
Path tableDir = getTableDir(htd.getTableName());
|
||||
Path tableDir = getTableDir(htd.getHTableDescriptor().getTableName());
|
||||
return createTableDescriptorForTableDirectory(tableDir, htd, forceCreation);
|
||||
}
|
||||
|
||||
|
@ -722,7 +765,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
* @throws IOException if a filesystem error occurs
|
||||
*/
|
||||
public boolean createTableDescriptorForTableDirectory(Path tableDir,
|
||||
HTableDescriptor htd, boolean forceCreation) throws IOException {
|
||||
TableDescriptor htd, boolean forceCreation) throws IOException {
|
||||
if (fsreadonly) {
|
||||
throw new NotImplementedException("Cannot create a table descriptor - in read only mode");
|
||||
}
|
||||
|
@ -743,4 +786,3 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -71,7 +71,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -84,6 +83,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
|
|||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -107,13 +107,13 @@ import org.apache.hadoop.hbase.client.RowMutations;
|
|||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.io.FileLink;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
|
@ -128,9 +128,6 @@ import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler;
|
|||
import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl;
|
||||
import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
||||
|
@ -1337,9 +1334,9 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
modTInfo = new TableInfo(tableName);
|
||||
tablesInfo.put(tableName, modTInfo);
|
||||
try {
|
||||
HTableDescriptor htd =
|
||||
TableDescriptor htd =
|
||||
FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName);
|
||||
modTInfo.htds.add(htd);
|
||||
modTInfo.htds.add(htd.getHTableDescriptor());
|
||||
} catch (IOException ioe) {
|
||||
if (!orphanTableDirs.containsKey(tableName)) {
|
||||
LOG.warn("Unable to read .tableinfo from " + hbaseRoot, ioe);
|
||||
|
@ -1394,7 +1391,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
for (String columnfamimly : columns) {
|
||||
htd.addFamily(new HColumnDescriptor(columnfamimly));
|
||||
}
|
||||
fstd.createTableDescriptor(htd, true);
|
||||
fstd.createTableDescriptor(new TableDescriptor(htd, TableState.State.ENABLED), true);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -1442,7 +1439,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
if (tableName.equals(htds[j].getTableName())) {
|
||||
HTableDescriptor htd = htds[j];
|
||||
LOG.info("fixing orphan table: " + tableName + " from cache");
|
||||
fstd.createTableDescriptor(htd, true);
|
||||
fstd.createTableDescriptor(new TableDescriptor(htd, TableState.State.ENABLED), true);
|
||||
j++;
|
||||
iter.remove();
|
||||
}
|
||||
|
@ -1802,19 +1799,16 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
* @throws IOException
|
||||
*/
|
||||
private void loadDisabledTables()
|
||||
throws ZooKeeperConnectionException, IOException {
|
||||
throws IOException {
|
||||
HConnectionManager.execute(new HConnectable<Void>(getConf()) {
|
||||
@Override
|
||||
public Void connect(HConnection connection) throws IOException {
|
||||
try {
|
||||
for (TableName tableName :
|
||||
ZKTableStateClientSideReader.getDisabledOrDisablingTables(zkw)) {
|
||||
disabledTables.add(tableName);
|
||||
TableName[] tables = connection.listTableNames();
|
||||
for (TableName table : tables) {
|
||||
if (connection.getTableState(table)
|
||||
.inStates(TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
disabledTables.add(table);
|
||||
}
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException(ke);
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -3546,12 +3540,15 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
/**
|
||||
* Check whether a orphaned table ZNode exists and fix it if requested.
|
||||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private void checkAndFixOrphanedTableZNodes()
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
Set<TableName> enablingTables = ZKTableStateClientSideReader.getEnablingTables(zkw);
|
||||
throws IOException {
|
||||
Set<TableName> enablingTables = new HashSet<>();
|
||||
for (TableName tableName: admin.listTableNames()) {
|
||||
if (connection.getTableState(tableName).getState().equals(TableState.State.ENABLING)) {
|
||||
enablingTables.add(tableName);
|
||||
}
|
||||
}
|
||||
String msg;
|
||||
TableInfo tableInfo;
|
||||
|
||||
|
@ -3570,21 +3567,12 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
}
|
||||
|
||||
if (orphanedTableZNodes.size() > 0 && this.fixTableZNodes) {
|
||||
ZKTableStateManager zkTableStateMgr = new ZKTableStateManager(zkw);
|
||||
|
||||
for (TableName tableName : orphanedTableZNodes) {
|
||||
try {
|
||||
// Set the table state to be disabled so that if we made mistake, we can trace
|
||||
// the history and figure it out.
|
||||
// Another choice is to call checkAndRemoveTableState() to delete the orphaned ZNode.
|
||||
// Both approaches works.
|
||||
zkTableStateMgr.setTableState(tableName, ZooKeeperProtos.Table.State.DISABLED);
|
||||
} catch (CoordinatedStateException e) {
|
||||
// This exception should not happen here
|
||||
LOG.error(
|
||||
"Got a CoordinatedStateException while fixing the ENABLING table znode " + tableName,
|
||||
e);
|
||||
}
|
||||
// Set the table state to be disabled so that if we made mistake, we can trace
|
||||
// the history and figure it out.
|
||||
// Another choice is to call checkAndRemoveTableState() to delete the orphaned ZNode.
|
||||
// Both approaches works.
|
||||
admin.disableTable(tableName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -155,7 +155,8 @@ class HMerge {
|
|||
|
||||
this.rootDir = FSUtils.getRootDir(conf);
|
||||
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;
|
||||
|
||||
final Configuration walConf = new Configuration(conf);
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configured;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -153,9 +154,9 @@ public class Merge extends Configured implements Tool {
|
|||
if (info2 == null) {
|
||||
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);
|
||||
HRegion merged = merge(htd, meta, info1, info2);
|
||||
HRegion merged = merge(htd.getHTableDescriptor(), meta, info1, info2);
|
||||
|
||||
LOG.info("Adding " + merged.getRegionInfo() + " to " +
|
||||
meta.getRegionInfo());
|
||||
|
|
|
@ -18,8 +18,11 @@
|
|||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -27,6 +30,9 @@ import org.apache.hadoop.conf.Configured;
|
|||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
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.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
|
@ -153,8 +159,9 @@ public class ZKDataMigrator extends Configured implements Tool {
|
|||
}
|
||||
byte[] data = ZKUtil.getData(zkw, znode);
|
||||
if (ProtobufUtil.isPBMagicPrefix(data)) continue;
|
||||
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
|
||||
builder.setState(ZooKeeperProtos.Table.State.valueOf(Bytes.toString(data)));
|
||||
ZooKeeperProtos.DeprecatedTableState.Builder builder =
|
||||
ZooKeeperProtos.DeprecatedTableState.newBuilder();
|
||||
builder.setState(ZooKeeperProtos.DeprecatedTableState.State.valueOf(Bytes.toString(data)));
|
||||
data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
|
||||
ZKUtil.setData(zkw, znode, data);
|
||||
}
|
||||
|
@ -232,15 +239,14 @@ public class ZKDataMigrator extends Configured implements Tool {
|
|||
}
|
||||
|
||||
private void migrateClusterKeyToPB(ZooKeeperWatcher zkw, String peerZnode, byte[] data)
|
||||
throws KeeperException, NoNodeException {
|
||||
throws KeeperException {
|
||||
ReplicationPeer peer = ZooKeeperProtos.ReplicationPeer.newBuilder()
|
||||
.setClusterkey(Bytes.toString(data)).build();
|
||||
ZKUtil.setData(zkw, peerZnode, ProtobufUtil.prependPBMagic(peer.toByteArray()));
|
||||
}
|
||||
|
||||
private void migratePeerStateToPB(ZooKeeperWatcher zkw, byte[] data,
|
||||
String peerStatePath)
|
||||
throws KeeperException, NoNodeException {
|
||||
String peerStatePath) throws KeeperException {
|
||||
String state = Bytes.toString(data);
|
||||
if (ZooKeeperProtos.ReplicationState.State.ENABLED.name().equals(state)) {
|
||||
ZKUtil.setData(zkw, peerStatePath, ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
|
||||
|
@ -249,6 +255,80 @@ 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 {
|
||||
System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZKDataMigrator(), args));
|
||||
}
|
||||
|
|
|
@ -50,6 +50,9 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
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.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -61,7 +64,6 @@ import org.apache.hadoop.fs.PathFilter;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -71,7 +73,6 @@ import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ConnectionUtils;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -82,6 +83,7 @@ import org.apache.hadoop.hbase.client.Mutation;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
|
||||
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.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager;
|
||||
|
@ -98,7 +100,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStor
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.LastSequenceId;
|
||||
|
@ -123,9 +124,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
|||
import org.apache.hadoop.io.MultipleIOException;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
||||
/**
|
||||
|
@ -335,13 +333,14 @@ public class WALSplitter {
|
|||
LOG.warn("Nothing to split in log file " + logPath);
|
||||
return true;
|
||||
}
|
||||
if (csm != null) {
|
||||
try {
|
||||
TableStateManager tsm = csm.getTableStateManager();
|
||||
disablingOrDisabledTables = tsm.getTablesInStates(
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING);
|
||||
} catch (CoordinatedStateException e) {
|
||||
throw new IOException("Can't get disabling/disabled tables", e);
|
||||
if(csm != null) {
|
||||
HConnection scc = csm.getServer().getConnection();
|
||||
TableName[] tables = scc.listTableNames();
|
||||
for (TableName table : tables) {
|
||||
if (scc.getTableState(table)
|
||||
.inStates(TableState.State.DISABLED, TableState.State.DISABLING)) {
|
||||
disablingOrDisabledTables.add(table);
|
||||
}
|
||||
}
|
||||
}
|
||||
int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3);
|
||||
|
|
|
@ -1,369 +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 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;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
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;
|
||||
|
||||
/**
|
||||
* 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.info("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)) {
|
||||
// If the table is in the one of the states from the states list, the cache
|
||||
// might be out-of-date, try to find it out from the master source (zookeeper server).
|
||||
//
|
||||
// Note: this adds extra zookeeper server calls and might have performance impact.
|
||||
// However, this is not the happy path so we should not reach here often. Therefore,
|
||||
// the performance impact should be minimal to none.
|
||||
try {
|
||||
ZooKeeperProtos.Table.State curstate = getTableState(watcher, tableName);
|
||||
|
||||
if (isTableInState(Arrays.asList(states), curstate)) {
|
||||
return false;
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
}
|
||||
}
|
||||
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 (using cache only). {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean isTableState(final TableName tableName,
|
||||
final ZooKeeperProtos.Table.State... states) {
|
||||
return isTableState(tableName, false, states); // only check cache
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if table is marked in specified state in ZK. {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean isTableState(final TableName tableName, final boolean checkSource,
|
||||
final ZooKeeperProtos.Table.State... states) {
|
||||
boolean isTableInSpecifiedState;
|
||||
synchronized (this.cache) {
|
||||
ZooKeeperProtos.Table.State currentState = this.cache.get(tableName);
|
||||
if (checkSource) {
|
||||
// The cache might be out-of-date, try to find it out from the master source (zookeeper
|
||||
// server) and update the cache.
|
||||
try {
|
||||
ZooKeeperProtos.Table.State stateInZK = getTableState(watcher, tableName);
|
||||
|
||||
if (currentState != stateInZK) {
|
||||
if (stateInZK != null) {
|
||||
this.cache.put(tableName, stateInZK);
|
||||
} else {
|
||||
this.cache.remove(tableName);
|
||||
}
|
||||
currentState = stateInZK;
|
||||
}
|
||||
} catch (KeeperException | InterruptedException e) {
|
||||
// Contacting zookeeper failed. Let us just trust the value in cache.
|
||||
}
|
||||
}
|
||||
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();
|
||||
ProtobufUtil.mergeFrom(builder, data, magicLen, data.length - magicLen);
|
||||
return builder.getState();
|
||||
} catch (IOException 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);
|
||||
}
|
||||
}
|
|
@ -3390,6 +3390,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Make sure that at least the specified number of region servers
|
||||
* are running
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
|
|||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.TableStateManager;
|
||||
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -54,6 +55,7 @@ import java.util.Set;
|
|||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -98,70 +100,72 @@ public class TestDrainingServer {
|
|||
final HRegionInfo REGIONINFO = new HRegionInfo(TableName.valueOf("table_test"),
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
|
||||
|
||||
ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
|
||||
"zkWatcher-Test", abortable, true);
|
||||
try (ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
|
||||
"zkWatcher-Test", abortable, true)) {
|
||||
|
||||
Map<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
|
||||
Map<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
|
||||
|
||||
onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
|
||||
onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
|
||||
onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
|
||||
onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
|
||||
|
||||
Mockito.when(server.getConfiguration()).thenReturn(conf);
|
||||
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
|
||||
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
|
||||
Mockito.when(server.getRegionServerVersion(Mockito.any(ServerName.class))).thenReturn("0.0.0");
|
||||
Mockito.when(server.getConfiguration()).thenReturn(conf);
|
||||
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
|
||||
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
|
||||
Mockito.when(server.getRegionServerVersion(Mockito.any(ServerName.class))).thenReturn("0.0.0");
|
||||
|
||||
CoordinatedStateManager cp = new ZkCoordinatedStateManager();
|
||||
cp.initialize(server);
|
||||
cp.start();
|
||||
CoordinatedStateManager cp = new ZkCoordinatedStateManager();
|
||||
cp.initialize(server);
|
||||
cp.start();
|
||||
|
||||
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
|
||||
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
|
||||
|
||||
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
|
||||
Mockito.when(serverManager.getOnlineServersList())
|
||||
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
|
||||
Mockito.when(serverManager.getOnlineServersList())
|
||||
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
|
||||
Mockito.when(serverManager.createDestinationServersList())
|
||||
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList(null))
|
||||
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList(Mockito.anyList())).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList())
|
||||
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList(null))
|
||||
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList(Mockito.anyList())).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
|
||||
for (ServerName sn : onlineServers.keySet()) {
|
||||
Mockito.when(serverManager.isServerOnline(sn)).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionClose(sn, REGIONINFO, -1)).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionClose(sn, REGIONINFO, -1, null, false)).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, new ArrayList<ServerName>()))
|
||||
.thenReturn(RegionOpeningState.OPENED);
|
||||
Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, null))
|
||||
.thenReturn(RegionOpeningState.OPENED);
|
||||
Mockito.when(serverManager.addServerToDrainList(sn)).thenReturn(true);
|
||||
for (ServerName sn : onlineServers.keySet()) {
|
||||
Mockito.when(serverManager.isServerOnline(sn)).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionClose(sn, REGIONINFO, -1)).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionClose(sn, REGIONINFO, -1, null, false)).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, new ArrayList<ServerName>()))
|
||||
.thenReturn(RegionOpeningState.OPENED);
|
||||
Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, null))
|
||||
.thenReturn(RegionOpeningState.OPENED);
|
||||
Mockito.when(serverManager.addServerToDrainList(sn)).thenReturn(true);
|
||||
}
|
||||
|
||||
Mockito.when(master.getServerManager()).thenReturn(serverManager);
|
||||
|
||||
TableStateManager tsm = mock(TableStateManager.class);
|
||||
am = new AssignmentManager(server, serverManager,
|
||||
balancer, startupMasterExecutor("mockExecutorService"), null, null, tsm);
|
||||
|
||||
Mockito.when(master.getAssignmentManager()).thenReturn(am);
|
||||
Mockito.when(master.getZooKeeper()).thenReturn(zkWatcher);
|
||||
|
||||
am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_A));
|
||||
|
||||
zkWatcher.registerListenerFirst(am);
|
||||
|
||||
addServerToDrainedList(SERVERNAME_A, onlineServers, serverManager);
|
||||
|
||||
am.assign(REGIONINFO, true);
|
||||
|
||||
setRegionOpenedOnZK(zkWatcher, SERVERNAME_A, REGIONINFO);
|
||||
setRegionOpenedOnZK(zkWatcher, SERVERNAME_B, REGIONINFO);
|
||||
|
||||
am.waitForAssignment(REGIONINFO);
|
||||
|
||||
assertTrue(am.getRegionStates().isRegionOnline(REGIONINFO));
|
||||
assertNotEquals(am.getRegionStates().getRegionServerOfRegion(REGIONINFO), SERVERNAME_A);
|
||||
}
|
||||
|
||||
Mockito.when(master.getServerManager()).thenReturn(serverManager);
|
||||
|
||||
am = new AssignmentManager(server, serverManager,
|
||||
balancer, startupMasterExecutor("mockExecutorService"), null, null);
|
||||
|
||||
Mockito.when(master.getAssignmentManager()).thenReturn(am);
|
||||
Mockito.when(master.getZooKeeper()).thenReturn(zkWatcher);
|
||||
|
||||
am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_A));
|
||||
|
||||
zkWatcher.registerListenerFirst(am);
|
||||
|
||||
addServerToDrainedList(SERVERNAME_A, onlineServers, serverManager);
|
||||
|
||||
am.assign(REGIONINFO, true);
|
||||
|
||||
setRegionOpenedOnZK(zkWatcher, SERVERNAME_A, REGIONINFO);
|
||||
setRegionOpenedOnZK(zkWatcher, SERVERNAME_B, REGIONINFO);
|
||||
|
||||
am.waitForAssignment(REGIONINFO);
|
||||
|
||||
assertTrue(am.getRegionStates().isRegionOnline(REGIONINFO));
|
||||
assertNotEquals(am.getRegionStates().getRegionServerOfRegion(REGIONINFO), SERVERNAME_A);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -207,80 +211,82 @@ public class TestDrainingServer {
|
|||
bulk.put(REGIONINFO_D, SERVERNAME_D);
|
||||
bulk.put(REGIONINFO_E, SERVERNAME_E);
|
||||
|
||||
ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
|
||||
"zkWatcher-BulkAssignTest", abortable, true);
|
||||
try (ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
|
||||
"zkWatcher-BulkAssignTest", abortable, true)) {
|
||||
|
||||
Mockito.when(server.getConfiguration()).thenReturn(conf);
|
||||
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
|
||||
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
|
||||
Mockito.when(server.getConfiguration()).thenReturn(conf);
|
||||
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
|
||||
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
|
||||
|
||||
CoordinatedStateManager cp = new ZkCoordinatedStateManager();
|
||||
cp.initialize(server);
|
||||
cp.start();
|
||||
CoordinatedStateManager cp = new ZkCoordinatedStateManager();
|
||||
cp.initialize(server);
|
||||
cp.start();
|
||||
|
||||
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
|
||||
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
|
||||
|
||||
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
|
||||
Mockito.when(serverManager.getOnlineServersList()).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
|
||||
Mockito.when(serverManager.getOnlineServersList()).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
|
||||
Mockito.when(serverManager.createDestinationServersList()).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList(null)).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList(Mockito.anyList())).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList()).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList(null)).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
Mockito.when(serverManager.createDestinationServersList(Mockito.anyList())).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
|
||||
for (Entry<HRegionInfo, ServerName> entry : bulk.entrySet()) {
|
||||
Mockito.when(serverManager.isServerOnline(entry.getValue())).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionClose(entry.getValue(),
|
||||
entry.getKey(), -1)).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionOpen(entry.getValue(),
|
||||
entry.getKey(), -1, null)).thenReturn(RegionOpeningState.OPENED);
|
||||
Mockito.when(serverManager.addServerToDrainList(entry.getValue())).thenReturn(true);
|
||||
}
|
||||
for (Entry<HRegionInfo, ServerName> entry : bulk.entrySet()) {
|
||||
Mockito.when(serverManager.isServerOnline(entry.getValue())).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionClose(entry.getValue(),
|
||||
entry.getKey(), -1)).thenReturn(true);
|
||||
Mockito.when(serverManager.sendRegionOpen(entry.getValue(),
|
||||
entry.getKey(), -1, null)).thenReturn(RegionOpeningState.OPENED);
|
||||
Mockito.when(serverManager.addServerToDrainList(entry.getValue())).thenReturn(true);
|
||||
}
|
||||
|
||||
Mockito.when(master.getServerManager()).thenReturn(serverManager);
|
||||
Mockito.when(master.getServerManager()).thenReturn(serverManager);
|
||||
|
||||
drainedServers.add(SERVERNAME_A);
|
||||
drainedServers.add(SERVERNAME_B);
|
||||
drainedServers.add(SERVERNAME_C);
|
||||
drainedServers.add(SERVERNAME_D);
|
||||
drainedServers.add(SERVERNAME_A);
|
||||
drainedServers.add(SERVERNAME_B);
|
||||
drainedServers.add(SERVERNAME_C);
|
||||
drainedServers.add(SERVERNAME_D);
|
||||
|
||||
am = new AssignmentManager(server, serverManager,
|
||||
balancer, startupMasterExecutor("mockExecutorServiceBulk"), null, null);
|
||||
TableStateManager tsm = mock(TableStateManager.class);
|
||||
am = new AssignmentManager(server, serverManager, balancer,
|
||||
startupMasterExecutor("mockExecutorServiceBulk"), null, null, tsm);
|
||||
|
||||
Mockito.when(master.getAssignmentManager()).thenReturn(am);
|
||||
Mockito.when(master.getAssignmentManager()).thenReturn(am);
|
||||
|
||||
zkWatcher.registerListener(am);
|
||||
zkWatcher.registerListener(am);
|
||||
|
||||
for (ServerName drained : drainedServers) {
|
||||
addServerToDrainedList(drained, onlineServers, serverManager);
|
||||
}
|
||||
for (ServerName drained : drainedServers) {
|
||||
addServerToDrainedList(drained, onlineServers, serverManager);
|
||||
}
|
||||
|
||||
am.assign(bulk);
|
||||
am.assign(bulk);
|
||||
|
||||
Set<RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
|
||||
for (RegionState rs : regionsInTransition) {
|
||||
setRegionOpenedOnZK(zkWatcher, rs.getServerName(), rs.getRegion());
|
||||
}
|
||||
Set<RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
|
||||
for (RegionState rs : regionsInTransition) {
|
||||
setRegionOpenedOnZK(zkWatcher, rs.getServerName(), rs.getRegion());
|
||||
}
|
||||
|
||||
am.waitForAssignment(REGIONINFO_A);
|
||||
am.waitForAssignment(REGIONINFO_B);
|
||||
am.waitForAssignment(REGIONINFO_C);
|
||||
am.waitForAssignment(REGIONINFO_D);
|
||||
am.waitForAssignment(REGIONINFO_E);
|
||||
am.waitForAssignment(REGIONINFO_A);
|
||||
am.waitForAssignment(REGIONINFO_B);
|
||||
am.waitForAssignment(REGIONINFO_C);
|
||||
am.waitForAssignment(REGIONINFO_D);
|
||||
am.waitForAssignment(REGIONINFO_E);
|
||||
|
||||
Map<HRegionInfo, ServerName> regionAssignments = am.getRegionStates().getRegionAssignments();
|
||||
for (Entry<HRegionInfo, ServerName> entry : regionAssignments.entrySet()) {
|
||||
LOG.info("Region Assignment: "
|
||||
+ entry.getKey().getRegionNameAsString() + " Server: " + entry.getValue());
|
||||
bunchServersAssigned.add(entry.getValue());
|
||||
}
|
||||
Map<HRegionInfo, ServerName> regionAssignments = am.getRegionStates().getRegionAssignments();
|
||||
for (Entry<HRegionInfo, ServerName> entry : regionAssignments.entrySet()) {
|
||||
LOG.info("Region Assignment: "
|
||||
+ entry.getKey().getRegionNameAsString() + " Server: " + entry.getValue());
|
||||
bunchServersAssigned.add(entry.getValue());
|
||||
}
|
||||
|
||||
for (ServerName sn : drainedServers) {
|
||||
assertFalse(bunchServersAssigned.contains(sn));
|
||||
}
|
||||
for (ServerName sn : drainedServers) {
|
||||
assertFalse(bunchServersAssigned.contains(sn));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void addServerToDrainedList(ServerName serverName,
|
||||
|
|
|
@ -42,8 +42,8 @@ public class TestFSTableDescriptorForceCreation {
|
|||
Path rootdir = new Path(UTIL.getDataTestDir(), name);
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
|
||||
assertTrue("Should create new table descriptor", fstd.createTableDescriptor(htd, false));
|
||||
assertTrue("Should create new table descriptor",
|
||||
fstd.createTableDescriptor(new TableDescriptor(htd), false));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -56,7 +56,8 @@ public class TestFSTableDescriptorForceCreation {
|
|||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor htd = new HTableDescriptor(name);
|
||||
fstd.add(htd);
|
||||
assertFalse("Should not create new table descriptor", fstd.createTableDescriptor(htd, false));
|
||||
assertFalse("Should not create new table descriptor",
|
||||
fstd.createTableDescriptor(new TableDescriptor(htd), false));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -67,9 +68,10 @@ public class TestFSTableDescriptorForceCreation {
|
|||
Path rootdir = new Path(UTIL.getDataTestDir(), name);
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
fstd.createTableDescriptor(htd, false);
|
||||
TableDescriptor td = new TableDescriptor(htd);
|
||||
fstd.createTableDescriptor(td, false);
|
||||
assertTrue("Should create new table descriptor",
|
||||
fstd.createTableDescriptor(htd, true));
|
||||
fstd.createTableDescriptor(td, true));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -160,8 +160,8 @@ public class TestHColumnDescriptorDefaultVersions {
|
|||
// Verify descriptor from HDFS
|
||||
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
|
||||
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
|
||||
htd = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
hcds = htd.getColumnFamilies();
|
||||
TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
hcds = td.getHTableDescriptor().getColumnFamilies();
|
||||
verifyHColumnDescriptor(expected, hcds, tableName, families);
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -49,11 +50,8 @@ import org.apache.hadoop.hbase.TableNotEnabledException;
|
|||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.Waiter;
|
||||
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -255,7 +253,7 @@ public class TestAdmin1 {
|
|||
this.admin.disableTable(ht.getName());
|
||||
assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
|
||||
.getMaster().getAssignmentManager().getTableStateManager().isTableState(
|
||||
ht.getName(), ZooKeeperProtos.Table.State.DISABLED));
|
||||
ht.getName(), TableState.State.DISABLED));
|
||||
|
||||
// Test that table is disabled
|
||||
get = new Get(row);
|
||||
|
@ -282,7 +280,7 @@ public class TestAdmin1 {
|
|||
this.admin.enableTable(table);
|
||||
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
|
||||
.getMaster().getAssignmentManager().getTableStateManager().isTableState(
|
||||
ht.getName(), ZooKeeperProtos.Table.State.ENABLED));
|
||||
ht.getName(), TableState.State.ENABLED));
|
||||
|
||||
// Test that table is enabled
|
||||
try {
|
||||
|
@ -354,7 +352,7 @@ public class TestAdmin1 {
|
|||
assertEquals(numTables + 1, tables.length);
|
||||
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
|
||||
.getMaster().getAssignmentManager().getTableStateManager().isTableState(
|
||||
TableName.valueOf("testCreateTable"), ZooKeeperProtos.Table.State.ENABLED));
|
||||
TableName.valueOf("testCreateTable"), TableState.State.ENABLED));
|
||||
}
|
||||
|
||||
@Test (timeout=300000)
|
||||
|
@ -1340,11 +1338,9 @@ public class TestAdmin1 {
|
|||
|
||||
@Test (timeout=300000)
|
||||
public void testEnableDisableAddColumnDeleteColumn() throws Exception {
|
||||
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
|
||||
TableName tableName = TableName.valueOf("testEnableDisableAddColumnDeleteColumn");
|
||||
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
|
||||
while (!ZKTableStateClientSideReader.isEnabledTable(zkw,
|
||||
TableName.valueOf("testEnableDisableAddColumnDeleteColumn"))) {
|
||||
while (!this.admin.isTableEnabled(tableName)) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
this.admin.disableTable(tableName);
|
||||
|
@ -1487,16 +1483,4 @@ public class TestAdmin1 {
|
|||
this.admin.deleteTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
@Test (timeout=30000)
|
||||
public void testTableNotFoundException() throws Exception {
|
||||
ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||
TableName table = TableName.valueOf("tableNotExists");
|
||||
try {
|
||||
ZKTableStateClientSideReader.isDisabledTable(zkw, table);
|
||||
fail("Shouldn't be here");
|
||||
} catch (TableNotFoundException e) {
|
||||
// This is expected.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -70,6 +70,11 @@ public class MockNoopMasterServices implements MasterServices, Server {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableStateManager getTableStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MasterCoprocessorHost getMasterCoprocessorHost() {
|
||||
return null;
|
||||
|
|
|
@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.client.HTable;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
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.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
|
@ -69,7 +70,6 @@ import org.apache.hadoop.hbase.executor.EventType;
|
|||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
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.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -156,10 +156,9 @@ public class TestAssignmentManagerOnCluster {
|
|||
Bytes.toBytes(metaServerName.getServerName()));
|
||||
master.assignmentManager.waitUntilNoRegionsInTransition(60000);
|
||||
}
|
||||
RegionState metaState =
|
||||
MetaTableLocator.getMetaRegionState(master.getZooKeeper());
|
||||
assertEquals("Meta should be not in transition",
|
||||
metaState.getState(), RegionState.State.OPEN);
|
||||
RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper());
|
||||
assertEquals("Meta should be not in transition",
|
||||
metaState.getState(), RegionState.State.OPEN);
|
||||
assertNotEquals("Meta should be moved off master",
|
||||
metaServerName, master.getServerName());
|
||||
cluster.killRegionServer(metaServerName);
|
||||
|
@ -289,7 +288,8 @@ public class TestAssignmentManagerOnCluster {
|
|||
String table = "testAssignRegionOnRestartedServer";
|
||||
TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 20);
|
||||
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;
|
||||
HMaster master = null;
|
||||
|
@ -888,7 +888,7 @@ public class TestAssignmentManagerOnCluster {
|
|||
}
|
||||
}
|
||||
|
||||
am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLING);
|
||||
am.getTableStateManager().setTableState(table, TableState.State.DISABLING);
|
||||
List<HRegionInfo> toAssignRegions = am.cleanOutCrashedServerReferences(destServerName);
|
||||
assertTrue("Regions to be assigned should be empty.", toAssignRegions.isEmpty());
|
||||
assertTrue("Regions to be assigned should be empty.", am.getRegionStates()
|
||||
|
@ -897,7 +897,7 @@ public class TestAssignmentManagerOnCluster {
|
|||
if (hri != null && serverName != null) {
|
||||
am.regionOnline(hri, serverName);
|
||||
}
|
||||
am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLED);
|
||||
am.getTableStateManager().setTableState(table, TableState.State.DISABLED);
|
||||
TEST_UTIL.deleteTable(table);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,6 +41,8 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ChoreService;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -54,13 +56,13 @@ import org.apache.hadoop.hbase.Server;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
|
||||
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.io.Reference;
|
||||
import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
|
||||
|
@ -352,13 +354,18 @@ public class TestCatalogJanitor {
|
|||
return new TableDescriptors() {
|
||||
@Override
|
||||
public HTableDescriptor remove(TableName tablename) throws IOException {
|
||||
// TODO Auto-generated method stub
|
||||
// noop
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
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;
|
||||
}
|
||||
|
||||
|
@ -368,6 +375,12 @@ public class TestCatalogJanitor {
|
|||
return createHTableDescriptor();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableDescriptor getDescriptor(TableName tablename)
|
||||
throws IOException {
|
||||
return createTableDescriptor();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException {
|
||||
return null;
|
||||
|
@ -375,8 +388,12 @@ public class TestCatalogJanitor {
|
|||
|
||||
@Override
|
||||
public void add(HTableDescriptor htd) throws IOException {
|
||||
// TODO Auto-generated method stub
|
||||
// noop
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(TableDescriptor htd) throws IOException {
|
||||
// noop
|
||||
}
|
||||
@Override
|
||||
public void setCacheOn() throws IOException {
|
||||
|
@ -540,6 +557,11 @@ public class TestCatalogJanitor {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableStateManager getTableStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
|
||||
boolean forcible, User user) throws IOException {
|
||||
|
@ -1169,6 +1191,11 @@ public class TestCatalogJanitor {
|
|||
return htd;
|
||||
}
|
||||
|
||||
private TableDescriptor createTableDescriptor() {
|
||||
TableDescriptor htd = new TableDescriptor(createHTableDescriptor(), TableState.State.ENABLED);
|
||||
return htd;
|
||||
}
|
||||
|
||||
private MultiResponse buildMultiResponse(MultiRequest req) {
|
||||
MultiResponse.Builder builder = MultiResponse.newBuilder();
|
||||
RegionActionResult.Builder regionActionResultBuilder =
|
||||
|
|
|
@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
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.Pair;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
@ -84,7 +84,7 @@ public class TestMaster {
|
|||
|
||||
try (HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME)) {
|
||||
assertTrue(m.assignmentManager.getTableStateManager().isTableState(TABLENAME,
|
||||
ZooKeeperProtos.Table.State.ENABLED));
|
||||
TableState.State.ENABLED));
|
||||
TEST_UTIL.loadTable(ht, FAMILYNAME, false);
|
||||
}
|
||||
|
||||
|
|
|
@ -43,20 +43,19 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.RegionTransition;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
|
@ -71,10 +70,8 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
|||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
|
@ -302,8 +299,8 @@ public class TestMasterFailover {
|
|||
log("Beginning to mock scenarios");
|
||||
|
||||
// Disable the disabledTable in ZK
|
||||
TableStateManager zktable = new ZKTableStateManager(zkw);
|
||||
zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED);
|
||||
TableStateManager tsm = master.getTableStateManager();
|
||||
tsm.setTableState(disabledTable, TableState.State.DISABLED);
|
||||
|
||||
/*
|
||||
* ZK = OFFLINE
|
||||
|
@ -619,7 +616,7 @@ public class TestMasterFailover {
|
|||
|
||||
assertTrue(" Table must be enabled.", master.getAssignmentManager()
|
||||
.getTableStateManager().isTableState(TableName.valueOf("enabledTable"),
|
||||
ZooKeeperProtos.Table.State.ENABLED));
|
||||
TableState.State.ENABLED));
|
||||
// we also need regions assigned out on the dead server
|
||||
List<HRegionInfo> enabledAndOnDeadRegions = new ArrayList<HRegionInfo>();
|
||||
enabledAndOnDeadRegions.addAll(enabledRegions.subList(0, 6));
|
||||
|
@ -679,13 +676,11 @@ public class TestMasterFailover {
|
|||
log("Beginning to mock scenarios");
|
||||
|
||||
// Disable the disabledTable in ZK
|
||||
TableStateManager zktable = new ZKTableStateManager(zkw);
|
||||
zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED);
|
||||
TableStateManager tsm = master.getTableStateManager();
|
||||
tsm.setTableState(disabledTable, TableState.State.DISABLED);
|
||||
|
||||
assertTrue(" The enabled table should be identified on master fail over.",
|
||||
zktable.isTableState(TableName.valueOf("enabledTable"),
|
||||
ZooKeeperProtos.Table.State.ENABLED));
|
||||
|
||||
tsm.isTableState(TableName.valueOf("enabledTable"), TableState.State.ENABLED));
|
||||
/*
|
||||
* ZK = CLOSING
|
||||
*/
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.client.Admin;
|
|||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
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.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
|
||||
|
@ -102,8 +102,8 @@ public class TestMasterRestartAfterDisablingTable {
|
|||
|
||||
assertTrue("The table should not be in enabled state", cluster.getMaster()
|
||||
.getAssignmentManager().getTableStateManager().isTableState(
|
||||
TableName.valueOf("tableRestart"), ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING));
|
||||
TableName.valueOf("tableRestart"), TableState.State.DISABLED,
|
||||
TableState.State.DISABLING));
|
||||
log("Enabling table\n");
|
||||
// Need a new Admin, the previous one is on the old master
|
||||
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
|
||||
|
@ -118,7 +118,7 @@ public class TestMasterRestartAfterDisablingTable {
|
|||
6, regions.size());
|
||||
assertTrue("The table should be in enabled state", cluster.getMaster()
|
||||
.getAssignmentManager().getTableStateManager()
|
||||
.isTableState(TableName.valueOf("tableRestart"), ZooKeeperProtos.Table.State.ENABLED));
|
||||
.isTableState(TableName.valueOf("tableRestart"), TableState.State.ENABLED));
|
||||
ht.close();
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
|
|
@ -42,9 +42,9 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
|||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.MockServer;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
@ -140,7 +140,10 @@ public class TestOpenedRegionHandler {
|
|||
// create a node with OPENED state
|
||||
zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
|
||||
region, server.getServerName());
|
||||
when(am.getTableStateManager()).thenReturn(new ZKTableStateManager(zkw));
|
||||
MasterServices masterServices = Mockito.mock(MasterServices.class);
|
||||
when(masterServices.getTableDescriptors()).thenReturn(new FSTableDescriptors(conf));
|
||||
TableStateManager tsm = new TableStateManager(masterServices);
|
||||
when(am.getTableStateManager()).thenReturn(tsm);
|
||||
Stat stat = new Stat();
|
||||
String nodeName = ZKAssign.getNodeName(zkw, region.getRegionInfo()
|
||||
.getEncodedName());
|
||||
|
@ -171,8 +174,8 @@ public class TestOpenedRegionHandler {
|
|||
} catch (Exception e) {
|
||||
expectedException = true;
|
||||
}
|
||||
assertFalse("The process method should not throw any exception.",
|
||||
expectedException);
|
||||
assertFalse("The process method should not throw any exception. "
|
||||
, expectedException);
|
||||
List<String> znodes = ZKUtil.listChildrenAndWatchForNewChildren(zkw,
|
||||
zkw.assignmentZNode);
|
||||
String regionName = znodes.get(0);
|
||||
|
|
|
@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.master;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
|
|
@ -36,7 +36,6 @@ import java.util.concurrent.Future;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ChoreService;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -53,7 +52,6 @@ import org.apache.hadoop.hbase.client.Admin;
|
|||
import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.exceptions.LockTimeoutException;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -388,12 +386,14 @@ public class TestTableLockManager {
|
|||
choreService.scheduleChore(alterThread);
|
||||
choreService.scheduleChore(splitThread);
|
||||
TEST_UTIL.waitTableEnabled(tableName);
|
||||
|
||||
while (true) {
|
||||
List<HRegionInfo> regions = admin.getTableRegions(tableName);
|
||||
LOG.info(String.format("Table #regions: %d regions: %s:", regions.size(), regions));
|
||||
assertEquals(admin.getTableDescriptor(tableName), desc);
|
||||
for (HRegion region : TEST_UTIL.getMiniHBaseCluster().getRegions(tableName)) {
|
||||
assertEquals(desc, region.getTableDesc());
|
||||
HTableDescriptor regionTableDesc = region.getTableDesc();
|
||||
assertEquals(desc, regionTableDesc);
|
||||
}
|
||||
if (regions.size() >= 5) {
|
||||
break;
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
|
|||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
|
@ -45,9 +45,9 @@ import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
|||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.TableStateManager;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
@ -188,13 +188,13 @@ public class MasterProcedureTestingUtility {
|
|||
public static void validateTableIsEnabled(final HMaster master, final TableName tableName)
|
||||
throws IOException {
|
||||
TableStateManager tsm = master.getAssignmentManager().getTableStateManager();
|
||||
assertTrue(tsm.isTableState(tableName, ZooKeeperProtos.Table.State.ENABLED));
|
||||
assertTrue(tsm.isTableState(tableName, TableState.State.ENABLED));
|
||||
}
|
||||
|
||||
public static void validateTableIsDisabled(final HMaster master, final TableName tableName)
|
||||
throws IOException {
|
||||
TableStateManager tsm = master.getAssignmentManager().getTableStateManager();
|
||||
assertTrue(tsm.isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED));
|
||||
assertTrue(tsm.isTableState(tableName, TableState.State.DISABLED));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -23,10 +23,10 @@ import static org.junit.Assert.assertTrue;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -45,6 +45,8 @@ public class TestCreateTableProcedure2 {
|
|||
TEST_UTIL.shutdownMiniZKCluster();
|
||||
}
|
||||
|
||||
/*
|
||||
Note: Relevant fix was undone by HBASE-7767.
|
||||
@Test
|
||||
public void testMasterRestartAfterNameSpaceEnablingNodeIsCreated() throws Exception {
|
||||
// Step 1: start mini zk cluster.
|
||||
|
@ -54,8 +56,9 @@ public class TestCreateTableProcedure2 {
|
|||
TableName tableName = TableName.valueOf("hbase:namespace");
|
||||
ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||
String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
|
||||
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
|
||||
builder.setState(ZooKeeperProtos.Table.State.ENABLED);
|
||||
HBaseProtos.TableState.Builder builder = HBaseProtos.TableState.newBuilder();
|
||||
builder.setState(HBaseProtos.TableState.State.ENABLED);
|
||||
builder.setTable(ProtobufUtil.toProtoTableName(tableName));
|
||||
byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
|
||||
ZKUtil.createSetData(zkw, znode, data);
|
||||
LOG.info("Create an orphaned Znode " + znode);
|
||||
|
@ -65,4 +68,5 @@ public class TestCreateTableProcedure2 {
|
|||
TEST_UTIL.startMiniCluster();
|
||||
assertTrue(TEST_UTIL.getHBaseCluster().getLiveMasterThreads().size() == 1);
|
||||
}
|
||||
*/
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
|
@ -273,8 +274,9 @@ public class TestTableDescriptorModificationFromClient {
|
|||
// Verify descriptor from HDFS
|
||||
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
|
||||
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
|
||||
htd = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
verifyTableDescriptor(htd, tableName, families);
|
||||
TableDescriptor td =
|
||||
FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
verifyTableDescriptor(td.getHTableDescriptor(), tableName, families);
|
||||
}
|
||||
|
||||
private void verifyTableDescriptor(final HTableDescriptor htd,
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -501,8 +502,8 @@ public final class SnapshotTestingUtils {
|
|||
this.desc = desc;
|
||||
this.tableRegions = tableRegions;
|
||||
this.snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
|
||||
new FSTableDescriptors(conf, snapshotDir.getFileSystem(conf), rootDir)
|
||||
.createTableDescriptorForTableDirectory(snapshotDir, htd, false);
|
||||
new FSTableDescriptors(conf).createTableDescriptorForTableDirectory(snapshotDir,
|
||||
new TableDescriptor(htd), false);
|
||||
}
|
||||
|
||||
public HTableDescriptor getTableDescriptor() {
|
||||
|
@ -719,7 +720,8 @@ public final class SnapshotTestingUtils {
|
|||
private RegionData[] createTable(final HTableDescriptor htd, final int nregions)
|
||||
throws IOException {
|
||||
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);
|
||||
RegionData[] regions = new RegionData[nregions];
|
||||
|
|
|
@ -35,14 +35,16 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
|
@ -71,6 +73,7 @@ public class TestFSTableDescriptors {
|
|||
public void testCreateAndUpdate() throws IOException {
|
||||
Path testdir = UTIL.getDataTestDir("testCreateAndUpdate");
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCreate"));
|
||||
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
|
||||
assertTrue(fstd.createTableDescriptor(htd));
|
||||
|
@ -78,7 +81,7 @@ public class TestFSTableDescriptors {
|
|||
FileStatus[] statuses = fs.listStatus(testdir);
|
||||
assertTrue("statuses.length=" + statuses.length, statuses.length == 1);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
fstd.updateTableDescriptor(htd);
|
||||
fstd.updateTableDescriptor(td);
|
||||
}
|
||||
statuses = fs.listStatus(testdir);
|
||||
assertTrue(statuses.length == 1);
|
||||
|
@ -92,20 +95,29 @@ public class TestFSTableDescriptors {
|
|||
Path testdir = UTIL.getDataTestDir("testSequenceidAdvancesOnTableInfo");
|
||||
HTableDescriptor htd = new HTableDescriptor(
|
||||
TableName.valueOf("testSequenceidAdvancesOnTableInfo"));
|
||||
TableDescriptor td = new TableDescriptor(htd);
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
|
||||
Path p0 = fstd.updateTableDescriptor(htd);
|
||||
Path p0 = fstd.updateTableDescriptor(td);
|
||||
int i0 = FSTableDescriptors.getTableInfoSequenceId(p0);
|
||||
Path p1 = fstd.updateTableDescriptor(htd);
|
||||
Path p1 = fstd.updateTableDescriptor(td);
|
||||
// Assert we cleaned up the old file.
|
||||
assertTrue(!fs.exists(p0));
|
||||
int i1 = FSTableDescriptors.getTableInfoSequenceId(p1);
|
||||
assertTrue(i1 == i0 + 1);
|
||||
Path p2 = fstd.updateTableDescriptor(htd);
|
||||
Path p2 = fstd.updateTableDescriptor(td);
|
||||
// Assert we cleaned up the old file.
|
||||
assertTrue(!fs.exists(p1));
|
||||
int i2 = FSTableDescriptors.getTableInfoSequenceId(p2);
|
||||
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
|
||||
|
@ -158,12 +170,13 @@ public class TestFSTableDescriptors {
|
|||
final String name = "testReadingHTDFromFS";
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
|
||||
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
|
||||
Path rootdir = UTIL.getDataTestDir(name);
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
|
||||
fstd.createTableDescriptor(htd);
|
||||
HTableDescriptor htd2 =
|
||||
TableDescriptor td2 =
|
||||
FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName());
|
||||
assertTrue(htd.equals(htd2));
|
||||
assertTrue(td.equals(td2));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -177,7 +190,8 @@ public class TestFSTableDescriptors {
|
|||
final int count = 10;
|
||||
// Write out table infos.
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -191,7 +205,7 @@ public class TestFSTableDescriptors {
|
|||
for (int i = 0; i < count; i++) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + 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.
|
||||
Thread.sleep(100);
|
||||
|
@ -232,7 +246,7 @@ public class TestFSTableDescriptors {
|
|||
for (int i = 0; i < count; i++) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + 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.
|
||||
Thread.sleep(100);
|
||||
|
@ -378,18 +392,19 @@ public class TestFSTableDescriptors {
|
|||
Path testdir = UTIL.getDataTestDir("testCreateTableDescriptorUpdatesIfThereExistsAlready");
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(
|
||||
"testCreateTableDescriptorUpdatesIfThereExistsAlready"));
|
||||
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
|
||||
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
|
||||
assertTrue(fstd.createTableDescriptor(htd));
|
||||
assertFalse(fstd.createTableDescriptor(htd));
|
||||
htd.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue"));
|
||||
assertTrue(fstd.createTableDescriptor(htd)); //this will re-create
|
||||
assertTrue(fstd.createTableDescriptor(td)); //this will re-create
|
||||
Path tableDir = fstd.getTableDir(htd.getTableName());
|
||||
Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR);
|
||||
FileStatus[] statuses = fs.listStatus(tmpTableDir);
|
||||
assertTrue(statuses.length == 0);
|
||||
|
||||
assertEquals(htd, FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir));
|
||||
assertEquals(td, FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir));
|
||||
}
|
||||
|
||||
private static class FSTableDescriptorsTest
|
||||
|
|
|
@ -68,7 +68,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Waiter.Predicate;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
|
@ -106,7 +105,6 @@ import org.apache.hadoop.hbase.master.TableLockManager;
|
|||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
|
@ -2891,55 +2889,6 @@ public class TestHBaseFsck {
|
|||
tableLockManager.tableDeleted(tableName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test orphaned table ZNode (for table states)
|
||||
*/
|
||||
@Test
|
||||
public void testOrphanedTableZNode() throws Exception {
|
||||
TableName table = TableName.valueOf("testOrphanedZKTableEntry");
|
||||
|
||||
try {
|
||||
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getTableStateManager()
|
||||
.setTableState(table, ZooKeeperProtos.Table.State.ENABLING);
|
||||
|
||||
try {
|
||||
setupTable(table);
|
||||
Assert.fail(
|
||||
"Create table should fail when its ZNode has already existed with ENABLING state.");
|
||||
} catch(TableExistsException t) {
|
||||
//Expected exception
|
||||
}
|
||||
// The setup table was interrupted in some state that needs to some cleanup.
|
||||
try {
|
||||
cleanupTable(table);
|
||||
} catch (IOException e) {
|
||||
// Because create table failed, it is expected that the cleanup table would
|
||||
// throw some exception. Ignore and continue.
|
||||
}
|
||||
|
||||
HBaseFsck hbck = doFsck(conf, false);
|
||||
assertTrue(hbck.getErrors().getErrorList().contains(ERROR_CODE.ORPHANED_ZK_TABLE_ENTRY));
|
||||
|
||||
// fix the orphaned ZK entry
|
||||
hbck = doFsck(conf, true);
|
||||
|
||||
// check that orpahned ZK table entry is gone.
|
||||
hbck = doFsck(conf, false);
|
||||
assertFalse(hbck.getErrors().getErrorList().contains(ERROR_CODE.ORPHANED_ZK_TABLE_ENTRY));
|
||||
// Now create table should succeed.
|
||||
setupTable(table);
|
||||
} finally {
|
||||
// This code could be called that either a table was created successfully or set up
|
||||
// table failed in some unknown state. Therefore, clean up can either succeed or fail.
|
||||
try {
|
||||
cleanupTable(table);
|
||||
} catch (IOException e) {
|
||||
// The cleanup table would throw some exception if create table failed in some state.
|
||||
// Ignore this exception
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test (timeout=180000)
|
||||
public void testMetaOffline() throws Exception {
|
||||
// check no errors
|
||||
|
|
|
@ -1,114 +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.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
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(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));
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue