HBASE-21559 The RestoreSnapshotFromClientTestBase related UT are flaky

Signed-off-by: zhangduo <zhangduo@apache.org>
This commit is contained in:
huzheng 2018-12-06 20:35:30 +08:00 committed by zhangduo
parent 1a1a65b565
commit 5cb8c3e9c7
1 changed files with 3 additions and 2 deletions

View File

@ -27,6 +27,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -150,7 +151,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
// The map is always accessed and modified under the object lock using synchronized.
// snapshotTable() will insert an Handler in the table.
// isSnapshotDone() will remove the handler requested if the operation is finished.
private Map<TableName, SnapshotSentinel> snapshotHandlers = new HashMap<>();
private Map<TableName, SnapshotSentinel> snapshotHandlers = new ConcurrentHashMap<>();
// Restore map, with table name as key, procedure ID as value.
// The map is always accessed and modified under the object lock using synchronized.
@ -419,7 +420,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
* @param tableName name of the table being snapshotted.
* @return <tt>true</tt> if there is a snapshot in progress on the specified table.
*/
public synchronized boolean isTakingSnapshot(final TableName tableName) {
public boolean isTakingSnapshot(final TableName tableName) {
SnapshotSentinel handler = this.snapshotHandlers.get(tableName);
return handler != null && !handler.isFinished();
}