mirror of https://github.com/apache/nifi.git
NIFI-1187: Fixing issue of possible assigment reordering causing uninitalized values to be possibly returned
Signed-off-by: Aldrin Piri <aldrin@apache.org>
This commit is contained in:
parent
c541c82c35
commit
ab7940368a
|
@ -137,14 +137,12 @@ public abstract class AbstractCouchbaseProcessor extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
private CouchbaseClusterControllerService getClusterService(final ProcessContext context) {
|
private CouchbaseClusterControllerService getClusterService(final ProcessContext context) {
|
||||||
if (clusterService == null) {
|
|
||||||
synchronized (AbstractCouchbaseProcessor.class) {
|
synchronized (AbstractCouchbaseProcessor.class) {
|
||||||
if (clusterService == null) {
|
if (clusterService == null) {
|
||||||
clusterService = context.getProperty(COUCHBASE_CLUSTER_SERVICE)
|
clusterService = context.getProperty(COUCHBASE_CLUSTER_SERVICE)
|
||||||
.asControllerService(CouchbaseClusterControllerService.class);
|
.asControllerService(CouchbaseClusterControllerService.class);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
return clusterService;
|
return clusterService;
|
||||||
}
|
}
|
||||||
|
|
|
@ -59,12 +59,8 @@ public class FlowFileEvent implements Event {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<String, String> getHeaders() {
|
public Map<String, String> getHeaders() {
|
||||||
if (!headersLoaded) {
|
|
||||||
synchronized (headers) {
|
synchronized (headers) {
|
||||||
if (headersLoaded) {
|
if (!headersLoaded) {
|
||||||
return headers;
|
|
||||||
}
|
|
||||||
|
|
||||||
headers.putAll(flowFile.getAttributes());
|
headers.putAll(flowFile.getAttributes());
|
||||||
headers.put(ENTRY_DATE_HEADER, Long.toString(flowFile.getEntryDate()));
|
headers.put(ENTRY_DATE_HEADER, Long.toString(flowFile.getEntryDate()));
|
||||||
headers.put(ID_HEADER, Long.toString(flowFile.getId()));
|
headers.put(ID_HEADER, Long.toString(flowFile.getId()));
|
||||||
|
@ -76,7 +72,6 @@ public class FlowFileEvent implements Event {
|
||||||
}
|
}
|
||||||
headers.put(LINEAGE_START_DATE_HEADER, Long.toString(flowFile.getLineageStartDate()));
|
headers.put(LINEAGE_START_DATE_HEADER, Long.toString(flowFile.getLineageStartDate()));
|
||||||
headers.put(SIZE_HEADER, Long.toString(flowFile.getSize()));
|
headers.put(SIZE_HEADER, Long.toString(flowFile.getSize()));
|
||||||
|
|
||||||
headersLoaded = true;
|
headersLoaded = true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -94,11 +89,7 @@ public class FlowFileEvent implements Event {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] getBody() {
|
public byte[] getBody() {
|
||||||
if (bodyLoaded) {
|
synchronized (bodyLock) {
|
||||||
return body;
|
|
||||||
}
|
|
||||||
|
|
||||||
synchronized (bodyLock ) {
|
|
||||||
if (!bodyLoaded) {
|
if (!bodyLoaded) {
|
||||||
if (flowFile.getSize() > Integer.MAX_VALUE) {
|
if (flowFile.getSize() > Integer.MAX_VALUE) {
|
||||||
throw new RuntimeException("Can't get body of Event because the backing FlowFile is too large (" + flowFile.getSize() + " bytes)");
|
throw new RuntimeException("Can't get body of Event because the backing FlowFile is too large (" + flowFile.getSize() + " bytes)");
|
||||||
|
|
Loading…
Reference in New Issue