HBASE-21025 Add cache for TableStateManager
This commit is contained in:
parent
28635d6101
commit
397388316e
|
@ -201,7 +201,7 @@ public class BucketCache implements BlockCache, HeapSize {
|
||||||
* Key set of offsets in BucketCache is limited so soft reference is the best choice here.
|
* Key set of offsets in BucketCache is limited so soft reference is the best choice here.
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
final IdReadWriteLock offsetLock = new IdReadWriteLock(ReferenceType.SOFT);
|
final IdReadWriteLock<Long> offsetLock = new IdReadWriteLock<>(ReferenceType.SOFT);
|
||||||
|
|
||||||
private final NavigableSet<BlockCacheKey> blocksByHFile =
|
private final NavigableSet<BlockCacheKey> blocksByHFile =
|
||||||
new ConcurrentSkipListSet<>(new Comparator<BlockCacheKey>() {
|
new ConcurrentSkipListSet<>(new Comparator<BlockCacheKey>() {
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
|
import org.apache.hadoop.hbase.client.TableState.State;
|
||||||
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;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||||
|
@ -45,8 +46,8 @@ public class MirroringTableStateManager extends TableStateManager {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(MirroringTableStateManager.class);
|
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
|
* Set this key to true in Configuration to enable mirroring of table state out to zookeeper so
|
||||||
* so hbase-1.x clients can pick-up table state.
|
* hbase-1.x clients can pick-up table state.
|
||||||
*/
|
*/
|
||||||
static final String MIRROR_TABLE_STATE_TO_ZK_KEY = "hbase.mirror.table.state.to.zookeeper";
|
static final String MIRROR_TABLE_STATE_TO_ZK_KEY = "hbase.mirror.table.state.to.zookeeper";
|
||||||
|
|
||||||
|
@ -54,26 +55,14 @@ public class MirroringTableStateManager extends TableStateManager {
|
||||||
super(master);
|
super(master);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void updateMetaState(TableName tableName, TableState.State newState)
|
@Override
|
||||||
throws IOException {
|
protected void metaStateUpdated(TableName tableName, State newState) throws IOException {
|
||||||
// Take the lock. Its reentrant. Calls to super will take same lock.
|
updateZooKeeper(new TableState(tableName, newState));
|
||||||
lock.writeLock().lock();
|
|
||||||
try {
|
|
||||||
super.updateMetaState(tableName, newState);
|
|
||||||
updateZooKeeper(new TableState(tableName, newState));
|
|
||||||
} finally {
|
|
||||||
lock.writeLock().unlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setDeletedTable(TableName tableName) throws IOException {
|
@Override
|
||||||
lock.writeLock().lock();
|
protected void metaStateDeleted(TableName tableName) throws IOException {
|
||||||
try {
|
deleteZooKeeper(tableName);
|
||||||
super.setDeletedTable(tableName);
|
|
||||||
deleteZooKeeper(tableName);
|
|
||||||
} finally {
|
|
||||||
lock.writeLock().unlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void updateZooKeeper(TableState tableState) throws IOException {
|
private void updateZooKeeper(TableState tableState) throws IOException {
|
||||||
|
@ -81,7 +70,7 @@ public class MirroringTableStateManager extends TableStateManager {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
String znode = ZNodePaths.joinZNode(this.master.getZooKeeper().getZNodePaths().tableZNode,
|
String znode = ZNodePaths.joinZNode(this.master.getZooKeeper().getZNodePaths().tableZNode,
|
||||||
tableState.getTableName().getNameAsString());
|
tableState.getTableName().getNameAsString());
|
||||||
try {
|
try {
|
||||||
// Make sure znode exists.
|
// Make sure znode exists.
|
||||||
if (ZKUtil.checkExists(this.master.getZooKeeper(), znode) == -1) {
|
if (ZKUtil.checkExists(this.master.getZooKeeper(), znode) == -1) {
|
||||||
|
@ -89,10 +78,10 @@ public class MirroringTableStateManager extends TableStateManager {
|
||||||
}
|
}
|
||||||
// Now set newState
|
// Now set newState
|
||||||
ZooKeeperProtos.DeprecatedTableState.Builder builder =
|
ZooKeeperProtos.DeprecatedTableState.Builder builder =
|
||||||
ZooKeeperProtos.DeprecatedTableState.newBuilder();
|
ZooKeeperProtos.DeprecatedTableState.newBuilder();
|
||||||
builder.setState(ZooKeeperProtos.DeprecatedTableState.State.
|
builder.setState(
|
||||||
valueOf(tableState.getState().toString()));
|
ZooKeeperProtos.DeprecatedTableState.State.valueOf(tableState.getState().toString()));
|
||||||
byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
|
byte[] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
|
||||||
ZKUtil.setData(this.master.getZooKeeper(), znode, data);
|
ZKUtil.setData(this.master.getZooKeeper(), znode, data);
|
||||||
} catch (KeeperException e) {
|
} catch (KeeperException e) {
|
||||||
// Only hbase1 clients suffer if this fails.
|
// Only hbase1 clients suffer if this fails.
|
||||||
|
|
|
@ -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,65 +17,68 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.master;
|
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.io.IOException;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.locks.ReadWriteLock;
|
import java.util.concurrent.locks.ReadWriteLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
|
||||||
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 edu.umd.cs.findbugs.annotations.NonNull;
|
|
||||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
import org.apache.hadoop.hbase.TableDescriptors;
|
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.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.ZKDataMigrator;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.zookeeper.KeeperException;
|
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.Result;
|
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
|
||||||
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. This class uses hbase:meta as its store for
|
||||||
* This class uses hbase:meta as its store for table state so hbase:meta must be online before
|
* table state so hbase:meta must be online before {@link #start()} is called.
|
||||||
* {@link #start()} is called.
|
|
||||||
* TODO: Cache state. Cut down on meta looksups.
|
|
||||||
*/
|
*/
|
||||||
// TODO: Make this a guava Service
|
// 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
|
* Set this key to false in Configuration to disable migrating table state from zookeeper so
|
||||||
* so hbase:meta table.
|
* hbase:meta table.
|
||||||
*/
|
*/
|
||||||
static final String MIGRATE_TABLE_STATE_FROM_ZK_KEY = "hbase.migrate.table.state.from.zookeeper";
|
private static final String MIGRATE_TABLE_STATE_FROM_ZK_KEY =
|
||||||
|
"hbase.migrate.table.state.from.zookeeper";
|
||||||
|
|
||||||
final ReadWriteLock lock = new ReentrantReadWriteLock();
|
private final IdReadWriteLock<TableName> tnLock = new IdReadWriteLock<>();
|
||||||
final MasterServices master;
|
protected final MasterServices master;
|
||||||
|
|
||||||
|
private final ConcurrentMap<TableName, TableState.State> tableName2State =
|
||||||
|
new ConcurrentHashMap<>();
|
||||||
|
|
||||||
public TableStateManager(MasterServices master) {
|
public TableStateManager(MasterServices master) {
|
||||||
this.master = master;
|
this.master = master;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set table state to provided.
|
* Set table state to provided. Caller should lock table on write.
|
||||||
* Caller should lock table on write.
|
|
||||||
* @param tableName table to change state for
|
* @param tableName table to change state for
|
||||||
* @param newState new state
|
* @param newState new state
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
public void setTableState(TableName tableName, TableState.State newState) throws IOException {
|
public void setTableState(TableName tableName, TableState.State newState) throws IOException {
|
||||||
|
ReadWriteLock lock = tnLock.getLock(tableName);
|
||||||
lock.writeLock().lock();
|
lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
updateMetaState(tableName, newState);
|
updateMetaState(tableName, newState);
|
||||||
|
@ -85,18 +88,16 @@ public class TableStateManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set table state to provided but only if table in specified states
|
* Set table state to provided but only if table in specified states Caller should lock table on
|
||||||
* Caller should lock table on write.
|
* write.
|
||||||
* @param tableName table to change state for
|
* @param tableName table to change state for
|
||||||
* @param newState new state
|
* @param newState new state
|
||||||
* @param states states to check against
|
* @param states states to check against
|
||||||
* @return null if succeed or table state if failed
|
* @return null if succeed or table state if failed
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
public TableState setTableStateIfInStates(TableName tableName,
|
public TableState setTableStateIfInStates(TableName tableName, TableState.State newState,
|
||||||
TableState.State newState,
|
TableState.State... states) throws IOException {
|
||||||
TableState.State... states)
|
ReadWriteLock lock = tnLock.getLock(tableName);
|
||||||
throws IOException {
|
|
||||||
lock.writeLock().lock();
|
lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
TableState currentState = readMetaState(tableName);
|
TableState currentState = readMetaState(tableName);
|
||||||
|
@ -115,26 +116,29 @@ public class TableStateManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set table state to provided but only if table not in specified states
|
* Set table state to provided but only if table not in specified states Caller should lock table
|
||||||
* Caller should lock table on write.
|
* on write.
|
||||||
* @param tableName table to change state for
|
* @param tableName table to change state for
|
||||||
* @param newState new state
|
* @param newState new state
|
||||||
* @param states states to check against
|
* @param states states to check against
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
public boolean setTableStateIfNotInStates(TableName tableName,
|
public boolean setTableStateIfNotInStates(TableName tableName, TableState.State newState,
|
||||||
TableState.State newState,
|
TableState.State... states) throws IOException {
|
||||||
TableState.State... states)
|
ReadWriteLock lock = tnLock.getLock(tableName);
|
||||||
throws IOException {
|
lock.writeLock().lock();
|
||||||
TableState currentState = readMetaState(tableName);
|
try {
|
||||||
if (currentState == null) {
|
TableState currentState = readMetaState(tableName);
|
||||||
throw new TableNotFoundException(tableName);
|
if (currentState == null) {
|
||||||
}
|
throw new TableNotFoundException(tableName);
|
||||||
if (!currentState.inStates(states)) {
|
}
|
||||||
updateMetaState(tableName, newState);
|
if (!currentState.inStates(states)) {
|
||||||
return true;
|
updateMetaState(tableName, newState);
|
||||||
} else {
|
return true;
|
||||||
return false;
|
} else {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
lock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -144,6 +148,7 @@ public class TableStateManager {
|
||||||
return tableState.isInStates(states);
|
return tableState.isInStates(states);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.error("Unable to get table " + tableName + " state", e);
|
LOG.error("Unable to get table " + tableName + " state", e);
|
||||||
|
// XXX: is it safe to just return false here?
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -152,28 +157,42 @@ public class TableStateManager {
|
||||||
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
MetaTableAccessor.deleteTableState(master.getConnection(), tableName);
|
ReadWriteLock lock = tnLock.getLock(tableName);
|
||||||
|
lock.writeLock().lock();
|
||||||
|
try {
|
||||||
|
MetaTableAccessor.deleteTableState(master.getConnection(), tableName);
|
||||||
|
metaStateDeleted(tableName);
|
||||||
|
} finally {
|
||||||
|
tableName2State.remove(tableName);
|
||||||
|
lock.writeLock().unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isTablePresent(TableName tableName) throws IOException {
|
public boolean isTablePresent(TableName tableName) throws IOException {
|
||||||
return readMetaState(tableName) != null;
|
ReadWriteLock lock = tnLock.getLock(tableName);
|
||||||
|
lock.readLock().lock();
|
||||||
|
try {
|
||||||
|
return readMetaState(tableName) != null;
|
||||||
|
} finally {
|
||||||
|
lock.readLock().unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return all tables in given states.
|
* Return all tables in given states.
|
||||||
*
|
|
||||||
* @param states filter by states
|
* @param states filter by states
|
||||||
* @return tables in given states
|
* @return tables in given states
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
public Set<TableName> getTablesInStates(final TableState.State... states) throws IOException {
|
public Set<TableName> getTablesInStates(TableState.State... states) throws IOException {
|
||||||
|
// Only be called in region normalizer, will not use cache.
|
||||||
final Set<TableName> rv = Sets.newHashSet();
|
final Set<TableName> rv = Sets.newHashSet();
|
||||||
MetaTableAccessor.fullScanTables(master.getConnection(), new MetaTableAccessor.Visitor() {
|
MetaTableAccessor.fullScanTables(master.getConnection(), new MetaTableAccessor.Visitor() {
|
||||||
@Override
|
@Override
|
||||||
public boolean visit(Result r) throws IOException {
|
public boolean visit(Result r) throws IOException {
|
||||||
TableState tableState = MetaTableAccessor.getTableState(r);
|
TableState tableState = MetaTableAccessor.getTableState(r);
|
||||||
if (tableState != null && tableState.inStates(states))
|
if (tableState != null && tableState.inStates(states)) {
|
||||||
rv.add(tableState.getTableName());
|
rv.add(tableState.getTableName());
|
||||||
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -188,29 +207,58 @@ public class TableStateManager {
|
||||||
|
|
||||||
@NonNull
|
@NonNull
|
||||||
public TableState getTableState(TableName tableName) throws IOException {
|
public TableState getTableState(TableName tableName) throws IOException {
|
||||||
TableState currentState = readMetaState(tableName);
|
ReadWriteLock lock = tnLock.getLock(tableName);
|
||||||
if (currentState == null) {
|
lock.readLock().lock();
|
||||||
throw new TableStateNotFoundException(tableName);
|
try {
|
||||||
|
TableState currentState = readMetaState(tableName);
|
||||||
|
if (currentState == null) {
|
||||||
|
throw new TableStateNotFoundException(tableName);
|
||||||
|
}
|
||||||
|
return currentState;
|
||||||
|
} finally {
|
||||||
|
lock.readLock().unlock();
|
||||||
}
|
}
|
||||||
return currentState;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void updateMetaState(TableName tableName, TableState.State newState)
|
private void updateMetaState(TableName tableName, TableState.State newState) throws IOException {
|
||||||
throws IOException {
|
|
||||||
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
||||||
if (TableState.State.DISABLING.equals(newState) ||
|
if (TableState.State.DISABLING.equals(newState) ||
|
||||||
TableState.State.DISABLED.equals(newState)) {
|
TableState.State.DISABLED.equals(newState)) {
|
||||||
throw new IllegalArgumentIOException("Cannot disable the meta table; " + newState);
|
throw new IllegalArgumentIOException("Cannot disable the meta table; " + newState);
|
||||||
}
|
}
|
||||||
// Otherwise, just return; no need to set ENABLED on meta -- it is always ENABLED.
|
// Otherwise, just return; no need to set ENABLED on meta -- it is always ENABLED.
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState);
|
boolean succ = false;
|
||||||
|
try {
|
||||||
|
MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState);
|
||||||
|
tableName2State.put(tableName, newState);
|
||||||
|
} finally {
|
||||||
|
if (!succ) {
|
||||||
|
tableName2State.remove(tableName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
metaStateUpdated(tableName, newState);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void metaStateUpdated(TableName tableName, TableState.State newState)
|
||||||
|
throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void metaStateDeleted(TableName tableName) throws IOException {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
protected TableState readMetaState(TableName tableName) throws IOException {
|
private TableState readMetaState(TableName tableName) throws IOException {
|
||||||
return MetaTableAccessor.getTableState(master.getConnection(), tableName);
|
TableState.State state = tableName2State.get(tableName);
|
||||||
|
if (state != null) {
|
||||||
|
return new TableState(tableName, state);
|
||||||
|
}
|
||||||
|
TableState tableState = MetaTableAccessor.getTableState(master.getConnection(), tableName);
|
||||||
|
if (tableState != null) {
|
||||||
|
tableName2State.putIfAbsent(tableName, tableState.getState());
|
||||||
|
}
|
||||||
|
return tableState;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void start() throws IOException {
|
public void start() throws IOException {
|
||||||
|
@ -222,8 +270,8 @@ public class TableStateManager {
|
||||||
|
|
||||||
private void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
|
private void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
final Map<String, TableDescriptor> allDescriptors = tableDescriptors.getAll();
|
Map<String, TableDescriptor> allDescriptors = tableDescriptors.getAll();
|
||||||
final Map<String, TableState> states = new HashMap<>();
|
Map<String, TableState> states = new HashMap<>();
|
||||||
// NOTE: Ful hbase:meta table scan!
|
// NOTE: Ful hbase:meta table scan!
|
||||||
MetaTableAccessor.fullScanTables(connection, new MetaTableAccessor.Visitor() {
|
MetaTableAccessor.fullScanTables(connection, new MetaTableAccessor.Visitor() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -233,22 +281,23 @@ public class TableStateManager {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
for (Map.Entry<String, TableDescriptor> entry: allDescriptors.entrySet()) {
|
for (Map.Entry<String, TableDescriptor> entry : allDescriptors.entrySet()) {
|
||||||
String table = entry.getKey();
|
TableName tableName = TableName.valueOf(entry.getKey());
|
||||||
if (table.equals(TableName.META_TABLE_NAME.getNameAsString())) {
|
if (TableName.isMetaTableName(tableName)) {
|
||||||
// This table is always enabled. No fixup needed. No entry in hbase:meta needed.
|
// 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.
|
// Call through to fixTableState though in case a super class wants to do something.
|
||||||
fixTableState(new TableState(TableName.valueOf(table), TableState.State.ENABLED));
|
fixTableState(new TableState(tableName, TableState.State.ENABLED));
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
TableState tableState = states.get(table);
|
TableState tableState = states.get(entry.getKey());
|
||||||
if (tableState == null) {
|
if (tableState == null) {
|
||||||
LOG.warn(table + " has no table state in hbase:meta, assuming ENABLED");
|
LOG.warn(tableName + " has no table state in hbase:meta, assuming ENABLED");
|
||||||
MetaTableAccessor.updateTableState(connection, TableName.valueOf(table),
|
MetaTableAccessor.updateTableState(connection, tableName, TableState.State.ENABLED);
|
||||||
TableState.State.ENABLED);
|
fixTableState(new TableState(tableName, TableState.State.ENABLED));
|
||||||
fixTableState(new TableState(TableName.valueOf(table), TableState.State.ENABLED));
|
tableName2State.put(tableName, TableState.State.ENABLED);
|
||||||
} else {
|
} else {
|
||||||
fixTableState(tableState);
|
fixTableState(tableState);
|
||||||
|
tableName2State.put(tableName, tableState.getState());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -256,31 +305,31 @@ public class TableStateManager {
|
||||||
/**
|
/**
|
||||||
* For subclasses in case they want to do fixup post hbase:meta.
|
* For subclasses in case they want to do fixup post hbase:meta.
|
||||||
*/
|
*/
|
||||||
protected void fixTableState(TableState tableState) throws IOException {}
|
protected void fixTableState(TableState tableState) throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This code is for case where a hbase2 Master is starting for the first time. ZooKeeper is
|
* This code is for case where a hbase2 Master is starting for the first time. ZooKeeper is where
|
||||||
* where we used to keep table state. On first startup, read zookeeper and update hbase:meta
|
* we used to keep table state. On first startup, read zookeeper and update hbase:meta with the
|
||||||
* with the table states found in zookeeper. This is tricky as we'll do this check every time we
|
* table states found in zookeeper. This is tricky as we'll do this check every time we startup
|
||||||
* startup until mirroring is disabled. See the {@link #MIGRATE_TABLE_STATE_FROM_ZK_KEY} flag.
|
* until mirroring is disabled. See the {@link #MIGRATE_TABLE_STATE_FROM_ZK_KEY} flag. Original
|
||||||
* Original form of this migration came in with HBASE-13032. It deleted all znodes when done.
|
* form of this migration came in with HBASE-13032. It deleted all znodes when done. We can't do
|
||||||
* We can't do that if we want to support hbase-1.x clients who need to be able to read table
|
* that if we want to support hbase-1.x clients who need to be able to read table state out of zk.
|
||||||
* state out of zk. See {@link MirroringTableStateManager}.
|
* See {@link MirroringTableStateManager}.
|
||||||
* @deprecated Since 2.0.0. Remove in hbase-3.0.0.
|
* @deprecated Since 2.0.0. Remove in hbase-3.0.0.
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
private void migrateZooKeeper() throws IOException {
|
private void migrateZooKeeper() throws IOException {
|
||||||
if (!this.master.getConfiguration().getBoolean(MIGRATE_TABLE_STATE_FROM_ZK_KEY,
|
if (!this.master.getConfiguration().getBoolean(MIGRATE_TABLE_STATE_FROM_ZK_KEY, true)) {
|
||||||
true)) {
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
for (Map.Entry<TableName, TableState.State> entry:
|
for (Map.Entry<TableName, TableState.State> entry : ZKDataMigrator
|
||||||
ZKDataMigrator.queryForTableStates(this.master.getZooKeeper()).entrySet()) {
|
.queryForTableStates(this.master.getZooKeeper()).entrySet()) {
|
||||||
if (this.master.getTableDescriptors().get(entry.getKey()) == null) {
|
if (this.master.getTableDescriptors().get(entry.getKey()) == null) {
|
||||||
deleteZooKeeper(entry.getKey());
|
deleteZooKeeper(entry.getKey());
|
||||||
LOG.info("Purged table state entry from zookeepr for table not in hbase:meta: " +
|
LOG.info("Purged table state entry from zookeepr for table not in hbase:meta: " +
|
||||||
entry.getKey());
|
entry.getKey());
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
TableState ts = null;
|
TableState ts = null;
|
||||||
|
@ -295,26 +344,26 @@ public class TableStateManager {
|
||||||
// DISABLING then we have a problem; we are starting up an hbase-2 on a cluster with
|
// DISABLING then we have a problem; we are starting up an hbase-2 on a cluster with
|
||||||
// RIT. It is going to be rough!
|
// RIT. It is going to be rough!
|
||||||
if (zkstate.equals(TableState.State.ENABLED) ||
|
if (zkstate.equals(TableState.State.ENABLED) ||
|
||||||
zkstate.equals(TableState.State.DISABLED)) {
|
zkstate.equals(TableState.State.DISABLED)) {
|
||||||
LOG.info("Migrating table state from zookeeper to hbase:meta; tableName=" +
|
LOG.info("Migrating table state from zookeeper to hbase:meta; tableName=" +
|
||||||
entry.getKey() + ", state=" + entry.getValue());
|
entry.getKey() + ", state=" + entry.getValue());
|
||||||
updateMetaState(entry.getKey(), entry.getValue());
|
updateMetaState(entry.getKey(), entry.getValue());
|
||||||
} else {
|
} else {
|
||||||
LOG.warn("Table={} has no state and zookeeper state is in-between={} (neither " +
|
LOG.warn("Table={} has no state and zookeeper state is in-between={} (neither " +
|
||||||
"ENABLED or DISABLED); NOT MIGRATING table state", entry.getKey(), zkstate);
|
"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
|
// What if the table states disagree? Defer to the hbase:meta setting rather than have the
|
||||||
// hbase-1.x support prevail.
|
// hbase-1.x support prevail.
|
||||||
}
|
}
|
||||||
} catch (KeeperException |InterruptedException e) {
|
} catch (KeeperException | InterruptedException e) {
|
||||||
LOG.warn("Failed reading table state from zookeeper", 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.
|
* Utility method that knows how to delete the old hbase-1.x table state znode. Used also by the
|
||||||
* Used also by the Mirroring subclass.
|
* Mirroring subclass.
|
||||||
* @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
|
* @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
|
@ -322,7 +371,7 @@ public class TableStateManager {
|
||||||
try {
|
try {
|
||||||
// Delete from ZooKeeper
|
// Delete from ZooKeeper
|
||||||
String znode = ZNodePaths.joinZNode(this.master.getZooKeeper().getZNodePaths().tableZNode,
|
String znode = ZNodePaths.joinZNode(this.master.getZooKeeper().getZNodePaths().tableZNode,
|
||||||
tableName.getNameAsString());
|
tableName.getNameAsString());
|
||||||
ZKUtil.deleteNodeFailSilent(this.master.getZooKeeper(), znode);
|
ZKUtil.deleteNodeFailSilent(this.master.getZooKeeper(), znode);
|
||||||
} catch (KeeperException e) {
|
} catch (KeeperException e) {
|
||||||
LOG.warn("Failed deleting table state from zookeeper", e);
|
LOG.warn("Failed deleting table state from zookeeper", e);
|
||||||
|
|
|
@ -42,14 +42,14 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
|
||||||
* For write lock, use lock.writeLock()
|
* For write lock, use lock.writeLock()
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class IdReadWriteLock {
|
public class IdReadWriteLock<T> {
|
||||||
// The number of lock we want to easily support. It's not a maximum.
|
// The number of lock we want to easily support. It's not a maximum.
|
||||||
private static final int NB_CONCURRENT_LOCKS = 1000;
|
private static final int NB_CONCURRENT_LOCKS = 1000;
|
||||||
/**
|
/**
|
||||||
* The pool to get entry from, entries are mapped by {@link Reference} and will be automatically
|
* The pool to get entry from, entries are mapped by {@link Reference} and will be automatically
|
||||||
* garbage-collected by JVM
|
* garbage-collected by JVM
|
||||||
*/
|
*/
|
||||||
private final ObjectPool<Long, ReentrantReadWriteLock> lockPool;
|
private final ObjectPool<T, ReentrantReadWriteLock> lockPool;
|
||||||
private final ReferenceType refType;
|
private final ReferenceType refType;
|
||||||
|
|
||||||
public IdReadWriteLock() {
|
public IdReadWriteLock() {
|
||||||
|
@ -65,22 +65,22 @@ public class IdReadWriteLock {
|
||||||
public IdReadWriteLock(ReferenceType referenceType) {
|
public IdReadWriteLock(ReferenceType referenceType) {
|
||||||
this.refType = referenceType;
|
this.refType = referenceType;
|
||||||
switch (referenceType) {
|
switch (referenceType) {
|
||||||
case SOFT:
|
case SOFT:
|
||||||
lockPool = new SoftObjectPool<>(new ObjectPool.ObjectFactory<Long, ReentrantReadWriteLock>() {
|
lockPool = new SoftObjectPool<>(new ObjectPool.ObjectFactory<T, ReentrantReadWriteLock>() {
|
||||||
@Override
|
@Override
|
||||||
public ReentrantReadWriteLock createObject(Long id) {
|
public ReentrantReadWriteLock createObject(T id) {
|
||||||
return new ReentrantReadWriteLock();
|
return new ReentrantReadWriteLock();
|
||||||
}
|
}
|
||||||
}, NB_CONCURRENT_LOCKS);
|
}, NB_CONCURRENT_LOCKS);
|
||||||
break;
|
break;
|
||||||
case WEAK:
|
case WEAK:
|
||||||
default:
|
default:
|
||||||
lockPool = new WeakObjectPool<>(new ObjectPool.ObjectFactory<Long, ReentrantReadWriteLock>() {
|
lockPool = new WeakObjectPool<>(new ObjectPool.ObjectFactory<T, ReentrantReadWriteLock>() {
|
||||||
@Override
|
@Override
|
||||||
public ReentrantReadWriteLock createObject(Long id) {
|
public ReentrantReadWriteLock createObject(T id) {
|
||||||
return new ReentrantReadWriteLock();
|
return new ReentrantReadWriteLock();
|
||||||
}
|
}
|
||||||
}, NB_CONCURRENT_LOCKS);
|
}, NB_CONCURRENT_LOCKS);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -92,7 +92,7 @@ public class IdReadWriteLock {
|
||||||
* Get the ReentrantReadWriteLock corresponding to the given id
|
* Get the ReentrantReadWriteLock corresponding to the given id
|
||||||
* @param id an arbitrary number to identify the lock
|
* @param id an arbitrary number to identify the lock
|
||||||
*/
|
*/
|
||||||
public ReentrantReadWriteLock getLock(long id) {
|
public ReentrantReadWriteLock getLock(T id) {
|
||||||
lockPool.purge();
|
lockPool.purge();
|
||||||
ReentrantReadWriteLock readWriteLock = lockPool.get(id);
|
ReentrantReadWriteLock readWriteLock = lockPool.get(id);
|
||||||
return readWriteLock;
|
return readWriteLock;
|
||||||
|
@ -113,7 +113,7 @@ public class IdReadWriteLock {
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public void waitForWaiters(long id, int numWaiters) throws InterruptedException {
|
public void waitForWaiters(T id, int numWaiters) throws InterruptedException {
|
||||||
for (ReentrantReadWriteLock readWriteLock;;) {
|
for (ReentrantReadWriteLock readWriteLock;;) {
|
||||||
readWriteLock = lockPool.get(id);
|
readWriteLock = lockPool.get(id);
|
||||||
if (readWriteLock != null) {
|
if (readWriteLock != null) {
|
||||||
|
|
|
@ -60,12 +60,12 @@ public class TestIdReadWriteLock {
|
||||||
private static final int NUM_SECONDS = 15;
|
private static final int NUM_SECONDS = 15;
|
||||||
|
|
||||||
@Parameterized.Parameter
|
@Parameterized.Parameter
|
||||||
public IdReadWriteLock idLock;
|
public IdReadWriteLock<Long> idLock;
|
||||||
|
|
||||||
@Parameterized.Parameters
|
@Parameterized.Parameters
|
||||||
public static Iterable<Object[]> data() {
|
public static Iterable<Object[]> data() {
|
||||||
return Arrays.asList(new Object[][] { { new IdReadWriteLock(ReferenceType.WEAK) },
|
return Arrays.asList(new Object[][] { { new IdReadWriteLock<Long>(ReferenceType.WEAK) },
|
||||||
{ new IdReadWriteLock(ReferenceType.SOFT) } });
|
{ new IdReadWriteLock<Long>(ReferenceType.SOFT) } });
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<Long, String> idOwner = new ConcurrentHashMap<>();
|
private Map<Long, String> idOwner = new ConcurrentHashMap<>();
|
||||||
|
|
Loading…
Reference in New Issue