[
https://issues.apache.org/jira/browse/NIFI-7527?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Tamas Palfy updated NIFI-7527:
------------------------------
Description:
The fix for https://issues.apache.org/jira/browse/NIFI-7453 (PutKudu kerberos
issue after TGT expires) introduced a new bug: after TGT refresh the processor
ends up in a deadlock.
The reason is that the onTrigger initiates a read lock:
{code:java}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession
session) throws ProcessException {
kuduClientReadLock.lock();
try {
onTrigger(context, session, kuduClientR);
} finally {
kuduClientReadLock.unlock();
}
}
{code}
and while the read lock is in effect, later (in the same stack) - if TGT
refresh occurs - a write lock is attempted:
{code:java}
...
public synchronized boolean checkTGTAndRelogin() throws
LoginException {
boolean didRelogin = super.checkTGTAndRelogin();
if (didRelogin) {
createKuduClient(context);
}
return didRelogin;
}
...
protected void createKuduClient(ProcessContext context) {
kuduClientWriteLock.lock();
try {
if (this.kuduClientR.get() != null) {
try {
this.kuduClientR.get().close();
} catch (KuduException e) {
getLogger().error("Couldn't close Kudu client.");
}
}
if (kerberosUser != null) {
final KerberosAction<KuduClient> kerberosAction = new
KerberosAction<>(kerberosUser, () -> buildClient(context), getLogger());
this.kuduClientR.set(kerberosAction.execute());
} else {
this.kuduClientR.set(buildClient(context));
}
} finally {
kuduClientWriteLock.unlock();
}
}
{code}
This attempt at the write lock will get stuck, waiting for the previous read
lock to get released.
(Other threads may have acquired the same read lock but they can release it
eventually - unless they too try to acquire the write lock themselves.)
The fix should be fairly simple: need to release the read lock before trying to
acquire the write lock.
Need to take care of the following though:
* Need to reacquire the read lock after the new Kudu client has been created
because the current logic calls an unlock on the ReadLock object later in a
finally block. As this is a reentrant lock, that means decreasing the counter
for the number of locks. We need to make sure the that number of locks and
unlocks match.
* Need to make sure whenever we reference the Kudu client, it's actually the
latest one. volatile no longer suffice, we need to wrap it in an
AtomicReference.
was:
The fix for https://issues.apache.org/jira/browse/NIFI-7453 (PutKudu kerberos
issue after TGT expires) introduced a new bug: after TGT refresh the processor
ends up in a deadlock.
The reason is that the onTrigger initiates a read lock:
{code:java}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession
session) throws ProcessException {
kuduClientReadLock.lock();
try {
onTrigger(context, session, kuduClientR);
} finally {
kuduClientReadLock.unlock();
}
}
{code}
and while the read lock is in effect, later (in the same stack) - if TGT
refresh occurs - a write lock is attempted:
{code:java}
...
public synchronized boolean checkTGTAndRelogin() throws
LoginException {
boolean didRelogin = super.checkTGTAndRelogin();
if (didRelogin) {
createKuduClient(context);
}
return didRelogin;
}
...
protected void createKuduClient(ProcessContext context) {
kuduClientWriteLock.lock();
try {
if (this.kuduClientR.get() != null) {
try {
this.kuduClientR.get().close();
} catch (KuduException e) {
getLogger().error("Couldn't close Kudu client.");
}
}
if (kerberosUser != null) {
final KerberosAction<KuduClient> kerberosAction = new
KerberosAction<>(kerberosUser, () -> buildClient(context), getLogger());
this.kuduClientR.set(kerberosAction.execute());
} else {
this.kuduClientR.set(buildClient(context));
}
} finally {
kuduClientWriteLock.unlock();
}
}
{code}
This write lock will stuck waiting to the previous read lock to get release.
(Other threads may have acquired the same readlock but they can release it
eventually - unless they too try to acquire the write lock themselves.)
The fix should be fairly simple: need to release the read lock before trying to
acquire the write lock.
Need to take care of the following though:
* Need to reacquire the read lock after the new Kudu client has been created
because the current logic calls an unlock on the ReadLock object later in a
finally block. As this is a reentrant lock, that means decreasing the counter
for the number of locks. We need to make sure the that number of locks and
unlocks match.
* Need to make sure whenever we reference the Kudu client, it's actually the
latest one. volatile no longer suffice, we need to wrap it in an
AtomicReference.
> AbstractKuduProcessor deadlocks after TGT refresh
> --------------------------------------------------
>
> Key: NIFI-7527
> URL: https://issues.apache.org/jira/browse/NIFI-7527
> Project: Apache NiFi
> Issue Type: Bug
> Reporter: Tamas Palfy
> Priority: Major
>
> The fix for https://issues.apache.org/jira/browse/NIFI-7453 (PutKudu kerberos
> issue after TGT expires) introduced a new bug: after TGT refresh the
> processor ends up in a deadlock.
> The reason is that the onTrigger initiates a read lock:
> {code:java}
> @Override
> public void onTrigger(final ProcessContext context, final ProcessSession
> session) throws ProcessException {
> kuduClientReadLock.lock();
> try {
> onTrigger(context, session, kuduClientR);
> } finally {
> kuduClientReadLock.unlock();
> }
> }
> {code}
> and while the read lock is in effect, later (in the same stack) - if TGT
> refresh occurs - a write lock is attempted:
> {code:java}
> ...
> public synchronized boolean checkTGTAndRelogin() throws
> LoginException {
> boolean didRelogin = super.checkTGTAndRelogin();
> if (didRelogin) {
> createKuduClient(context);
> }
> return didRelogin;
> }
> ...
> protected void createKuduClient(ProcessContext context) {
> kuduClientWriteLock.lock();
> try {
> if (this.kuduClientR.get() != null) {
> try {
> this.kuduClientR.get().close();
> } catch (KuduException e) {
> getLogger().error("Couldn't close Kudu client.");
> }
> }
> if (kerberosUser != null) {
> final KerberosAction<KuduClient> kerberosAction = new
> KerberosAction<>(kerberosUser, () -> buildClient(context), getLogger());
> this.kuduClientR.set(kerberosAction.execute());
> } else {
> this.kuduClientR.set(buildClient(context));
> }
> } finally {
> kuduClientWriteLock.unlock();
> }
> }
> {code}
> This attempt at the write lock will get stuck, waiting for the previous read
> lock to get released.
> (Other threads may have acquired the same read lock but they can release it
> eventually - unless they too try to acquire the write lock themselves.)
> The fix should be fairly simple: need to release the read lock before trying
> to acquire the write lock.
> Need to take care of the following though:
> * Need to reacquire the read lock after the new Kudu client has been created
> because the current logic calls an unlock on the ReadLock object later in a
> finally block. As this is a reentrant lock, that means decreasing the counter
> for the number of locks. We need to make sure the that number of locks and
> unlocks match.
> * Need to make sure whenever we reference the Kudu client, it's actually the
> latest one. volatile no longer suffice, we need to wrap it in an
> AtomicReference.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)