diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE
index 794a9d562d..9d548857f7 100644
--- a/nifi-assembly/NOTICE
+++ b/nifi-assembly/NOTICE
@@ -1013,6 +1013,11 @@ The following binary components are provided under the Apache Software License v
Expert Group and released to the public domain, as explained at
http://creativecommons.org/publicdomain/zero/1.0/
+ (ASLv2) ParCEFone
+ The following NOTICE information applies:
+ ParCEFone
+ Copyright 2016 Fluenda
+
This includes derived works from the Apache Software License V2 library python-evtx (https://github.com/williballenthin/python-evtx)
Copyright 2012, 2013 Willi Ballenthin william.ballenthin@mandiant.com
while at Mandiant http://www.mandiant.com
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
index 205d9ecc50..0887920fcf 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
@@ -164,6 +164,11 @@ The following binary components are provided under the Apache Software License v
"GCC RUntime Library Exception"
http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
+ (ASLv2) ParCEFone
+ The following NOTICE information applies:
+ ParCEFone
+ Copyright 2016 Fluenda
+
************************
Common Development and Distribution License 1.1
************************
@@ -175,6 +180,7 @@ The following binary components are provided under the Common Development and Di
(CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:jar:1.19 - https://jersey.java.net/jersey-server/)
(CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
(CDDL 1.1) (GPL2 w/ CPE) Javax JMS Api (javax.jms:javax.jms-api:jar:2.0.1 - http://java.net/projects/jms-spec/pages/Home)
+ (CDDL 1.1) (GPL2 w/ CPE) Expression Language 3.0 API (javax.el:javax.el-api:jar:3.0.0 - http://uel-spec.java.net)
*****************
Common Development and Distribution License v1.0:
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index b3bea5fa7d..cce9c57b15 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -254,6 +254,17 @@ language governing permissions and limitations under the License. -->
org.everit.json.schema
1.4.0
+
+ com.fluenda
+ ParCEFone
+ 1.2.0
+
+
+ org.slf4j
+ slf4j-log4j12
+
+
+
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ParseCEF.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ParseCEF.java
new file mode 100644
index 0000000000..3417c0d02f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ParseCEF.java
@@ -0,0 +1,327 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.fluenda.parcefone.event.CEFHandlingException;
+import com.fluenda.parcefone.event.CommonEvent;
+import com.fluenda.parcefone.parser.CEFParser;
+
+import com.martiansoftware.macnificent.MacAddress;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.text.SimpleDateFormat;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TimeZone;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"logs", "cef", "attributes", "system", "event", "message"})
+@CapabilityDescription("Parses the contents of a CEF formatted message and adds attributes to the FlowFile for " +
+ "headers and extensions of the parts of the CEF message.\n" +
+ "Note: This Processor expects CEF messages WITHOUT the syslog headers (i.e. starting at \"CEF:0\"")
+@WritesAttributes({@WritesAttribute(attribute = "cef.header.version", description = "The version of the CEF message."),
+ @WritesAttribute(attribute = "cef.header.deviceVendor", description = "The Device Vendor of the CEF message."),
+ @WritesAttribute(attribute = "cef.header.deviceProduct", description = "The Device Product of the CEF message."),
+ @WritesAttribute(attribute = "cef.header.deviceVersion", description = "The Device Version of the CEF message."),
+ @WritesAttribute(attribute = "cef.header.deviceEventClassId", description = "The Device Event Class ID of the CEF message."),
+ @WritesAttribute(attribute = "cef.header.name", description = "The name of the CEF message."),
+ @WritesAttribute(attribute = "cef.header.severity", description = "The severity of the CEF message."),
+ @WritesAttribute(attribute = "cef.extension.*", description = "The key and value generated by the parsing of the message.")})
+@SeeAlso({ParseSyslog.class})
+
+public class ParseCEF extends AbstractProcessor {
+
+ // There should be no date format other than internationally agreed formats...
+ // flowfile-attributes uses Java 8 time to parse data (as Date objects are not timezoned)
+ private final static DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
+
+ // for some reason Jackson doesnt seem to be able to use DateTieFormater
+ // so we use a SimpleDateFormat to format within flowfile-content
+ private final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
+
+
+ // add a TZ object to be used by flowfile-attribute routine
+ private String tzId = null;
+
+ // Add serializer and mapper
+ private static final ObjectMapper mapper = new ObjectMapper();
+
+ public static final String DESTINATION_CONTENT = "flowfile-content";
+ public static final String DESTINATION_ATTRIBUTES = "flowfile-attribute";
+ public static final PropertyDescriptor FIELDS_DESTINATION = new PropertyDescriptor.Builder()
+ .name("FIELDS_DESTINATION")
+ .displayName("Parsed fields destination")
+ .description(
+ "Indicates whether the results of the CEF parser are written " +
+ "to the FlowFile content or a FlowFile attribute; if using " + DESTINATION_ATTRIBUTES +
+ "attribute, fields will be populated as attributes. " +
+ "If set to " + DESTINATION_CONTENT + ", the CEF extension field will be converted into " +
+ "a flat JSON object.")
+ .required(true)
+ .allowableValues(DESTINATION_CONTENT, DESTINATION_ATTRIBUTES)
+ .defaultValue(DESTINATION_CONTENT)
+ .build();
+
+ public static final PropertyDescriptor APPEND_RAW_MESSAGE_TO_JSON = new PropertyDescriptor.Builder()
+ .name("APPEND_RAW_MESSAGE_TO_JSON")
+ .displayName("Append raw message to JSON")
+ .description("When using flowfile-content (i.e. JSON output), add the original CEF message to " +
+ "the resulting JSON object. The original message is added as a string to _raw.")
+ .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+ .required(true)
+ .defaultValue("true")
+ .build();
+
+ public static final String UTC = "UTC";
+ public static final String LOCAL_TZ = "Local Timezone (system Default)";
+ public static final PropertyDescriptor TIME_REPRESENTATION = new PropertyDescriptor.Builder()
+ .name("TIME_REPRESENTATION")
+ .displayName("Timezone")
+ .description("Timezone to be used when representing date fields. UTC will convert all " +
+ "dates to UTC, while Local Timezone will convert them to the timezone used by NiFi.")
+ .allowableValues(UTC, LOCAL_TZ)
+ .required(true)
+ .defaultValue(LOCAL_TZ)
+ .build();
+
+ static final Relationship REL_FAILURE = new Relationship.Builder()
+ .name("failure")
+ .description("Any FlowFile that could not be parsed as a CEF message will be transferred to this Relationship without any attributes being added")
+ .build();
+ static final Relationship REL_SUCCESS = new Relationship.Builder()
+ .name("success")
+ .description("Any FlowFile that is successfully parsed as a CEF message will be to this Relationship.")
+ .build();
+
+ @Override
+ public List getSupportedPropertyDescriptors() {
+ final Listproperties =new ArrayList<>();
+ properties.add(FIELDS_DESTINATION);
+ properties.add(APPEND_RAW_MESSAGE_TO_JSON);
+ properties.add(TIME_REPRESENTATION);
+ return properties;
+ }
+
+ @Override
+ public Set getRelationships() {
+ final Set relationships = new HashSet<>();
+ relationships.add(REL_FAILURE);
+ relationships.add(REL_SUCCESS);
+ return relationships;
+ }
+
+ @OnScheduled
+ public void OnScheduled(final ProcessContext context) {
+
+ // Configure jackson mapper before spawning onTriggers
+ final SimpleModule module = new SimpleModule()
+ .addSerializer(MacAddress.class, new MacAddressToStringSerializer());
+ this.mapper.registerModule(module);
+ this.mapper.setDateFormat(this.simpleDateFormat);
+
+ switch (context.getProperty(TIME_REPRESENTATION).getValue()) {
+ case LOCAL_TZ:
+ // set the mapper TZ to local TZ
+ this.mapper.setTimeZone(TimeZone.getDefault());
+ tzId = TimeZone.getDefault().getID();
+ break;
+ case UTC:
+ // set the mapper TZ to local TZ
+ this.mapper.setTimeZone(TimeZone.getTimeZone(UTC));
+ tzId = UTC;
+ break;
+ }
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+ FlowFile flowFile = session.get();
+ if (flowFile == null) {
+ return;
+ }
+
+ final CEFParser parser = new CEFParser();
+ final byte[] buffer = new byte[(int) flowFile.getSize()];
+ session.read(flowFile, new InputStreamCallback() {
+ @Override
+ public void process(final InputStream in) throws IOException {
+ StreamUtils.fillBuffer(in, buffer);
+ }
+ });
+
+ CommonEvent event;
+
+ try {
+ event = parser.parse(buffer, true);
+ } catch (Exception e) {
+ // This should never trigger but adding in here as a fencing mechanism to
+ // address possible ParCEFone bugs.
+ getLogger().error("Parser returned unexpected Exception {} while processing {}; routing to failure", new Object[] {e, flowFile});
+ session.transfer(flowFile, REL_FAILURE);
+ return;
+ }
+
+
+ // ParCEFone returns null every time it cannot parse an
+ // event, so we test
+ if (event==null) {
+ getLogger().error("Failed to parse {} as a CEF message: it does not conform to the CEF standard; routing to failure", new Object[] {flowFile});
+ session.transfer(flowFile, REL_FAILURE);
+ return;
+ }
+
+
+ try {
+ final String destination = context.getProperty(FIELDS_DESTINATION).getValue();
+
+ switch (destination) {
+ case DESTINATION_ATTRIBUTES:
+
+ final Map attributes = new HashMap<>();
+
+ // Process KVs of the Header field
+ for (Map.Entry entry : event.getHeader().entrySet()) {
+ attributes.put("cef.header."+entry.getKey(), prettyResult(entry.getValue(), tzId));
+ }
+
+ // Process KVs composing the Extension field
+ for (Map.Entry entry : event.getExtension(true).entrySet()) {
+ attributes.put("cef.extension." + entry.getKey(), prettyResult(entry.getValue(), tzId));
+
+ flowFile = session.putAllAttributes(flowFile, attributes);
+ }
+ break;
+
+ case DESTINATION_CONTENT:
+
+ ObjectNode results = mapper.createObjectNode();
+
+ // Add two JSON objects containing one CEF field each
+ results.set("header", mapper.valueToTree(event.getHeader()));
+ results.set("extension", mapper.valueToTree(event.getExtension(true)));
+
+ // Add the original content to original CEF content
+ // to the resulting JSON
+ if (context.getProperty(APPEND_RAW_MESSAGE_TO_JSON).asBoolean()) {
+ results.set("_raw", mapper.valueToTree(new String(buffer)));
+ }
+
+ flowFile = session.write(flowFile, new OutputStreamCallback() {
+ @Override
+ public void process(OutputStream out) throws IOException {
+ try (OutputStream outputStream = new BufferedOutputStream(out)) {
+ outputStream.write(mapper.writeValueAsBytes(results));
+ }
+ }
+ });
+
+ // Adjust the FlowFile mime.type attribute
+ flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), "application/json");
+
+ // Update the provenance for good measure
+ session.getProvenanceReporter().modifyContent(flowFile, "Replaced content with parsed CEF fields and values");
+ break;
+ }
+
+ // whatever the parsing stratgy, ready to transfer to success and commit
+ session.transfer(flowFile, REL_SUCCESS);
+ session.commit();
+ } catch (CEFHandlingException e) {
+ // The flowfile has failed parsing & validation, routing to failure and committing
+ getLogger().error("Failed to parse {} as a CEF message due to {}; routing to failure", new Object[] {flowFile, e});
+ session.transfer(flowFile, REL_FAILURE);
+ session.commit();
+ return;
+ } finally {
+ session.rollback();
+ }
+ }
+
+ private String prettyResult(Object entryValue, String tzID) {
+
+ if (entryValue instanceof InetAddress ) {
+ return ((InetAddress) entryValue).getHostAddress();
+ } else if (entryValue instanceof Date) {
+ ZonedDateTime zdt = ZonedDateTime.from(((Date) entryValue).toInstant().atZone(ZoneId.of(tzID)));
+ return(String.valueOf(zdt.format(dateTimeFormatter)));
+ } else {
+ return String.valueOf(entryValue);
+ }
+ }
+
+
+ // Serialize MacAddress as plain string
+ private class MacAddressToStringSerializer extends JsonSerializer {
+
+ @Override
+ public void serialize(MacAddress macAddress,
+ JsonGenerator jsonGenerator,
+ SerializerProvider serializerProvider)
+ throws IOException, JsonProcessingException {
+ jsonGenerator.writeObject(macAddress.toString());
+ }
+
+ }
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index b27efdd39f..35d03d1d31 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -58,6 +58,7 @@ org.apache.nifi.processors.standard.LogAttribute
org.apache.nifi.processors.standard.MergeContent
org.apache.nifi.processors.standard.ModifyBytes
org.apache.nifi.processors.standard.MonitorActivity
+org.apache.nifi.processors.standard.ParseCEF
org.apache.nifi.processors.standard.ParseSyslog
org.apache.nifi.processors.standard.PostHTTP
org.apache.nifi.processors.standard.PutEmail
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestParseCEF.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestParseCEF.java
new file mode 100644
index 0000000000..0da4d2b0a6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestParseCEF.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
+
+public class TestParseCEF {
+ private SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
+
+ private final static String sample1 = "CEF:0|TestVendor|TestProduct|TestVersion|TestEventClassID|TestName|Low|" +
+ // TimeStamp, String and Long
+ "rt=Feb 09 2015 00:27:43 UTC cn3Label=Test Long cn3=9223372036854775807 " +
+ // FloatPoint and MacAddress
+ "cfp1=1.234 cfp1Label=Test FP Number smac=00:00:0c:07:ac:00 " +
+ // IPv6 and String
+ "c6a3=2001:cdba::3257:9652 c6a3Label=Test IPv6 cs1Label=Test String cs1=test test test chocolate " +
+ // IPv4
+ "destinationTranslatedAddress=123.123.123.123 " +
+ // Date without TZ
+ "deviceCustomDate1=Feb 06 2015 13:27:43 " +
+ // Integer and IP Address (from v4)
+ "dpt=1234 agt=123.123.0.124 dlat=40.366633";
+
+ private final static String sample2 = "CEF:0|TestVendor|TestProduct|TestVersion|TestEventClassID|TestName|Low|" +
+ // TimeStamp, String and Long
+ "rt=Feb 09 2015 00:27:43 UTC cn3Label=Test Long cn3=9223372036854775807 " +
+ // FloatPoint and MacAddress
+ "cfp1=1.234 cfp1Label=Test FP Number smac=00:00:0c:07:ac:00 " +
+ // IPv6 and String
+ "c6a3=2001:cdba::3257:9652 c6a3Label=Test IPv6 cs1Label=Test String cs1=test test test chocolate " +
+ // IPv4
+ "destinationTranslatedAddress=123.123.123.123 " +
+ // Date without TZ
+ "deviceCustomDate1=Feb 06 2015 13:27:43 " +
+ // Integer and IP Address (from v4)
+ "dpt=1234 agt=123.123.0.124 dlat=40.366633 " +
+ // A JSON object inside one of CEF's custom Strings
+ "cs2Label=JSON payload " +
+ "cs2={\"test_test_test\": \"chocolate!\", \"what?!?\": \"Simple! test test test chocolate!\"}";
+
+
+ @Test
+ public void testInvalidMessage() {
+ final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
+ runner.enqueue("test test test chocolate\n".getBytes());
+ runner.run();
+
+ runner.assertAllFlowFilesTransferred(ParseCEF.REL_FAILURE, 1);
+ }
+
+ @Test
+ public void testSuccessfulParseToAttributes() throws IOException {
+ final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
+ runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_ATTRIBUTES);
+ runner.enqueue(sample1.getBytes());
+ runner.run();
+
+ runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
+ final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
+ mff.assertAttributeEquals("cef.extension.rt", sdf.format(new Date(1423441663000L)));
+ mff.assertAttributeEquals("cef.extension.cn3Label", "Test Long");
+ mff.assertAttributeEquals("cef.extension.cn3", "9223372036854775807");
+ mff.assertAttributeEquals("cef.extension.cfp1", "1.234");
+ mff.assertAttributeEquals("cef.extension.cfp1Label", "Test FP Number");
+ mff.assertAttributeEquals("cef.extension.smac", "00:00:0c:07:ac:00");
+ mff.assertAttributeEquals("cef.extension.c6a3", "2001:cdba:0:0:0:0:3257:9652");
+ mff.assertAttributeEquals("cef.extension.c6a3Label", "Test IPv6");
+ mff.assertAttributeEquals("cef.extension.cs1Label", "Test String");
+ mff.assertAttributeEquals("cef.extension.cs1", "test test test chocolate");
+ mff.assertAttributeEquals("cef.extension.destinationTranslatedAddress", "123.123.123.123");
+ mff.assertContentEquals(sample1.getBytes());
+
+
+ // Converting a field without timezone will always result on render time being dependent
+ // on locale of the machine running this test.
+ long eventTime = 1423229263000L;
+ int offset = TimeZone.getDefault().getOffset(eventTime);
+ sdf.setTimeZone(TimeZone.getDefault());
+
+ String prettyEvent = sdf.format(new Date(eventTime - offset));
+
+ mff.assertAttributeEquals("cef.extension.deviceCustomDate1",prettyEvent);
+ mff.assertAttributeEquals("cef.extension.dpt", "1234");
+ mff.assertAttributeEquals("cef.extension.agt", "123.123.0.124");
+ mff.assertAttributeEquals("cef.extension.dlat", "40.366633");
+ }
+
+ @Test
+ public void testSuccessfulParseToAttributesWithUTC() throws IOException {
+ final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
+ runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_ATTRIBUTES);
+ runner.setProperty(ParseCEF.TIME_REPRESENTATION, ParseCEF.UTC);
+ runner.enqueue(sample1.getBytes());
+ runner.run();
+
+ sdf.setTimeZone(TimeZone.getTimeZone(ParseCEF.UTC));
+
+ runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
+ final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
+ mff.assertAttributeEquals("cef.extension.rt", sdf.format(new Date(1423441663000L)));
+
+ // Converting a field without timezone will always result on render time being dependent
+ // on locale of the machine running this test.
+ long eventTime = 1423229263000L;
+ int offset = TimeZone.getDefault().getOffset(eventTime);
+ sdf.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+ String prettyEvent = sdf.format(new Date(eventTime - offset));
+
+ mff.assertAttributeEquals("cef.extension.deviceCustomDate1",prettyEvent);
+ mff.assertContentEquals(sample1.getBytes());
+ }
+
+ @Test
+ public void testSuccessfulParseToContent() throws IOException {
+ final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
+ runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_CONTENT);
+ runner.enqueue(sample1.getBytes());
+ runner.run();
+
+ runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
+ final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
+
+ byte [] rawJson = mff.toByteArray();
+
+ JsonNode results = new ObjectMapper().readTree(rawJson);
+
+ JsonNode header = results.get("header");
+ JsonNode extension = results.get("extension");
+
+ Assert.assertEquals("TestVendor", header.get("deviceVendor").asText());
+ Assert.assertEquals(sdf.format(new Date(1423441663000L)),
+ extension.get("rt").asText());
+ Assert.assertEquals("Test Long", extension.get("cn3Label").asText());
+ Assert.assertEquals( 9223372036854775807L, extension.get("cn3").asLong());
+ Assert.assertTrue(extension.get("cfp1").floatValue() == 1.234F);
+ Assert.assertEquals("Test FP Number", extension.get("cfp1Label").asText());
+ Assert.assertEquals("00:00:0c:07:ac:00", extension.get("smac").asText());
+ Assert.assertEquals("2001:cdba:0:0:0:0:3257:9652", extension.get("c6a3").asText());
+ Assert.assertEquals("Test IPv6", extension.get("c6a3Label").asText());
+ Assert.assertEquals("123.123.123.123", extension.get("destinationTranslatedAddress").asText());
+ Assert.assertEquals("Test String", extension.get("cs1Label").asText());
+ Assert.assertEquals("test test test chocolate", extension.get("cs1").asText());
+ }
+
+ @Test
+ public void testSuccessfulParseToContentWhenCEFContainsJSON() throws IOException {
+ final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
+ runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_CONTENT);
+ runner.enqueue(sample2.getBytes());
+ runner.run();
+
+ runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
+ final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
+
+ byte [] rawJson = mff.toByteArray();
+
+ JsonNode results = new ObjectMapper().readTree(rawJson);
+
+ JsonNode header = results.get("header");
+ JsonNode extension = results.get("extension");
+
+ Assert.assertEquals("TestVendor", header.get("deviceVendor").asText());
+ Assert.assertEquals(sdf.format(new Date(1423441663000L)),
+ extension.get("rt").asText());
+ Assert.assertEquals("Test Long", extension.get("cn3Label").asText());
+ Assert.assertEquals( 9223372036854775807L, extension.get("cn3").asLong());
+ Assert.assertTrue(extension.get("cfp1").floatValue() == 1.234F);
+ Assert.assertEquals("Test FP Number", extension.get("cfp1Label").asText());
+ Assert.assertEquals("00:00:0c:07:ac:00", extension.get("smac").asText());
+ Assert.assertEquals("2001:cdba:0:0:0:0:3257:9652", extension.get("c6a3").asText());
+ Assert.assertEquals("Test IPv6", extension.get("c6a3Label").asText());
+ Assert.assertEquals("Test String", extension.get("cs1Label").asText());
+ Assert.assertEquals("test test test chocolate", extension.get("cs1").asText());
+ Assert.assertEquals("123.123.123.123", extension.get("destinationTranslatedAddress").asText());
+
+ JsonNode inner = new ObjectMapper().readTree(extension.get("cs2").asText());
+ Assert.assertEquals("chocolate!", inner.get("test_test_test").asText());
+ }
+
+
+ @Test
+ public void testSuccessfulParseToContentUTC() throws IOException {
+ final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
+ runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_CONTENT);
+ runner.setProperty(ParseCEF.TIME_REPRESENTATION, ParseCEF.UTC);
+ runner.enqueue(sample1.getBytes());
+ runner.run();
+
+ runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
+ final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
+
+ byte [] rawJson = mff.toByteArray();
+
+ JsonNode results = new ObjectMapper().readTree(rawJson);
+
+ JsonNode extension = results.get("extension");
+
+ sdf.setTimeZone(TimeZone.getTimeZone("UTC"));
+ Assert.assertEquals(sdf.format(new Date(1423441663000L)),
+ extension.get("rt").asText());
+
+ // Converting a field without timezone will always result on render time being dependent
+ // on locale of the machine running this test.
+ long eventTime = 1423229263000L;
+ int offset = TimeZone.getDefault().getOffset(eventTime);
+
+ // Set TZ to UTC
+ sdf.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+ String prettyEvent = sdf.format(new Date(eventTime - offset));
+ Assert.assertEquals(prettyEvent, extension.get("deviceCustomDate1").asText());
+ }
+
+
+}
+