HBASE-17232 Replace HashSet with ArrayList to accumulate delayed scanners in KVHeap and StoreScanner.

This commit is contained in:
binlijin 2016-12-02 21:10:37 +08:00
parent 4b3ffffa09
commit 9c13219f0d
2 changed files with 5 additions and 6 deletions

View File

@ -20,11 +20,10 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.PriorityQueue;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -53,7 +52,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
// Holds the scanners when a ever a eager close() happens. All such eagerly closed
// scans are collected and when the final scanner.close() happens will perform the
// actual close.
protected Set<KeyValueScanner> scannersForDelayedClose = new HashSet<KeyValueScanner>();
protected List<KeyValueScanner> scannersForDelayedClose = null;
/**
* The current sub-scanner, i.e. the one that contains the next key/value
@ -89,6 +88,8 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
KeyValueHeap(List<? extends KeyValueScanner> scanners,
KVScannerComparator comparator) throws IOException {
this.comparator = comparator;
this.scannersForDelayedClose = new ArrayList<KeyValueScanner>(
scanners.size());
if (!scanners.isEmpty()) {
this.heap = new PriorityQueue<KeyValueScanner>(scanners.size(),
this.comparator);

View File

@ -24,10 +24,8 @@ import com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.NavigableSet;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.locks.ReentrantLock;
@ -94,7 +92,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
// Collects all the KVHeap that are eagerly getting closed during the
// course of a scan
protected Set<KeyValueHeap> heapsForDelayedClose = new HashSet<KeyValueHeap>();
protected List<KeyValueHeap> heapsForDelayedClose = new ArrayList<KeyValueHeap>();
/**
* The number of KVs seen by the scanner. Includes explicitly skipped KVs, but not