HBASE-9087 Handlers being blocked during reads
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1509886 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
90da9dd4a2
commit
0108ff5323
|
@ -27,10 +27,11 @@ import java.util.Collections;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.CompletionService;
|
||||
import java.util.concurrent.CopyOnWriteArraySet;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorCompletionService;
|
||||
import java.util.concurrent.Future;
|
||||
|
@ -145,8 +146,8 @@ public class HStore implements Store {
|
|||
final List<StoreFile> filesCompacting = Lists.newArrayList();
|
||||
|
||||
// All access must be synchronized.
|
||||
private final CopyOnWriteArraySet<ChangedReadersObserver> changedReaderObservers =
|
||||
new CopyOnWriteArraySet<ChangedReadersObserver>();
|
||||
private final Set<ChangedReadersObserver> changedReaderObservers =
|
||||
Collections.newSetFromMap(new ConcurrentHashMap<ChangedReadersObserver, Boolean>());
|
||||
|
||||
private final int blocksize;
|
||||
private HFileDataBlockEncoder dataBlockEncoder;
|
||||
|
|
Loading…
Reference in New Issue