[
https://issues.apache.org/jira/browse/NIFI-1942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15396586#comment-15396586
]
ASF GitHub Bot commented on NIFI-1942:
--------------------------------------
Github user JPercivall commented on a diff in the pull request:
https://github.com/apache/nifi/pull/476#discussion_r72539247
--- Diff:
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java
---
@@ -0,0 +1,613 @@
+/*
+ * 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 java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.supercsv.cellprocessor.Optional;
+import org.supercsv.cellprocessor.ParseBigDecimal;
+import org.supercsv.cellprocessor.ParseBool;
+import org.supercsv.cellprocessor.ParseChar;
+import org.supercsv.cellprocessor.ParseDate;
+import org.supercsv.cellprocessor.ParseDouble;
+import org.supercsv.cellprocessor.ParseInt;
+import org.supercsv.cellprocessor.ParseLong;
+import org.supercsv.cellprocessor.constraint.DMinMax;
+import org.supercsv.cellprocessor.constraint.Equals;
+import org.supercsv.cellprocessor.constraint.ForbidSubStr;
+import org.supercsv.cellprocessor.constraint.IsIncludedIn;
+import org.supercsv.cellprocessor.constraint.LMinMax;
+import org.supercsv.cellprocessor.constraint.NotNull;
+import org.supercsv.cellprocessor.constraint.RequireHashCode;
+import org.supercsv.cellprocessor.constraint.RequireSubStr;
+import org.supercsv.cellprocessor.constraint.StrMinMax;
+import org.supercsv.cellprocessor.constraint.StrNotNullOrEmpty;
+import org.supercsv.cellprocessor.constraint.StrRegEx;
+import org.supercsv.cellprocessor.constraint.Strlen;
+import org.supercsv.cellprocessor.constraint.Unique;
+import org.supercsv.cellprocessor.constraint.UniqueHashCode;
+import org.supercsv.cellprocessor.ift.CellProcessor;
+import org.supercsv.exception.SuperCsvCellProcessorException;
+import org.supercsv.io.CsvListReader;
+import org.supercsv.prefs.CsvPreference;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"csv", "schema", "validation"})
+@CapabilityDescription("Validates the contents of FlowFiles against a
user-specified CSV schema. " +
+ "Take a look at the additional documentation of this processor for
some schema examples.")
+public class ValidateCsv extends AbstractProcessor {
+
+ private final static List<String> allowedOperators =
Arrays.asList("ParseBigDecimal", "ParseBool", "ParseChar", "ParseDate",
+ "ParseDouble", "ParseInt", "ParseLong", "Optional", "DMinMax",
"Equals", "ForbidSubStr", "LMinMax", "NotNull", "Null",
+ "RequireHashCode", "RequireSubStr", "Strlen", "StrMinMax",
"StrNotNullOrEmpty", "StrRegEx", "Unique",
+ "UniqueHashCode", "IsIncludedIn");
+
+ private static final String routeWholeFlowFile = "Route to 'valid' the
whole FlowFile if the CSV is valid";
+ private static final String routeLinesIndividually = "Route to 'valid'
a FlowFile containing the valid lines and"
+ + " to 'invalid' a FlowFile containing the invalid lines";
+
+ public static final AllowableValue VALIDATE_WHOLE_FLOWFILE = new
AllowableValue(routeWholeFlowFile, routeWholeFlowFile,
+ "In case an error is found in the CSV file, the whole flow
file will be routed to the 'invalid' relationship. "
+ + "This option offers best performances.");
+
+ public static final AllowableValue VALIDATE_LINES_INDIVIDUALLY = new
AllowableValue(routeLinesIndividually, routeLinesIndividually,
+ "In case an error is found, the input CSV file will be split
into two FlowFiles: one routed to the 'valid' "
+ + "relationship containing all the correct lines and
one routed to the 'invalid' relationship containing all "
+ + "the incorrect lines. Take care if choosing this
option while using Unique cell processors in schema definition.");
--- End diff --
I like calling out the Unique processor here but it'd nice to have more
explicit why and what happens if you do use it.
> Create a processor to validate CSV against a user-supplied schema
> -----------------------------------------------------------------
>
> Key: NIFI-1942
> URL: https://issues.apache.org/jira/browse/NIFI-1942
> Project: Apache NiFi
> Issue Type: Improvement
> Components: Extensions
> Reporter: Pierre Villard
> Assignee: Pierre Villard
> Priority: Minor
> Fix For: 1.0.0
>
> Attachments: ValidateCSV.xml
>
>
> In order to extend the set of "quality control" processors, it would be
> interesting to have a processor validating CSV formatted flow files against a
> user-specified schema.
> Flow file validated against schema would be routed to "valid" relationship
> although flow file not validated against schema would be routed to "invalid"
> relationship.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)