HBASE-16060 1.x clients cannot access table state talking to 2.0 cluster
This patch adds mirroring of table state out to zookeeper. HBase-1.x clients look for table state in zookeeper, not in hbase:meta where hbase-2.x maintains table state. The patch also moves and refactors the 'migration' code that was put in place by HBASE-13032. D hbase-client/src/main/java/org/apache/hadoop/hbase/CoordinatedStateException.java Unused. M hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Move table state migration code from Master startup out to TableStateManager where it belongs. Also start MirroringTableStateManager dependent on config. A hbase-server/src/main/java/org/apache/hadoop/hbase/master/MirroringTableStateManager.java M hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java Move migration from zookeeper of table state in here. Also plumb in mechanism so subclass can get a chance to look at table state as we do the startup fixup full-table scan of meta. M hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java Bug-fix. Now we create regions in CLOSED state but we fail to check table state; were presuming table always enabled. Meant on startup there'd be an unassigned region that never got assigned. A hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java Test migration and mirroring.
This commit is contained in:
parent
43f507dfa8
commit
c7473df2c3
|
@ -1,46 +0,0 @@
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase;
|
|
||||||
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
import org.apache.hadoop.hbase.exceptions.HBaseException;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Thrown by operations requiring coordination state access or manipulation
|
|
||||||
* when internal error within coordination engine (or other internal implementation) occurs.
|
|
||||||
*/
|
|
||||||
@InterfaceAudience.Private
|
|
||||||
@SuppressWarnings("serial")
|
|
||||||
public class CoordinatedStateException extends HBaseException {
|
|
||||||
public CoordinatedStateException() {
|
|
||||||
super();
|
|
||||||
}
|
|
||||||
|
|
||||||
public CoordinatedStateException(final String message) {
|
|
||||||
super(message);
|
|
||||||
}
|
|
||||||
|
|
||||||
public CoordinatedStateException(final String message, final Throwable t) {
|
|
||||||
super(message, t);
|
|
||||||
}
|
|
||||||
|
|
||||||
public CoordinatedStateException(final Throwable t) {
|
|
||||||
super(t);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -59,6 +59,9 @@ public class ZNodePaths {
|
||||||
// znode containing the current cluster state
|
// znode containing the current cluster state
|
||||||
public final String clusterStateZNode;
|
public final String clusterStateZNode;
|
||||||
// znode used for table disabling/enabling
|
// znode used for table disabling/enabling
|
||||||
|
// Still used in hbase2 by MirroringTableStateManager; it mirrors internal table state out to
|
||||||
|
// zookeeper for hbase1 clients to make use of. If no hbase1 clients disable. See
|
||||||
|
// MirroringTableStateManager. To be removed in hbase3.
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public final String tableZNode;
|
public final String tableZNode;
|
||||||
// znode containing the unique cluster ID
|
// znode containing the unique cluster ID
|
||||||
|
|
|
@ -85,7 +85,7 @@ 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.
|
* Deprected, table state is stored in hbase:meta since 2.0.0.
|
||||||
*/
|
*/
|
||||||
message DeprecatedTableState {
|
message DeprecatedTableState {
|
||||||
// Table's current state
|
// Table's current state
|
||||||
|
|
|
@ -61,7 +61,6 @@ import org.apache.hadoop.hbase.ClusterId;
|
||||||
import org.apache.hadoop.hbase.ClusterMetrics;
|
import org.apache.hadoop.hbase.ClusterMetrics;
|
||||||
import org.apache.hadoop.hbase.ClusterMetrics.Option;
|
import org.apache.hadoop.hbase.ClusterMetrics.Option;
|
||||||
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
|
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
|
||||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
|
||||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||||
import org.apache.hadoop.hbase.HBaseIOException;
|
import org.apache.hadoop.hbase.HBaseIOException;
|
||||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
|
@ -180,7 +179,6 @@ 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.LoadBalancerTracker;
|
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
|
||||||
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
|
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
|
||||||
import org.apache.hadoop.hbase.zookeeper.MasterMaintenanceModeTracker;
|
import org.apache.hadoop.hbase.zookeeper.MasterMaintenanceModeTracker;
|
||||||
|
@ -716,7 +714,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
* Initialize all ZK based system trackers.
|
* Initialize all ZK based system trackers.
|
||||||
*/
|
*/
|
||||||
void initializeZKBasedSystemTrackers() throws IOException,
|
void initializeZKBasedSystemTrackers() throws IOException,
|
||||||
InterruptedException, KeeperException, CoordinatedStateException {
|
InterruptedException, KeeperException {
|
||||||
this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
|
this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
|
||||||
this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
|
this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
|
||||||
this.normalizer.setMasterServices(this);
|
this.normalizer.setMasterServices(this);
|
||||||
|
@ -780,7 +778,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
* </ol>
|
* </ol>
|
||||||
*/
|
*/
|
||||||
private void finishActiveMasterInitialization(MonitoredTask status)
|
private void finishActiveMasterInitialization(MonitoredTask status)
|
||||||
throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
|
throws IOException, InterruptedException, KeeperException {
|
||||||
|
|
||||||
Thread zombieDetector = new Thread(new InitializationMonitor(this),
|
Thread zombieDetector = new Thread(new InitializationMonitor(this),
|
||||||
"ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
|
"ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
|
||||||
|
@ -820,7 +818,12 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
|
|
||||||
// This manager is started AFTER hbase:meta is confirmed on line.
|
// This manager is started AFTER hbase:meta is confirmed on line.
|
||||||
// See inside metaBootstrap.recoverMeta(); below. Shouldn't be so cryptic!
|
// See inside metaBootstrap.recoverMeta(); below. Shouldn't be so cryptic!
|
||||||
this.tableStateManager = new TableStateManager(this);
|
// hbase.mirror.table.state.to.zookeeper is so hbase1 clients can connect. They read table
|
||||||
|
// state from zookeeper while hbase2 reads it from hbase:meta. Disable if no hbase1 clients.
|
||||||
|
this.tableStateManager =
|
||||||
|
this.conf.getBoolean(MirroringTableStateManager.MIRROR_TABLE_STATE_TO_ZK_KEY, true)?
|
||||||
|
new MirroringTableStateManager(this):
|
||||||
|
new TableStateManager(this);
|
||||||
|
|
||||||
status.setStatus("Initializing ZK system trackers");
|
status.setStatus("Initializing ZK system trackers");
|
||||||
initializeZKBasedSystemTrackers();
|
initializeZKBasedSystemTrackers();
|
||||||
|
@ -875,12 +878,11 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// we recover hbase:meta region servers inside master initialization and
|
// Bring up hbase:meta. recoverMeta is a blocking call waiting until hbase:meta is deployed.
|
||||||
// handle other failed servers in SSH in order to start up master node ASAP
|
// It also starts the TableStateManager.
|
||||||
MasterMetaBootstrap metaBootstrap = createMetaBootstrap(this, status);
|
MasterMetaBootstrap metaBootstrap = createMetaBootstrap(this, status);
|
||||||
metaBootstrap.recoverMeta();
|
metaBootstrap.recoverMeta();
|
||||||
|
|
||||||
|
|
||||||
//Initialize after meta as it scans meta
|
//Initialize after meta as it scans meta
|
||||||
if (favoredNodesManager != null) {
|
if (favoredNodesManager != null) {
|
||||||
SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
|
SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
|
||||||
|
@ -889,15 +891,6 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
favoredNodesManager.initialize(snapshotOfRegionAssignment);
|
favoredNodesManager.initialize(snapshotOfRegionAssignment);
|
||||||
}
|
}
|
||||||
|
|
||||||
// migrating existent table state from zk, so splitters
|
|
||||||
// and recovery process treat states properly.
|
|
||||||
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().znodePaths.tableZNode);
|
|
||||||
|
|
||||||
status.setStatus("Submitting log splitting work for previously failed region servers");
|
status.setStatus("Submitting log splitting work for previously failed region servers");
|
||||||
metaBootstrap.processDeadServers();
|
metaBootstrap.processDeadServers();
|
||||||
|
|
||||||
|
|
|
@ -53,8 +53,11 @@ public class MasterMetaBootstrap {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void recoverMeta() throws InterruptedException, IOException {
|
public void recoverMeta() throws InterruptedException, IOException {
|
||||||
|
// This is a blocking call that waits until hbase:meta is deployed.
|
||||||
master.recoverMeta();
|
master.recoverMeta();
|
||||||
|
// Now we can start the TableStateManager. It is backed by hbase:meta.
|
||||||
master.getTableStateManager().start();
|
master.getTableStateManager().start();
|
||||||
|
// Enable server crash procedure handling
|
||||||
enableCrashedServerProcessing(false);
|
enableCrashedServerProcessing(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,109 @@
|
||||||
|
/*
|
||||||
|
* 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 org.apache.hadoop.hbase.client.TableState;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A subclass of TableStateManager that mirrors change in state out to zookeeper for hbase-1.x
|
||||||
|
* clients to pick up; hbase-1.x clients read table state of zookeeper rather than from hbase:meta
|
||||||
|
* as hbase-2.x clients do. Set "hbase.mirror.table.state.to.zookeeper" to false to disable
|
||||||
|
* mirroring. See in HMaster where we make the choice. The below does zk updates on a best-effort
|
||||||
|
* basis only. If we fail updating zk we keep going because only hbase1 clients suffer; we'll just
|
||||||
|
* log at WARN level.
|
||||||
|
* @deprecated Since 2.0.0. To be removed in 3.0.0.
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class MirroringTableStateManager extends TableStateManager {
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(MirroringTableStateManager.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set this key to true in Configuration to enable mirroring of table state out to zookeeper
|
||||||
|
* so hbase-1.x clients can pick-up table state.
|
||||||
|
*/
|
||||||
|
static final String MIRROR_TABLE_STATE_TO_ZK_KEY = "hbase.mirror.table.state.to.zookeeper";
|
||||||
|
|
||||||
|
public MirroringTableStateManager(MasterServices master) {
|
||||||
|
super(master);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void updateMetaState(TableName tableName, TableState.State newState)
|
||||||
|
throws IOException {
|
||||||
|
// Take the lock. Its reentrant. Calls to super will take same lock.
|
||||||
|
lock.writeLock().lock();
|
||||||
|
try {
|
||||||
|
super.updateMetaState(tableName, newState);
|
||||||
|
updateZooKeeper(new TableState(tableName, newState));
|
||||||
|
} finally {
|
||||||
|
lock.writeLock().unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setDeletedTable(TableName tableName) throws IOException {
|
||||||
|
lock.writeLock().lock();
|
||||||
|
try {
|
||||||
|
super.setDeletedTable(tableName);
|
||||||
|
deleteZooKeeper(tableName);
|
||||||
|
} finally {
|
||||||
|
lock.writeLock().unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateZooKeeper(TableState tableState) throws IOException {
|
||||||
|
if (tableState == null || tableState.getState() == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
String znode = ZNodePaths.joinZNode(this.master.getZooKeeper().getZNodePaths().tableZNode,
|
||||||
|
tableState.getTableName().getNameAsString());
|
||||||
|
try {
|
||||||
|
// Make sure znode exists.
|
||||||
|
if (ZKUtil.checkExists(this.master.getZooKeeper(), znode) == -1) {
|
||||||
|
ZKUtil.createAndFailSilent(this.master.getZooKeeper(), znode);
|
||||||
|
}
|
||||||
|
// Now set newState
|
||||||
|
ZooKeeperProtos.DeprecatedTableState.Builder builder =
|
||||||
|
ZooKeeperProtos.DeprecatedTableState.newBuilder();
|
||||||
|
builder.setState(ZooKeeperProtos.DeprecatedTableState.State.
|
||||||
|
valueOf(tableState.getState().toString()));
|
||||||
|
byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
|
||||||
|
ZKUtil.setData(this.master.getZooKeeper(), znode, data);
|
||||||
|
} catch (KeeperException e) {
|
||||||
|
// Only hbase1 clients suffer if this fails.
|
||||||
|
LOG.warn("Failed setting table state to zookeeper mirrored for hbase-1.x clients", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// This method is called by the super class on each row it finds in the hbase:meta table with
|
||||||
|
// table state in it.
|
||||||
|
@Override
|
||||||
|
protected void fixTableState(TableState tableState) throws IOException {
|
||||||
|
updateZooKeeper(tableState);
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,4 +1,4 @@
|
||||||
/**
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -26,6 +26,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
|
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
|
||||||
|
import org.apache.hadoop.hbase.util.ZKDataMigrator;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
|
||||||
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
|
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
|
||||||
import edu.umd.cs.findbugs.annotations.NonNull;
|
import edu.umd.cs.findbugs.annotations.NonNull;
|
||||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
import edu.umd.cs.findbugs.annotations.Nullable;
|
||||||
|
@ -35,6 +38,7 @@ import org.apache.hadoop.hbase.TableDescriptors;
|
||||||
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.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
import org.apache.zookeeper.KeeperException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
|
@ -43,15 +47,22 @@ import org.apache.hadoop.hbase.client.TableState;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is a helper class used to manage table states.
|
* This is a helper class used to manage table states.
|
||||||
* States persisted in tableinfo and cached internally.
|
* This class uses hbase:meta as its store for table state so hbase:meta must be online before
|
||||||
|
* {@link #start()} is called.
|
||||||
* TODO: Cache state. Cut down on meta looksups.
|
* TODO: Cache state. Cut down on meta looksups.
|
||||||
*/
|
*/
|
||||||
|
// TODO: Make this a guava Service
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class TableStateManager {
|
public class TableStateManager {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(TableStateManager.class);
|
private static final Logger LOG = LoggerFactory.getLogger(TableStateManager.class);
|
||||||
|
/**
|
||||||
|
* Set this key to false in Configuration to disable migrating table state from zookeeper
|
||||||
|
* so hbase:meta table.
|
||||||
|
*/
|
||||||
|
static final String MIGRATE_TABLE_STATE_FROM_ZK_KEY = "hbase.migrate.table.state.from.zookeeper";
|
||||||
|
|
||||||
private final ReadWriteLock lock = new ReentrantReadWriteLock();
|
final ReadWriteLock lock = new ReentrantReadWriteLock();
|
||||||
private final MasterServices master;
|
final MasterServices master;
|
||||||
|
|
||||||
public TableStateManager(MasterServices master) {
|
public TableStateManager(MasterServices master) {
|
||||||
this.master = master;
|
this.master = master;
|
||||||
|
@ -71,7 +82,6 @@ public class TableStateManager {
|
||||||
} finally {
|
} finally {
|
||||||
lock.writeLock().unlock();
|
lock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -140,8 +150,9 @@ public class TableStateManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setDeletedTable(TableName tableName) throws IOException {
|
public void setDeletedTable(TableName tableName) throws IOException {
|
||||||
if (tableName.equals(TableName.META_TABLE_NAME))
|
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
||||||
return;
|
return;
|
||||||
|
}
|
||||||
MetaTableAccessor.deleteTableState(master.getConnection(), tableName);
|
MetaTableAccessor.deleteTableState(master.getConnection(), tableName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -170,11 +181,17 @@ public class TableStateManager {
|
||||||
return rv;
|
return rv;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static class TableStateNotFoundException extends TableNotFoundException {
|
||||||
|
TableStateNotFoundException(TableName tableName) {
|
||||||
|
super(tableName.getNameAsString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@NonNull
|
@NonNull
|
||||||
public TableState.State getTableState(TableName tableName) throws IOException {
|
public TableState.State getTableState(TableName tableName) throws IOException {
|
||||||
TableState currentState = readMetaState(tableName);
|
TableState currentState = readMetaState(tableName);
|
||||||
if (currentState == null) {
|
if (currentState == null) {
|
||||||
throw new TableNotFoundException(tableName);
|
throw new TableStateNotFoundException(tableName);
|
||||||
}
|
}
|
||||||
return currentState.getState();
|
return currentState.getState();
|
||||||
}
|
}
|
||||||
|
@ -194,42 +211,121 @@ public class TableStateManager {
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
protected TableState readMetaState(TableName tableName) throws IOException {
|
protected TableState readMetaState(TableName tableName) throws IOException {
|
||||||
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
|
||||||
return new TableState(tableName, TableState.State.ENABLED);
|
|
||||||
}
|
|
||||||
return MetaTableAccessor.getTableState(master.getConnection(), tableName);
|
return MetaTableAccessor.getTableState(master.getConnection(), tableName);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void start() throws IOException {
|
public void start() throws IOException {
|
||||||
TableDescriptors tableDescriptors = master.getTableDescriptors();
|
TableDescriptors tableDescriptors = master.getTableDescriptors();
|
||||||
|
migrateZooKeeper();
|
||||||
Connection connection = master.getConnection();
|
Connection connection = master.getConnection();
|
||||||
fixTableStates(tableDescriptors, connection);
|
fixTableStates(tableDescriptors, connection);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
|
private void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
final Map<String, TableDescriptor> allDescriptors =
|
final Map<String, TableDescriptor> allDescriptors = tableDescriptors.getAllDescriptors();
|
||||||
tableDescriptors.getAllDescriptors();
|
|
||||||
final Map<String, TableState> states = new HashMap<>();
|
final Map<String, TableState> states = new HashMap<>();
|
||||||
|
// NOTE: Ful hbase:meta table scan!
|
||||||
MetaTableAccessor.fullScanTables(connection, new MetaTableAccessor.Visitor() {
|
MetaTableAccessor.fullScanTables(connection, new MetaTableAccessor.Visitor() {
|
||||||
@Override
|
@Override
|
||||||
public boolean visit(Result r) throws IOException {
|
public boolean visit(Result r) throws IOException {
|
||||||
TableState state = MetaTableAccessor.getTableState(r);
|
TableState state = MetaTableAccessor.getTableState(r);
|
||||||
if (state != null)
|
states.put(state.getTableName().getNameAsString(), state);
|
||||||
states.put(state.getTableName().getNameAsString(), state);
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
for (Map.Entry<String, TableDescriptor> entry : allDescriptors.entrySet()) {
|
for (Map.Entry<String, TableDescriptor> entry: allDescriptors.entrySet()) {
|
||||||
String table = entry.getKey();
|
String table = entry.getKey();
|
||||||
if (table.equals(TableName.META_TABLE_NAME.getNameAsString())) {
|
if (table.equals(TableName.META_TABLE_NAME.getNameAsString())) {
|
||||||
|
// This table is always enabled. No fixup needed. No entry in hbase:meta needed.
|
||||||
|
// Call through to fixTableState though in case a super class wants to do something.
|
||||||
|
fixTableState(new TableState(TableName.valueOf(table), TableState.State.ENABLED));
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
if (!states.containsKey(table)) {
|
TableState tableState = states.get(table);
|
||||||
LOG.warn(table + " has no state, assuming ENABLED");
|
if (tableState == null || tableState.getState() == null) {
|
||||||
|
LOG.warn(table + " has no table state in hbase:meta, assuming ENABLED");
|
||||||
MetaTableAccessor.updateTableState(connection, TableName.valueOf(table),
|
MetaTableAccessor.updateTableState(connection, TableName.valueOf(table),
|
||||||
TableState.State.ENABLED);
|
TableState.State.ENABLED);
|
||||||
|
fixTableState(new TableState(TableName.valueOf(table), TableState.State.ENABLED));
|
||||||
|
} else {
|
||||||
|
fixTableState(tableState);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* For subclasses in case they want to do fixup post hbase:meta.
|
||||||
|
*/
|
||||||
|
protected void fixTableState(TableState tableState) throws IOException {}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This code is for case where a hbase2 Master is starting for the first time. ZooKeeper is
|
||||||
|
* where we used to keep table state. On first startup, read zookeeper and update hbase:meta
|
||||||
|
* with the table states found in zookeeper. This is tricky as we'll do this check every time we
|
||||||
|
* startup until mirroring is disabled. See the {@link #MIGRATE_TABLE_STATE_FROM_ZK_KEY} flag.
|
||||||
|
* Original form of this migration came in with HBASE-13032. It deleted all znodes when done.
|
||||||
|
* We can't do that if we want to support hbase-1.x clients who need to be able to read table
|
||||||
|
* state out of zk. See {@link MirroringTableStateManager}.
|
||||||
|
* @deprecated Since 2.0.0. Remove in hbase-3.0.0.
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
|
private void migrateZooKeeper() throws IOException {
|
||||||
|
if (this.master.getConfiguration().getBoolean(MIGRATE_TABLE_STATE_FROM_ZK_KEY, false)) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
for (Map.Entry<TableName, TableState.State> entry:
|
||||||
|
ZKDataMigrator.queryForTableStates(this.master.getZooKeeper()).entrySet()) {
|
||||||
|
if (this.master.getTableDescriptors().get(entry.getKey()) == null) {
|
||||||
|
deleteZooKeeper(entry.getKey());
|
||||||
|
LOG.info("Purged table state entry from zookeepr for table not in hbase:meta: " +
|
||||||
|
entry.getKey());
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
TableState.State state = null;
|
||||||
|
try {
|
||||||
|
state = getTableState(entry.getKey());
|
||||||
|
} catch (TableStateNotFoundException e) {
|
||||||
|
// This can happen; table exists but no TableState.
|
||||||
|
}
|
||||||
|
if (state == null) {
|
||||||
|
TableState.State zkstate = entry.getValue();
|
||||||
|
// Only migrate if it is an enable or disabled table. If in-between -- ENABLING or
|
||||||
|
// DISABLING then we have a problem; we are starting up an hbase-2 on a cluster with
|
||||||
|
// RIT. It is going to be rough!
|
||||||
|
if (zkstate.equals(TableState.State.ENABLED) ||
|
||||||
|
zkstate.equals(TableState.State.DISABLED)) {
|
||||||
|
LOG.info("Migrating table state from zookeeper to hbase:meta; tableName=" +
|
||||||
|
entry.getKey() + ", state=" + entry.getValue());
|
||||||
|
updateMetaState(entry.getKey(), entry.getValue());
|
||||||
|
} else {
|
||||||
|
LOG.warn("Table={} has no state and zookeeper state is in-between={} (neither " +
|
||||||
|
"ENABLED or DISABLED); NOT MIGRATING table state", entry.getKey(), zkstate);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// What if the table states disagree? Defer to the hbase:meta setting rather than have the
|
||||||
|
// hbase-1.x support prevail.
|
||||||
|
}
|
||||||
|
} catch (KeeperException |InterruptedException e) {
|
||||||
|
LOG.warn("Failed reading table state from zookeeper", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method that knows how to delete the old hbase-1.x table state znode.
|
||||||
|
* Used also by the Mirroring subclass.
|
||||||
|
* @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
|
protected void deleteZooKeeper(TableName tableName) {
|
||||||
|
try {
|
||||||
|
// Delete from ZooKeeper
|
||||||
|
String znode = ZNodePaths.joinZNode(this.master.getZooKeeper().getZNodePaths().tableZNode,
|
||||||
|
tableName.getNameAsString());
|
||||||
|
ZKUtil.deleteNodeFailSilent(this.master.getZooKeeper(), znode);
|
||||||
|
} catch (KeeperException e) {
|
||||||
|
LOG.warn("Failed deleting table state from zookeeper", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1186,6 +1186,7 @@ public class AssignmentManager implements ServerListener {
|
||||||
LOG.debug("Joining cluster...");
|
LOG.debug("Joining cluster...");
|
||||||
|
|
||||||
// Scan hbase:meta to build list of existing regions, servers, and assignment
|
// Scan hbase:meta to build list of existing regions, servers, and assignment
|
||||||
|
// hbase:meta is online when we get to here and TableStateManager has been started.
|
||||||
loadMeta();
|
loadMeta();
|
||||||
|
|
||||||
for (int i = 0; master.getServerManager().countOfRegionServers() < 1; ++i) {
|
for (int i = 0; master.getServerManager().countOfRegionServers() < 1; ++i) {
|
||||||
|
@ -1232,6 +1233,10 @@ public class AssignmentManager implements ServerListener {
|
||||||
regionStates.addRegionToServer(regionNode);
|
regionStates.addRegionToServer(regionNode);
|
||||||
} else if (localState == State.OFFLINE || regionInfo.isOffline()) {
|
} else if (localState == State.OFFLINE || regionInfo.isOffline()) {
|
||||||
regionStates.addToOfflineRegions(regionNode);
|
regionStates.addToOfflineRegions(regionNode);
|
||||||
|
} else if (localState == State.CLOSED && getTableStateManager().
|
||||||
|
isTableState(regionNode.getTable(), TableState.State.DISABLED)) {
|
||||||
|
// The region is CLOSED and the table is DISABLED, there is nothing to schedule;
|
||||||
|
// the region is inert.
|
||||||
} else {
|
} else {
|
||||||
// These regions should have a procedure in replay
|
// These regions should have a procedure in replay
|
||||||
regionStates.addRegionInTransition(regionNode, null);
|
regionStates.addRegionInTransition(regionNode, null);
|
||||||
|
@ -1496,8 +1501,10 @@ public class AssignmentManager implements ServerListener {
|
||||||
synchronized (regionNode) {
|
synchronized (regionNode) {
|
||||||
regionNode.transitionState(State.OPEN, RegionStates.STATES_EXPECTED_ON_OPEN);
|
regionNode.transitionState(State.OPEN, RegionStates.STATES_EXPECTED_ON_OPEN);
|
||||||
if (isMetaRegion(hri)) {
|
if (isMetaRegion(hri)) {
|
||||||
master.getTableStateManager().setTableState(TableName.META_TABLE_NAME,
|
// Usually we'd set a table ENABLED at this stage but hbase:meta is ALWAYs enabled, it
|
||||||
TableState.State.ENABLED);
|
// can't be disabled -- so skip the RPC (besides... enabled is managed by TableStateManager
|
||||||
|
// which is backed by hbase:meta... Avoid setting ENABLED to avoid having to update state
|
||||||
|
// on table that contains state.
|
||||||
setMetaInitialized(hri, true);
|
setMetaInitialized(hri, true);
|
||||||
}
|
}
|
||||||
regionStates.addRegionToServer(regionNode);
|
regionStates.addRegionToServer(regionNode);
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
/**
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -37,7 +37,6 @@ 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.exceptions.HBaseException;
|
|
||||||
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
|
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
|
||||||
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;
|
||||||
|
@ -133,7 +132,7 @@ public class DeleteTableProcedure
|
||||||
default:
|
default:
|
||||||
throw new UnsupportedOperationException("unhandled state=" + state);
|
throw new UnsupportedOperationException("unhandled state=" + state);
|
||||||
}
|
}
|
||||||
} catch (HBaseException|IOException e) {
|
} catch (IOException e) {
|
||||||
if (isRollbackSupported(state)) {
|
if (isRollbackSupported(state)) {
|
||||||
setFailure("master-delete-table", e);
|
setFailure("master-delete-table", e);
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -27,7 +27,6 @@ import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
|
||||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
|
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
|
||||||
|
@ -207,7 +206,7 @@ public final class ProcedureSyncWait {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static void waitRegionInTransition(final MasterProcedureEnv env,
|
protected static void waitRegionInTransition(final MasterProcedureEnv env,
|
||||||
final List<RegionInfo> regions) throws IOException, CoordinatedStateException {
|
final List<RegionInfo> regions) throws IOException {
|
||||||
final RegionStates states = env.getAssignmentManager().getRegionStates();
|
final RegionStates states = env.getAssignmentManager().getRegionStates();
|
||||||
for (final RegionInfo region : regions) {
|
for (final RegionInfo region : regions) {
|
||||||
ProcedureSyncWait.waitFor(env, "regions " + region.getRegionNameAsString() + " in transition",
|
ProcedureSyncWait.waitFor(env, "regions " + region.getRegionNameAsString() + " in transition",
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
/**
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.TableNotFoundException;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
import org.apache.hadoop.hbase.exceptions.HBaseException;
|
|
||||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
|
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
|
||||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||||
|
@ -141,7 +140,7 @@ public class TruncateTableProcedure
|
||||||
default:
|
default:
|
||||||
throw new UnsupportedOperationException("unhandled state=" + state);
|
throw new UnsupportedOperationException("unhandled state=" + state);
|
||||||
}
|
}
|
||||||
} catch (HBaseException|IOException e) {
|
} catch (IOException e) {
|
||||||
if (isRollbackSupported(state)) {
|
if (isRollbackSupported(state)) {
|
||||||
setFailure("master-truncate-table", e);
|
setFailure("master-truncate-table", e);
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -36,13 +36,17 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* utlity method to migrate zookeeper data across HBase versions.
|
* Utlity method to migrate zookeeper data across HBase versions.
|
||||||
|
* @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
|
||||||
*/
|
*/
|
||||||
|
@Deprecated
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class ZKDataMigrator {
|
public class ZKDataMigrator {
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(ZKDataMigrator.class);
|
private static final Logger LOG = LoggerFactory.getLogger(ZKDataMigrator.class);
|
||||||
|
|
||||||
|
// Shutdown constructor.
|
||||||
|
private ZKDataMigrator() {}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Method for table states migration.
|
* Method for table states migration.
|
||||||
* Used when upgrading from pre-2.0 to 2.0
|
* Used when upgrading from pre-2.0 to 2.0
|
||||||
|
@ -50,6 +54,7 @@ public class ZKDataMigrator {
|
||||||
* and delete.
|
* and delete.
|
||||||
* Used by master to clean migration from zk based states to
|
* Used by master to clean migration from zk based states to
|
||||||
* table descriptor based states.
|
* table descriptor based states.
|
||||||
|
* @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public static Map<TableName, TableState.State> queryForTableStates(ZKWatcher zkw)
|
public static Map<TableName, TableState.State> queryForTableStates(ZKWatcher zkw)
|
||||||
|
@ -90,6 +95,7 @@ public class ZKDataMigrator {
|
||||||
* @param tableName table we're checking
|
* @param tableName table we're checking
|
||||||
* @return Null or {@link ZooKeeperProtos.DeprecatedTableState.State} found in znode.
|
* @return Null or {@link ZooKeeperProtos.DeprecatedTableState.State} found in znode.
|
||||||
* @throws KeeperException
|
* @throws KeeperException
|
||||||
|
* @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
private static ZooKeeperProtos.DeprecatedTableState.State getTableState(
|
private static ZooKeeperProtos.DeprecatedTableState.State getTableState(
|
||||||
|
@ -113,5 +119,4 @@ public class ZKDataMigrator {
|
||||||
throw ZKUtil.convert(e);
|
throw ZKUtil.convert(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,7 +27,6 @@ import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.Abortable;
|
import org.apache.hadoop.hbase.Abortable;
|
||||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
@ -275,7 +274,7 @@ public class TestMasterNoCluster {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void initializeZKBasedSystemTrackers() throws IOException, InterruptedException,
|
void initializeZKBasedSystemTrackers() throws IOException, InterruptedException,
|
||||||
KeeperException, CoordinatedStateException {
|
KeeperException {
|
||||||
super.initializeZKBasedSystemTrackers();
|
super.initializeZKBasedSystemTrackers();
|
||||||
// Record a newer server in server manager at first
|
// Record a newer server in server manager at first
|
||||||
getServerManager().recordNewServerWithLock(newServer,
|
getServerManager().recordNewServerWithLock(newServer,
|
||||||
|
|
|
@ -0,0 +1,105 @@
|
||||||
|
/*
|
||||||
|
* 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 org.apache.hadoop.hbase.client.TableState;
|
||||||
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
|
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||||
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
|
||||||
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.junit.rules.TestName;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import static junit.framework.TestCase.assertTrue;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that table state is mirrored out to zookeeper for hbase-1.x clients.
|
||||||
|
* Also tests that table state gets migrated from zookeeper on master start.
|
||||||
|
*/
|
||||||
|
@Category({ MasterTests.class, LargeTests.class })
|
||||||
|
public class TestMirroringTableStateManager {
|
||||||
|
@ClassRule
|
||||||
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
|
HBaseClassTestRule.forClass(TestMirroringTableStateManager.class);
|
||||||
|
@Rule
|
||||||
|
public TestName name = new TestName();
|
||||||
|
|
||||||
|
private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void before() throws Exception {
|
||||||
|
TEST_UTIL.startMiniCluster();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void after() throws Exception {
|
||||||
|
TEST_UTIL.shutdownMiniCluster();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMirroring() throws Exception {
|
||||||
|
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||||
|
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY_STR);
|
||||||
|
ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||||
|
assertTrue(TableState.State.ENABLED.equals(getTableStateInZK(zkw, tableName)));
|
||||||
|
TEST_UTIL.getAdmin().disableTable(tableName);
|
||||||
|
assertTrue(TableState.State.DISABLED.equals(getTableStateInZK(zkw, tableName)));
|
||||||
|
TEST_UTIL.getAdmin().deleteTable(tableName);
|
||||||
|
assertTrue(getTableStateInZK(zkw, tableName) == null);
|
||||||
|
}
|
||||||
|
|
||||||
|
private TableState.State getTableStateInZK(ZKWatcher watcher, final TableName tableName)
|
||||||
|
throws KeeperException, IOException, InterruptedException {
|
||||||
|
String znode = ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, tableName.getNameAsString());
|
||||||
|
byte [] data = ZKUtil.getData(watcher, znode);
|
||||||
|
if (data == null || data.length <= 0) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
ProtobufUtil.expectPBMagicPrefix(data);
|
||||||
|
ZooKeeperProtos.DeprecatedTableState.Builder builder =
|
||||||
|
ZooKeeperProtos.DeprecatedTableState.newBuilder();
|
||||||
|
int magicLen = ProtobufUtil.lengthOfPBMagic();
|
||||||
|
ProtobufUtil.mergeFrom(builder, data, magicLen, data.length - magicLen);
|
||||||
|
return TableState.State.valueOf(builder.getState().toString());
|
||||||
|
} catch (IOException e) {
|
||||||
|
KeeperException ke = new KeeperException.DataInconsistencyException();
|
||||||
|
ke.initCause(e);
|
||||||
|
throw ke;
|
||||||
|
} catch (DeserializationException e) {
|
||||||
|
throw ZKUtil.convert(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,4 +1,4 @@
|
||||||
/**
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -17,27 +17,27 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.master;
|
package org.apache.hadoop.hbase.master;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
|
|
||||||
import org.apache.zookeeper.KeeperException;
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
import org.junit.ClassRule;
|
import org.junit.ClassRule;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
import org.junit.rules.TestName;
|
import org.junit.rules.TestName;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
|
import static junit.framework.TestCase.assertTrue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests the default table lock manager
|
* Tests the default table lock manager
|
||||||
|
@ -54,37 +54,41 @@ public class TestTableStateManager {
|
||||||
@Rule
|
@Rule
|
||||||
public TestName name = new TestName();
|
public TestName name = new TestName();
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void before() throws Exception {
|
||||||
|
TEST_UTIL.startMiniCluster();
|
||||||
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
public void tearDown() throws Exception {
|
public void after() throws Exception {
|
||||||
TEST_UTIL.shutdownMiniCluster();
|
TEST_UTIL.shutdownMiniCluster();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUpgradeFromZk() throws Exception {
|
public void testMigration() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||||
TEST_UTIL.startMiniCluster(2, 1);
|
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY_STR);
|
||||||
TEST_UTIL.shutdownMiniHBaseCluster();
|
TEST_UTIL.getAdmin().disableTable(tableName);
|
||||||
ZKWatcher watcher = TEST_UTIL.getZooKeeperWatcher();
|
// Table is disabled. Now remove the DISABLED column from the hbase:meta for this table's
|
||||||
setTableStateInZK(watcher, tableName, ZooKeeperProtos.DeprecatedTableState.State.DISABLED);
|
// region. We want to see if Master will read the DISABLED from zk and make use of it as
|
||||||
TEST_UTIL.restartHBaseCluster(1);
|
// though it were reading the zk table state written by a hbase-1.x cluster.
|
||||||
|
TableState state = MetaTableAccessor.getTableState(TEST_UTIL.getConnection(), tableName);
|
||||||
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
|
assertTrue("State=" + state, state.getState().equals(TableState.State.DISABLED));
|
||||||
Assert.assertEquals(TableState.State.DISABLED,
|
MetaTableAccessor.deleteTableState(TEST_UTIL.getConnection(), tableName);
|
||||||
master.getTableStateManager().getTableState(tableName));
|
assertTrue(MetaTableAccessor.getTableState(TEST_UTIL.getConnection(), tableName) == null);
|
||||||
}
|
// Now kill Master so a new one can come up and run through the zk migration.
|
||||||
|
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
|
||||||
private void setTableStateInZK(ZKWatcher watcher, final TableName tableName,
|
master.stop("Restarting");
|
||||||
final ZooKeeperProtos.DeprecatedTableState.State state)
|
while (!master.isStopped()) {
|
||||||
throws KeeperException, IOException {
|
Threads.sleep(1);
|
||||||
String znode = ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, tableName.getNameAsString());
|
|
||||||
if (ZKUtil.checkExists(watcher, znode) == -1) {
|
|
||||||
ZKUtil.createAndFailSilent(watcher, znode);
|
|
||||||
}
|
}
|
||||||
ZooKeeperProtos.DeprecatedTableState.Builder builder =
|
assertTrue(master.isStopped());
|
||||||
ZooKeeperProtos.DeprecatedTableState.newBuilder();
|
JVMClusterUtil.MasterThread newMasterThread = TEST_UTIL.getMiniHBaseCluster().startMaster();
|
||||||
builder.setState(state);
|
master = newMasterThread.getMaster();
|
||||||
byte[] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
|
while (!master.isInitialized()) {
|
||||||
ZKUtil.setData(watcher, znode, data);
|
Threads.sleep(1);
|
||||||
|
}
|
||||||
|
assertTrue(MetaTableAccessor.getTableState(TEST_UTIL.getConnection(),
|
||||||
|
tableName).getState().equals(TableState.State.DISABLED));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue