You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2016/10/21 19:59:06 UTC

[21/27] nifi git commit: NIFI-2341 - Introduce ParseCEF processor

NIFI-2341 - Introduce ParseCEF processor

This closes #785


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/b864d49f
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/b864d49f
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/b864d49f

Branch: refs/heads/appveyor-improvement
Commit: b864d49f66692dabeb805965a08e83e14d27fd58
Parents: 9f6fb61
Author: Andre F de Miranda <tr...@users.noreply.github.com>
Authored: Thu Aug 4 23:30:37 2016 +1000
Committer: Matt Burgess <ma...@apache.org>
Committed: Thu Oct 20 17:16:57 2016 -0400

----------------------------------------------------------------------
 nifi-assembly/NOTICE                            |   5 +
 .../src/main/resources/META-INF/NOTICE          |   6 +
 .../nifi-standard-processors/pom.xml            |  11 +
 .../nifi/processors/standard/ParseCEF.java      | 327 +++++++++++++++++++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../nifi/processors/standard/TestParseCEF.java  | 244 ++++++++++++++
 6 files changed, 594 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/b864d49f/nifi-assembly/NOTICE
----------------------------------------------------------------------
diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE
index 794a9d5..9d54885 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

http://git-wip-us.apache.org/repos/asf/nifi/blob/b864d49f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
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 205d9ec..0887920 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:

http://git-wip-us.apache.org/repos/asf/nifi/blob/b864d49f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
----------------------------------------------------------------------
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 b3bea5f..cce9c57 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. -->
             <artifactId>org.everit.json.schema</artifactId>
             <version>1.4.0</version>
         </dependency>
+        <dependency>
+            <groupId>com.fluenda</groupId>
+            <artifactId>ParCEFone</artifactId>
+            <version>1.2.0</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/nifi/blob/b864d49f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ParseCEF.java
----------------------------------------------------------------------
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 0000000..3417c0d
--- /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<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor>properties =new ArrayList<>();
+        properties.add(FIELDS_DESTINATION);
+        properties.add(APPEND_RAW_MESSAGE_TO_JSON);
+        properties.add(TIME_REPRESENTATION);
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        final Set<Relationship> 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<String, String> attributes = new HashMap<>();
+
+                    // Process KVs of the Header field
+                    for (Map.Entry<String, Object> entry : event.getHeader().entrySet()) {
+                        attributes.put("cef.header."+entry.getKey(), prettyResult(entry.getValue(), tzId));
+                    }
+
+                    // Process KVs composing the Extension field
+                    for (Map.Entry<String, Object> 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<MacAddress> {
+
+        @Override
+        public void serialize(MacAddress macAddress,
+                              JsonGenerator jsonGenerator,
+                              SerializerProvider serializerProvider)
+                throws IOException, JsonProcessingException {
+            jsonGenerator.writeObject(macAddress.toString());
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/b864d49f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
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 b27efdd..35d03d1 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

http://git-wip-us.apache.org/repos/asf/nifi/blob/b864d49f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestParseCEF.java
----------------------------------------------------------------------
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 0000000..0da4d2b
--- /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());
+    }
+
+
+}
+