You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2023/01/23 15:54:45 UTC
[nifi] branch main updated: NIFI-11076: Resolving deadlock issue in StandardParameterContext (#6865)
This is an automated email from the ASF dual-hosted git repository.
markap14 pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git
The following commit(s) were added to refs/heads/main by this push:
new 29618ae5c6 NIFI-11076: Resolving deadlock issue in StandardParameterContext (#6865)
29618ae5c6 is described below
commit 29618ae5c64a210fb9b497512fcc5d2e26575c75
Author: Joe Gresock <jg...@gmail.com>
AuthorDate: Mon Jan 23 10:54:38 2023 -0500
NIFI-11076: Resolving deadlock issue in StandardParameterContext (#6865)
* NIFI-11076: Resolving deadlock issue in StandardParameterContext
* NIFI-11076: Resolving deadlock issue in StandardParameterContext
---
.../nifi/parameter/StandardParameterContext.java | 22 +++++++++-------------
1 file changed, 9 insertions(+), 13 deletions(-)
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java
index 63aa3e50d6..339317fdd9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java
@@ -47,6 +47,7 @@ import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Stack;
+import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -61,7 +62,7 @@ public class StandardParameterContext implements ParameterContext {
private final Authorizable parentAuthorizable;
private String name;
- private long version = 0L;
+ private AtomicLong version = new AtomicLong(0L);
private final Map<ParameterDescriptor, Parameter> parameters = new LinkedHashMap<>();
private final List<ParameterContext> inheritedParameterContexts = new ArrayList<>();
private ParameterProvider parameterProvider;
@@ -105,7 +106,7 @@ public class StandardParameterContext implements ParameterContext {
public void setName(final String name) {
writeLock.lock();
try {
- this.version++;
+ this.version.incrementAndGet();
this.name = name;
} finally {
writeLock.unlock();
@@ -125,8 +126,9 @@ public class StandardParameterContext implements ParameterContext {
@Override
public void setParameters(final Map<String, Parameter> updatedParameters) {
writeLock.lock();
+ final Map<String, ParameterUpdate> parameterUpdates = new HashMap<>();
try {
- this.version++;
+ this.version.incrementAndGet();
final Map<ParameterDescriptor, Parameter> currentEffectiveParameters = getEffectiveParameters();
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(getProposedParameters(updatedParameters));
@@ -139,12 +141,11 @@ public class StandardParameterContext implements ParameterContext {
updateParameters(parameters, updatedParameters, true);
// Get a list of all effective updates in order to alert referencing components
- final Map<String, ParameterUpdate> parameterUpdates = new HashMap<>(updateParameters(currentEffectiveParameters, effectiveParameterUpdates, false));
-
- alertReferencingComponents(parameterUpdates);
+ parameterUpdates.putAll(updateParameters(currentEffectiveParameters, effectiveParameterUpdates, false));
} finally {
writeLock.unlock();
}
+ alertReferencingComponents(parameterUpdates);
}
private Map<ParameterDescriptor, Parameter> getProposedParameters(final Map<String, Parameter> proposedParameterUpdates) {
@@ -270,12 +271,7 @@ public class StandardParameterContext implements ParameterContext {
@Override
public long getVersion() {
- readLock.lock();
- try {
- return version;
- } finally {
- readLock.unlock();
- }
+ return version.get();
}
public Optional<Parameter> getParameter(final String parameterName) {
@@ -597,7 +593,7 @@ public class StandardParameterContext implements ParameterContext {
writeLock.lock();
try {
- this.version++;
+ this.version.incrementAndGet();
final Map<ParameterDescriptor, Parameter> currentEffectiveParameters = getEffectiveParameters();
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(inheritedParameterContexts);