Enis Soztutar created PHOENIX-2892:
--------------------------------------
Summary: Scan for pre-warming the block cache for 2ndary index
should be removed
Key: PHOENIX-2892
URL: https://issues.apache.org/jira/browse/PHOENIX-2892
Project: Phoenix
Issue Type: Bug
Reporter: Enis Soztutar
Assignee: Enis Soztutar
Fix For: 4.8.0
We have run into an issue in a mid-sized cluster with secondary indexes. The
problem is that all handlers for doing writes were blocked waiting on a single
scan from the secondary index to complete for > 5mins, thus causing all
incoming RPCs to timeout and causing write un-availability and further problems
(disabling the index, etc). We've taken jstack outputs continuously from the
servers to understand what is going on.
In the jstack outputs from that particular server, we can see three types of
stacks (this is raw jstack so the thread names are not there unfortunately).
- First, there are a lot of threads waiting for the MVCC transactions started
previously:
{code}
Thread 15292: (state = BLOCKED)
- java.lang.Object.wait(long) @bci=0 (Compiled frame; information may be
imprecise)
-
org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl.waitForPreviousTransactionsComplete(org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl$WriteEntry)
@bci=86, line=253 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl.completeMemstoreInsertWithSeqNum(org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl$WriteEntry,
org.apache.hadoop.hbase.regionserver.SequenceId) @bci=29, line=135 (Compiled
frame)
-
org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(org.apache.hadoop.hbase.regionserver.HRegion$BatchOperationInProgress)
@bci=1906, line=3187 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(org.apache.hadoop.hbase.regionserver.HRegion$BatchOperationInProgress)
@bci=79, line=2819 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(org.apache.hadoop.hbase.client.Mutation[],
long, long) @bci=12, line=2761 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(org.apache.hadoop.hbase.protobuf.generated.ClientProtos$RegionActionResult$Builder,
org.apache.hadoop.hbase.regionserver.Region,
org.apache.hadoop.hbase.quotas.OperationQuota, java.util.List,
org.apache.hadoop.hbase.CellScanner) @bci=150, line=692 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(org.apache.hadoop.hbase.regionserver.Region,
org.apache.hadoop.hbase.quotas.OperationQuota,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$RegionAction,
org.apache.hadoop.hbase.CellScanner,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$RegionActionResult$Builder,
java.util.List, long) @bci=547, line=654 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(com.google.protobuf.RpcController,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$MultiRequest)
@bci=407, line=2032 (Compiled frame)
-
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(com.google.protobuf.Descriptors$MethodDescriptor,
com.google.protobuf.RpcController, com.google.protobuf.Message) @bci=167,
line=32213 (Compiled frame)
-
org.apache.hadoop.hbase.ipc.RpcServer.call(com.google.protobuf.BlockingService,
com.google.protobuf.Descriptors$MethodDescriptor, com.google.protobuf.Message,
org.apache.hadoop.hbase.CellScanner, long,
org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler) @bci=59, line=2114
(Compiled frame)
- org.apache.hadoop.hbase.ipc.CallRunner.run() @bci=345, line=101 (Compiled
frame)
-
org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(java.util.concurrent.BlockingQueue)
@bci=54, line=130 (Compiled frame)
- org.apache.hadoop.hbase.ipc.RpcExecutor$1.run() @bci=20, line=107
(Interpreted frame)
- java.lang.Thread.run() @bci=11, line=745 (Interpreted frame)
{code}
The way MVCC works is that it assumes that transactions are short living, and
it guarantees that transactions are committed in strict serial order.
Transactions in this case are write requests coming in and being executed from
handlers. Each handler will start a transaction, get a mvcc write index (which
is the mvcc trx number) and does the WAL append + memstore append. Then it
marks the mvcc trx to be complete, and before returning to the user, we have to
guarantee that the transaction is visible. So we wait for the mvcc read point
to be advanced beyond our own write trx number. This is done at the above stack
trace (waitForPreviousTransactionsComplete). A lot of threads with this stack
means that one or more handlers have started a mvcc transaction, but did not
finish the work and thus did not complete their transactions. MVCC read point
can only advance serially, otherwise ongoing transactions will be visible, thus
we just wait for the previously started transactions to complete.
- second set of threads are waiting with this stack trace:
{code}
Thread 15274: (state = BLOCKED)
- sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may
be imprecise)
- java.util.concurrent.locks.LockSupport.parkNanos(java.lang.Object, long)
@bci=20, line=226 (Interpreted frame)
-
java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(int,
long) @bci=122, line=1033 (Interpreted frame)
-
java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(int,
long) @bci=25, line=1326 (Interpreted frame)
- java.util.concurrent.CountDownLatch.await(long,
java.util.concurrent.TimeUnit) @bci=10, line=282 (Interpreted frame)
- org.apache.hadoop.hbase.regionserver.HRegion.getRowLockInternal(byte[],
boolean) @bci=94, line=5044 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(org.apache.hadoop.hbase.regionserver.HRegion$BatchOperationInProgress)
@bci=386, line=2962 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(org.apache.hadoop.hbase.regionserver.HRegion$BatchOperationInProgress)
@bci=79, line=2819 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(org.apache.hadoop.hbase.client.Mutation[],
long, long) @bci=12, line=2761 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(org.apache.hadoop.hbase.protobuf.generated.ClientProtos$RegionActionResult$Builder,
org.apache.hadoop.hbase.regionserver.Region,
org.apache.hadoop.hbase.quotas.OperationQuota, java.util.List,
org.apache.hadoop.hbase.CellScanner) @bci=150, line=692 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(org.apache.hadoop.hbase.regionserver.Region,
org.apache.hadoop.hbase.quotas.OperationQuota,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$RegionAction,
org.apache.hadoop.hbase.CellScanner,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$RegionActionResult$Builder,
java.util.List, long) @bci=547, line=654 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(com.google.protobuf.RpcController,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$MultiRequest)
@bci=407, line=2032 (Compiled frame)
-
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(com.google.protobuf.Descriptors$MethodDescriptor,
com.google.protobuf.RpcController, com.google.protobuf.Message) @bci=167,
line=32213 (Compiled frame)
-
org.apache.hadoop.hbase.ipc.RpcServer.call(com.google.protobuf.BlockingService,
com.google.protobuf.Descriptors$MethodDescriptor, com.google.protobuf.Message,
org.apache.hadoop.hbase.CellScanner, long,
org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler) @bci=59, line=2114
(Compiled frame)
- org.apache.hadoop.hbase.ipc.CallRunner.run() @bci=345, line=101 (Compiled
frame)
-
org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(java.util.concurrent.BlockingQueue)
@bci=54, line=130 (Compiled frame)
- org.apache.hadoop.hbase.ipc.RpcExecutor$1.run() @bci=20, line=107
(Interpreted frame)
- java.lang.Thread.run() @bci=11, line=745 (Interpreted frame)
{code}
These are the threads that want to write something, but they have to wait for
the row lock(s) that are held from the other threads. It is very likely that
the threads in the previous set are holding the actual row locks that these set
of threads wants to acquire. This is normal behavior, nothing concerning.
However, if the handler cannot acquire the row lock in 60 seconds, then the
handler will throw an exception, and RPC will be retried.
- Finally the only thread in the third set is doing a scan for the index
update:
{code}
Thread 15276: (state = IN_JAVA)
- org.apache.hadoop.hbase.regionserver.DefaultMemStore$MemStoreScanner.peek()
@bci=0, line=847 (Compiled frame; information may be imprecise)
-
org.apache.hadoop.hbase.regionserver.KeyValueHeap$KVScannerComparator.compare(org.apache.hadoop.hbase.regionserver.KeyValueScanner,
org.apache.hadoop.hbase.regionserver.KeyValueScanner) @bci=2, line=181
(Compiled frame)
-
org.apache.hadoop.hbase.regionserver.KeyValueHeap$KVScannerComparator.compare(java.lang.Object,
java.lang.Object) @bci=9, line=171 (Compiled frame)
- java.util.PriorityQueue.siftUpUsingComparator(int, java.lang.Object)
@bci=25, line=649 (Compiled frame)
- java.util.PriorityQueue.siftUp(int, java.lang.Object) @bci=10, line=627
(Compiled frame)
- java.util.PriorityQueue.offer(java.lang.Object) @bci=67, line=329 (Compiled
frame)
- java.util.PriorityQueue.add(java.lang.Object) @bci=2, line=306 (Compiled
frame)
- org.apache.hadoop.hbase.regionserver.KeyValueHeap.generalizedSeek(boolean,
org.apache.hadoop.hbase.Cell, boolean, boolean) @bci=36, line=289 (Compiled
frame)
-
org.apache.hadoop.hbase.regionserver.KeyValueHeap.reseek(org.apache.hadoop.hbase.Cell)
@bci=5, line=256 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.StoreScanner.reseek(org.apache.hadoop.hbase.Cell)
@bci=53, line=817 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.StoreScanner.seekAsDirection(org.apache.hadoop.hbase.Cell)
@bci=2, line=803 (Compiled frame)
- org.apache.hadoop.hbase.regionserver.StoreScanner.next(java.util.List,
org.apache.hadoop.hbase.regionserver.ScannerContext) @bci=876, line=636
(Compiled frame)
- org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(java.util.List,
org.apache.hadoop.hbase.regionserver.ScannerContext) @bci=29, line=147
(Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(java.util.List,
org.apache.hadoop.hbase.regionserver.KeyValueHeap,
org.apache.hadoop.hbase.regionserver.ScannerContext, byte[], int, short)
@bci=22, line=5483 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(java.util.List,
org.apache.hadoop.hbase.regionserver.ScannerContext) @bci=385, line=5634
(Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(java.util.List,
org.apache.hadoop.hbase.regionserver.ScannerContext) @bci=29, line=5421
(Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(java.util.List)
@bci=6, line=5407 (Compiled frame)
-
org.apache.phoenix.index.PhoenixIndexBuilder.batchStarted(org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress)
@bci=279, line=88 (Compiled frame)
-
org.apache.phoenix.hbase.index.builder.IndexBuildManager.getIndexUpdate(org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress,
java.util.Collection) @bci=5, line=121 (Compiled frame)
-
org.apache.phoenix.hbase.index.Indexer.preBatchMutateWithExceptions(org.apache.hadoop.hbase.coprocessor.ObserverContext,
org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress) @bci=300,
line=278 (Compiled frame)
-
org.apache.phoenix.hbase.index.Indexer.preBatchMutate(org.apache.hadoop.hbase.coprocessor.ObserverContext,
org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress) @bci=17,
line=207 (Interpreted frame)
-
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$35.call(org.apache.hadoop.hbase.coprocessor.RegionObserver,
org.apache.hadoop.hbase.coprocessor.ObserverContext) @bci=6, line=991
(Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$RegionOperation.call(org.apache.hadoop.hbase.Coprocessor,
org.apache.hadoop.hbase.coprocessor.ObserverContext) @bci=6, line=1673
(Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(boolean,
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$CoprocessorOperation)
@bci=88, line=1748 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$CoprocessorOperation)
@bci=3, line=1705 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.preBatchMutate(org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress)
@bci=26, line=987 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(org.apache.hadoop.hbase.regionserver.HRegion$BatchOperationInProgress)
@bci=996, line=3044 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(org.apache.hadoop.hbase.regionserver.HRegion$BatchOperationInProgress)
@bci=79, line=2819 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(org.apache.hadoop.hbase.client.Mutation[],
long, long) @bci=12, line=2761 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(org.apache.hadoop.hbase.protobuf.generated.ClientProtos$RegionActionResult$Builder,
org.apache.hadoop.hbase.regionserver.Region,
org.apache.hadoop.hbase.quotas.OperationQuota, java.util.List,
org.apache.hadoop.hbase.CellScanner) @bci=150, line=692 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(org.apache.hadoop.hbase.regionserver.Region,
org.apache.hadoop.hbase.quotas.OperationQuota,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$RegionAction,
org.apache.hadoop.hbase.CellScanner,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$RegionActionResult$Builder,
java.util.List, long) @bci=547, line=654 (Compiled frame)
-
org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(com.google.protobuf.RpcController,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$MultiRequest)
@bci=407, line=2032 (Compiled frame)
-
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(com.google.protobuf.Descriptors$MethodDescriptor,
com.google.protobuf.RpcController, com.google.protobuf.Message) @bci=167,
line=32213 (Compiled frame)
-
org.apache.hadoop.hbase.ipc.RpcServer.call(com.google.protobuf.BlockingService,
com.google.protobuf.Descriptors$MethodDescriptor, com.google.protobuf.Message,
org.apache.hadoop.hbase.CellScanner, long,
org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler) @bci=59, line=2114
(Compiled frame)
- org.apache.hadoop.hbase.ipc.CallRunner.run() @bci=345, line=101 (Compiled
frame)
-
org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(java.util.concurrent.BlockingQueue)
@bci=54, line=130 (Compiled frame)
- org.apache.hadoop.hbase.ipc.RpcExecutor$1.run() @bci=20, line=107
(Interpreted frame)
- java.lang.Thread.run() @bci=11, line=745 (Interpreted frame)
{code}
This thread is doing a "scan" to be able to compute the index updates (note
HRegion.doMiniBatchMutation -> IndexBuildManager.getIndexUpdate -> Region
Scan).
Normally, this scan should finish in a short interval, but for whatever reason
it takes more than 5 minutes. We know that it takes more than 5 minutes,
because in the jstacks from epoch 1462985796 to epoch 1462985958, we see the
exact same thread (Thread 15276) continuing the scan.
So, we do a skip scan which takes >5 minutes while holding some row locks, and
holding the MVCC write transaction open. All the other handlers that start
write transactions after this also holds some other row locks and waits for the
handler doing the scan to finish. The other handlers are just waiting for some
of the same row locks. So in effect everybody is waiting on one handler thread
to finish the scan.
So, why we are doing a scan in the index builder code, and why it is taking
more than 5 minutes?
Apparently, we are doing a skip scan in PhoenixIndexBuilder:
{code}
public void batchStarted(MiniBatchOperationInProgress<Mutation>
miniBatchOp, IndexMetaData context) throws IOException {
// The entire purpose of this method impl is to get the existing rows
for the
// table rows being indexed into the block cache, as the index
maintenance code
// does a point scan per row.
List<IndexMaintainer> indexMaintainers =
((PhoenixIndexMetaData)context).getIndexMaintainers();
List<KeyRange> keys =
Lists.newArrayListWithExpectedSize(miniBatchOp.size());
Map<ImmutableBytesWritable, IndexMaintainer> maintainers =
new HashMap<ImmutableBytesWritable, IndexMaintainer>();
ImmutableBytesWritable indexTableName = new ImmutableBytesWritable();
for (int i = 0; i < miniBatchOp.size(); i++) {
Mutation m = miniBatchOp.getOperation(i);
keys.add(PVarbinary.INSTANCE.getKeyRange(m.getRow()));
for(IndexMaintainer indexMaintainer: indexMaintainers) {
if (indexMaintainer.isImmutableRows()) continue;
indexTableName.set(indexMaintainer.getIndexTableName());
if (maintainers.get(indexTableName) != null) continue;
maintainers.put(indexTableName, indexMaintainer);
}
}
if (maintainers.isEmpty()) return;
Scan scan = IndexManagementUtil.newLocalStateScan(new
ArrayList<IndexMaintainer>(maintainers.values()));
ScanRanges scanRanges = ScanRanges.createPointLookup(keys);
scanRanges.initializeScan(scan);
scan.setFilter(new SkipScanFilter(scanRanges.getSkipScanFilter(),true));
Region region = env.getRegion();
RegionScanner scanner = region.getScanner(scan);
// Run through the scanner using internal nextRaw method
region.startRegionOperation();
try {
synchronized (scanner) {
boolean hasMore;
do {
List<Cell> results = Lists.newArrayList();
// Results are potentially returned even when the return
value of s.next is
// false since this is an indication of whether or not
there are more values
// after the ones returned
hasMore = scanner.nextRaw(results);
} while (hasMore);
}
}
{code}
I think that for some cases, this skip scan turns into an expensive and long
scan (instead of point lookup scan). However, there is no easy way to debug the
actual rows and the schema from that.
The comments say that it is done to bring the results to the block cache for
the following gets to come. However, there should not be a need for that at all
since the reads themselves will bring those results to the block cache. The
only explanation maybe to use prefetching from the skip scan, but it is still
not correct to do double-work.
The other thing is that we execute the getIndexUpdate()'s in parallel later,
but we are still processing the skip scan serially.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)