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:
Tony Kurc 2015-11-17 22:16:24 -05:00 committed by Aldrin Piri
parent c541c82c35
commit ab7940368a
2 changed files with 6 additions and 17 deletions

View File

@ -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;
} }

View File

@ -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,10 +89,6 @@ public class FlowFileEvent implements Event {
@Override @Override
public byte[] getBody() { public byte[] getBody() {
if (bodyLoaded) {
return body;
}
synchronized (bodyLock) { synchronized (bodyLock) {
if (!bodyLoaded) { if (!bodyLoaded) {
if (flowFile.getSize() > Integer.MAX_VALUE) { if (flowFile.getSize() > Integer.MAX_VALUE) {