HBASE-21559 The RestoreSnapshotFromClientTestBase related UT are flaky
Signed-off-by: zhangduo <zhangduo@apache.org>
This commit is contained in:
parent
170df27b88
commit
dfb9ae8e0e
|
@ -27,6 +27,7 @@ import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
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.ThreadPoolExecutor;
|
import java.util.concurrent.ThreadPoolExecutor;
|
||||||
import java.util.concurrent.locks.ReadWriteLock;
|
import java.util.concurrent.locks.ReadWriteLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
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.
|
// The map is always accessed and modified under the object lock using synchronized.
|
||||||
// snapshotTable() will insert an Handler in the table.
|
// snapshotTable() will insert an Handler in the table.
|
||||||
// isSnapshotDone() will remove the handler requested if the operation is finished.
|
// 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.
|
// Restore map, with table name as key, procedure ID as value.
|
||||||
// The map is always accessed and modified under the object lock using synchronized.
|
// 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.
|
* @param tableName name of the table being snapshotted.
|
||||||
* @return <tt>true</tt> if there is a snapshot in progress on the specified table.
|
* @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);
|
SnapshotSentinel handler = this.snapshotHandlers.get(tableName);
|
||||||
return handler != null && !handler.isFinished();
|
return handler != null && !handler.isFinished();
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue