Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
NIFI-2341 - Introduce ParseCEF processor
- Loading branch information
Showing
6 changed files
with
284 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
172 changes: 172 additions & 0 deletions
172
.../nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ParseCEF.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,172 @@ | ||
/* | ||
* 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.fluenda.parcefone.event.CEFHandlingException; | ||
import com.fluenda.parcefone.event.CommonEvent; | ||
import com.fluenda.parcefone.parser.CEFParser; | ||
|
||
|
||
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.flowfile.FlowFile; | ||
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.stream.io.StreamUtils; | ||
|
||
import java.io.IOException; | ||
import java.io.InputStream; | ||
import java.net.InetAddress; | ||
import java.text.SimpleDateFormat; | ||
import java.util.Date; | ||
import java.util.HashMap; | ||
import java.util.HashSet; | ||
import java.util.Map; | ||
import java.util.Set; | ||
|
||
@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 deviceProduct of the CEF message."), | ||
@WritesAttribute(attribute = "cef.header.deviceVersion", description = "The deviceVersion of the CEF message."), | ||
@WritesAttribute(attribute = "cef.header.deviceEventClassId", description = "The deviceEventClassId 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 { | ||
|
||
|
||
|
||
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 Set<Relationship> getRelationships() { | ||
final Set<Relationship> relationships = new HashSet<>(); | ||
relationships.add(REL_FAILURE); | ||
relationships.add(REL_SUCCESS); | ||
return relationships; | ||
} | ||
|
||
@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 = null; | ||
|
||
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); | ||
} | ||
|
||
|
||
// 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; | ||
} | ||
|
||
final Map<String, Object> headerFieldMap; | ||
final Map<String, Object> extensionFieldMap; | ||
try { | ||
headerFieldMap = event.getHeader(); | ||
extensionFieldMap = event.getExtension(true); | ||
} catch (CEFHandlingException e) { | ||
getLogger().error("Failed to parse {} as a CEF message due to {}; routing to failure", new Object[] {flowFile, e}); | ||
session.transfer(flowFile, REL_FAILURE); | ||
return; | ||
} | ||
|
||
|
||
final Map<String, String> attributes = new HashMap<>(); | ||
|
||
// Process KVs of the Header field | ||
for (Map.Entry<String, Object> entry : headerFieldMap.entrySet()) { | ||
attributes.put("cef.header."+entry.getKey(), prettyResult(entry.getValue())); | ||
} | ||
|
||
for (Map.Entry<String, Object> entry : extensionFieldMap.entrySet()) { | ||
attributes.put("cef.extension." + entry.getKey(), prettyResult(entry.getValue())); | ||
} | ||
|
||
flowFile = session.putAllAttributes(flowFile, attributes); | ||
session.transfer(flowFile, REL_SUCCESS); | ||
} | ||
|
||
private String prettyResult(Object entryValue) { | ||
|
||
if (entryValue instanceof InetAddress ) { | ||
return ((InetAddress) entryValue).getHostAddress(); | ||
} else if (entryValue instanceof Date) { | ||
// Not to discuss. THere should be no date format other than internationally agreed formats... | ||
final SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ"); | ||
// Discuss this | ||
// formatter.setTimeZone(TimeZone.getTimeZone("UTC")); | ||
return String.valueOf(formatter.format(entryValue)); | ||
} else { | ||
return String.valueOf(entryValue); | ||
} | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
90 changes: 90 additions & 0 deletions
90
...i-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestParseCEF.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,90 @@ | ||
/* | ||
* 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 org.apache.nifi.util.MockFlowFile; | ||
import org.apache.nifi.util.TestRunner; | ||
import org.apache.nifi.util.TestRunners; | ||
import org.junit.Test; | ||
|
||
import java.text.SimpleDateFormat; | ||
import java.util.Date; | ||
import java.util.TimeZone; | ||
|
||
|
||
public class TestParseCEF { | ||
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 " + | ||
// IPv4 | ||
"destinationTranslatedAddress=123.123.123.123 " + | ||
// Date without TZ | ||
"deviceCustomDate1=Feb 08 2015 13:27:43 " + | ||
// Integer and IP Address (from v4) | ||
"dpt=1234 agt=123.123.0.124 dlat=40.366633"; | ||
|
||
@Test | ||
public void testSuccessfulParse() { | ||
final TestRunner runner = TestRunners.newTestRunner(new ParseCEF()); | ||
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", new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ").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.destinationTranslatedAddress", "123.123.123.123"); | ||
|
||
|
||
// Converting a field without timezone will always result on render time being dependent | ||
// on locale of the machine running this test. | ||
long eventTime = 1423402063000L; | ||
int offset = TimeZone.getDefault().getOffset(eventTime); | ||
|
||
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ"); | ||
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 testInvalidMessage() { | ||
final TestRunner runner = TestRunners.newTestRunner(new ParseCEF()); | ||
runner.enqueue("test test test chocolate\n".getBytes()); | ||
runner.run(); | ||
|
||
runner.assertAllFlowFilesTransferred(ParseCEF.REL_FAILURE, 1); | ||
} | ||
|
||
} | ||
|