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:
stack 2014-09-15 09:34:10 -07:00 committed by Bharath Vissapragada
parent 18200b09e2
commit cb756629b0
No known key found for this signature in database
GPG Key ID: 18AE42A0B5A93FA7
83 changed files with 4756 additions and 2035 deletions

View File

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

View File

@ -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.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest; 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.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.GetProcedureResultRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
@ -1006,7 +1008,7 @@ class ConnectionManager {
@Override @Override
public boolean isTableEnabled(TableName tableName) throws IOException { public boolean isTableEnabled(TableName tableName) throws IOException {
return this.registry.isTableOnlineState(tableName, true); return getTableState(tableName).inStates(TableState.State.ENABLED);
} }
@Override @Override
@ -1016,7 +1018,7 @@ class ConnectionManager {
@Override @Override
public boolean isTableDisabled(TableName tableName) throws IOException { public boolean isTableDisabled(TableName tableName) throws IOException {
return this.registry.isTableOnlineState(tableName, false); return getTableState(tableName).inStates(TableState.State.DISABLED);
} }
@Override @Override
@ -2173,6 +2175,13 @@ class ConnectionManager {
return stub.listTableNamesByNamespace(controller, request); return stub.listTableNamesByNamespace(controller, request);
} }
@Override
public GetTableStateResponse getTableState(
RpcController controller, GetTableStateRequest request)
throws ServiceException {
return stub.getTableState(controller, request);
}
@Override @Override
public void close() { public void close() {
release(this.mss); release(this.mss);
@ -2800,6 +2809,19 @@ class ConnectionManager {
public RpcControllerFactory getRpcControllerFactory() { public RpcControllerFactory getRpcControllerFactory() {
return this.rpcControllerFactory; 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();
}
}
} }
/** /**

View File

@ -212,6 +212,13 @@ public interface HConnection extends Connection {
@Deprecated @Deprecated
boolean isTableDisabled(byte[] tableName) throws IOException; boolean isTableDisabled(byte[] tableName) throws IOException;
/**
* Retrieve TableState, represent current table state.
* @param tableName table state for
* @return state of the table
*/
public TableState getTableState(TableName tableName) throws IOException;
/** /**
* @param tableName table name * @param tableName table name
* @return true if all regions of the table are available, false otherwise * @return true if all regions of the table are available, false otherwise

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
/** /**
* Cluster registry. * Cluster registry.
@ -46,12 +45,6 @@ interface Registry {
*/ */
String getClusterId(); String getClusterId();
/**
* @param enabled Return true if table is enabled
* @throws IOException
*/
boolean isTableOnlineState(TableName tableName, boolean enabled) throws IOException;
/** /**
* @return Count of 'running' regionservers * @return Count of 'running' regionservers
* @throws IOException * @throws IOException

View File

@ -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;
}
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.List; import java.util.List;
import org.apache.commons.logging.Log; 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.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -116,24 +113,6 @@ class ZooKeeperRegistry implements Registry {
return this.clusterId; return this.clusterId;
} }
@Override
public boolean isTableOnlineState(TableName tableName, boolean enabled)
throws IOException {
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
try {
if (enabled) {
return ZKTableStateClientSideReader.isEnabledTable(zkw, tableName);
}
return ZKTableStateClientSideReader.isDisabledTable(zkw, tableName);
} catch (KeeperException e) {
throw new IOException("Enable/Disable failed", e);
} catch (InterruptedException e) {
throw new InterruptedIOException();
} finally {
zkw.close();
}
}
@Override @Override
public int getCurrentNrHRS() throws IOException { public int getCurrentNrHRS() throws IOException {
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher(); ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();

View File

@ -22,6 +22,8 @@ import java.util.List;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.CellScannable;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HColumnDescriptor; 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.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; 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.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.IsBalancerEnabledRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest; 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.TruncateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest; 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.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
@ -1411,6 +1413,18 @@ public final class RequestConverter {
return builder.build(); return builder.build();
} }
/*
* Creates a protocol buffer GetTableStateRequest
*
* @param tableName table to get request for
* @return a GetTableStateRequest
*/
public static GetTableStateRequest buildGetTableStateRequest(final TableName tableName) {
return GetTableStateRequest.newBuilder()
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.build();
}
/** /**
* Creates a protocol buffer GetTableDescriptorsRequest for a single table * Creates a protocol buffer GetTableDescriptorsRequest for a single table
* *

View File

@ -1,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);
}
}
}

View File

@ -138,6 +138,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
// znode used for region transitioning and assignment // znode used for region transitioning and assignment
public String assignmentZNode; public String assignmentZNode;
// znode used for table disabling/enabling // znode used for table disabling/enabling
@Deprecated
public String tableZNode; public String tableZNode;
// znode containing the unique cluster ID // znode containing the unique cluster ID
public String clusterIdZNode; public String clusterIdZNode;

View File

@ -474,11 +474,6 @@ public class TestAsyncProcess {
return "testClusterId"; return "testClusterId";
} }
@Override
public boolean isTableOnlineState(TableName tableName, boolean enabled) throws IOException {
return false;
}
@Override @Override
public int getCurrentNrHRS() throws IOException { public int getCurrentNrHRS() throws IOException {
return 1; return 1;

View File

@ -129,12 +129,6 @@ public class TestClientNoCluster extends Configured implements Tool {
return HConstants.CLUSTER_ID_DEFAULT; return HConstants.CLUSTER_ID_DEFAULT;
} }
@Override
public boolean isTableOnlineState(TableName tableName, boolean enabled)
throws IOException {
return enabled;
}
@Override @Override
public int getCurrentNrHRS() throws IOException { public int getCurrentNrHRS() throws IOException {
return 1; return 1;

View File

@ -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
}
}
}

View File

@ -4419,12 +4419,12 @@ public final class ZooKeeperProtos {
// @@protoc_insertion_point(class_scope:hbase.pb.SplitLogTask) // @@protoc_insertion_point(class_scope:hbase.pb.SplitLogTask)
} }
public interface TableOrBuilder public interface DeprecatedTableStateOrBuilder
extends com.google.protobuf.MessageOrBuilder { 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> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -4434,7 +4434,7 @@ public final class ZooKeeperProtos {
*/ */
boolean hasState(); 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> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -4442,32 +4442,33 @@ public final class ZooKeeperProtos {
* for more. * for more.
* </pre> * </pre>
*/ */
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState(); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State getState();
} }
/** /**
* Protobuf type {@code hbase.pb.Table} * Protobuf type {@code hbase.pb.DeprecatedTableState}
* *
* <pre> * <pre>
** **
* The znode that holds state of table. * The znode that holds state of table.
* Deprected, table state is stored in table descriptor on HDFS.
* </pre> * </pre>
*/ */
public static final class Table extends public static final class DeprecatedTableState extends
com.google.protobuf.GeneratedMessage com.google.protobuf.GeneratedMessage
implements TableOrBuilder { implements DeprecatedTableStateOrBuilder {
// Use Table.newBuilder() to construct. // Use DeprecatedTableState.newBuilder() to construct.
private Table(com.google.protobuf.GeneratedMessage.Builder<?> builder) { private DeprecatedTableState(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
super(builder); super(builder);
this.unknownFields = builder.getUnknownFields(); this.unknownFields = builder.getUnknownFields();
} }
private Table(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } private DeprecatedTableState(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
private static final Table defaultInstance; private static final DeprecatedTableState defaultInstance;
public static Table getDefaultInstance() { public static DeprecatedTableState getDefaultInstance() {
return defaultInstance; return defaultInstance;
} }
public Table getDefaultInstanceForType() { public DeprecatedTableState getDefaultInstanceForType() {
return defaultInstance; return defaultInstance;
} }
@ -4477,7 +4478,7 @@ public final class ZooKeeperProtos {
getUnknownFields() { getUnknownFields() {
return this.unknownFields; return this.unknownFields;
} }
private Table( private DeprecatedTableState(
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
@ -4502,7 +4503,7 @@ public final class ZooKeeperProtos {
} }
case 8: { case 8: {
int rawValue = input.readEnum(); int rawValue = input.readEnum();
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State value = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.valueOf(rawValue); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State value = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.valueOf(rawValue);
if (value == null) { if (value == null) {
unknownFields.mergeVarintField(1, rawValue); unknownFields.mergeVarintField(1, rawValue);
} else { } else {
@ -4525,33 +4526,33 @@ public final class ZooKeeperProtos {
} }
public static final com.google.protobuf.Descriptors.Descriptor public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() { getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Table_descriptor; return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_descriptor;
} }
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() { internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Table_fieldAccessorTable return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable
.ensureFieldAccessorsInitialized( .ensureFieldAccessorsInitialized(
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder.class); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.Builder.class);
} }
public static com.google.protobuf.Parser<Table> PARSER = public static com.google.protobuf.Parser<DeprecatedTableState> PARSER =
new com.google.protobuf.AbstractParser<Table>() { new com.google.protobuf.AbstractParser<DeprecatedTableState>() {
public Table parsePartialFrom( public DeprecatedTableState parsePartialFrom(
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return new Table(input, extensionRegistry); return new DeprecatedTableState(input, extensionRegistry);
} }
}; };
@java.lang.Override @java.lang.Override
public com.google.protobuf.Parser<Table> getParserForType() { public com.google.protobuf.Parser<DeprecatedTableState> getParserForType() {
return PARSER; return PARSER;
} }
/** /**
* Protobuf enum {@code hbase.pb.Table.State} * Protobuf enum {@code hbase.pb.DeprecatedTableState.State}
* *
* <pre> * <pre>
* Table's current state * Table's current state
@ -4629,7 +4630,7 @@ public final class ZooKeeperProtos {
} }
public static final com.google.protobuf.Descriptors.EnumDescriptor public static final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptor() { getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDescriptor().getEnumTypes().get(0); return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.getDescriptor().getEnumTypes().get(0);
} }
private static final State[] VALUES = values(); private static final State[] VALUES = values();
@ -4651,15 +4652,15 @@ public final class ZooKeeperProtos {
this.value = value; this.value = value;
} }
// @@protoc_insertion_point(enum_scope:hbase.pb.Table.State) // @@protoc_insertion_point(enum_scope:hbase.pb.DeprecatedTableState.State)
} }
private int bitField0_; 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; 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> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -4671,7 +4672,7 @@ public final class ZooKeeperProtos {
return ((bitField0_ & 0x00000001) == 0x00000001); 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> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -4679,12 +4680,12 @@ public final class ZooKeeperProtos {
* for more. * for more.
* </pre> * </pre>
*/ */
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State getState() {
return state_; return state_;
} }
private void initFields() { private void initFields() {
state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED; state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.ENABLED;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -4735,10 +4736,10 @@ public final class ZooKeeperProtos {
if (obj == this) { if (obj == this) {
return true; return true;
} }
if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table)) { if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState)) {
return super.equals(obj); return super.equals(obj);
} }
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table) obj; org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState) obj;
boolean result = true; boolean result = true;
result = result && (hasState() == other.hasState()); result = result && (hasState() == other.hasState());
@ -4768,53 +4769,53 @@ public final class ZooKeeperProtos {
return hash; return hash;
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
com.google.protobuf.ByteString data) com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data); return PARSER.parseFrom(data);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
com.google.protobuf.ByteString data, com.google.protobuf.ByteString data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry); return PARSER.parseFrom(data, extensionRegistry);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom(byte[] data) public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data); return PARSER.parseFrom(data);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
byte[] data, byte[] data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry); return PARSER.parseFrom(data, extensionRegistry);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom(java.io.InputStream input) public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(java.io.InputStream input)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseFrom(input); return PARSER.parseFrom(input);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
java.io.InputStream input, java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseFrom(input, extensionRegistry); return PARSER.parseFrom(input, extensionRegistry);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseDelimitedFrom(java.io.InputStream input) public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseDelimitedFrom(input); return PARSER.parseDelimitedFrom(input);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseDelimitedFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseDelimitedFrom(
java.io.InputStream input, java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseDelimitedFrom(input, extensionRegistry); return PARSER.parseDelimitedFrom(input, extensionRegistry);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
com.google.protobuf.CodedInputStream input) com.google.protobuf.CodedInputStream input)
throws java.io.IOException { throws java.io.IOException {
return PARSER.parseFrom(input); return PARSER.parseFrom(input);
} }
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parseFrom( public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
@ -4823,7 +4824,7 @@ public final class ZooKeeperProtos {
public static Builder newBuilder() { return Builder.create(); } public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); } public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table prototype) { public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState prototype) {
return newBuilder().mergeFrom(prototype); return newBuilder().mergeFrom(prototype);
} }
public Builder toBuilder() { return newBuilder(this); } public Builder toBuilder() { return newBuilder(this); }
@ -4835,29 +4836,30 @@ public final class ZooKeeperProtos {
return builder; return builder;
} }
/** /**
* Protobuf type {@code hbase.pb.Table} * Protobuf type {@code hbase.pb.DeprecatedTableState}
* *
* <pre> * <pre>
** **
* The znode that holds state of table. * The znode that holds state of table.
* Deprected, table state is stored in table descriptor on HDFS.
* </pre> * </pre>
*/ */
public static final class Builder extends public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder> com.google.protobuf.GeneratedMessage.Builder<Builder>
implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableOrBuilder { implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableStateOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() { getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Table_descriptor; return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_descriptor;
} }
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() { internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Table_fieldAccessorTable return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable
.ensureFieldAccessorsInitialized( .ensureFieldAccessorsInitialized(
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder.class); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.Builder.class);
} }
// Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.newBuilder() // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.newBuilder()
private Builder() { private Builder() {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -4877,7 +4879,7 @@ public final class ZooKeeperProtos {
public Builder clear() { public Builder clear() {
super.clear(); super.clear();
state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED; state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.ENABLED;
bitField0_ = (bitField0_ & ~0x00000001); bitField0_ = (bitField0_ & ~0x00000001);
return this; return this;
} }
@ -4888,23 +4890,23 @@ public final class ZooKeeperProtos {
public com.google.protobuf.Descriptors.Descriptor public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() { getDescriptorForType() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_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() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState getDefaultInstanceForType() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.getDefaultInstance();
} }
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table build() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState build() {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table result = buildPartial(); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState result = buildPartial();
if (!result.isInitialized()) { if (!result.isInitialized()) {
throw newUninitializedMessageException(result); throw newUninitializedMessageException(result);
} }
return result; return result;
} }
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table buildPartial() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState buildPartial() {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table(this); org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState(this);
int from_bitField0_ = bitField0_; int from_bitField0_ = bitField0_;
int to_bitField0_ = 0; int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) { if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@ -4917,16 +4919,16 @@ public final class ZooKeeperProtos {
} }
public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table) { if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState) {
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table)other); return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState)other);
} else { } else {
super.mergeFrom(other); super.mergeFrom(other);
return this; return this;
} }
} }
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table other) { public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState other) {
if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance()) return this; if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.getDefaultInstance()) return this;
if (other.hasState()) { if (other.hasState()) {
setState(other.getState()); setState(other.getState());
} }
@ -4946,11 +4948,11 @@ public final class ZooKeeperProtos {
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table parsedMessage = null; org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parsedMessage = null;
try { try {
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table) e.getUnfinishedMessage(); parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState) e.getUnfinishedMessage();
throw e; throw e;
} finally { } finally {
if (parsedMessage != null) { if (parsedMessage != null) {
@ -4961,10 +4963,10 @@ public final class ZooKeeperProtos {
} }
private int bitField0_; private int bitField0_;
// required .hbase.pb.Table.State state = 1 [default = ENABLED]; // required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];
private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED; private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.ENABLED;
/** /**
* <code>required .hbase.pb.Table.State state = 1 [default = ENABLED];</code> * <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -4976,7 +4978,7 @@ public final class ZooKeeperProtos {
return ((bitField0_ & 0x00000001) == 0x00000001); 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> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -4984,11 +4986,11 @@ public final class ZooKeeperProtos {
* for more. * for more.
* </pre> * </pre>
*/ */
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState() { public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State getState() {
return state_; return state_;
} }
/** /**
* <code>required .hbase.pb.Table.State state = 1 [default = ENABLED];</code> * <code>required .hbase.pb.DeprecatedTableState.State state = 1 [default = ENABLED];</code>
* *
* <pre> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -4996,7 +4998,7 @@ public final class ZooKeeperProtos {
* for more. * for more.
* </pre> * </pre>
*/ */
public Builder setState(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State value) { public Builder setState(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State value) {
if (value == null) { if (value == null) {
throw new NullPointerException(); throw new NullPointerException();
} }
@ -5006,7 +5008,7 @@ public final class ZooKeeperProtos {
return this; 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> * <pre>
* This is the table's state. If no znode for a table, * This is the table's state. If no znode for a table,
@ -5016,20 +5018,20 @@ public final class ZooKeeperProtos {
*/ */
public Builder clearState() { public Builder clearState() {
bitField0_ = (bitField0_ & ~0x00000001); bitField0_ = (bitField0_ & ~0x00000001);
state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED; state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.State.ENABLED;
onChanged(); onChanged();
return this; return this;
} }
// @@protoc_insertion_point(builder_scope:hbase.pb.Table) // @@protoc_insertion_point(builder_scope:hbase.pb.DeprecatedTableState)
} }
static { static {
defaultInstance = new Table(true); defaultInstance = new DeprecatedTableState(true);
defaultInstance.initFields(); defaultInstance.initFields();
} }
// @@protoc_insertion_point(class_scope:hbase.pb.Table) // @@protoc_insertion_point(class_scope:hbase.pb.DeprecatedTableState)
} }
public interface TableCFOrBuilder public interface TableCFOrBuilder
@ -10934,10 +10936,10 @@ public final class ZooKeeperProtos {
com.google.protobuf.GeneratedMessage.FieldAccessorTable com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_hbase_pb_SplitLogTask_fieldAccessorTable; internal_static_hbase_pb_SplitLogTask_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor private static com.google.protobuf.Descriptors.Descriptor
internal_static_hbase_pb_Table_descriptor; internal_static_hbase_pb_DeprecatedTableState_descriptor;
private static private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_hbase_pb_Table_fieldAccessorTable; internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor private static com.google.protobuf.Descriptors.Descriptor
internal_static_hbase_pb_TableCF_descriptor; internal_static_hbase_pb_TableCF_descriptor;
private static 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" + "\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" + "\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" + "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.", "AY\020\002\"\225\001\n\024DeprecatedTableState\022<\n\005state\030\001",
"Table.State:\007ENABLED\"?\n\005State\022\013\n\007ENABLED" + " \002(\0162$.hbase.pb.DeprecatedTableState.Sta" +
"\020\000\022\014\n\010DISABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABL" + "te:\007ENABLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DIS" +
"ING\020\003\"D\n\007TableCF\022\'\n\ntable_name\030\001 \001(\0132\023.h" + "ABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"D\n\007" +
"base.pb.TableName\022\020\n\010families\030\002 \003(\014\"\330\001\n\017" + "TableCF\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.T" +
"ReplicationPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027r" + "ableName\022\020\n\010families\030\002 \003(\014\"\330\001\n\017Replicati" +
"eplicationEndpointImpl\030\002 \001(\t\022&\n\004data\030\003 \003" + "onPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027replicatio" +
"(\0132\030.hbase.pb.BytesBytesPair\022/\n\rconfigur" + "nEndpointImpl\030\002 \001(\t\022&\n\004data\030\003 \003(\0132\030.hbas" +
"ation\030\004 \003(\0132\030.hbase.pb.NameStringPair\022$\n" + "e.pb.BytesBytesPair\022/\n\rconfiguration\030\004 \003" +
"\ttable_cfs\030\005 \003(\0132\021.hbase.pb.TableCF\022\021\n\tb" + "(\0132\030.hbase.pb.NameStringPair\022$\n\ttable_cf" +
"andwidth\030\006 \001(\003\"g\n\020ReplicationState\022/\n\005st", "s\030\005 \003(\0132\021.hbase.pb.TableCF\022\021\n\tbandwidth\030",
"ate\030\001 \002(\0162 .hbase.pb.ReplicationState.St" + "\006 \001(\003\"g\n\020ReplicationState\022/\n\005state\030\001 \002(\016" +
"ate\"\"\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001\"" + "2 .hbase.pb.ReplicationState.State\"\"\n\005St" +
"+\n\027ReplicationHLogPosition\022\020\n\010position\030\001" + "ate\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001\"+\n\027Replic" +
" \002(\003\"%\n\017ReplicationLock\022\022\n\nlock_owner\030\001 " + "ationHLogPosition\022\020\n\010position\030\001 \002(\003\"%\n\017R" +
"\002(\t\"\252\001\n\tTableLock\022\'\n\ntable_name\030\001 \001(\0132\023." + "eplicationLock\022\022\n\nlock_owner\030\001 \002(\t\"\252\001\n\tT" +
"hbase.pb.TableName\022(\n\nlock_owner\030\002 \001(\0132\024" + "ableLock\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb." +
".hbase.pb.ServerName\022\021\n\tthread_id\030\003 \001(\003\022" + "TableName\022(\n\nlock_owner\030\002 \001(\0132\024.hbase.pb" +
"\021\n\tis_shared\030\004 \001(\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013c" + ".ServerName\022\021\n\tthread_id\030\003 \001(\003\022\021\n\tis_sha" +
"reate_time\030\006 \001(\003\"\036\n\013SwitchState\022\017\n\007enabl" + "red\030\004 \001(\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013create_tim" +
"ed\030\001 \001(\010BE\n*org.apache.hadoop.hbase.prot", "e\030\006 \001(\003\"\036\n\013SwitchState\022\017\n\007enabled\030\001 \001(\010B",
"obuf.generatedB\017ZooKeeperProtosH\001\210\001\001\240\001\001" "E\n*org.apache.hadoop.hbase.protobuf.gene" +
"ratedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -11059,11 +11062,11 @@ public final class ZooKeeperProtos {
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_SplitLogTask_descriptor, internal_static_hbase_pb_SplitLogTask_descriptor,
new java.lang.String[] { "State", "ServerName", "Mode", }); new java.lang.String[] { "State", "ServerName", "Mode", });
internal_static_hbase_pb_Table_descriptor = internal_static_hbase_pb_DeprecatedTableState_descriptor =
getDescriptor().getMessageTypes().get(5); getDescriptor().getMessageTypes().get(5);
internal_static_hbase_pb_Table_fieldAccessorTable = new internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_Table_descriptor, internal_static_hbase_pb_DeprecatedTableState_descriptor,
new java.lang.String[] { "State", }); new java.lang.String[] { "State", });
internal_static_hbase_pb_TableCF_descriptor = internal_static_hbase_pb_TableCF_descriptor =
getDescriptor().getMessageTypes().get(6); getDescriptor().getMessageTypes().get(6);

View File

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

View File

@ -456,6 +456,14 @@ message GetTableNamesResponse {
repeated TableName table_names = 1; repeated TableName table_names = 1;
} }
message GetTableStateRequest {
required TableName table_name = 1;
}
message GetTableStateResponse {
required TableState table_state = 1;
}
message GetClusterStatusRequest { message GetClusterStatusRequest {
} }
@ -901,7 +909,9 @@ service MasterService {
rpc IsSnapshotCleanupEnabled (IsSnapshotCleanupEnabledRequest) rpc IsSnapshotCleanupEnabled (IsSnapshotCleanupEnabledRequest)
returns (IsSnapshotCleanupEnabledResponse); returns (IsSnapshotCleanupEnabledResponse);
/** returns table state */
rpc GetTableState(GetTableStateRequest)
returns(GetTableStateResponse);
} }
/** Request and response to get the clusterID for this cluster */ /** Request and response to get the clusterID for this cluster */

View File

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

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.LoadBalancer; 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.ServerManager;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock; import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.net.Address; import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
/** /**
* Service to support Region Server Grouping (HBase-6721) * Service to support Region Server Grouping (HBase-6721)
@ -269,8 +269,8 @@ public class RSGroupAdminServer implements RSGroupAdmin {
} }
for(TableName table: tables) { for(TableName table: tables) {
if (master.getAssignmentManager().getTableStateManager().isTableState(table, if (master.getAssignmentManager().getTableStateManager().isTableState(table,
ZooKeeperProtos.Table.State.DISABLED, TableState.State.DISABLED,
ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLING)) {
LOG.debug("Skipping move regions because the table" + table + " is disabled."); LOG.debug("Skipping move regions because the table" + table + " is disabled.");
continue; continue;
} }

View File

@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
@ -61,11 +60,13 @@ import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.ServerListener; 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.CreateTableProcedure;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.net.Address; 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.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos;
import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos; 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.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -646,7 +646,7 @@ public class RSGroupInfoManagerImpl implements RSGroupInfoManager, ServerListene
if (sn == null) { if (sn == null) {
found.set(false); found.set(false);
} else if (tsm.isTableState(RSGROUP_TABLE_NAME, } else if (tsm.isTableState(RSGROUP_TABLE_NAME,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
try { try {
ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn); ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn);
ClientProtos.GetRequest request = ClientProtos.GetRequest request =
@ -670,7 +670,7 @@ public class RSGroupInfoManagerImpl implements RSGroupInfoManager, ServerListene
if (sn == null) { if (sn == null) {
nsFound.set(false); nsFound.set(false);
} else if (tsm.isTableState(TableName.NAMESPACE_TABLE_NAME, } else if (tsm.isTableState(TableName.NAMESPACE_TABLE_NAME,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
try { try {
ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn); ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn);
ClientProtos.GetRequest request = ClientProtos.GetRequest request =

View File

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

View File

@ -0,0 +1,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 +
'}';
}
}

View File

@ -36,6 +36,14 @@ public interface TableDescriptors {
HTableDescriptor get(final TableName tableName) HTableDescriptor get(final TableName tableName)
throws IOException; throws IOException;
/**
* @param tableName
* @return TableDescriptor for tablename
* @throws IOException
*/
TableDescriptor getDescriptor(final TableName tableName)
throws IOException;
/** /**
* Get Map of all NamespaceDescriptors for a given namespace. * Get Map of all NamespaceDescriptors for a given namespace.
* @return Map of all descriptors. * @return Map of all descriptors.
@ -53,6 +61,15 @@ public interface TableDescriptors {
Map<String, HTableDescriptor> getAll() Map<String, HTableDescriptor> getAll()
throws IOException; throws IOException;
/**
* Get Map of all TableDescriptors. Populates the descriptor cache as a
* side effect.
* @return Map of all descriptors.
* @throws IOException
*/
Map<String, TableDescriptor> getAllDescriptors()
throws IOException;
/** /**
* Add or update descriptor * Add or update descriptor
* @param htd Descriptor to set into TableDescriptors * @param htd Descriptor to set into TableDescriptors
@ -61,6 +78,14 @@ public interface TableDescriptors {
void add(final HTableDescriptor htd) void add(final HTableDescriptor htd)
throws IOException; throws IOException;
/**
* Add or update descriptor
* @param htd Descriptor to set into TableDescriptors
* @throws IOException
*/
void add(final TableDescriptor htd)
throws IOException;
/** /**
* @param tablename * @param tablename
* @return Instance of table descriptor or null if none found. * @return Instance of table descriptor or null if none found.

View File

@ -1,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;
}

View File

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

View File

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

View File

@ -23,11 +23,11 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; 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.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@ -309,7 +309,7 @@ public class ZkOpenRegionCoordination implements OpenRegionCoordination {
} }
if (!openedNodeDeleted) { if (!openedNodeDeleted) {
if (assignmentManager.getTableStateManager().isTableState(regionInfo.getTable(), if (assignmentManager.getTableStateManager().isTableState(regionInfo.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
debugLog(regionInfo, "Opened region " debugLog(regionInfo, "Opened region "
+ regionInfo.getShortNameToLog() + " but " + regionInfo.getShortNameToLog() + " but "
+ "this table is disabled, triggering close of region"); + "this table is disabled, triggering close of region");

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.master; package org.apache.hadoop.hbase.master;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; 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.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Admin.MasterSwitchType; 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.ZkOpenRegionCoordination;
import org.apache.hadoop.hbase.coordination.ZkRegionMergeCoordination; import org.apache.hadoop.hbase.coordination.ZkRegionMergeCoordination;
import org.apache.hadoop.hbase.exceptions.DeserializationException; 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.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
@ -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.master.handler.OpenedRegionHandler;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException; import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.ConfigUtil; import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 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.FSUtils;
import org.apache.hadoop.hbase.util.KeyLocker; import org.apache.hadoop.hbase.util.KeyLocker;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
@ -286,14 +285,11 @@ public class AssignmentManager extends ZooKeeperListener {
* @param service Executor service * @param service Executor service
* @param metricsMaster metrics manager * @param metricsMaster metrics manager
* @param tableLockManager TableLock manager * @param tableLockManager TableLock manager
* @throws KeeperException
* @throws IOException
*/ */
public AssignmentManager(MasterServices server, ServerManager serverManager, public AssignmentManager(MasterServices server, ServerManager serverManager,
final LoadBalancer balancer, final LoadBalancer balancer,
final ExecutorService service, MetricsMaster metricsMaster, final ExecutorService service, MetricsMaster metricsMaster,
final TableLockManager tableLockManager) throws KeeperException, final TableLockManager tableLockManager, final TableStateManager tableStateManager) {
IOException, CoordinatedStateException {
super(server.getZooKeeper()); super(server.getZooKeeper());
this.server = server; this.server = server;
this.serverManager = serverManager; this.serverManager = serverManager;
@ -306,15 +302,9 @@ public class AssignmentManager extends ZooKeeperListener {
this.shouldAssignRegionsWithFavoredNodes = conf.getClass( this.shouldAssignRegionsWithFavoredNodes = conf.getClass(
HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals( HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals(
FavoredNodeLoadBalancer.class); FavoredNodeLoadBalancer.class);
try {
if (server.getCoordinatedStateManager() != null) { this.tableStateManager = tableStateManager;
this.tableStateManager = server.getCoordinatedStateManager().getTableStateManager();
} else {
this.tableStateManager = null;
}
} catch (InterruptedException e) {
throw new InterruptedIOException();
}
// This is the max attempts, not retries, so it should be at least 1. // This is the max attempts, not retries, so it should be at least 1.
this.maximumAttempts = Math.max(1, this.maximumAttempts = Math.max(1,
this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10)); this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10));
@ -392,7 +382,7 @@ public class AssignmentManager extends ZooKeeperListener {
} }
/** /**
* @return Instance of ZKTableStateManager. * @return Instance of TableStateManager.
*/ */
public TableStateManager getTableStateManager() { public TableStateManager getTableStateManager() {
// These are 'expensive' to make involving trip to zk ensemble so allow // These are 'expensive' to make involving trip to zk ensemble so allow
@ -516,10 +506,9 @@ public class AssignmentManager extends ZooKeeperListener {
* @throws IOException * @throws IOException
* @throws KeeperException * @throws KeeperException
* @throws InterruptedException * @throws InterruptedException
* @throws CoordinatedStateException
*/ */
void joinCluster() throws IOException, void joinCluster() throws IOException,
KeeperException, InterruptedException, CoordinatedStateException { KeeperException, CoordinatedStateException {
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
// Concurrency note: In the below the accesses on regionsInTransition are // Concurrency note: In the below the accesses on regionsInTransition are
// outside of a synchronization block where usually all accesses to RIT are // outside of a synchronization block where usually all accesses to RIT are
@ -560,7 +549,7 @@ public class AssignmentManager extends ZooKeeperListener {
* @throws InterruptedException * @throws InterruptedException
*/ */
boolean processDeadServersAndRegionsInTransition(final Set<ServerName> deadServers) boolean processDeadServersAndRegionsInTransition(final Set<ServerName> deadServers)
throws KeeperException, IOException, InterruptedException, CoordinatedStateException { throws KeeperException, IOException {
List<String> nodes = ZKUtil.listChildrenNoWatch(watcher, watcher.assignmentZNode); List<String> nodes = ZKUtil.listChildrenNoWatch(watcher, watcher.assignmentZNode);
if (useZKForAssignment && nodes == null) { if (useZKForAssignment && nodes == null) {
@ -568,7 +557,6 @@ public class AssignmentManager extends ZooKeeperListener {
server.abort(errorMessage, new IOException(errorMessage)); server.abort(errorMessage, new IOException(errorMessage));
return true; // Doesn't matter in this case return true; // Doesn't matter in this case
} }
boolean failover = !serverManager.getDeadServers().isEmpty(); boolean failover = !serverManager.getDeadServers().isEmpty();
if (failover) { if (failover) {
// This may not be a failover actually, especially if meta is on this master. // This may not be a failover actually, especially if meta is on this master.
@ -689,7 +677,11 @@ public class AssignmentManager extends ZooKeeperListener {
if (!failover) { if (!failover) {
// Fresh cluster startup. // Fresh cluster startup.
LOG.info("Clean cluster startup. Assigning user regions"); 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 // unassign replicas of the split parents and the merged regions
// the daughter replicas are opened in assignAllUserRegions if it was // the daughter replicas are opened in assignAllUserRegions if it was
@ -707,11 +699,10 @@ public class AssignmentManager extends ZooKeeperListener {
* locations are returned. * locations are returned.
*/ */
private Map<HRegionInfo, ServerName> getUserRegionsToAssign() private Map<HRegionInfo, ServerName> getUserRegionsToAssign()
throws InterruptedIOException, CoordinatedStateException { throws IOException {
Set<TableName> disabledOrDisablingOrEnabling = Set<TableName> disabledOrDisablingOrEnabling =
tableStateManager.getTablesInStates(ZooKeeperProtos.Table.State.DISABLED, tableStateManager.getTablesInStates(TableState.State.DISABLED,
ZooKeeperProtos.Table.State.DISABLING, ZooKeeperProtos.Table.State.ENABLING); TableState.State.DISABLING, TableState.State.ENABLING);
// Clean re/start, mark all user regions closed before reassignment // Clean re/start, mark all user regions closed before reassignment
return regionStates.closeAllUserRegions(disabledOrDisablingOrEnabling); return regionStates.closeAllUserRegions(disabledOrDisablingOrEnabling);
} }
@ -739,7 +730,7 @@ public class AssignmentManager extends ZooKeeperListener {
try { try {
// Assign the regions // Assign the regions
assignAllUserRegions(getUserRegionsToAssign()); assignAllUserRegions(getUserRegionsToAssign());
} catch (CoordinatedStateException | IOException | InterruptedException e) { } catch (IOException | InterruptedException e) {
LOG.error("Exception occured while assigning user regions.", 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); LOG.debug("Znode " + regionNameStr + " deleted, state: " + rs);
boolean disabled = getTableStateManager().isTableState(regionInfo.getTable(), boolean disabled = getTableStateManager().isTableState(regionInfo.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING); TableState.State.DISABLED, TableState.State.DISABLING);
ServerName serverName = rs.getServerName(); ServerName serverName = rs.getServerName();
if (serverManager.isServerOnline(serverName)) { if (serverManager.isServerOnline(serverName)) {
@ -2269,7 +2260,7 @@ public class AssignmentManager extends ZooKeeperListener {
// will not be in ENABLING or ENABLED state. // will not be in ENABLING or ENABLED state.
TableName tableName = region.getTable(); TableName tableName = region.getTable();
if (!tableStateManager.isTableState(tableName, if (!tableStateManager.isTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED, ZooKeeperProtos.Table.State.ENABLING)) { TableState.State.ENABLED, TableState.State.ENABLING)) {
LOG.debug("Setting table " + tableName + " to ENABLED state."); LOG.debug("Setting table " + tableName + " to ENABLED state.");
setEnabledTable(tableName); setEnabledTable(tableName);
} }
@ -2495,8 +2486,8 @@ public class AssignmentManager extends ZooKeeperListener {
private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) { private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
if (this.tableStateManager.isTableState(region.getTable(), if (this.tableStateManager.isTableState(region.getTable(),
ZooKeeperProtos.Table.State.DISABLED, TableState.State.DISABLED,
ZooKeeperProtos.Table.State.DISABLING) || replicasToClose.contains(region)) { TableState.State.DISABLING) || replicasToClose.contains(region)) {
LOG.info("Table " + region.getTable() + " is disabled or disabling;" LOG.info("Table " + region.getTable() + " is disabled or disabling;"
+ " skipping assign of " + region.getRegionNameAsString()); + " skipping assign of " + region.getRegionNameAsString());
offlineDisabledRegion(region); offlineDisabledRegion(region);
@ -3127,7 +3118,7 @@ public class AssignmentManager extends ZooKeeperListener {
for (HRegionInfo hri : regionsFromMetaScan) { for (HRegionInfo hri : regionsFromMetaScan) {
TableName tableName = hri.getTable(); TableName tableName = hri.getTable();
if (!tableStateManager.isTableState(tableName, if (!tableStateManager.isTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
setEnabledTable(tableName); setEnabledTable(tableName);
} }
} }
@ -3194,14 +3185,14 @@ public class AssignmentManager extends ZooKeeperListener {
* @throws IOException * @throws IOException
*/ */
Set<ServerName> rebuildUserRegions() throws Set<ServerName> rebuildUserRegions() throws
IOException, KeeperException, CoordinatedStateException { IOException, KeeperException {
Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates( Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates(
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.ENABLING); TableState.State.DISABLED, TableState.State.ENABLING);
Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates( Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
ZooKeeperProtos.Table.State.DISABLED, TableState.State.DISABLED,
ZooKeeperProtos.Table.State.DISABLING, TableState.State.DISABLING,
ZooKeeperProtos.Table.State.ENABLING); TableState.State.ENABLING);
// Region assignment from META // Region assignment from META
List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getConnection()); List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getConnection());
@ -3253,7 +3244,7 @@ public class AssignmentManager extends ZooKeeperListener {
ServerName lastHost = hrl.getServerName(); ServerName lastHost = hrl.getServerName();
ServerName regionLocation = RegionStateStore.getRegionServer(result, replicaId); ServerName regionLocation = RegionStateStore.getRegionServer(result, replicaId);
if (tableStateManager.isTableState(regionInfo.getTable(), if (tableStateManager.isTableState(regionInfo.getTable(),
ZooKeeperProtos.Table.State.DISABLED)) { TableState.State.DISABLED)) {
// force region to forget it hosts for disabled/disabling tables. // force region to forget it hosts for disabled/disabling tables.
// see HBASE-13326 // see HBASE-13326
lastHost = null; lastHost = null;
@ -3283,7 +3274,7 @@ public class AssignmentManager extends ZooKeeperListener {
// this will be used in rolling restarts // this will be used in rolling restarts
if (!disabledOrDisablingOrEnabling.contains(tableName) if (!disabledOrDisablingOrEnabling.contains(tableName)
&& !getTableStateManager().isTableState(tableName, && !getTableStateManager().isTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
setEnabledTable(tableName); setEnabledTable(tableName);
} }
} }
@ -3300,9 +3291,9 @@ public class AssignmentManager extends ZooKeeperListener {
* @throws IOException * @throws IOException
*/ */
private void recoverTableInDisablingState() private void recoverTableInDisablingState()
throws KeeperException, IOException, CoordinatedStateException { throws KeeperException, IOException {
Set<TableName> disablingTables = Set<TableName> disablingTables =
tableStateManager.getTablesInStates(ZooKeeperProtos.Table.State.DISABLING); tableStateManager.getTablesInStates(TableState.State.DISABLING);
if (disablingTables.size() != 0) { if (disablingTables.size() != 0) {
for (TableName tableName : disablingTables) { for (TableName tableName : disablingTables) {
// Recover by calling DisableTableHandler // Recover by calling DisableTableHandler
@ -3324,9 +3315,9 @@ public class AssignmentManager extends ZooKeeperListener {
* @throws IOException * @throws IOException
*/ */
private void recoverTableInEnablingState() private void recoverTableInEnablingState()
throws KeeperException, IOException, CoordinatedStateException { throws KeeperException, IOException {
Set<TableName> enablingTables = tableStateManager. Set<TableName> enablingTables = tableStateManager.
getTablesInStates(ZooKeeperProtos.Table.State.ENABLING); getTablesInStates(TableState.State.ENABLING);
if (enablingTables.size() != 0) { if (enablingTables.size() != 0) {
for (TableName tableName : enablingTables) { for (TableName tableName : enablingTables) {
// Recover by calling EnableTableHandler // Recover by calling EnableTableHandler
@ -3398,9 +3389,9 @@ public class AssignmentManager extends ZooKeeperListener {
LOG.info("Server " + serverName + " isn't online. SSH will handle this"); LOG.info("Server " + serverName + " isn't online. SSH will handle this");
continue; continue;
} }
RegionState.State state = regionState.getState();
HRegionInfo regionInfo = regionState.getRegion(); HRegionInfo regionInfo = regionState.getRegion();
State state = regionState.getState(); LOG.info("Processing " + regionState);
switch (state) { switch (state) {
case CLOSED: case CLOSED:
invokeAssign(regionInfo); invokeAssign(regionInfo);
@ -3790,7 +3781,7 @@ public class AssignmentManager extends ZooKeeperListener {
server.abort("Unexpected ZK exception deleting node " + hri, ke); server.abort("Unexpected ZK exception deleting node " + hri, ke);
} }
if (tableStateManager.isTableState(hri.getTable(), if (tableStateManager.isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
regionStates.regionOffline(hri); regionStates.regionOffline(hri);
it.remove(); it.remove();
continue; continue;
@ -3813,7 +3804,7 @@ public class AssignmentManager extends ZooKeeperListener {
HRegionInfo hri = plan.getRegionInfo(); HRegionInfo hri = plan.getRegionInfo();
TableName tableName = hri.getTable(); TableName tableName = hri.getTable();
if (tableStateManager.isTableState(tableName, if (tableStateManager.isTableState(tableName,
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
LOG.info("Ignored moving region of disabling/disabled table " LOG.info("Ignored moving region of disabling/disabled table "
+ tableName); + tableName);
return; return;
@ -3861,8 +3852,8 @@ public class AssignmentManager extends ZooKeeperListener {
protected void setEnabledTable(TableName tableName) { protected void setEnabledTable(TableName tableName) {
try { try {
this.tableStateManager.setTableState(tableName, this.tableStateManager.setTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED); TableState.State.ENABLED);
} catch (CoordinatedStateException e) { } catch (IOException e) {
// here we can abort as it is the start up flow // here we can abort as it is the start up flow
String errorMsg = "Unable to ensure that the table " + tableName String errorMsg = "Unable to ensure that the table " + tableName
+ " will be" + " enabled because of a ZooKeeper issue"; + " will be" + " enabled because of a ZooKeeper issue";
@ -3967,8 +3958,8 @@ public class AssignmentManager extends ZooKeeperListener {
// When there are more than one region server a new RS is selected as the // When there are more than one region server a new RS is selected as the
// destination and the same is updated in the region plan. (HBASE-5546) // destination and the same is updated in the region plan. (HBASE-5546)
if (getTableStateManager().isTableState(hri.getTable(), if (getTableStateManager().isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) || TableState.State.DISABLED, TableState.State.DISABLING) ||
replicasToClose.contains(hri)) { replicasToClose.contains(hri)) {
offlineDisabledRegion(hri); offlineDisabledRegion(hri);
return; return;
} }
@ -3996,15 +3987,14 @@ public class AssignmentManager extends ZooKeeperListener {
// reset the count, if any // reset the count, if any
failedOpenTracker.remove(hri.getEncodedName()); failedOpenTracker.remove(hri.getEncodedName());
if (getTableStateManager().isTableState(hri.getTable(), if (getTableStateManager().isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(hri); invokeUnAssign(hri);
} }
} }
private void onRegionClosed(final HRegionInfo hri) { private void onRegionClosed(final HRegionInfo hri) {
if (getTableStateManager().isTableState(hri.getTable(), if (getTableStateManager().isTableState(hri.getTable(), TableState.State.DISABLED,
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) || TableState.State.DISABLING) || replicasToClose.contains(hri)) {
replicasToClose.contains(hri)) {
offlineDisabledRegion(hri); offlineDisabledRegion(hri);
return; return;
} }
@ -4050,7 +4040,7 @@ public class AssignmentManager extends ZooKeeperListener {
} }
if (getTableStateManager().isTableState(p.getTable(), if (getTableStateManager().isTableState(p.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(p); invokeUnAssign(p);
} }
return null; return null;
@ -4076,7 +4066,7 @@ public class AssignmentManager extends ZooKeeperListener {
// User could disable the table before master knows the new region. // User could disable the table before master knows the new region.
if (getTableStateManager().isTableState(p.getTable(), if (getTableStateManager().isTableState(p.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(a); invokeUnAssign(a);
invokeUnAssign(b); invokeUnAssign(b);
} else { } else {
@ -4130,7 +4120,7 @@ public class AssignmentManager extends ZooKeeperListener {
// User could disable the table before master knows the new region. // User could disable the table before master knows the new region.
if (getTableStateManager().isTableState(p.getTable(), if (getTableStateManager().isTableState(p.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(p); invokeUnAssign(p);
} else { } else {
Callable<Object> mergeReplicasCallable = new Callable<Object>() { Callable<Object> mergeReplicasCallable = new Callable<Object>() {
@ -4170,7 +4160,7 @@ public class AssignmentManager extends ZooKeeperListener {
} }
if (getTableStateManager().isTableState(p.getTable(), if (getTableStateManager().isTableState(p.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
invokeUnAssign(a); invokeUnAssign(a);
invokeUnAssign(b); invokeUnAssign(b);
} }
@ -4291,7 +4281,7 @@ public class AssignmentManager extends ZooKeeperListener {
// User could disable the table before master knows the new region. // User could disable the table before master knows the new region.
if (tableStateManager.isTableState(p.getTable(), if (tableStateManager.isTableState(p.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
unassign(p); unassign(p);
} }
} }
@ -4421,7 +4411,7 @@ public class AssignmentManager extends ZooKeeperListener {
// User could disable the table before master knows the new region. // User could disable the table before master knows the new region.
if (tableStateManager.isTableState(p.getTable(), if (tableStateManager.isTableState(p.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
unassign(hri_a); unassign(hri_a);
unassign(hri_b); unassign(hri_b);
} }
@ -4692,7 +4682,7 @@ public class AssignmentManager extends ZooKeeperListener {
errorMsg = hri.getShortNameToLog() errorMsg = hri.getShortNameToLog()
+ " is not pending close on " + serverName; + " is not pending close on " + serverName;
} else { } else {
onRegionClosed(hri); onRegionClosed(hri);
} }
break; break;

View File

@ -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.MetaScanner.MetaScannerVisitorBase;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.executor.ExecutorType; import org.apache.hadoop.hbase.executor.ExecutorType;
@ -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.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos; 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.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager; import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; 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.Pair;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.util.ZKDataMigrator;
import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker; import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker; import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
@ -385,6 +386,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
private long splitPlanCount; private long splitPlanCount;
private long mergePlanCount; private long mergePlanCount;
// handle table states
private TableStateManager tableStateManager;
/** flag used in test cases in order to simulate RS failures during master initialization */ /** flag used in test cases in order to simulate RS failures during master initialization */
private volatile boolean initializationBeforeMetaAssignment = false; private volatile boolean initializationBeforeMetaAssignment = false;
@ -694,9 +698,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
this.assignmentManager = new AssignmentManager(this, serverManager, this.assignmentManager = new AssignmentManager(this, serverManager,
this.balancer, this.service, this.metricsMaster, this.balancer, this.service, this.metricsMaster,
this.tableLockManager); this.tableLockManager, tableStateManager);
zooKeeper.registerListenerFirst(assignmentManager); zooKeeper.registerListenerFirst(assignmentManager);
this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
this.serverManager); this.serverManager);
this.regionServerTracker.start(); this.regionServerTracker.start();
@ -728,6 +731,14 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
this.mpmHost.register(new MasterFlushTableProcedureManager()); this.mpmHost.register(new MasterFlushTableProcedureManager());
this.mpmHost.loadProcedures(conf); this.mpmHost.loadProcedures(conf);
this.mpmHost.initialize(this, this.metricsMaster); this.mpmHost.initialize(this, this.metricsMaster);
// migrating existent table state from zk
for (Map.Entry<TableName, TableState.State> entry : ZKDataMigrator
.queryForTableStates(getZooKeeper()).entrySet()) {
LOG.info("Converting state from zk to new states:" + entry);
tableStateManager.setTableState(entry.getKey(), entry.getValue());
}
ZKUtil.deleteChildrenRecursively(getZooKeeper(), getZooKeeper().tableZNode);
} }
/** /**
@ -792,6 +803,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Invalidate all write locks held previously // Invalidate all write locks held previously
this.tableLockManager.reapWriteLocks(); this.tableLockManager.reapWriteLocks();
this.tableStateManager = new TableStateManager(this);
this.tableStateManager.start();
status.setStatus("Initializing ZK system trackers"); status.setStatus("Initializing ZK system trackers");
initializeZKBasedSystemTrackers(); initializeZKBasedSystemTrackers();
@ -1186,8 +1200,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
} }
private void enableMeta(TableName metaTableName) { private void enableMeta(TableName metaTableName) {
if (!this.assignmentManager.getTableStateManager().isTableState(metaTableName, if (!this.tableStateManager.isTableState(metaTableName,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
this.assignmentManager.setEnabledTable(metaTableName); this.assignmentManager.setEnabledTable(metaTableName);
} }
} }
@ -1231,6 +1245,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return tableNamespaceManager; return tableNamespaceManager;
} }
@Override
public TableStateManager getTableStateManager() {
return tableStateManager;
}
/* /*
* Start up all services. If any of these threads gets an unhandled exception * Start up all services. If any of these threads gets an unhandled exception
* then they just die with a logged message. This should be fine because * then they just die with a logged message. This should be fine because
@ -1663,7 +1682,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Don't run the normalizer concurrently // Don't run the normalizer concurrently
List<TableName> allEnabledTables = new ArrayList<>( List<TableName> allEnabledTables = new ArrayList<>(
this.assignmentManager.getTableStateManager().getTablesInStates( this.assignmentManager.getTableStateManager().getTablesInStates(
ZooKeeperProtos.Table.State.ENABLED)); TableState.State.ENABLED));
Collections.shuffle(allEnabledTables); Collections.shuffle(allEnabledTables);
@ -2508,7 +2527,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
throw new TableNotFoundException(tableName); throw new TableNotFoundException(tableName);
} }
if (!getAssignmentManager().getTableStateManager(). if (!getAssignmentManager().getTableStateManager().
isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED)) { isTableState(tableName, TableState.State.DISABLED)) {
throw new TableNotDisabledException(tableName); throw new TableNotDisabledException(tableName);
} }
} }

View File

@ -547,7 +547,6 @@ public class MasterFileSystem {
fsd.createTableDescriptor( fsd.createTableDescriptor(
new HTableDescriptor(fsd.get(TableName.META_TABLE_NAME))); new HTableDescriptor(fsd.get(TableName.META_TABLE_NAME)));
} }
return rd; return rd;
} }

View File

@ -40,10 +40,12 @@ import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.errorhandling.ForeignException; 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.MergeRegionException;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.ipc.PriorityFunction; import org.apache.hadoop.hbase.ipc.PriorityFunction;
@ -952,13 +954,11 @@ public class MasterRpcServices extends RSRpcServices
public GetTableNamesResponse getTableNames(RpcController controller, public GetTableNamesResponse getTableNames(RpcController controller,
GetTableNamesRequest req) throws ServiceException { GetTableNamesRequest req) throws ServiceException {
try { try {
master.checkInitialized(); master.checkServiceStarted();
final String regex = req.hasRegex() ? req.getRegex() : null; final String regex = req.hasRegex() ? req.getRegex() : null;
final String namespace = req.hasNamespace() ? req.getNamespace() : null; final String namespace = req.hasNamespace() ? req.getNamespace() : null;
List<TableName> tableNames = master.listTableNames(namespace, regex, List<TableName> tableNames = master.listTableNames(namespace, regex,
req.getIncludeSysTables()); req.getIncludeSysTables());
GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder(); GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder();
if (tableNames != null && tableNames.size() > 0) { if (tableNames != null && tableNames.size() > 0) {
// Add the table names to the response // 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 @Override
public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c, public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c,
IsCatalogJanitorEnabledRequest req) throws ServiceException { IsCatalogJanitorEnabledRequest req) throws ServiceException {

View File

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

View File

@ -31,6 +31,8 @@ import java.util.SortedSet;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.TreeSet; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -42,14 +44,13 @@ import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConfigUtil; import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.util.FSUtils; 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.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException; 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. * Region state accountant. It holds the states of all regions in the memory.
* In normal scenario, it should match the meta table and the true region states. * In normal scenario, it should match the meta table and the true region states.
@ -720,7 +718,7 @@ public class RegionStates {
if (oldServerName != null && serverHoldings.containsKey(oldServerName)) { if (oldServerName != null && serverHoldings.containsKey(oldServerName)) {
if (newState == State.MERGED || newState == State.SPLIT if (newState == State.MERGED || newState == State.SPLIT
|| hri.isMetaRegion() || tableStateManager.isTableState(hri.getTable(), || hri.isMetaRegion() || tableStateManager.isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { TableState.State.DISABLED, TableState.State.DISABLING)) {
// Offline the region only if it's merged/split, or the table is disabled/disabling. // Offline the region only if it's merged/split, or the table is disabled/disabling.
// Otherwise, offline it from this server only when it is online on a different server. // Otherwise, offline it from this server only when it is online on a different server.
LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName); LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
@ -1295,8 +1293,8 @@ public class RegionStates {
* Update a region state. It will be put in transition if not already there. * Update a region state. It will be put in transition if not already there.
*/ */
private RegionState updateRegionState(final HRegionInfo hri, private RegionState updateRegionState(final HRegionInfo hri,
final State state, final ServerName serverName, long openSeqNum) { final RegionState.State state, final ServerName serverName, long openSeqNum) {
if (state == State.FAILED_CLOSE || state == State.FAILED_OPEN) { if (state == RegionState.State.FAILED_CLOSE || state == RegionState.State.FAILED_OPEN) {
LOG.warn("Failed to open/close " + hri.getShortNameToLog() LOG.warn("Failed to open/close " + hri.getShortNameToLog()
+ " on " + serverName + ", set to " + state); + " on " + serverName + ", set to " + state);
} }

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure; import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; 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 // Now check if the table is assigned, if not then fail fast
if (isTableAssigned()) { if (isTableAssigned() && isTableEnabled()) {
try { try {
boolean initGoodSofar = true; boolean initGoodSofar = true;
nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME); nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
@ -297,6 +298,12 @@ public class TableNamespaceManager {
return false; return false;
} }
private boolean isTableEnabled() throws IOException {
return masterServices.getTableStateManager().getTableState(
TableName.NAMESPACE_TABLE_NAME
).equals(TableState.State.ENABLED);
}
private boolean isTableAssigned() { private boolean isTableAssigned() {
return !masterServices.getAssignmentManager().getRegionStates(). return !masterServices.getAssignmentManager().getRegionStates().
getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty(); getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty();

View File

@ -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;
}
}

View File

@ -23,11 +23,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
/** /**
* Handles CLOSED region event on Master. * 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()); LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName());
// Check if this table is being disabled or not // Check if this table is being disabled or not
if (this.assignmentManager.getTableStateManager().isTableState(this.regionInfo.getTable(), 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.getReplicasToClose().contains(regionInfo)) {
assignmentManager.offlineDisabledRegion(regionInfo); assignmentManager.offlineDisabledRegion(regionInfo);
return; return;

View File

@ -30,14 +30,16 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.ipc.RpcServer; 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.MasterServices;
import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock; import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSTableDescriptors;
@ -119,13 +120,6 @@ public class CreateTableHandler extends EventHandler {
if (MetaTableAccessor.tableExists(this.server.getConnection(), tableName)) { if (MetaTableAccessor.tableExists(this.server.getConnection(), tableName)) {
throw new TableExistsException(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; success = true;
} finally { } finally {
if (!success) { if (!success) {
@ -135,52 +129,6 @@ public class CreateTableHandler extends EventHandler {
return this; 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 @Override
public String toString() { public String toString() {
String name = "UnknownServerName"; String name = "UnknownServerName";
@ -228,9 +176,6 @@ public class CreateTableHandler extends EventHandler {
releaseTableLock(); releaseTableLock();
LOG.info("Table, " + this.hTableDescriptor.getTableName() + ", creation " + LOG.info("Table, " + this.hTableDescriptor.getTableName() + ", creation " +
(exception == null ? "successful" : "failed. " + exception)); (exception == null ? "successful" : "failed. " + exception));
if (exception != null) {
removeEnablingTable(this.assignmentManager, this.hTableDescriptor.getTableName());
}
} }
/** /**
@ -253,9 +198,12 @@ public class CreateTableHandler extends EventHandler {
FileSystem fs = fileSystemManager.getFileSystem(); FileSystem fs = fileSystemManager.getFileSystem();
// 1. Create Table Descriptor // 1. Create Table Descriptor
// using a copy of descriptor, table will be created enabling first
TableDescriptor underConstruction = new TableDescriptor(
this.hTableDescriptor, TableState.State.ENABLING);
Path tempTableDir = FSUtils.getTableDir(tempdir, tableName); Path tempTableDir = FSUtils.getTableDir(tempdir, tableName);
new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory( new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory(
tempTableDir, this.hTableDescriptor, false); tempTableDir, underConstruction, false);
Path tableDir = FSUtils.getTableDir(fileSystemManager.getRootDir(), tableName); Path tableDir = FSUtils.getTableDir(fileSystemManager.getRootDir(), tableName);
// 2. Create Regions // 2. Create Regions
@ -280,24 +228,18 @@ public class CreateTableHandler extends EventHandler {
// 7. Trigger immediate assignment of the regions in round-robin fashion // 7. Trigger immediate assignment of the regions in round-robin fashion
ModifyRegionUtils.assignRegions(assignmentManager, regionInfos); 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. // 8. Update the tabledescriptor cache.
((HMaster) this.server).getTableDescriptors().get(tableName); ((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 * Create any replicas for the regions (the default replicas that was
* already created is passed to the method) * already created is passed to the method)
* @param hTableDescriptor * @param hTableDescriptor descriptor to use
* @param regions default replicas * @param regions default replicas
* @return the combined list of default and non-default replicas * @return the combined list of default and non-default replicas
*/ */

View File

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

View File

@ -26,15 +26,15 @@ import java.util.Map;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock; import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@ -97,14 +97,9 @@ public class EnableTableHandler extends EventHandler {
if (!this.skipTableStateCheck) { if (!this.skipTableStateCheck) {
throw new TableNotFoundException(tableName); throw new TableNotFoundException(tableName);
} }
try { TableStateManager tsm = assignmentManager.getTableStateManager();
this.assignmentManager.getTableStateManager().checkAndRemoveTableState(tableName, if (tsm.isTableState(tableName, TableState.State.ENABLING)) {
ZooKeeperProtos.Table.State.ENABLING, true); tsm.setDeletedTable(tableName);
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.");
} }
} }
@ -113,16 +108,11 @@ public class EnableTableHandler extends EventHandler {
// After that, no other requests can be accepted until the table reaches // After that, no other requests can be accepted until the table reaches
// DISABLED or ENABLED. // DISABLED or ENABLED.
if (!skipTableStateCheck) { if (!skipTableStateCheck) {
try { if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates( this.tableName, TableState.State.ENABLING,
this.tableName, ZooKeeperProtos.Table.State.ENABLING, TableState.State.DISABLED)) {
ZooKeeperProtos.Table.State.DISABLED)) { LOG.info("Table " + tableName + " isn't disabled; skipping enable");
LOG.info("Table " + tableName + " isn't disabled; skipping enable"); throw new TableNotDisabledException(this.tableName);
throw new TableNotDisabledException(this.tableName);
}
} catch (CoordinatedStateException e) {
throw new IOException("Unable to ensure that the table will be" +
" enabling because of a coordination engine issue", e);
} }
} }
success = true; success = true;
@ -158,11 +148,7 @@ public class EnableTableHandler extends EventHandler {
if (cpHost != null) { if (cpHost != null) {
cpHost.postEnableTableHandler(this.tableName, null); cpHost.postEnableTableHandler(this.tableName, null);
} }
} catch (IOException e) { } catch (IOException | InterruptedException e) {
LOG.error("Error trying to enable the table " + this.tableName, e);
} catch (CoordinatedStateException e) {
LOG.error("Error trying to enable the table " + this.tableName, e);
} catch (InterruptedException e) {
LOG.error("Error trying to enable the table " + this.tableName, e); LOG.error("Error trying to enable the table " + this.tableName, e);
} finally { } finally {
releaseTableLock(); releaseTableLock();
@ -179,14 +165,13 @@ public class EnableTableHandler extends EventHandler {
} }
} }
private void handleEnableTable() throws IOException, CoordinatedStateException, private void handleEnableTable() throws IOException,
InterruptedException { InterruptedException {
// I could check table is disabling and if so, not enable but require // I could check table is disabling and if so, not enable but require
// that user first finish disabling but that might be obnoxious. // that user first finish disabling but that might be obnoxious.
// Set table enabling flag up in zk.
this.assignmentManager.getTableStateManager().setTableState(this.tableName, this.assignmentManager.getTableStateManager().setTableState(this.tableName,
ZooKeeperProtos.Table.State.ENABLING); TableState.State.ENABLING);
boolean done = false; boolean done = false;
ServerManager serverManager = ((HMaster)this.server).getServerManager(); ServerManager serverManager = ((HMaster)this.server).getServerManager();
// Get the regions of this table. We're done when all listed // Get the regions of this table. We're done when all listed
@ -251,7 +236,7 @@ public class EnableTableHandler extends EventHandler {
if (done) { if (done) {
// Flip the table to enabled. // Flip the table to enabled.
this.assignmentManager.getTableStateManager().setTableState( this.assignmentManager.getTableStateManager().setTableState(
this.tableName, ZooKeeperProtos.Table.State.ENABLED); this.tableName, TableState.State.ENABLED);
LOG.info("Table '" + this.tableName LOG.info("Table '" + this.tableName
+ "' was successfully enabled. Status: done=" + done); + "' was successfully enabled. Status: done=" + done);
} else { } else {

View File

@ -29,6 +29,8 @@ import java.util.TreeMap;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CoordinatedStateException; 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.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor; 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.MetaTableAccessor;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionLocator; 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.EventHandler;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.BulkReOpen; import org.apache.hadoop.hbase.master.BulkReOpen;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock; import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -137,7 +138,7 @@ public abstract class TableEventHandler extends EventHandler {
handleTableOperation(hris); handleTableOperation(hris);
if (eventType.isOnlineSchemaChangeSupported() && this.masterServices. if (eventType.isOnlineSchemaChangeSupported() && this.masterServices.
getAssignmentManager().getTableStateManager().isTableState( getAssignmentManager().getTableStateManager().isTableState(
tableName, ZooKeeperProtos.Table.State.ENABLED)) { tableName, TableState.State.ENABLED)) {
if (reOpenAllRegions(hris)) { if (reOpenAllRegions(hris)) {
LOG.info("Completed table operation " + eventType + " on table " + LOG.info("Completed table operation " + eventType + " on table " +
tableName); tableName);
@ -236,10 +237,10 @@ public abstract class TableEventHandler extends EventHandler {
* @throws FileNotFoundException * @throws FileNotFoundException
* @throws IOException * @throws IOException
*/ */
public HTableDescriptor getTableDescriptor() public TableDescriptor getTableDescriptor()
throws FileNotFoundException, IOException { throws FileNotFoundException, IOException {
HTableDescriptor htd = TableDescriptor htd =
this.masterServices.getTableDescriptors().get(tableName); this.masterServices.getTableDescriptors().getDescriptor(tableName);
if (htd == null) { if (htd == null) {
throw new IOException("HTableDescriptor missing for " + tableName); throw new IOException("HTableDescriptor missing for " + tableName);
} }

View File

@ -32,12 +32,12 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
/** /**
@ -336,7 +336,7 @@ public class AddColumnFamilyProcedure
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException { private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
// This operation only run when the table is enabled. // This operation only run when the table is enabled.
if (!env.getMasterServices().getAssignmentManager().getTableStateManager() if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) { .isTableState(getTableName(), TableState.State.ENABLED)) {
return; return;
} }

View File

@ -33,20 +33,21 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; 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.exceptions.HBaseException;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState; 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.security.User;
import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
@ -299,8 +300,8 @@ public class CreateTableProcedure
!(env.getMasterServices().isInitialized()) && tableName.isSystemTable(); !(env.getMasterServices().isInitialized()) && tableName.isSystemTable();
if (!skipTableStateCheck) { if (!skipTableStateCheck) {
TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager(); TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager();
if (tsm.isTableState(tableName, true, ZooKeeperProtos.Table.State.ENABLING, if (tsm.isTableState(tableName, TableState.State.ENABLING,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
LOG.warn("The table " + tableName + " does not exist in meta but has a znode. " + LOG.warn("The table " + tableName + " does not exist in meta but has a znode. " +
"run hbck to fix inconsistencies."); "run hbck to fix inconsistencies.");
setFailure("master-create-table", new TableExistsException(getTableName())); 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 // using a copy of descriptor, table will be created enabling first
final Path tempTableDir = FSUtils.getTableDir(tempdir, hTableDescriptor.getTableName()); final Path tempTableDir = FSUtils.getTableDir(tempdir, hTableDescriptor.getTableName());
new FSTableDescriptors(env.getMasterConfiguration()).createTableDescriptorForTableDirectory( new FSTableDescriptors(env.getMasterConfiguration()).createTableDescriptorForTableDirectory(
tempTableDir, hTableDescriptor, false); tempTableDir, new TableDescriptor(hTableDescriptor), false);
// 2. Create Regions // 2. Create Regions
newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir, newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir,
@ -448,14 +449,14 @@ public class CreateTableProcedure
// Mark the table as Enabling // Mark the table as Enabling
assignmentManager.getTableStateManager().setTableState(tableName, assignmentManager.getTableStateManager().setTableState(tableName,
ZooKeeperProtos.Table.State.ENABLING); TableState.State.ENABLING);
// Trigger immediate assignment of the regions in round-robin fashion // Trigger immediate assignment of the regions in round-robin fashion
ModifyRegionUtils.assignRegions(assignmentManager, regions); ModifyRegionUtils.assignRegions(assignmentManager, regions);
// Enable table // Enable table
assignmentManager.getTableStateManager() assignmentManager.getTableStateManager()
.setTableState(tableName, ZooKeeperProtos.Table.State.ENABLED); .setTableState(tableName, TableState.State.ENABLED);
} }
/** /**

View File

@ -31,12 +31,12 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState; 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.security.User;
import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -357,7 +357,7 @@ public class DeleteColumnFamilyProcedure
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException { private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
// This operation only run when the table is enabled. // This operation only run when the table is enabled.
if (!env.getMasterServices().getAssignmentManager().getTableStateManager() if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) { .isTableState(getTableName(), TableState.State.ENABLED)) {
return; return;
} }

View File

@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.constraint.ConstraintException;
import org.apache.hadoop.hbase.exceptions.HBaseException; import org.apache.hadoop.hbase.exceptions.HBaseException;
import org.apache.hadoop.hbase.master.AssignmentManager; 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.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState; 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.security.User;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.htrace.Trace; import org.apache.htrace.Trace;
@ -286,8 +286,8 @@ public class DisableTableProcedure
// this issue. // this issue.
TableStateManager tsm = TableStateManager tsm =
env.getMasterServices().getAssignmentManager().getTableStateManager(); env.getMasterServices().getAssignmentManager().getTableStateManager();
if (!tsm.setTableStateIfInStates(tableName, ZooKeeperProtos.Table.State.DISABLING, if (!tsm.setTableStateIfInStates(tableName, TableState.State.DISABLING,
ZooKeeperProtos.Table.State.DISABLING, ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.DISABLING, TableState.State.ENABLED)) {
LOG.info("Table " + tableName + " isn't enabled; skipping disable"); LOG.info("Table " + tableName + " isn't enabled; skipping disable");
setFailure("master-disable-table", new TableNotEnabledException(tableName)); setFailure("master-disable-table", new TableNotEnabledException(tableName));
canTableBeDisabled = false; canTableBeDisabled = false;
@ -311,7 +311,7 @@ public class DisableTableProcedure
try { try {
// If the state was changed, undo it. // If the state was changed, undo it.
if (env.getMasterServices().getAssignmentManager().getTableStateManager().isTableState( if (env.getMasterServices().getAssignmentManager().getTableStateManager().isTableState(
tableName, ZooKeeperProtos.Table.State.DISABLING)) { tableName, TableState.State.DISABLING)) {
EnableTableProcedure.setTableStateToEnabled(env, tableName); EnableTableProcedure.setTableStateToEnabled(env, tableName);
} }
} catch (Exception e) { } catch (Exception e) {
@ -344,7 +344,7 @@ public class DisableTableProcedure
// Set table disabling flag up in zk. // Set table disabling flag up in zk.
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState( env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
tableName, tableName,
ZooKeeperProtos.Table.State.DISABLING); TableState.State.DISABLING);
} }
/** /**
@ -435,7 +435,7 @@ public class DisableTableProcedure
// Flip the table to disabled // Flip the table to disabled
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState( env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
tableName, tableName,
ZooKeeperProtos.Table.State.DISABLED); TableState.State.DISABLED);
LOG.info("Disabled table, " + tableName + ", is completed."); LOG.info("Disabled table, " + tableName + ", is completed.");
} }

View File

@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.exceptions.HBaseException;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.BulkAssigner; 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.MasterServices;
import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState; 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.security.User;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; 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 // 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. // set the state later on). A quick state check should be enough for us to move forward.
TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager(); 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"); LOG.info("Table " + tableName + " isn't disabled; skipping enable");
setFailure("master-enable-table", new TableNotDisabledException(this.tableName)); setFailure("master-enable-table", new TableNotDisabledException(this.tableName));
canTableBeEnabled = false; canTableBeEnabled = false;
@ -344,8 +344,7 @@ public class EnableTableProcedure
// Set table disabling flag up in zk. // Set table disabling flag up in zk.
LOG.info("Attempting to enable the table " + tableName); LOG.info("Attempting to enable the table " + tableName);
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState( env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
tableName, tableName, TableState.State.ENABLING);
ZooKeeperProtos.Table.State.ENABLING);
} }
/** /**
@ -490,8 +489,7 @@ public class EnableTableProcedure
final TableName tableName) throws HBaseException, IOException { final TableName tableName) throws HBaseException, IOException {
// Flip the table to Enabled // Flip the table to Enabled
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState( env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
tableName, tableName, TableState.State.ENABLED);
ZooKeeperProtos.Table.State.ENABLED);
LOG.info("Table '" + tableName + "' was successfully enabled."); LOG.info("Table '" + tableName + "' was successfully enabled.");
} }

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionLocator; 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.AssignmentManager;
import org.apache.hadoop.hbase.master.BulkReOpen; import org.apache.hadoop.hbase.master.BulkReOpen;
import org.apache.hadoop.hbase.master.MasterFileSystem; 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. // We only execute this procedure with table online if online schema change config is set.
if (!env.getMasterServices().getAssignmentManager().getTableStateManager() if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
.isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED) .isTableState(tableName, TableState.State.DISABLED)
&& !MasterDDLOperationHelper.isOnlineSchemaChangeAllowed(env)) { && !MasterDDLOperationHelper.isOnlineSchemaChangeAllowed(env)) {
throw new TableNotDisabledException(tableName); throw new TableNotDisabledException(tableName);
} }

View File

@ -32,12 +32,12 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
/** /**
@ -316,7 +316,7 @@ public class ModifyColumnFamilyProcedure
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException { private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
// This operation only run when the table is enabled. // This operation only run when the table is enabled.
if (!env.getMasterServices().getAssignmentManager().getTableStateManager() if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) { .isTableState(getTableName(), TableState.State.ENABLED)) {
return; return;
} }

View File

@ -42,11 +42,11 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableState; 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.security.User;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@ -294,7 +294,7 @@ public class ModifyTableProcedure
env.getMasterServices().getTableDescriptors().get(getTableName()); env.getMasterServices().getTableDescriptors().get(getTableName());
if (env.getMasterServices().getAssignmentManager().getTableStateManager() 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. // We only execute this procedure with table online if online schema change config is set.
if (!MasterDDLOperationHelper.isOnlineSchemaChangeAllowed(env)) { if (!MasterDDLOperationHelper.isOnlineSchemaChangeAllowed(env)) {
throw new TableNotDisabledException(getTableName()); throw new TableNotDisabledException(getTableName());
@ -432,7 +432,7 @@ public class ModifyTableProcedure
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException { private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
// This operation only run when the table is enabled. // This operation only run when the table is enabled.
if (!env.getMasterServices().getAssignmentManager().getTableStateManager() if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) { .isTableState(getTableName(), TableState.State.ENABLED)) {
return; return;
} }

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; 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.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
@ -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.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState; 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.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@ -526,7 +526,7 @@ implements ServerProcedureInterface {
} else if (rit != null) { } else if (rit != null) {
if ((rit.isPendingCloseOrClosing() || rit.isOffline()) if ((rit.isPendingCloseOrClosing() || rit.isOffline())
&& am.getTableStateManager().isTableState(hri.getTable(), && am.getTableStateManager().isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) || TableState.State.DISABLED, TableState.State.DISABLING) ||
am.getReplicasToClose().contains(hri)) { am.getReplicasToClose().contains(hri)) {
// If the table was partially disabled and the RS went down, we should clear the // If the table was partially disabled and the RS went down, we should clear the
// RIT and remove the node for the region. // RIT and remove the node for the region.
@ -713,7 +713,7 @@ implements ServerProcedureInterface {
} }
// If table is not disabled but the region is offlined, // If table is not disabled but the region is offlined,
boolean disabled = assignmentManager.getTableStateManager().isTableState(hri.getTable(), boolean disabled = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLED); TableState.State.DISABLED);
if (disabled){ if (disabled){
LOG.info("The table " + hri.getTable() + " was disabled. Hence not proceeding."); LOG.info("The table " + hri.getTable() + " was disabled. Hence not proceeding.");
return false; return false;
@ -725,7 +725,7 @@ implements ServerProcedureInterface {
return false; return false;
} }
boolean disabling = assignmentManager.getTableStateManager().isTableState(hri.getTable(), boolean disabling = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
ZooKeeperProtos.Table.State.DISABLING); TableState.State.DISABLING);
if (disabling) { if (disabling) {
LOG.info("The table " + hri.getTable() + " is disabled. Hence not assigning region" + LOG.info("The table " + hri.getTable() + " is disabled. Hence not assigning region" +
hri.getEncodedName()); hri.getEncodedName());

View File

@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; 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.errorhandling.ForeignException;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.ipc.RpcServer; 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.ProcedureDescription;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type; 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.quotas.QuotaExceededException;
import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
@ -622,7 +622,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
TableName snapshotTable = TableName.valueOf(snapshot.getTable()); TableName snapshotTable = TableName.valueOf(snapshot.getTable());
AssignmentManager assignmentMgr = master.getAssignmentManager(); AssignmentManager assignmentMgr = master.getAssignmentManager();
if (assignmentMgr.getTableStateManager().isTableState(snapshotTable, if (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
ZooKeeperProtos.Table.State.ENABLED)) { TableState.State.ENABLED)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Table enabled, starting distributed snapshot for " LOG.debug("Table enabled, starting distributed snapshot for "
+ ClientSnapshotDescriptionUtils.toString(snapshot)); + ClientSnapshotDescriptionUtils.toString(snapshot));
@ -634,7 +634,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
} }
// For disabled table, snapshot is created by the master // For disabled table, snapshot is created by the master
else if (assignmentMgr.getTableStateManager().isTableState(snapshotTable, else if (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
ZooKeeperProtos.Table.State.DISABLED)) { TableState.State.DISABLED)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Table is disabled, running snapshot entirely on master " LOG.debug("Table is disabled, running snapshot entirely on master "
+ ClientSnapshotDescriptionUtils.toString(snapshot)); + ClientSnapshotDescriptionUtils.toString(snapshot));
@ -801,7 +801,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
// Execute the restore/clone operation // Execute the restore/clone operation
if (MetaTableAccessor.tableExists(master.getConnection(), tableName)) { if (MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
if (master.getAssignmentManager().getTableStateManager().isTableState( if (master.getAssignmentManager().getTableStateManager().isTableState(
TableName.valueOf(snapshot.getTable()), ZooKeeperProtos.Table.State.ENABLED)) { TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) {
throw new UnsupportedOperationException("Table '" + throw new UnsupportedOperationException("Table '" +
TableName.valueOf(snapshot.getTable()) + "' must be disabled in order to " + TableName.valueOf(snapshot.getTable()) + "' must be disabled in order to " +
"perform a restore operation" + "perform a restore operation" +

View File

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

View File

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

View File

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

View File

@ -43,7 +43,9 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -305,7 +307,8 @@ public final class SnapshotManifest {
private void load() throws IOException { private void load() throws IOException {
switch (getSnapshotFormat(desc)) { switch (getSnapshotFormat(desc)) {
case SnapshotManifestV1.DESCRIPTOR_VERSION: { case SnapshotManifestV1.DESCRIPTOR_VERSION: {
this.htd = FSTableDescriptors.getTableDescriptorFromFs(workingDirFs, workingDir); this.htd = FSTableDescriptors.getTableDescriptorFromFs(workingDirFs, workingDir)
.getHTableDescriptor();
ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader"); ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
try { try {
this.regionManifests = this.regionManifests =
@ -410,7 +413,8 @@ public final class SnapshotManifest {
LOG.info("Using old Snapshot Format"); LOG.info("Using old Snapshot Format");
// write a copy of descriptor to the snapshot directory // write a copy of descriptor to the snapshot directory
new FSTableDescriptors(conf, workingDirFs, rootDir) new FSTableDescriptors(conf, workingDirFs, rootDir)
.createTableDescriptorForTableDirectory(workingDir, htd, false); .createTableDescriptorForTableDirectory(workingDir, new TableDescriptor(
htd, TableState.State.ENABLED), false);
} else { } else {
LOG.debug("Convert to Single Snapshot Manifest for " + this.desc.getName()); LOG.debug("Convert to Single Snapshot Manifest for " + this.desc.getName());
convertToV2SingleManifest(); convertToV2SingleManifest();

View File

@ -38,7 +38,9 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
@ -88,15 +90,10 @@ public class FSTableDescriptors implements TableDescriptors {
// This cache does not age out the old stuff. Thinking is that the amount // 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. // of data we keep up in here is so small, no need to do occasional purge.
// TODO. // TODO.
private final Map<TableName, HTableDescriptor> cache = private final Map<TableName, TableDescriptor> cache =
new ConcurrentHashMap<TableName, HTableDescriptor>(); 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 * Construct a FSTableDescriptors instance using the hbase root dir of the given
* conf and the filesystem where that root dir lives. * conf and the filesystem where that root dir lives.
* This instance can do write operations (is not read only). * This instance can do write operations (is not read only).
@ -121,7 +118,6 @@ public class FSTableDescriptors implements TableDescriptors {
this.rootdir = rootdir; this.rootdir = rootdir;
this.fsreadonly = fsreadonly; this.fsreadonly = fsreadonly;
this.usecache = usecache; this.usecache = usecache;
this.metaTableDescriptor = HTableDescriptor.metaTableDescriptor(conf);
} }
@Override @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. * to see if a newer file has been created since the cached one was read.
*/ */
@Override @Override
public HTableDescriptor get(final TableName tablename) public TableDescriptor getDescriptor(final TableName tablename)
throws IOException { throws IOException {
invocations++; invocations++;
if (TableName.META_TABLE_NAME.equals(tablename)) { if (TableName.META_TABLE_NAME.equals(tablename)) {
cachehits++; 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 // hbase:meta is already handled. If some one tries to get the descriptor for
// .logs, .oldlogs or .corrupt throw an exception. // .logs, .oldlogs or .corrupt throw an exception.
@ -163,73 +159,100 @@ public class FSTableDescriptors implements TableDescriptors {
if (usecache) { if (usecache) {
// Look in cache of descriptors. // Look in cache of descriptors.
HTableDescriptor cachedtdm = this.cache.get(tablename); TableDescriptor cachedtdm = this.cache.get(tablename);
if (cachedtdm != null) { if (cachedtdm != null) {
cachehits++; cachehits++;
return cachedtdm; return cachedtdm;
} }
} }
HTableDescriptor tdmt = null; TableDescriptor tdmt = null;
try { try {
tdmt = getTableDescriptorFromFs(fs, rootdir, tablename, !fsreadonly); tdmt = getTableDescriptorFromFs(fs, rootdir, tablename);
} catch (NullPointerException e) {
LOG.debug("Exception during readTableDecriptor. Current table name = "
+ tablename, e);
} catch (TableInfoMissingException e) { } catch (TableInfoMissingException e) {
// ignore. This is regular operation // ignore. This is regular operation
} catch (IOException ioe) { } catch (NullPointerException | IOException e) {
LOG.debug("Exception during readTableDecriptor. Current table name = " LOG.debug("Exception during readTableDecriptor. Current table name = "
+ tablename, ioe); + tablename, e);
} }
// last HTD written wins // last HTD written wins
if (usecache && tdmt != null) { if (usecache && tdmt != null) {
this.cache.put(tablename, tdmt); this.cache.put(tablename, tdmt);
} }
return 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. * Returns a map from table name to table descriptor for all tables.
*/ */
@Override @Override
public Map<String, HTableDescriptor> getAll() public Map<String, TableDescriptor> getAllDescriptors()
throws IOException { throws IOException {
Map<String, HTableDescriptor> htds = new TreeMap<String, HTableDescriptor>(); Map<String, TableDescriptor> tds = new TreeMap<String, TableDescriptor>();
if (fsvisited && usecache) { if (fsvisited && usecache) {
for (Map.Entry<TableName, HTableDescriptor> entry: this.cache.entrySet()) { for (Map.Entry<TableName, TableDescriptor> entry: this.cache.entrySet()) {
htds.put(entry.getKey().toString(), entry.getValue()); tds.put(entry.getKey().toString(), entry.getValue());
} }
// add hbase:meta to the response // add hbase:meta to the response
htds.put(HTableDescriptor.META_TABLEDESC.getTableName().getNameAsString(), tds.put(HTableDescriptor.META_TABLEDESC.getTableName().getNameAsString(),
HTableDescriptor.META_TABLEDESC); new TableDescriptor(HTableDescriptor.META_TABLEDESC, TableState.State.ENABLED));
} else { } else {
LOG.debug("Fetching table descriptors from the filesystem."); LOG.debug("Fetching table descriptors from the filesystem.");
boolean allvisited = true; boolean allvisited = true;
for (Path d : FSUtils.getTableDirs(fs, rootdir)) { for (Path d : FSUtils.getTableDirs(fs, rootdir)) {
HTableDescriptor htd = null; TableDescriptor td = null;
try { try {
htd = get(FSUtils.getTableName(d)); td = getDescriptor(FSUtils.getTableName(d));
} catch (FileNotFoundException fnfe) { } catch (FileNotFoundException fnfe) {
// inability of retrieving one HTD shouldn't stop getting the remaining // inability of retrieving one HTD shouldn't stop getting the remaining
LOG.warn("Trouble retrieving htd", fnfe); LOG.warn("Trouble retrieving htd", fnfe);
} }
if (htd == null) { if (td == null) {
allvisited = false; allvisited = false;
continue; continue;
} else { } else {
htds.put(htd.getTableName().getNameAsString(), htd); tds.put(td.getHTableDescriptor().getTableName().getNameAsString(), td);
} }
fsvisited = allvisited; 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; return htds;
} }
/* (non-Javadoc) /**
* @see org.apache.hadoop.hbase.TableDescriptors#getTableDescriptors(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path) * Find descriptors by namespace.
*/ * @see #get(org.apache.hadoop.hbase.TableName)
*/
@Override @Override
public Map<String, HTableDescriptor> getByNamespace(String name) public Map<String, HTableDescriptor> getByNamespace(String name)
throws IOException { throws IOException {
@ -250,6 +273,27 @@ public class FSTableDescriptors implements TableDescriptors {
return htds; return htds;
} }
/**
* Adds (or updates) the table descriptor to the FileSystem
* and updates the local cache with it.
*/
@Override
public void add(TableDescriptor htd) throws IOException {
if (fsreadonly) {
throw new NotImplementedException("Cannot add a table descriptor - in read only mode");
}
TableName tableName = htd.getHTableDescriptor().getTableName();
if (TableName.META_TABLE_NAME.equals(tableName)) {
throw new NotImplementedException();
}
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tableName.getNameAsString())) {
throw new NotImplementedException(
"Cannot add a table descriptor for a reserved subdirectory name: "
+ htd.getHTableDescriptor().getNameAsString());
}
updateTableDescriptor(htd);
}
/** /**
* Adds (or updates) the table descriptor to the FileSystem * Adds (or updates) the table descriptor to the FileSystem
* and updates the local cache with it. * and updates the local cache with it.
@ -259,14 +303,23 @@ public class FSTableDescriptors implements TableDescriptors {
if (fsreadonly) { if (fsreadonly) {
throw new NotImplementedException("Cannot add a table descriptor - in read only mode"); throw new NotImplementedException("Cannot add a table descriptor - in read only mode");
} }
if (TableName.META_TABLE_NAME.equals(htd.getTableName())) { TableName tableName = htd.getTableName();
if (TableName.META_TABLE_NAME.equals(tableName)) {
throw new NotImplementedException(); throw new NotImplementedException();
} }
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(htd.getTableName().getNameAsString())) { if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tableName.getNameAsString())) {
throw new NotImplementedException( throw new NotImplementedException(
"Cannot add a table descriptor for a reserved subdirectory name: " + htd.getNameAsString()); "Cannot add a table descriptor for a reserved subdirectory name: "
+ htd.getNameAsString());
} }
updateTableDescriptor(htd); TableDescriptor descriptor = getDescriptor(htd.getTableName());
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()); throw new IOException("Failed delete of " + tabledir.toString());
} }
} }
HTableDescriptor descriptor = this.cache.remove(tablename); TableDescriptor descriptor = this.cache.remove(tablename);
if (descriptor == null) { if (descriptor == null) {
return null; return null;
} else { } else {
return descriptor; return descriptor.getHTableDescriptor();
} }
} }
@ -474,8 +527,8 @@ public class FSTableDescriptors implements TableDescriptors {
* if it exists, bypassing the local cache. * if it exists, bypassing the local cache.
* Returns null if it's not found. * Returns null if it's not found.
*/ */
public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs, public static TableDescriptor getTableDescriptorFromFs(FileSystem fs,
Path hbaseRootDir, TableName tableName) throws IOException { Path hbaseRootDir, TableName tableName) throws IOException {
Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName); Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
return getTableDescriptorFromFs(fs, tableDir); return getTableDescriptorFromFs(fs, tableDir);
} }
@ -485,37 +538,16 @@ public class FSTableDescriptors implements TableDescriptors {
* directly from the file system if it exists. * directly from the file system if it exists.
* @throws TableInfoMissingException if there is no descriptor * @throws TableInfoMissingException if there is no descriptor
*/ */
public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs, public static TableDescriptor getTableDescriptorFromFs(FileSystem fs, Path tableDir)
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)
throws IOException { 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); FileStatus status = getTableInfoPath(fs, tableDir, false);
if (status == null) { if (status == null) {
throw new TableInfoMissingException("No table descriptor file under " + tableDir); 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 { boolean rewritePb) throws IOException {
int len = Ints.checkedCast(status.getLen()); int len = Ints.checkedCast(status.getLen());
byte [] content = new byte[len]; byte [] content = new byte[len];
@ -525,30 +557,32 @@ public class FSTableDescriptors implements TableDescriptors {
} finally { } finally {
fsDataInputStream.close(); fsDataInputStream.close();
} }
HTableDescriptor htd = null; TableDescriptor td = null;
try { try {
htd = HTableDescriptor.parseFrom(content); td = TableDescriptor.parseFrom(content);
} catch (DeserializationException e) { } catch (DeserializationException e) {
// we have old HTableDescriptor here // we have old HTableDescriptor here
try { try {
HTableDescriptor ohtd = HTableDescriptor.parseFrom(content); HTableDescriptor ohtd = HTableDescriptor.parseFrom(content);
LOG.warn("Found old table descriptor, converting to new format for table " + LOG.warn("Found old table descriptor, converting to new format for table " +
ohtd.getTableName()); ohtd.getTableName());
htd = new HTableDescriptor(ohtd); td = new TableDescriptor(ohtd);
if (rewritePb) rewriteTableDescriptor(fs, status, htd); if (rewritePb) {
rewriteTableDescriptor(fs, status, td);
}
} catch (DeserializationException e1) { } catch (DeserializationException e1) {
throw new IOException("content=" + Bytes.toShort(content), e1); throw new IOException("content=" + Bytes.toShort(content), e1);
} }
} }
if (rewritePb && !ProtobufUtil.isPBMagicPrefix(content)) { if (rewritePb && !ProtobufUtil.isPBMagicPrefix(content)) {
// Convert the file over to be pb before leaving here. // 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, private static void rewriteTableDescriptor(final FileSystem fs, final FileStatus status,
final HTableDescriptor td) final TableDescriptor td)
throws IOException { throws IOException {
Path tableInfoDir = status.getPath().getParent(); Path tableInfoDir = status.getPath().getParent();
Path tableDir = tableInfoDir.getParent(); Path tableDir = tableInfoDir.getParent();
@ -560,17 +594,18 @@ public class FSTableDescriptors implements TableDescriptors {
* @throws IOException Thrown if failed update. * @throws IOException Thrown if failed update.
* @throws NotImplementedException if in read only mode * @throws NotImplementedException if in read only mode
*/ */
@VisibleForTesting Path updateTableDescriptor(HTableDescriptor htd) @VisibleForTesting Path updateTableDescriptor(TableDescriptor td)
throws IOException { throws IOException {
if (fsreadonly) { if (fsreadonly) {
throw new NotImplementedException("Cannot update a table descriptor - in read only mode"); throw new NotImplementedException("Cannot update a table descriptor - in read only mode");
} }
Path tableDir = getTableDir(htd.getTableName()); TableName tableName = td.getHTableDescriptor().getTableName();
Path p = writeTableDescriptor(fs, htd, tableDir, getTableInfoPath(tableDir)); Path tableDir = getTableDir(tableName);
Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(tableDir));
if (p == null) throw new IOException("Failed update"); if (p == null) throw new IOException("Failed update");
LOG.info("Updated tableinfo=" + p); LOG.info("Updated tableinfo=" + p);
if (usecache) { if (usecache) {
this.cache.put(htd.getTableName(), htd); this.cache.put(td.getHTableDescriptor().getTableName(), td);
} }
return p; return p;
} }
@ -621,9 +656,8 @@ public class FSTableDescriptors implements TableDescriptors {
* @return Descriptor file or null if we failed write. * @return Descriptor file or null if we failed write.
*/ */
private static Path writeTableDescriptor(final FileSystem fs, private static Path writeTableDescriptor(final FileSystem fs,
final HTableDescriptor htd, final Path tableDir, final TableDescriptor htd, final Path tableDir,
final FileStatus currentDescriptorFile) final FileStatus currentDescriptorFile) throws IOException {
throws IOException {
// Get temporary dir into which we'll first write a file to avoid half-written file phenomenon. // Get temporary dir into which we'll first write a file to avoid half-written file phenomenon.
// This directory is never removed to avoid removing it out from under a concurrent writer. // This directory is never removed to avoid removing it out from under a concurrent writer.
Path tmpTableDir = new Path(tableDir, TMP_DIR); Path tmpTableDir = new Path(tableDir, TMP_DIR);
@ -652,7 +686,7 @@ public class FSTableDescriptors implements TableDescriptors {
} }
tableInfoDirPath = new Path(tableInfoDir, filename); tableInfoDirPath = new Path(tableInfoDir, filename);
try { try {
writeHTD(fs, tempPath, htd); writeTD(fs, tempPath, htd);
fs.mkdirs(tableInfoDirPath.getParent()); fs.mkdirs(tableInfoDirPath.getParent());
if (!fs.rename(tempPath, tableInfoDirPath)) { if (!fs.rename(tempPath, tableInfoDirPath)) {
throw new IOException("Failed rename of " + tempPath + " to " + tableInfoDirPath); throw new IOException("Failed rename of " + tempPath + " to " + tableInfoDirPath);
@ -676,7 +710,7 @@ public class FSTableDescriptors implements TableDescriptors {
return tableInfoDirPath; return tableInfoDirPath;
} }
private static void writeHTD(final FileSystem fs, final Path p, final HTableDescriptor htd) private static void writeTD(final FileSystem fs, final Path p, final TableDescriptor htd)
throws IOException { throws IOException {
FSDataOutputStream out = fs.create(p, false); FSDataOutputStream out = fs.create(p, false);
try { try {
@ -693,10 +727,19 @@ public class FSTableDescriptors implements TableDescriptors {
* Used by tests. * Used by tests.
* @return True if we successfully created file. * @return True if we successfully created file.
*/ */
public boolean createTableDescriptor(HTableDescriptor htd) throws IOException { public boolean createTableDescriptor(TableDescriptor htd) throws IOException {
return createTableDescriptor(htd, false); return createTableDescriptor(htd, false);
} }
/**
* Create new HTableDescriptor in HDFS. Happens when we are creating table.
* Used by tests.
* @return True if we successfully created file.
*/
public boolean createTableDescriptor(HTableDescriptor htd) throws IOException {
return createTableDescriptor(new TableDescriptor(htd), false);
}
/** /**
* Create new HTableDescriptor in HDFS. Happens when we are creating table. If * Create new HTableDescriptor in HDFS. Happens when we are creating table. If
* forceCreation is true then even if previous table descriptor is present it * forceCreation is true then even if previous table descriptor is present it
@ -704,9 +747,9 @@ public class FSTableDescriptors implements TableDescriptors {
* *
* @return True if we successfully created file. * @return True if we successfully created file.
*/ */
public boolean createTableDescriptor(HTableDescriptor htd, boolean forceCreation) public boolean createTableDescriptor(TableDescriptor htd, boolean forceCreation)
throws IOException { throws IOException {
Path tableDir = getTableDir(htd.getTableName()); Path tableDir = getTableDir(htd.getHTableDescriptor().getTableName());
return createTableDescriptorForTableDirectory(tableDir, htd, forceCreation); return createTableDescriptorForTableDirectory(tableDir, htd, forceCreation);
} }
@ -722,7 +765,7 @@ public class FSTableDescriptors implements TableDescriptors {
* @throws IOException if a filesystem error occurs * @throws IOException if a filesystem error occurs
*/ */
public boolean createTableDescriptorForTableDirectory(Path tableDir, public boolean createTableDescriptorForTableDirectory(Path tableDir,
HTableDescriptor htd, boolean forceCreation) throws IOException { TableDescriptor htd, boolean forceCreation) throws IOException {
if (fsreadonly) { if (fsreadonly) {
throw new NotImplementedException("Cannot create a table descriptor - in read only mode"); throw new NotImplementedException("Cannot create a table descriptor - in read only mode");
} }
@ -743,4 +786,3 @@ public class FSTableDescriptors implements TableDescriptors {
} }
} }

View File

@ -71,7 +71,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HColumnDescriptor; 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.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.client.Table;
import org.apache.hadoop.hbase.io.FileLink; import org.apache.hadoop.hbase.io.FileLink;
import org.apache.hadoop.hbase.io.HFileLink; 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.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface; 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.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo; 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.TableIntegrityErrorHandlerImpl;
import org.apache.hadoop.hbase.util.hbck.TableLockChecker; import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
import org.apache.hadoop.hbase.wal.WALSplitter; 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.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
@ -1337,9 +1334,9 @@ public class HBaseFsck extends Configured implements Closeable {
modTInfo = new TableInfo(tableName); modTInfo = new TableInfo(tableName);
tablesInfo.put(tableName, modTInfo); tablesInfo.put(tableName, modTInfo);
try { try {
HTableDescriptor htd = TableDescriptor htd =
FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName); FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName);
modTInfo.htds.add(htd); modTInfo.htds.add(htd.getHTableDescriptor());
} catch (IOException ioe) { } catch (IOException ioe) {
if (!orphanTableDirs.containsKey(tableName)) { if (!orphanTableDirs.containsKey(tableName)) {
LOG.warn("Unable to read .tableinfo from " + hbaseRoot, ioe); LOG.warn("Unable to read .tableinfo from " + hbaseRoot, ioe);
@ -1394,7 +1391,7 @@ public class HBaseFsck extends Configured implements Closeable {
for (String columnfamimly : columns) { for (String columnfamimly : columns) {
htd.addFamily(new HColumnDescriptor(columnfamimly)); htd.addFamily(new HColumnDescriptor(columnfamimly));
} }
fstd.createTableDescriptor(htd, true); fstd.createTableDescriptor(new TableDescriptor(htd, TableState.State.ENABLED), true);
return true; return true;
} }
@ -1442,7 +1439,7 @@ public class HBaseFsck extends Configured implements Closeable {
if (tableName.equals(htds[j].getTableName())) { if (tableName.equals(htds[j].getTableName())) {
HTableDescriptor htd = htds[j]; HTableDescriptor htd = htds[j];
LOG.info("fixing orphan table: " + tableName + " from cache"); LOG.info("fixing orphan table: " + tableName + " from cache");
fstd.createTableDescriptor(htd, true); fstd.createTableDescriptor(new TableDescriptor(htd, TableState.State.ENABLED), true);
j++; j++;
iter.remove(); iter.remove();
} }
@ -1802,19 +1799,16 @@ public class HBaseFsck extends Configured implements Closeable {
* @throws IOException * @throws IOException
*/ */
private void loadDisabledTables() private void loadDisabledTables()
throws ZooKeeperConnectionException, IOException { throws IOException {
HConnectionManager.execute(new HConnectable<Void>(getConf()) { HConnectionManager.execute(new HConnectable<Void>(getConf()) {
@Override @Override
public Void connect(HConnection connection) throws IOException { public Void connect(HConnection connection) throws IOException {
try { TableName[] tables = connection.listTableNames();
for (TableName tableName : for (TableName table : tables) {
ZKTableStateClientSideReader.getDisabledOrDisablingTables(zkw)) { if (connection.getTableState(table)
disabledTables.add(tableName); .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; 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. * Check whether a orphaned table ZNode exists and fix it if requested.
* @throws IOException * @throws IOException
* @throws KeeperException
* @throws InterruptedException
*/ */
private void checkAndFixOrphanedTableZNodes() private void checkAndFixOrphanedTableZNodes()
throws IOException, KeeperException, InterruptedException { throws IOException {
Set<TableName> enablingTables = ZKTableStateClientSideReader.getEnablingTables(zkw); Set<TableName> enablingTables = new HashSet<>();
for (TableName tableName: admin.listTableNames()) {
if (connection.getTableState(tableName).getState().equals(TableState.State.ENABLING)) {
enablingTables.add(tableName);
}
}
String msg; String msg;
TableInfo tableInfo; TableInfo tableInfo;
@ -3570,21 +3567,12 @@ public class HBaseFsck extends Configured implements Closeable {
} }
if (orphanedTableZNodes.size() > 0 && this.fixTableZNodes) { if (orphanedTableZNodes.size() > 0 && this.fixTableZNodes) {
ZKTableStateManager zkTableStateMgr = new ZKTableStateManager(zkw);
for (TableName tableName : orphanedTableZNodes) { for (TableName tableName : orphanedTableZNodes) {
try { // Set the table state to be disabled so that if we made mistake, we can trace
// Set the table state to be disabled so that if we made mistake, we can trace // the history and figure it out.
// the history and figure it out. // Another choice is to call checkAndRemoveTableState() to delete the orphaned ZNode.
// Another choice is to call checkAndRemoveTableState() to delete the orphaned ZNode. // Both approaches works.
// Both approaches works. admin.disableTable(tableName);
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);
}
} }
} }
} }

View File

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

View File

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

View File

@ -18,8 +18,11 @@
package org.apache.hadoop.hbase.util; package org.apache.hadoop.hbase.util;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -27,6 +30,9 @@ import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; 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); byte[] data = ZKUtil.getData(zkw, znode);
if (ProtobufUtil.isPBMagicPrefix(data)) continue; if (ProtobufUtil.isPBMagicPrefix(data)) continue;
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder(); ZooKeeperProtos.DeprecatedTableState.Builder builder =
builder.setState(ZooKeeperProtos.Table.State.valueOf(Bytes.toString(data))); ZooKeeperProtos.DeprecatedTableState.newBuilder();
builder.setState(ZooKeeperProtos.DeprecatedTableState.State.valueOf(Bytes.toString(data)));
data = ProtobufUtil.prependPBMagic(builder.build().toByteArray()); data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
ZKUtil.setData(zkw, znode, data); ZKUtil.setData(zkw, znode, data);
} }
@ -232,15 +239,14 @@ public class ZKDataMigrator extends Configured implements Tool {
} }
private void migrateClusterKeyToPB(ZooKeeperWatcher zkw, String peerZnode, byte[] data) private void migrateClusterKeyToPB(ZooKeeperWatcher zkw, String peerZnode, byte[] data)
throws KeeperException, NoNodeException { throws KeeperException {
ReplicationPeer peer = ZooKeeperProtos.ReplicationPeer.newBuilder() ReplicationPeer peer = ZooKeeperProtos.ReplicationPeer.newBuilder()
.setClusterkey(Bytes.toString(data)).build(); .setClusterkey(Bytes.toString(data)).build();
ZKUtil.setData(zkw, peerZnode, ProtobufUtil.prependPBMagic(peer.toByteArray())); ZKUtil.setData(zkw, peerZnode, ProtobufUtil.prependPBMagic(peer.toByteArray()));
} }
private void migratePeerStateToPB(ZooKeeperWatcher zkw, byte[] data, private void migratePeerStateToPB(ZooKeeperWatcher zkw, byte[] data,
String peerStatePath) String peerStatePath) throws KeeperException {
throws KeeperException, NoNodeException {
String state = Bytes.toString(data); String state = Bytes.toString(data);
if (ZooKeeperProtos.ReplicationState.State.ENABLED.name().equals(state)) { if (ZooKeeperProtos.ReplicationState.State.ENABLED.name().equals(state)) {
ZKUtil.setData(zkw, peerStatePath, ReplicationStateZKBase.ENABLED_ZNODE_BYTES); 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 { public static void main(String args[]) throws Exception {
System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZKDataMigrator(), args)); System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZKDataMigrator(), args));
} }

View File

@ -50,6 +50,9 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -61,7 +64,6 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -71,7 +73,6 @@ import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.ConnectionUtils; import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.Delete; 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.client.Put;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination; import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.RegionOpeningException; import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.master.SplitLogManager; 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.ClusterStatusProtos.StoreSequenceId;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.LastSequenceId; 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 org.apache.hadoop.io.MultipleIOException;
import com.google.common.annotations.VisibleForTesting; 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; import com.google.protobuf.TextFormat;
/** /**
@ -335,13 +333,14 @@ public class WALSplitter {
LOG.warn("Nothing to split in log file " + logPath); LOG.warn("Nothing to split in log file " + logPath);
return true; return true;
} }
if (csm != null) { if(csm != null) {
try { HConnection scc = csm.getServer().getConnection();
TableStateManager tsm = csm.getTableStateManager(); TableName[] tables = scc.listTableNames();
disablingOrDisabledTables = tsm.getTablesInStates( for (TableName table : tables) {
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING); if (scc.getTableState(table)
} catch (CoordinatedStateException e) { .inStates(TableState.State.DISABLED, TableState.State.DISABLING)) {
throw new IOException("Can't get disabling/disabled tables", e); disablingOrDisabledTables.add(table);
}
} }
} }
int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3); int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3);

View File

@ -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);
}
}

View File

@ -3390,6 +3390,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
} }
} }
/** /**
* Make sure that at least the specified number of region servers * Make sure that at least the specified number of region servers
* are running * are running

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.ServerManager; 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.master.balancer.LoadBalancerFactory;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.testclassification.MediumTests; 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.assertFalse;
import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue; 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"), final HRegionInfo REGIONINFO = new HRegionInfo(TableName.valueOf("table_test"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW); HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), try (ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
"zkWatcher-Test", abortable, true); "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_A, ServerLoad.EMPTY_SERVERLOAD);
onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD); onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
Mockito.when(server.getConfiguration()).thenReturn(conf); Mockito.when(server.getConfiguration()).thenReturn(conf);
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1")); Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher); Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
Mockito.when(server.getRegionServerVersion(Mockito.any(ServerName.class))).thenReturn("0.0.0"); Mockito.when(server.getRegionServerVersion(Mockito.any(ServerName.class))).thenReturn("0.0.0");
CoordinatedStateManager cp = new ZkCoordinatedStateManager(); CoordinatedStateManager cp = new ZkCoordinatedStateManager();
cp.initialize(server); cp.initialize(server);
cp.start(); cp.start();
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp); Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers); Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
Mockito.when(serverManager.getOnlineServersList()) Mockito.when(serverManager.getOnlineServersList())
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet())); .thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList()) Mockito.when(serverManager.createDestinationServersList())
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet())); .thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList(null)) Mockito.when(serverManager.createDestinationServersList(null))
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet())); .thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList(Mockito.anyList())).thenReturn( Mockito.when(serverManager.createDestinationServersList(Mockito.anyList())).thenReturn(
new ArrayList<ServerName>(onlineServers.keySet())); new ArrayList<ServerName>(onlineServers.keySet()));
for (ServerName sn : onlineServers.keySet()) { for (ServerName sn : onlineServers.keySet()) {
Mockito.when(serverManager.isServerOnline(sn)).thenReturn(true); Mockito.when(serverManager.isServerOnline(sn)).thenReturn(true);
Mockito.when(serverManager.sendRegionClose(sn, REGIONINFO, -1)).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.sendRegionClose(sn, REGIONINFO, -1, null, false)).thenReturn(true);
Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, new ArrayList<ServerName>())) Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, new ArrayList<ServerName>()))
.thenReturn(RegionOpeningState.OPENED); .thenReturn(RegionOpeningState.OPENED);
Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, null)) Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, null))
.thenReturn(RegionOpeningState.OPENED); .thenReturn(RegionOpeningState.OPENED);
Mockito.when(serverManager.addServerToDrainList(sn)).thenReturn(true); 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 @Test
@ -207,80 +211,82 @@ public class TestDrainingServer {
bulk.put(REGIONINFO_D, SERVERNAME_D); bulk.put(REGIONINFO_D, SERVERNAME_D);
bulk.put(REGIONINFO_E, SERVERNAME_E); bulk.put(REGIONINFO_E, SERVERNAME_E);
ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), try (ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
"zkWatcher-BulkAssignTest", abortable, true); "zkWatcher-BulkAssignTest", abortable, true)) {
Mockito.when(server.getConfiguration()).thenReturn(conf); Mockito.when(server.getConfiguration()).thenReturn(conf);
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1")); Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher); Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
CoordinatedStateManager cp = new ZkCoordinatedStateManager(); CoordinatedStateManager cp = new ZkCoordinatedStateManager();
cp.initialize(server); cp.initialize(server);
cp.start(); cp.start();
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp); Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers); Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
Mockito.when(serverManager.getOnlineServersList()).thenReturn( Mockito.when(serverManager.getOnlineServersList()).thenReturn(
new ArrayList<ServerName>(onlineServers.keySet())); new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList()).thenReturn( Mockito.when(serverManager.createDestinationServersList()).thenReturn(
new ArrayList<ServerName>(onlineServers.keySet())); new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList(null)).thenReturn( Mockito.when(serverManager.createDestinationServersList(null)).thenReturn(
new ArrayList<ServerName>(onlineServers.keySet())); new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList(Mockito.anyList())).thenReturn( Mockito.when(serverManager.createDestinationServersList(Mockito.anyList())).thenReturn(
new ArrayList<ServerName>(onlineServers.keySet())); new ArrayList<ServerName>(onlineServers.keySet()));
for (Entry<HRegionInfo, ServerName> entry : bulk.entrySet()) { for (Entry<HRegionInfo, ServerName> entry : bulk.entrySet()) {
Mockito.when(serverManager.isServerOnline(entry.getValue())).thenReturn(true); Mockito.when(serverManager.isServerOnline(entry.getValue())).thenReturn(true);
Mockito.when(serverManager.sendRegionClose(entry.getValue(), Mockito.when(serverManager.sendRegionClose(entry.getValue(),
entry.getKey(), -1)).thenReturn(true); entry.getKey(), -1)).thenReturn(true);
Mockito.when(serverManager.sendRegionOpen(entry.getValue(), Mockito.when(serverManager.sendRegionOpen(entry.getValue(),
entry.getKey(), -1, null)).thenReturn(RegionOpeningState.OPENED); entry.getKey(), -1, null)).thenReturn(RegionOpeningState.OPENED);
Mockito.when(serverManager.addServerToDrainList(entry.getValue())).thenReturn(true); 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_A);
drainedServers.add(SERVERNAME_B); drainedServers.add(SERVERNAME_B);
drainedServers.add(SERVERNAME_C); drainedServers.add(SERVERNAME_C);
drainedServers.add(SERVERNAME_D); drainedServers.add(SERVERNAME_D);
am = new AssignmentManager(server, serverManager, TableStateManager tsm = mock(TableStateManager.class);
balancer, startupMasterExecutor("mockExecutorServiceBulk"), null, null); 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) { for (ServerName drained : drainedServers) {
addServerToDrainedList(drained, onlineServers, serverManager); addServerToDrainedList(drained, onlineServers, serverManager);
} }
am.assign(bulk); am.assign(bulk);
Set<RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition(); Set<RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
for (RegionState rs : regionsInTransition) { for (RegionState rs : regionsInTransition) {
setRegionOpenedOnZK(zkWatcher, rs.getServerName(), rs.getRegion()); setRegionOpenedOnZK(zkWatcher, rs.getServerName(), rs.getRegion());
} }
am.waitForAssignment(REGIONINFO_A); am.waitForAssignment(REGIONINFO_A);
am.waitForAssignment(REGIONINFO_B); am.waitForAssignment(REGIONINFO_B);
am.waitForAssignment(REGIONINFO_C); am.waitForAssignment(REGIONINFO_C);
am.waitForAssignment(REGIONINFO_D); am.waitForAssignment(REGIONINFO_D);
am.waitForAssignment(REGIONINFO_E); am.waitForAssignment(REGIONINFO_E);
Map<HRegionInfo, ServerName> regionAssignments = am.getRegionStates().getRegionAssignments(); Map<HRegionInfo, ServerName> regionAssignments = am.getRegionStates().getRegionAssignments();
for (Entry<HRegionInfo, ServerName> entry : regionAssignments.entrySet()) { for (Entry<HRegionInfo, ServerName> entry : regionAssignments.entrySet()) {
LOG.info("Region Assignment: " LOG.info("Region Assignment: "
+ entry.getKey().getRegionNameAsString() + " Server: " + entry.getValue()); + entry.getKey().getRegionNameAsString() + " Server: " + entry.getValue());
bunchServersAssigned.add(entry.getValue()); bunchServersAssigned.add(entry.getValue());
} }
for (ServerName sn : drainedServers) { for (ServerName sn : drainedServers) {
assertFalse(bunchServersAssigned.contains(sn)); assertFalse(bunchServersAssigned.contains(sn));
} }
}
} }
private void addServerToDrainedList(ServerName serverName, private void addServerToDrainedList(ServerName serverName,

View File

@ -42,8 +42,8 @@ public class TestFSTableDescriptorForceCreation {
Path rootdir = new Path(UTIL.getDataTestDir(), name); Path rootdir = new Path(UTIL.getDataTestDir(), name);
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir); FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
assertTrue("Should create new table descriptor",
assertTrue("Should create new table descriptor", fstd.createTableDescriptor(htd, false)); fstd.createTableDescriptor(new TableDescriptor(htd), false));
} }
@Test @Test
@ -56,7 +56,8 @@ public class TestFSTableDescriptorForceCreation {
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir); FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
HTableDescriptor htd = new HTableDescriptor(name); HTableDescriptor htd = new HTableDescriptor(name);
fstd.add(htd); 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 @Test
@ -67,9 +68,10 @@ public class TestFSTableDescriptorForceCreation {
Path rootdir = new Path(UTIL.getDataTestDir(), name); Path rootdir = new Path(UTIL.getDataTestDir(), name);
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir); FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); 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", assertTrue("Should create new table descriptor",
fstd.createTableDescriptor(htd, true)); fstd.createTableDescriptor(td, true));
} }
} }

View File

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

View File

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

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; 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.TableNotFoundException;
import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.exceptions.MergeRegionException; 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.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; 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.AssignmentManager;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -255,7 +253,7 @@ public class TestAdmin1 {
this.admin.disableTable(ht.getName()); this.admin.disableTable(ht.getName());
assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster() assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
.getMaster().getAssignmentManager().getTableStateManager().isTableState( .getMaster().getAssignmentManager().getTableStateManager().isTableState(
ht.getName(), ZooKeeperProtos.Table.State.DISABLED)); ht.getName(), TableState.State.DISABLED));
// Test that table is disabled // Test that table is disabled
get = new Get(row); get = new Get(row);
@ -282,7 +280,7 @@ public class TestAdmin1 {
this.admin.enableTable(table); this.admin.enableTable(table);
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster() assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
.getMaster().getAssignmentManager().getTableStateManager().isTableState( .getMaster().getAssignmentManager().getTableStateManager().isTableState(
ht.getName(), ZooKeeperProtos.Table.State.ENABLED)); ht.getName(), TableState.State.ENABLED));
// Test that table is enabled // Test that table is enabled
try { try {
@ -354,7 +352,7 @@ public class TestAdmin1 {
assertEquals(numTables + 1, tables.length); assertEquals(numTables + 1, tables.length);
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster() assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
.getMaster().getAssignmentManager().getTableStateManager().isTableState( .getMaster().getAssignmentManager().getTableStateManager().isTableState(
TableName.valueOf("testCreateTable"), ZooKeeperProtos.Table.State.ENABLED)); TableName.valueOf("testCreateTable"), TableState.State.ENABLED));
} }
@Test (timeout=300000) @Test (timeout=300000)
@ -1340,11 +1338,9 @@ public class TestAdmin1 {
@Test (timeout=300000) @Test (timeout=300000)
public void testEnableDisableAddColumnDeleteColumn() throws Exception { public void testEnableDisableAddColumnDeleteColumn() throws Exception {
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
TableName tableName = TableName.valueOf("testEnableDisableAddColumnDeleteColumn"); TableName tableName = TableName.valueOf("testEnableDisableAddColumnDeleteColumn");
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close(); TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
while (!ZKTableStateClientSideReader.isEnabledTable(zkw, while (!this.admin.isTableEnabled(tableName)) {
TableName.valueOf("testEnableDisableAddColumnDeleteColumn"))) {
Thread.sleep(10); Thread.sleep(10);
} }
this.admin.disableTable(tableName); this.admin.disableTable(tableName);
@ -1487,16 +1483,4 @@ public class TestAdmin1 {
this.admin.deleteTable(tableName); 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.
}
}
} }

View File

@ -70,6 +70,11 @@ public class MockNoopMasterServices implements MasterServices, Server {
return null; return null;
} }
@Override
public TableStateManager getTableStateManager() {
return null;
}
@Override @Override
public MasterCoprocessorHost getMasterCoprocessorHost() { public MasterCoprocessorHost getMasterCoprocessorHost() {
return null; return null;

View File

@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@ -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.RegionState.State;
import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer; import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -156,10 +156,9 @@ public class TestAssignmentManagerOnCluster {
Bytes.toBytes(metaServerName.getServerName())); Bytes.toBytes(metaServerName.getServerName()));
master.assignmentManager.waitUntilNoRegionsInTransition(60000); master.assignmentManager.waitUntilNoRegionsInTransition(60000);
} }
RegionState metaState = RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper());
MetaTableLocator.getMetaRegionState(master.getZooKeeper()); assertEquals("Meta should be not in transition",
assertEquals("Meta should be not in transition", metaState.getState(), RegionState.State.OPEN);
metaState.getState(), RegionState.State.OPEN);
assertNotEquals("Meta should be moved off master", assertNotEquals("Meta should be moved off master",
metaServerName, master.getServerName()); metaServerName, master.getServerName());
cluster.killRegionServer(metaServerName); cluster.killRegionServer(metaServerName);
@ -289,7 +288,8 @@ public class TestAssignmentManagerOnCluster {
String table = "testAssignRegionOnRestartedServer"; String table = "testAssignRegionOnRestartedServer";
TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 20); TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 20);
TEST_UTIL.getMiniHBaseCluster().stopMaster(0); TEST_UTIL.getMiniHBaseCluster().stopMaster(0);
TEST_UTIL.getMiniHBaseCluster().startMaster(); //restart the master so that conf take into affect //restart the master so that conf take into affect
TEST_UTIL.getMiniHBaseCluster().startMaster();
ServerName deadServer = null; ServerName deadServer = null;
HMaster master = null; HMaster master = null;
@ -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); List<HRegionInfo> toAssignRegions = am.cleanOutCrashedServerReferences(destServerName);
assertTrue("Regions to be assigned should be empty.", toAssignRegions.isEmpty()); assertTrue("Regions to be assigned should be empty.", toAssignRegions.isEmpty());
assertTrue("Regions to be assigned should be empty.", am.getRegionStates() assertTrue("Regions to be assigned should be empty.", am.getRegionStates()
@ -897,7 +897,7 @@ public class TestAssignmentManagerOnCluster {
if (hri != null && serverName != null) { if (hri != null && serverName != null) {
am.regionOnline(hri, serverName); am.regionOnline(hri, serverName);
} }
am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLED); am.getTableStateManager().setTableState(table, TableState.State.DISABLED);
TEST_UTIL.deleteTable(table); TEST_UTIL.deleteTable(table);
} }
} }

View File

@ -41,6 +41,8 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.CoordinatedStateManager; 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.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; 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.ServerName;
import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.TableDescriptors; 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.ClusterConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination; import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails; 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.executor.ExecutorService;
import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator; import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
@ -352,13 +354,18 @@ public class TestCatalogJanitor {
return new TableDescriptors() { return new TableDescriptors() {
@Override @Override
public HTableDescriptor remove(TableName tablename) throws IOException { public HTableDescriptor remove(TableName tablename) throws IOException {
// TODO Auto-generated method stub // noop
return null; return null;
} }
@Override @Override
public Map<String, HTableDescriptor> getAll() throws IOException { public Map<String, HTableDescriptor> getAll() throws IOException {
// TODO Auto-generated method stub // noop
return null;
}
@Override public Map<String, TableDescriptor> getAllDescriptors() throws IOException {
// noop
return null; return null;
} }
@ -368,6 +375,12 @@ public class TestCatalogJanitor {
return createHTableDescriptor(); return createHTableDescriptor();
} }
@Override
public TableDescriptor getDescriptor(TableName tablename)
throws IOException {
return createTableDescriptor();
}
@Override @Override
public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException { public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException {
return null; return null;
@ -375,8 +388,12 @@ public class TestCatalogJanitor {
@Override @Override
public void add(HTableDescriptor htd) throws IOException { public void add(HTableDescriptor htd) throws IOException {
// TODO Auto-generated method stub // noop
}
@Override
public void add(TableDescriptor htd) throws IOException {
// noop
} }
@Override @Override
public void setCacheOn() throws IOException { public void setCacheOn() throws IOException {
@ -540,6 +557,11 @@ public class TestCatalogJanitor {
return null; return null;
} }
@Override
public TableStateManager getTableStateManager() {
return null;
}
@Override @Override
public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b, public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
boolean forcible, User user) throws IOException { boolean forcible, User user) throws IOException {
@ -1169,6 +1191,11 @@ public class TestCatalogJanitor {
return htd; return htd;
} }
private TableDescriptor createTableDescriptor() {
TableDescriptor htd = new TableDescriptor(createHTableDescriptor(), TableState.State.ENABLED);
return htd;
}
private MultiResponse buildMultiResponse(MultiRequest req) { private MultiResponse buildMultiResponse(MultiRequest req) {
MultiResponse.Builder builder = MultiResponse.newBuilder(); MultiResponse.Builder builder = MultiResponse.newBuilder();
RegionActionResult.Builder regionActionResultBuilder = RegionActionResult.Builder regionActionResultBuilder =

View File

@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
@ -84,7 +84,7 @@ public class TestMaster {
try (HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME)) { try (HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME)) {
assertTrue(m.assignmentManager.getTableStateManager().isTableState(TABLENAME, assertTrue(m.assignmentManager.getTableStateManager().isTableState(TABLENAME,
ZooKeeperProtos.Table.State.ENABLED)); TableState.State.ENABLED));
TEST_UTIL.loadTable(ht, FAMILYNAME, false); TEST_UTIL.loadTable(ht, FAMILYNAME, false);
} }

View File

@ -43,20 +43,19 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.RegionTransition; import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter; 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.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region; 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.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.data.Stat;
import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
@ -302,8 +299,8 @@ public class TestMasterFailover {
log("Beginning to mock scenarios"); log("Beginning to mock scenarios");
// Disable the disabledTable in ZK // Disable the disabledTable in ZK
TableStateManager zktable = new ZKTableStateManager(zkw); TableStateManager tsm = master.getTableStateManager();
zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED); tsm.setTableState(disabledTable, TableState.State.DISABLED);
/* /*
* ZK = OFFLINE * ZK = OFFLINE
@ -619,7 +616,7 @@ public class TestMasterFailover {
assertTrue(" Table must be enabled.", master.getAssignmentManager() assertTrue(" Table must be enabled.", master.getAssignmentManager()
.getTableStateManager().isTableState(TableName.valueOf("enabledTable"), .getTableStateManager().isTableState(TableName.valueOf("enabledTable"),
ZooKeeperProtos.Table.State.ENABLED)); TableState.State.ENABLED));
// we also need regions assigned out on the dead server // we also need regions assigned out on the dead server
List<HRegionInfo> enabledAndOnDeadRegions = new ArrayList<HRegionInfo>(); List<HRegionInfo> enabledAndOnDeadRegions = new ArrayList<HRegionInfo>();
enabledAndOnDeadRegions.addAll(enabledRegions.subList(0, 6)); enabledAndOnDeadRegions.addAll(enabledRegions.subList(0, 6));
@ -679,13 +676,11 @@ public class TestMasterFailover {
log("Beginning to mock scenarios"); log("Beginning to mock scenarios");
// Disable the disabledTable in ZK // Disable the disabledTable in ZK
TableStateManager zktable = new ZKTableStateManager(zkw); TableStateManager tsm = master.getTableStateManager();
zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED); tsm.setTableState(disabledTable, TableState.State.DISABLED);
assertTrue(" The enabled table should be identified on master fail over.", assertTrue(" The enabled table should be identified on master fail over.",
zktable.isTableState(TableName.valueOf("enabledTable"), tsm.isTableState(TableName.valueOf("enabledTable"), TableState.State.ENABLED));
ZooKeeperProtos.Table.State.ENABLED));
/* /*
* ZK = CLOSING * ZK = CLOSING
*/ */

View File

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

View File

@ -42,9 +42,9 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.MockServer; import org.apache.hadoop.hbase.util.MockServer;
import org.apache.hadoop.hbase.zookeeper.ZKAssign; 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.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -140,7 +140,10 @@ public class TestOpenedRegionHandler {
// create a node with OPENED state // create a node with OPENED state
zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL, zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
region, server.getServerName()); 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(); Stat stat = new Stat();
String nodeName = ZKAssign.getNodeName(zkw, region.getRegionInfo() String nodeName = ZKAssign.getNodeName(zkw, region.getRegionInfo()
.getEncodedName()); .getEncodedName());
@ -171,8 +174,8 @@ public class TestOpenedRegionHandler {
} catch (Exception e) { } catch (Exception e) {
expectedException = true; expectedException = true;
} }
assertFalse("The process method should not throw any exception.", assertFalse("The process method should not throw any exception. "
expectedException); , expectedException);
List<String> znodes = ZKUtil.listChildrenAndWatchForNewChildren(zkw, List<String> znodes = ZKUtil.listChildrenAndWatchForNewChildren(zkw,
zkw.assignmentZNode); zkw.assignmentZNode);
String regionName = znodes.get(0); String regionName = znodes.get(0);

View File

@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.master;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;

View File

@ -36,7 +36,6 @@ import java.util.concurrent.Future;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; 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.BaseMasterObserver;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.exceptions.LockTimeoutException;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -388,12 +386,14 @@ public class TestTableLockManager {
choreService.scheduleChore(alterThread); choreService.scheduleChore(alterThread);
choreService.scheduleChore(splitThread); choreService.scheduleChore(splitThread);
TEST_UTIL.waitTableEnabled(tableName); TEST_UTIL.waitTableEnabled(tableName);
while (true) { while (true) {
List<HRegionInfo> regions = admin.getTableRegions(tableName); List<HRegionInfo> regions = admin.getTableRegions(tableName);
LOG.info(String.format("Table #regions: %d regions: %s:", regions.size(), regions)); LOG.info(String.format("Table #regions: %d regions: %s:", regions.size(), regions));
assertEquals(admin.getTableDescriptor(tableName), desc); assertEquals(admin.getTableDescriptor(tableName), desc);
for (HRegion region : TEST_UTIL.getMiniHBaseCluster().getRegions(tableName)) { for (HRegion region : TEST_UTIL.getMiniHBaseCluster().getRegions(tableName)) {
assertEquals(desc, region.getTableDesc()); HTableDescriptor regionTableDesc = region.getTableDesc();
assertEquals(desc, regionTableDesc);
} }
if (regions.size() >= 5) { if (regions.size() >= 5) {
break; break;

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Durability; 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.MetaScanner.MetaScannerVisitorBase;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.HMaster; 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.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; 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.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
@ -188,13 +188,13 @@ public class MasterProcedureTestingUtility {
public static void validateTableIsEnabled(final HMaster master, final TableName tableName) public static void validateTableIsEnabled(final HMaster master, final TableName tableName)
throws IOException { throws IOException {
TableStateManager tsm = master.getAssignmentManager().getTableStateManager(); 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) public static void validateTableIsDisabled(final HMaster master, final TableName tableName)
throws IOException { throws IOException {
TableStateManager tsm = master.getAssignmentManager().getTableStateManager(); TableStateManager tsm = master.getAssignmentManager().getTableStateManager();
assertTrue(tsm.isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED)); assertTrue(tsm.isTableState(tableName, TableState.State.DISABLED));
} }
/** /**

View File

@ -23,10 +23,10 @@ import static org.junit.Assert.assertTrue;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility; 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.testclassification.MediumTests;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -45,6 +45,8 @@ public class TestCreateTableProcedure2 {
TEST_UTIL.shutdownMiniZKCluster(); TEST_UTIL.shutdownMiniZKCluster();
} }
/*
Note: Relevant fix was undone by HBASE-7767.
@Test @Test
public void testMasterRestartAfterNameSpaceEnablingNodeIsCreated() throws Exception { public void testMasterRestartAfterNameSpaceEnablingNodeIsCreated() throws Exception {
// Step 1: start mini zk cluster. // Step 1: start mini zk cluster.
@ -54,8 +56,9 @@ public class TestCreateTableProcedure2 {
TableName tableName = TableName.valueOf("hbase:namespace"); TableName tableName = TableName.valueOf("hbase:namespace");
ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString()); String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder(); HBaseProtos.TableState.Builder builder = HBaseProtos.TableState.newBuilder();
builder.setState(ZooKeeperProtos.Table.State.ENABLED); builder.setState(HBaseProtos.TableState.State.ENABLED);
builder.setTable(ProtobufUtil.toProtoTableName(tableName));
byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray()); byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
ZKUtil.createSetData(zkw, znode, data); ZKUtil.createSetData(zkw, znode, data);
LOG.info("Create an orphaned Znode " + znode); LOG.info("Create an orphaned Znode " + znode);
@ -65,4 +68,5 @@ public class TestCreateTableProcedure2 {
TEST_UTIL.startMiniCluster(); TEST_UTIL.startMiniCluster();
assertTrue(TEST_UTIL.getHBaseCluster().getLiveMasterThreads().size() == 1); assertTrue(TEST_UTIL.getHBaseCluster().getLiveMasterThreads().size() == 1);
} }
*/
} }

View File

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

View File

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

View File

@ -35,14 +35,16 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableExistsException; 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.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
@ -71,6 +73,7 @@ public class TestFSTableDescriptors {
public void testCreateAndUpdate() throws IOException { public void testCreateAndUpdate() throws IOException {
Path testdir = UTIL.getDataTestDir("testCreateAndUpdate"); Path testdir = UTIL.getDataTestDir("testCreateAndUpdate");
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCreate")); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCreate"));
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FileSystem fs = FileSystem.get(UTIL.getConfiguration());
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir); FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
assertTrue(fstd.createTableDescriptor(htd)); assertTrue(fstd.createTableDescriptor(htd));
@ -78,7 +81,7 @@ public class TestFSTableDescriptors {
FileStatus[] statuses = fs.listStatus(testdir); FileStatus[] statuses = fs.listStatus(testdir);
assertTrue("statuses.length=" + statuses.length, statuses.length == 1); assertTrue("statuses.length=" + statuses.length, statuses.length == 1);
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
fstd.updateTableDescriptor(htd); fstd.updateTableDescriptor(td);
} }
statuses = fs.listStatus(testdir); statuses = fs.listStatus(testdir);
assertTrue(statuses.length == 1); assertTrue(statuses.length == 1);
@ -92,20 +95,29 @@ public class TestFSTableDescriptors {
Path testdir = UTIL.getDataTestDir("testSequenceidAdvancesOnTableInfo"); Path testdir = UTIL.getDataTestDir("testSequenceidAdvancesOnTableInfo");
HTableDescriptor htd = new HTableDescriptor( HTableDescriptor htd = new HTableDescriptor(
TableName.valueOf("testSequenceidAdvancesOnTableInfo")); TableName.valueOf("testSequenceidAdvancesOnTableInfo"));
TableDescriptor td = new TableDescriptor(htd);
FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FileSystem fs = FileSystem.get(UTIL.getConfiguration());
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir); FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
Path p0 = fstd.updateTableDescriptor(htd); Path p0 = fstd.updateTableDescriptor(td);
int i0 = FSTableDescriptors.getTableInfoSequenceId(p0); int i0 = FSTableDescriptors.getTableInfoSequenceId(p0);
Path p1 = fstd.updateTableDescriptor(htd); Path p1 = fstd.updateTableDescriptor(td);
// Assert we cleaned up the old file. // Assert we cleaned up the old file.
assertTrue(!fs.exists(p0)); assertTrue(!fs.exists(p0));
int i1 = FSTableDescriptors.getTableInfoSequenceId(p1); int i1 = FSTableDescriptors.getTableInfoSequenceId(p1);
assertTrue(i1 == i0 + 1); assertTrue(i1 == i0 + 1);
Path p2 = fstd.updateTableDescriptor(htd); Path p2 = fstd.updateTableDescriptor(td);
// Assert we cleaned up the old file. // Assert we cleaned up the old file.
assertTrue(!fs.exists(p1)); assertTrue(!fs.exists(p1));
int i2 = FSTableDescriptors.getTableInfoSequenceId(p2); int i2 = FSTableDescriptors.getTableInfoSequenceId(p2);
assertTrue(i2 == i1 + 1); assertTrue(i2 == i1 + 1);
td = new TableDescriptor(htd, TableState.State.DISABLED);
Path p3 = fstd.updateTableDescriptor(td);
// Assert we cleaned up the old file.
assertTrue(!fs.exists(p2));
int i3 = FSTableDescriptors.getTableInfoSequenceId(p3);
assertTrue(i3 == i2 + 1);
TableDescriptor descriptor = fstd.getDescriptor(htd.getTableName());
assertEquals(descriptor, td);
} }
@Test @Test
@ -158,12 +170,13 @@ public class TestFSTableDescriptors {
final String name = "testReadingHTDFromFS"; final String name = "testReadingHTDFromFS";
FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FileSystem fs = FileSystem.get(UTIL.getConfiguration());
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
Path rootdir = UTIL.getDataTestDir(name); Path rootdir = UTIL.getDataTestDir(name);
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir); FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
fstd.createTableDescriptor(htd); fstd.createTableDescriptor(htd);
HTableDescriptor htd2 = TableDescriptor td2 =
FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName()); FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName());
assertTrue(htd.equals(htd2)); assertTrue(td.equals(td2));
} }
@Test @Test
@ -177,7 +190,8 @@ public class TestFSTableDescriptors {
final int count = 10; final int count = 10;
// Write out table infos. // Write out table infos.
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
HTableDescriptor htd = new HTableDescriptor(name + i); TableDescriptor htd = new TableDescriptor(new HTableDescriptor(name + i),
TableState.State.ENABLED);
htds.createTableDescriptor(htd); htds.createTableDescriptor(htd);
} }
@ -191,7 +205,7 @@ public class TestFSTableDescriptors {
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
htd.addFamily(new HColumnDescriptor("" + i)); htd.addFamily(new HColumnDescriptor("" + i));
htds.updateTableDescriptor(htd); htds.updateTableDescriptor(new TableDescriptor(htd));
} }
// Wait a while so mod time we write is for sure different. // Wait a while so mod time we write is for sure different.
Thread.sleep(100); Thread.sleep(100);
@ -232,7 +246,7 @@ public class TestFSTableDescriptors {
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
htd.addFamily(new HColumnDescriptor("" + i)); htd.addFamily(new HColumnDescriptor("" + i));
htds.updateTableDescriptor(htd); htds.updateTableDescriptor(new TableDescriptor(htd));
} }
// Wait a while so mod time we write is for sure different. // Wait a while so mod time we write is for sure different.
Thread.sleep(100); Thread.sleep(100);
@ -378,18 +392,19 @@ public class TestFSTableDescriptors {
Path testdir = UTIL.getDataTestDir("testCreateTableDescriptorUpdatesIfThereExistsAlready"); Path testdir = UTIL.getDataTestDir("testCreateTableDescriptorUpdatesIfThereExistsAlready");
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf( HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(
"testCreateTableDescriptorUpdatesIfThereExistsAlready")); "testCreateTableDescriptorUpdatesIfThereExistsAlready"));
TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FileSystem fs = FileSystem.get(UTIL.getConfiguration());
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir); FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
assertTrue(fstd.createTableDescriptor(htd)); assertTrue(fstd.createTableDescriptor(htd));
assertFalse(fstd.createTableDescriptor(htd)); assertFalse(fstd.createTableDescriptor(htd));
htd.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue")); htd.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue"));
assertTrue(fstd.createTableDescriptor(htd)); //this will re-create assertTrue(fstd.createTableDescriptor(td)); //this will re-create
Path tableDir = fstd.getTableDir(htd.getTableName()); Path tableDir = fstd.getTableDir(htd.getTableName());
Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR); Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR);
FileStatus[] statuses = fs.listStatus(tmpTableDir); FileStatus[] statuses = fs.listStatus(tmpTableDir);
assertTrue(statuses.length == 0); assertTrue(statuses.length == 0);
assertEquals(htd, FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir)); assertEquals(td, FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir));
} }
private static class FSTableDescriptorsTest private static class FSTableDescriptorsTest

View File

@ -68,7 +68,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Admin; 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.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; 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.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
@ -2891,55 +2889,6 @@ public class TestHBaseFsck {
tableLockManager.tableDeleted(tableName); 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) @Test (timeout=180000)
public void testMetaOffline() throws Exception { public void testMetaOffline() throws Exception {
// check no errors // check no errors

View File

@ -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));
}
}