mirror of https://github.com/apache/nifi.git
NIFI-11076: Resolving deadlock issue in StandardParameterContext (#6865)
* NIFI-11076: Resolving deadlock issue in StandardParameterContext * NIFI-11076: Resolving deadlock issue in StandardParameterContext
This commit is contained in:
parent
eb5d172693
commit
29618ae5c6
|
@ -47,6 +47,7 @@ import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Stack;
|
import java.util.Stack;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.concurrent.locks.Lock;
|
import java.util.concurrent.locks.Lock;
|
||||||
import java.util.concurrent.locks.ReadWriteLock;
|
import java.util.concurrent.locks.ReadWriteLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
@ -61,7 +62,7 @@ public class StandardParameterContext implements ParameterContext {
|
||||||
private final Authorizable parentAuthorizable;
|
private final Authorizable parentAuthorizable;
|
||||||
|
|
||||||
private String name;
|
private String name;
|
||||||
private long version = 0L;
|
private AtomicLong version = new AtomicLong(0L);
|
||||||
private final Map<ParameterDescriptor, Parameter> parameters = new LinkedHashMap<>();
|
private final Map<ParameterDescriptor, Parameter> parameters = new LinkedHashMap<>();
|
||||||
private final List<ParameterContext> inheritedParameterContexts = new ArrayList<>();
|
private final List<ParameterContext> inheritedParameterContexts = new ArrayList<>();
|
||||||
private ParameterProvider parameterProvider;
|
private ParameterProvider parameterProvider;
|
||||||
|
@ -105,7 +106,7 @@ public class StandardParameterContext implements ParameterContext {
|
||||||
public void setName(final String name) {
|
public void setName(final String name) {
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
try {
|
try {
|
||||||
this.version++;
|
this.version.incrementAndGet();
|
||||||
this.name = name;
|
this.name = name;
|
||||||
} finally {
|
} finally {
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
|
@ -125,8 +126,9 @@ public class StandardParameterContext implements ParameterContext {
|
||||||
@Override
|
@Override
|
||||||
public void setParameters(final Map<String, Parameter> updatedParameters) {
|
public void setParameters(final Map<String, Parameter> updatedParameters) {
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
|
final Map<String, ParameterUpdate> parameterUpdates = new HashMap<>();
|
||||||
try {
|
try {
|
||||||
this.version++;
|
this.version.incrementAndGet();
|
||||||
|
|
||||||
final Map<ParameterDescriptor, Parameter> currentEffectiveParameters = getEffectiveParameters();
|
final Map<ParameterDescriptor, Parameter> currentEffectiveParameters = getEffectiveParameters();
|
||||||
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(getProposedParameters(updatedParameters));
|
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(getProposedParameters(updatedParameters));
|
||||||
|
@ -139,12 +141,11 @@ public class StandardParameterContext implements ParameterContext {
|
||||||
updateParameters(parameters, updatedParameters, true);
|
updateParameters(parameters, updatedParameters, true);
|
||||||
|
|
||||||
// Get a list of all effective updates in order to alert referencing components
|
// Get a list of all effective updates in order to alert referencing components
|
||||||
final Map<String, ParameterUpdate> parameterUpdates = new HashMap<>(updateParameters(currentEffectiveParameters, effectiveParameterUpdates, false));
|
parameterUpdates.putAll(updateParameters(currentEffectiveParameters, effectiveParameterUpdates, false));
|
||||||
|
|
||||||
alertReferencingComponents(parameterUpdates);
|
|
||||||
} finally {
|
} finally {
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
|
alertReferencingComponents(parameterUpdates);
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<ParameterDescriptor, Parameter> getProposedParameters(final Map<String, Parameter> proposedParameterUpdates) {
|
private Map<ParameterDescriptor, Parameter> getProposedParameters(final Map<String, Parameter> proposedParameterUpdates) {
|
||||||
|
@ -270,12 +271,7 @@ public class StandardParameterContext implements ParameterContext {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getVersion() {
|
public long getVersion() {
|
||||||
readLock.lock();
|
return version.get();
|
||||||
try {
|
|
||||||
return version;
|
|
||||||
} finally {
|
|
||||||
readLock.unlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public Optional<Parameter> getParameter(final String parameterName) {
|
public Optional<Parameter> getParameter(final String parameterName) {
|
||||||
|
@ -597,7 +593,7 @@ public class StandardParameterContext implements ParameterContext {
|
||||||
|
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
try {
|
try {
|
||||||
this.version++;
|
this.version.incrementAndGet();
|
||||||
|
|
||||||
final Map<ParameterDescriptor, Parameter> currentEffectiveParameters = getEffectiveParameters();
|
final Map<ParameterDescriptor, Parameter> currentEffectiveParameters = getEffectiveParameters();
|
||||||
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(inheritedParameterContexts);
|
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(inheritedParameterContexts);
|
||||||
|
|
Loading…
Reference in New Issue