[
https://issues.apache.org/jira/browse/PHOENIX-7245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17842109#comment-17842109
]
Kadir Ozdemir edited comment on PHOENIX-7245 at 4/29/24 5:19 PM:
-----------------------------------------------------------------
NPE happens in IndexRegionObserver.postBatchMutateIndispensably :
{code:java}
if (context.waitList != null) {
for (CountDownLatch countDownLatch : context.waitList) {
countDownLatch.countDown(); <--- NPE happens here
}
}
{code}
I attempted to reproduce this NPE using some ITs but could not. However, I can
see why this can happen.
The handling of concurrent batches in IndexRegionObserver is explained
[here|https://docs.google.com/document/d/12H_MwsPtyM0ORiBHclBpBLZWtm4zpY_cc5y_pwtgMUk/edit]
as follows
{quote}Currently, we maintain a map of pending rows which maps a data table row
key to an instance of PendingRow which maintains a reference count of pending
mutations from different batches on the same row. Also for each mutation batch,
an instance of BatchMutateContext, the context of the batch, is maintained.
This context holds the row locks, pre and post states of the data table rows to
be updated by this batch, the index mutations corresponding to the data table
updates and so on. The context is a local thread variable so that it can be
accessed by coprocessor calls.
The proposed changes on these two classes are as follows. PendingRow will have
a new attribute of type BatchMutateContext to refer to the most recent batch
for a row. BatchMutateContext will have three new attributes: currentPhase,
lastConcurrentBatchContext, and waitList. The currentPhase attribute is an enum
with values PRE, POST and FAILED to represent if the data table has been
updated or not, or the update has failed. lastConcurrentBatchContext is a map
from a row key to a BatchMutateContext object of the batch that is the last
mutation batch on the row with this row key (i.e., has arrived before this
batch having the map). waitList is a list of latches for the batch mutate
threads waiting for this batch (having the waitList) to complete.
After acquiring row locks for the first time for a given batch of mutations (to
read the data table row states), the pending rows map is used to check if there
is already a pending mutation on this row and the phase of the mutation batch
for this pending mutation is PRE. If so, instead of reading the row state from
the table, the in-memory post row state is retrieved from the reference on the
PendingRow object and the referenced BatchMutateContext object is added to the
lastConcurrentBatchContext map to check the phase of referenced batch later.
After acquiring row locks for the first time for a given batch of mutations (to
update the data table rows), the lastConcurrentBatchContext is used to check if
the previous concurrent batches have completed their data table updates or
failed (i..e, their phase become POST or FAILED). If a previous batch is still
in phase PRE, then the current thread adds a latch to the waitList of these
previous batches and then starts waiting on the latch. The wait is a timed wait
and if the wait times out, the waiting batch thread fails the batch by throwing
an IOException. The HBase client retries this batch later.
When a batch mutate thread completes its data table update or fails to update,
it signals the threads using its waitList.
The concurrent batch of mutations is a set such that every pair of batches in
this set has at least one common row. Since a BatchMutateContext object of a
batch is modified only after the row locks for all the rows that are mutated by
this batch are acquired, there can be only one thread can acquire the locks for
its batch and safely access all the batch contexts in the set of concurrent
batches. Because of this, we do not read atomic variables or additional locks
to serialize the access to the BatchMutateContext objects.
{quote}
The issue with the above design is the assumption that BatchMutateContext is
modified only after the row locks for all the rows that are mutated by this
batch are acquired, there can be only one thread can acquire the locks for its
batch and safely access all the batch contexts in the set of concurrent
batches. However this assumption does not always hold. If the current thread is
interrupted (i.e., gets an exception) while the row locks are being acquired,
then BatchMutateContext can be updated by multiple threads and these updates
will not be safe.
The postBatchMutateIndispensably coproc hook is always called regardless if the
current operation is successful or not. The boolean input parameter called
success indicates if the operation is successful. If the operation is
successful then, it is safe to assume that all row locks are acquired. However,
it is not safe to assume that all row locks are acquired if the operation is
not successful.
To fix this problem, we need to protect the wait list using a separate lock.
Also, we can immediately remove the PendingRow objects from the memory when a
batch is failed (instead of checking their reference counts and then removing
them when their reference count becomes zero).
[~tkhurana], [~vjasani], [~stoty], I will post a PR on this soon.
was (Author: kadir):
NPE happens in IndexRegionObserver.postBatchMutateIndispensably :
if (context.waitList != null) \{
for (CountDownLatch countDownLatch : context.waitList) {
countDownLatch.countDown(); <--- NPE happens here
}
}
I attempted to reproduce this NPE using some ITs but could not. However, I can
see why this can happen.
The handling of concurrent batches in IndexRegionObserver is explained
[here|https://docs.google.com/document/d/12H_MwsPtyM0ORiBHclBpBLZWtm4zpY_cc5y_pwtgMUk/edit]
as follows
{quote}Currently, we maintain a map of pending rows which maps a data table row
key to an instance of PendingRow which maintains a reference count of pending
mutations from different batches on the same row. Also for each mutation batch,
an instance of BatchMutateContext, the context of the batch, is maintained.
This context holds the row locks, pre and post states of the data table rows to
be updated by this batch, the index mutations corresponding to the data table
updates and so on. The context is a local thread variable so that it can be
accessed by coprocessor calls.
The proposed changes on these two classes are as follows. PendingRow will have
a new attribute of type BatchMutateContext to refer to the most recent batch
for a row. BatchMutateContext will have three new attributes: currentPhase,
lastConcurrentBatchContext, and waitList. The currentPhase attribute is an enum
with values PRE, POST and FAILED to represent if the data table has been
updated or not, or the update has failed. lastConcurrentBatchContext is a map
from a row key to a BatchMutateContext object of the batch that is the last
mutation batch on the row with this row key (i.e., has arrived before this
batch having the map). waitList is a list of latches for the batch mutate
threads waiting for this batch (having the waitList) to complete.
After acquiring row locks for the first time for a given batch of mutations (to
read the data table row states), the pending rows map is used to check if there
is already a pending mutation on this row and the phase of the mutation batch
for this pending mutation is PRE. If so, instead of reading the row state from
the table, the in-memory post row state is retrieved from the reference on the
PendingRow object and the referenced BatchMutateContext object is added to the
lastConcurrentBatchContext map to check the phase of referenced batch later.
After acquiring row locks for the first time for a given batch of mutations (to
update the data table rows), the lastConcurrentBatchContext is used to check if
the previous concurrent batches have completed their data table updates or
failed (i..e, their phase become POST or FAILED). If a previous batch is still
in phase PRE, then the current thread adds a latch to the waitList of these
previous batches and then starts waiting on the latch. The wait is a timed wait
and if the wait times out, the waiting batch thread fails the batch by throwing
an IOException. The HBase client retries this batch later.
When a batch mutate thread completes its data table update or fails to update,
it signals the threads using its waitList.
The concurrent batch of mutations is a set such that every pair of batches in
this set has at least one common row. Since a BatchMutateContext object of a
batch is modified only after the row locks for all the rows that are mutated by
this batch are acquired, there can be only one thread can acquire the locks for
its batch and safely access all the batch contexts in the set of concurrent
batches. Because of this, we do not read atomic variables or additional locks
to serialize the access to the BatchMutateContext objects.
{quote}
The issue with the above design is the assumption that BatchMutateContext is
modified only after the row locks for all the rows that are mutated by this
batch are acquired, there can be only one thread can acquire the locks for its
batch and safely access all the batch contexts in the set of concurrent
batches. However this assumption does not always hold. If the current thread is
interrupted (i.e., gets an exception) while the row locks are being acquired,
then BatchMutateContext can be updated by multiple threads and these updates
will not be safe.
The postBatchMutateIndispensably coproc hook is always called regardless the
current operation is successful or not. The boolean input parameter called
success indicates if the operation is successful. If the operation is
successful then, it is safe to assume that all row locks are acquired. However,
it is not safe to assume that all row locks are acquired if the operation is
not successful.
To fix this problem, we need to protect the wait list using a separate lock.
Also, we can immediately remove the PendingRow objects from the memory when a
batch is failed (instead of checking their reference counts PendingRow objects
and then removing them when their reference count becomes zero).
[~tkhurana], [~vjasani], [~stoty], I will post a PR on this soon.
> NPE in Phoenix Coproc leading to Region Server crash
> ----------------------------------------------------
>
> Key: PHOENIX-7245
> URL: https://issues.apache.org/jira/browse/PHOENIX-7245
> Project: Phoenix
> Issue Type: Bug
> Components: phoenix
> Affects Versions: 5.1.1
> Reporter: Ravi Kishore Valeti
> Assignee: Kadir Ozdemir
> Priority: Major
>
> In our Production, while investigating Region Server crashes, we found that
> it is due to Phoenix coproc throwing Null Pointer Exception in
> IndexRegionObserver.postBatchMutateIndispensably() method.
> Below are the logs
> {code:java}
> 2024-02-26 13:52:40,716 ERROR
> [r.default.FPBQ.Fifo.handler=216,queue=8,port=xxxxx]
> coprocessor.CoprocessorHost - The coprocessor
> org.apache.phoenix.hbase.index.IndexRegionObserver threw
> java.lang.NullPointerExceptionjava.lang.NullPointerExceptionat
> org.apache.phoenix.hbase.index.IndexRegionObserver.postBatchMutateIndispensably(IndexRegionObserver.java:1301)at
>
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$30.call(RegionCoprocessorHost.java:1028)at
>
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$30.call(RegionCoprocessorHost.java:1025)at
>
> org.apache.hadoop.hbase.coprocessor.CoprocessorHost$ObserverOperationWithoutResult.callObserver(CoprocessorHost.java:558)at
>
> org.apache.hadoop.hbase.coprocessor.CoprocessorHost.execOperation(CoprocessorHost.java:631)at
>
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.postBatchMutateIndispensably(RegionCoprocessorHost.java:1025)at
>
> org.apache.hadoop.hbase.regionserver.HRegion$MutationBatchOperation.doPostOpCleanupForMiniBatch(HRegion.java:4134)at
>
> org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutate(HRegion.java:4573)at
>
> org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:4447)at
> org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:4369)at
> org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(RSRpcServices.java:1033)at
>
> org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicBatchOp(RSRpcServices.java:951)at
>
> org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(RSRpcServices.java:916)at
>
> org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(RSRpcServices.java:2892)at
>
> org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:45961)at
> org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:415)at
> org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:124)at
> org.apache.hadoop.hbase.ipc.RpcHandler.run(RpcHandler.java:102)at
> org.apache.hadoop.hbase.ipc.RpcHandler.run(RpcHandler.java:82)
> 2024-02-26 13:52:40,725 ERROR
> [r.default.FPBQ.Fifo.handler=216,queue=8,port=xxxxx]
> regionserver.HRegionServer - ***** ABORTING region server
> ssss,xxxxx,1708268161243: The coprocessor
> org.apache.phoenix.hbase.index.IndexRegionObserver threw
> java.lang.NullPointerException *****java.lang.NullPointerExceptionat
> org.apache.phoenix.hbase.index.IndexRegionObserver.postBatchMutateIndispensably(IndexRegionObserver.java:1301)at
>
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$30.call(RegionCoprocessorHost.java:1028)at
>
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$30.call(RegionCoprocessorHost.java:1025)at
>
> org.apache.hadoop.hbase.coprocessor.CoprocessorHost$ObserverOperationWithoutResult.callObserver(CoprocessorHost.java:558)at
>
> org.apache.hadoop.hbase.coprocessor.CoprocessorHost.execOperation(CoprocessorHost.java:631)at
>
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.postBatchMutateIndispensably(RegionCoprocessorHost.java:1025)at
>
> org.apache.hadoop.hbase.regionserver.HRegion$MutationBatchOperation.doPostOpCleanupForMiniBatch(HRegion.java:4134)at
>
> org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutate(HRegion.java:4573)at
>
> org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:4447)at
> org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:4369)at
> org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(RSRpcServices.java:1033)at
>
> org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicBatchOp(RSRpcServices.java:951)at
>
> org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(RSRpcServices.java:916)at
>
> org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(RSRpcServices.java:2892)at
>
> org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:45961)at
> org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:415)at
> org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:124)at
> org.apache.hadoop.hbase.ipc.RpcHandler.run(RpcHandler.java:102)at
> org.apache.hadoop.hbase.ipc.RpcHandler.run(RpcHandler.java:82){code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)