HBASE-25257 Remove MirroringTableStateManager (#2634)

Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
Duo Zhang 2020-11-09 21:36:44 +08:00 committed by GitHub
parent 5c7432f4a9
commit 58c974888f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 4 additions and 451 deletions

View File

@ -877,14 +877,8 @@ public class HMaster extends HRegionServer implements MasterServices {
procsByType.getOrDefault(ServerCrashProcedure.class, Collections.emptyList()).stream()
.map(p -> (ServerCrashProcedure) p).map(p -> p.getServerName()).collect(Collectors.toSet()),
walManager.getLiveServersFromWALDir(), walManager.getSplittingServersFromWALDir());
// This manager will be started AFTER hbase:meta is confirmed on line.
// 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);
// This manager must be accessed AFTER hbase:meta is confirmed on line..
this.tableStateManager = new TableStateManager(this);
status.setStatus("Initializing ZK system trackers");
initializeZKBasedSystemTrackers();
@ -970,7 +964,6 @@ public class HMaster extends HRegionServer implements MasterServices {
}
this.assignmentManager.joinCluster();
// The below depends on hbase:meta being online.
this.tableStateManager.start();
// for migrating from a version without HBASE-25099, and also for honoring the configuration
// first.
@ -1004,11 +997,6 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
}
// Below has to happen after tablestatemanager has started in the case where this hbase-2.x
// is being started over an hbase-1.x dataset. tablestatemanager runs a migration as part
// of its 'start' moving table state from zookeeper to hbase:meta. This migration needs to
// complete before we do this next step processing offline regions else it fails reading
// table states messing up master launch (namespace table, etc., are not assigned).
this.assignmentManager.processOfflineRegions();
// Initialize after meta is up as below scans meta
if (getFavoredNodesManager() != null && !maintenanceMode) {

View File

@ -1,98 +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.master;
import java.io.IOException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.client.TableState.State;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
/**
* 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);
}
@Override
protected void metaStateUpdated(TableName tableName, State newState) throws IOException {
updateZooKeeper(new TableState(tableName, newState));
}
@Override
protected void metaStateDeleted(TableName tableName) throws IOException {
deleteZooKeeper(tableName);
}
private void updateZooKeeper(TableState tableState) throws IOException {
if (tableState == 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);
}
}

View File

@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.master;
import edu.umd.cs.findbugs.annotations.NonNull;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@ -29,21 +27,14 @@ import java.util.concurrent.locks.ReadWriteLock;
import org.apache.hadoop.hbase.CatalogFamilyFormat;
import org.apache.hadoop.hbase.ClientMetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hadoop.hbase.util.IdReadWriteLock;
import org.apache.hadoop.hbase.util.IdReadWriteLockWithObjectPool;
import org.apache.hadoop.hbase.util.ZKDataMigrator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -51,22 +42,15 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
/**
* This is a helper class used to manage table states. This class uses hbase:meta as its store for
* table state so hbase:meta must be online before {@link #start()} is called.
* table state so hbase:meta must be online before accessing its methods.
*/
// TODO: Make this a guava Service
@InterfaceAudience.Private
public class TableStateManager {
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.
*/
private static final String MIGRATE_TABLE_STATE_FROM_ZK_KEY =
"hbase.migrate.table.state.from.zookeeper";
private final IdReadWriteLock<TableName> tnLock = new IdReadWriteLockWithObjectPool<>();
protected final MasterServices master;
private final MasterServices master;
private final ConcurrentMap<TableName, TableState.State> tableName2State =
new ConcurrentHashMap<>();
@ -110,7 +94,6 @@ public class TableStateManager {
lock.writeLock().lock();
try {
MetaTableAccessor.deleteTableState(master.getConnection(), tableName);
metaStateDeleted(tableName);
} finally {
tableName2State.remove(tableName);
lock.writeLock().unlock();
@ -182,14 +165,6 @@ public class TableStateManager {
this.tableName2State.remove(tableName);
}
}
metaStateUpdated(tableName, newState);
}
protected void metaStateUpdated(TableName tableName, TableState.State newState)
throws IOException {
}
protected void metaStateDeleted(TableName tableName) throws IOException {
}
@Nullable
@ -204,118 +179,4 @@ public class TableStateManager {
}
return tableState;
}
public void start() throws IOException {
migrateZooKeeper();
fixTableStates(master.getTableDescriptors(), master.getConnection());
}
private void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
throws IOException {
Map<String, TableState> states = new HashMap<>();
// NOTE: Full hbase:meta table scan!
MetaTableAccessor.fullScanTables(connection, new ClientMetaTableAccessor.Visitor() {
@Override
public boolean visit(Result r) throws IOException {
TableState state = CatalogFamilyFormat.getTableState(r);
states.put(state.getTableName().getNameAsString(), state);
return true;
}
});
for (TableDescriptor tableDesc : tableDescriptors.getAll().values()) {
TableName tableName = tableDesc.getTableName();
if (TableName.isMetaTableName(tableName)) {
// 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, TableState.State.ENABLED));
continue;
}
TableState tableState = states.get(tableName.getNameAsString());
if (tableState == null) {
LOG.warn(tableName + " has no table state in hbase:meta, assuming ENABLED");
MetaTableAccessor.updateTableState(connection, tableName, TableState.State.ENABLED);
fixTableState(new TableState(tableName, TableState.State.ENABLED));
tableName2State.put(tableName, TableState.State.ENABLED);
} else {
fixTableState(tableState);
tableName2State.put(tableName, tableState.getState());
}
}
}
/**
* 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, true)) {
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 ts = null;
try {
ts = getTableState(entry.getKey());
} catch (TableNotFoundException e) {
// This can happen; table exists but no TableState.
}
if (ts == 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);
}
}
}

View File

@ -1,105 +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.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.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.getZNodePaths().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);
}
}
}

View File

@ -1,93 +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.master;
import org.apache.hadoop.hbase.HBaseClassTestRule;
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.client.TableState;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.After;
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 static junit.framework.TestCase.assertTrue;
/**
* Tests the default table lock manager
*/
@Category({ MasterTests.class, LargeTests.class })
public class TestTableStateManager {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestTableStateManager.class);
private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@Rule
public TestName name = new TestName();
@Before
public void before() throws Exception {
TEST_UTIL.startMiniCluster();
}
@After
public void after() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testMigration() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY_STR);
TEST_UTIL.getAdmin().disableTable(tableName);
// Table is disabled. Now remove the DISABLED column from the hbase:meta for this table's
// region. We want to see if Master will read the DISABLED from zk and make use of it as
// though it were reading the zk table state written by a hbase-1.x cluster.
TableState state = MetaTableAccessor.getTableState(TEST_UTIL.getConnection(), tableName);
assertTrue("State=" + state, state.getState().equals(TableState.State.DISABLED));
MetaTableAccessor.deleteTableState(TEST_UTIL.getConnection(), 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();
master.stop("Restarting");
while (!master.isStopped()) {
Threads.sleep(1);
}
assertTrue(master.isStopped());
JVMClusterUtil.MasterThread newMasterThread = TEST_UTIL.getMiniHBaseCluster().startMaster();
master = newMasterThread.getMaster();
while (!master.isInitialized()) {
Threads.sleep(1);
}
assertTrue(MetaTableAccessor.getTableState(TEST_UTIL.getConnection(),
tableName).getState().equals(TableState.State.DISABLED));
}
}