HBASE-21025 Add cache for TableStateManager

This commit is contained in:
zhangduo 2018-08-10 14:03:28 +08:00
parent 28635d6101
commit 397388316e
5 changed files with 184 additions and 146 deletions

View File

@ -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>() {

View File

@ -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.
lock.writeLock().lock();
try {
super.updateMetaState(tableName, newState);
updateZooKeeper(new TableState(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 {
super.setDeletedTable(tableName);
deleteZooKeeper(tableName); deleteZooKeeper(tableName);
} finally {
lock.writeLock().unlock();
}
} }
private void updateZooKeeper(TableState tableState) throws IOException { private void updateZooKeeper(TableState tableState) throws IOException {
@ -90,9 +79,9 @@ 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.

View File

@ -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,17 +116,17 @@ 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();
try {
TableState currentState = readMetaState(tableName); TableState currentState = readMetaState(tableName);
if (currentState == null) { if (currentState == null) {
throw new TableNotFoundException(tableName); throw new TableNotFoundException(tableName);
@ -136,6 +137,9 @@ public class TableStateManager {
} else { } else {
return false; return false;
} }
} finally {
lock.writeLock().unlock();
}
} }
public boolean isTableState(TableName tableName, TableState.State... states) { public boolean isTableState(TableName tableName, TableState.State... states) {
@ -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;
} }
ReadWriteLock lock = tnLock.getLock(tableName);
lock.writeLock().lock();
try {
MetaTableAccessor.deleteTableState(master.getConnection(), tableName); 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 {
ReadWriteLock lock = tnLock.getLock(tableName);
lock.readLock().lock();
try {
return readMetaState(tableName) != null; 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,15 +207,20 @@ public class TableStateManager {
@NonNull @NonNull
public TableState getTableState(TableName tableName) throws IOException { public TableState getTableState(TableName tableName) throws IOException {
ReadWriteLock lock = tnLock.getLock(tableName);
lock.readLock().lock();
try {
TableState currentState = readMetaState(tableName); TableState currentState = readMetaState(tableName);
if (currentState == null) { if (currentState == null) {
throw new TableStateNotFoundException(tableName); throw new TableStateNotFoundException(tableName);
} }
return currentState; return currentState;
} finally {
lock.readLock().unlock();
}
} }
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)) {
@ -205,12 +229,36 @@ public class TableStateManager {
// 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;
} }
boolean succ = false;
try {
MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState); 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,27 +305,27 @@ 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: " +
@ -307,14 +356,14 @@ public class TableStateManager {
// 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

View File

@ -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() {
@ -66,18 +66,18 @@ public class IdReadWriteLock {
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) {

View File

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