mirror of https://github.com/apache/nifi.git
NIFI-4098 This closes #2013. Adjust ParseCEF to re-use bean validator in order to improve performance
This commit is contained in:
parent
844dbe4edb
commit
a307ea3a00
|
@ -44,6 +44,14 @@ Apache Software License v2
|
|||
|
||||
The following binary components are provided under the Apache Software License v2
|
||||
|
||||
(ASLv2) Apache BVal Project
|
||||
The following NOTICE information applies:
|
||||
Apache BVal project
|
||||
Copyright 2010-2016 The Apache Software Foundation.
|
||||
|
||||
This product includes software developed by Agimatec GmbH.
|
||||
Copyright 2007-2010 Agimatec GmbH. All rights reserved.
|
||||
|
||||
(ASLv2) Apache Commons IO
|
||||
The following NOTICE information applies:
|
||||
Apache Commons IO
|
||||
|
@ -255,7 +263,7 @@ The following binary components are provided under the Apache Software License v
|
|||
(ASLv2) Apache Commons Lang
|
||||
The following NOTICE information applies:
|
||||
Apache Commons Lang
|
||||
Copyright 2001-2015 The Apache Software Foundation
|
||||
Copyright 2001-2017 The Apache Software Foundation
|
||||
|
||||
This product includes software from the Spring Framework,
|
||||
under the Apache License 2.0 (see: StringUtils.containsWhitespace())
|
||||
|
@ -472,6 +480,12 @@ The following binary components are provided under the Apache Software License v
|
|||
|
||||
IPTC Photo Metadata descriptions Copyright 2010 International Press Telecommunications Council.
|
||||
|
||||
|
||||
(ASLv2) Apache Calcite -- Avatica
|
||||
The following NOTICE information applies:
|
||||
Apache Calcite -- Avatica
|
||||
Copyright 2012-2017 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Apache Calcite
|
||||
The following NOTICE information applies:
|
||||
Apache Calcite
|
||||
|
@ -1432,6 +1446,14 @@ The following binary components are provided under the Apache Software License v
|
|||
Hortonworks Schema Registry
|
||||
Copyright 2016-2017 Hortonworks, Inc.
|
||||
|
||||
(ASLv2) Aggregate Designer
|
||||
The following NOTICE information applies:
|
||||
|
||||
Aggregate Designer
|
||||
|
||||
Copyright 2006 - 2013 Pentaho Corporation. All rights reserved.
|
||||
Copyright 2000-2005, 2014-2016 Julian Hyde
|
||||
|
||||
************************
|
||||
Common Development and Distribution License 1.1
|
||||
************************
|
||||
|
|
|
@ -26,6 +26,14 @@ Apache Software License v2
|
|||
|
||||
The following binary components are provided under the Apache Software License v2
|
||||
|
||||
(ASLv2) Apache BVal Project
|
||||
The following NOTICE information applies:
|
||||
Apache BVal project
|
||||
Copyright 2010-2016 The Apache Software Foundation.
|
||||
|
||||
This product includes software developed by Agimatec GmbH.
|
||||
Copyright 2007-2010 Agimatec GmbH. All rights reserved.
|
||||
|
||||
(ASLv2) Apache Commons IO
|
||||
The following NOTICE information applies:
|
||||
Apache Commons IO
|
||||
|
@ -34,7 +42,7 @@ The following binary components are provided under the Apache Software License v
|
|||
(ASLv2) Apache Commons Lang
|
||||
The following NOTICE information applies:
|
||||
Apache Commons Lang
|
||||
Copyright 2001-2015 The Apache Software Foundation
|
||||
Copyright 2001-2017 The Apache Software Foundation
|
||||
|
||||
This product includes software from the Spring Framework,
|
||||
under the Apache License 2.0 (see: StringUtils.containsWhitespace())
|
||||
|
@ -185,6 +193,11 @@ The following binary components are provided under the Apache Software License v
|
|||
Grok
|
||||
Copyright 2014 Anthony Corbacho, and contributors.
|
||||
|
||||
(ASLv2) Apache Calcite -- Avatica
|
||||
The following NOTICE information applies:
|
||||
Apache Calcite -- Avatica
|
||||
Copyright 2012-2017 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Apache Calcite
|
||||
The following NOTICE information applies:
|
||||
Apache Calcite
|
||||
|
@ -197,6 +210,13 @@ The following binary components are provided under the Apache Software License v
|
|||
|
||||
The web site includes files generated by Jekyll.
|
||||
|
||||
(ASLv2) Aggregate Designer
|
||||
The following NOTICE information applies:
|
||||
|
||||
Aggregate Designer
|
||||
|
||||
Copyright 2006 - 2013 Pentaho Corporation. All rights reserved.
|
||||
Copyright 2000-2005, 2014-2016 Julian Hyde
|
||||
|
||||
************************
|
||||
Common Development and Distribution License 1.1
|
||||
|
|
|
@ -305,7 +305,7 @@
|
|||
<dependency>
|
||||
<groupId>com.fluenda</groupId>
|
||||
<artifactId>ParCEFone</artifactId>
|
||||
<version>1.2.4</version>
|
||||
<version>1.2.5</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.slf4j</groupId>
|
||||
|
@ -313,6 +313,33 @@
|
|||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.bval</groupId>
|
||||
<artifactId>bval-jsr</artifactId>
|
||||
<version>1.1.2</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.tomcat</groupId>
|
||||
<artifactId>tomcat-el-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.geronimo.specs</groupId>
|
||||
<artifactId>geronimo-annotation_1.2_spec</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.geronimo.specs</groupId>
|
||||
<artifactId>geronimo-jcdi_1.1.spec</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.geronimo.specs</groupId>
|
||||
<artifactId>geronimo-jpa_2.0.spec</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.bval</groupId>
|
||||
<artifactId>bval-xstream</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.thekraken</groupId>
|
||||
<artifactId>grok</artifactId>
|
||||
|
|
|
@ -25,12 +25,15 @@ 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.bval.jsr.ApacheValidationProvider;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
|
@ -59,6 +62,8 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
import org.apache.nifi.stream.io.BufferedOutputStream;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
|
||||
import javax.validation.Validation;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
@ -170,6 +175,9 @@ public class ParseCEF extends AbstractProcessor {
|
|||
.description("Any FlowFile that is successfully parsed as a CEF message will be transferred to this Relationship.")
|
||||
.build();
|
||||
|
||||
// Create a Bean validator to be shared by the parser instances.
|
||||
final javax.validation.Validator validator = Validation.byProvider(ApacheValidationProvider.class).configure().buildValidatorFactory().getValidator();;
|
||||
|
||||
@Override
|
||||
public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
final List<PropertyDescriptor>properties = new ArrayList<>();
|
||||
|
@ -219,7 +227,8 @@ public class ParseCEF extends AbstractProcessor {
|
|||
return;
|
||||
}
|
||||
|
||||
final CEFParser parser = new CEFParser();
|
||||
final CEFParser parser = new CEFParser(validator);
|
||||
|
||||
final byte[] buffer = new byte[(int) flowFile.getSize()];
|
||||
session.read(flowFile, new InputStreamCallback() {
|
||||
@Override
|
||||
|
@ -312,6 +321,8 @@ public class ParseCEF extends AbstractProcessor {
|
|||
} 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});
|
||||
// Create a provenance event recording the routing to failure
|
||||
session.getProvenanceReporter().route(flowFile, REL_FAILURE);
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
session.commit();
|
||||
return;
|
||||
|
|
Loading…
Reference in New Issue