exceptionfactory commented on code in PR #6480:
URL: https://github.com/apache/nifi/pull/6480#discussion_r1048583178


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateRecord.java:
##########
@@ -0,0 +1,649 @@
+/*
+ * 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.github.javafaker.Faker;
+import com.github.javafaker.service.files.EnFile;
+import org.apache.avro.Schema;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.avro.AvroSchemaValidator;
+import org.apache.nifi.avro.AvroTypeUtil;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.ChoiceDataType;
+import org.apache.nifi.serialization.record.type.DecimalDataType;
+import org.apache.nifi.serialization.record.type.EnumDataType;
+import org.apache.nifi.serialization.record.type.MapDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.util.StringUtils;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.math.BigInteger;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+@SupportsBatching
+@Tags({"test", "random", "generate", "fake"})
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the 
mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number 
of records in the FlowFile"),
+})
+@CapabilityDescription("This processor creates FlowFiles with records having 
random value for the specified fields. GenerateFakeRecord is useful " +
+        "for testing, configuration, and simulation. It uses either 
user-defined properties to define a record schema or a provided schema and 
generates the specified number of records using " +
+        "random data for the fields in the schema.")
+public class GenerateRecord extends AbstractProcessor {
+
+    // Additional Faker datatypes that don't use predetermined data files 
(i.e. they generate data or have non-String types)
+    static final AllowableValue FT_BOOL = new AllowableValue("Boolean.bool", 
"Boolean - bool (true/false)", "A value of 'true' or 'false'");
+    static final AllowableValue FT_FUTURE_DATE = new 
AllowableValue("DateAndTime.futureDate", "Date And Time - Future Date", 
"Generates a date up to one year in the " +
+            "future from the time the processor is executed");
+    static final AllowableValue FT_PAST_DATE = new 
AllowableValue("DateAndTime.pastDate", "Date And Time - Past Date", "Generates 
a date up to one year in the past from the time the " +
+            "processor is executed");
+    static final AllowableValue FT_BIRTHDAY = new 
AllowableValue("DateAndTime.birthday", "Date And Time - Birthday", "Generates a 
random birthday between 65 and 18 years ago");
+    static final AllowableValue FT_MD5 = new AllowableValue("Crypto.MD5", 
"Crypto - MD5", "An MD5 hash");
+    static final AllowableValue FT_NUMBER = new 
AllowableValue("Number.Integer", "Number - Integer", "A integer number");
+    static final AllowableValue FT_SHA1 = new AllowableValue("Crypto.SHA-1", 
"Crypto - SHA-1", "A SHA-1 hash");
+    static final AllowableValue FT_SHA256 = new 
AllowableValue("Crypto.SHA-256", "Crypto - SHA-256", "A SHA-256 hash");
+    static final AllowableValue FT_SHA512 = new 
AllowableValue("Crypto.SHA-512", "Crypto - SHA-512", "A SHA-512 hash");
+
+    static final String FT_LATITUDE_ALLOWABLE_VALUE_NAME = "Address.latitude";
+    static final String FT_LONGITUDE_ALLOWABLE_VALUE_NAME = 
"Address.longitude";
+
+    static final String[] SUPPORTED_LOCALES = {
+            "bg",
+            "ca",
+            "ca-CAT",
+            "da-DK",
+            "de",
+            "de-AT",
+            "de-CH",
+            "en",
+            "en-AU",
+            "en-au-ocker",
+            "en-BORK",
+            "en-CA",
+            "en-GB",
+            "en-IND",
+            "en-MS",
+            "en-NEP",
+            "en-NG",
+            "en-NZ",
+            "en-PAK",
+            "en-SG",
+            "en-UG",
+            "en-US",
+            "en-ZA",
+            "es",
+            "es-MX",
+            "fa",
+            "fi-FI",
+            "fr",
+            "he",
+            "hu",
+            "in-ID",
+            "it",
+            "ja",
+            "ko",
+            "nb-NO",
+            "nl",
+            "pl",
+            "pt",
+            "pt-BR",
+            "ru",
+            "sk",
+            "sv",
+            "sv-SE",
+            "tr",
+            "uk",
+            "vi",
+            "zh-CN",
+            "zh-TW"
+    };
+
+    private static final String PACKAGE_PREFIX = "com.github.javafaker";
+
+    private volatile Faker faker = new Faker();
+
+    private static final AllowableValue[] fakerDatatypeValues;
+
+    protected static final Map<String, FakerMethodHolder> datatypeFunctionMap 
= new HashMap<>();
+
+    static {
+        final List<EnFile> fakerFiles = EnFile.getFiles();
+        final Map<String, Class<?>> possibleFakerTypeMap = new 
HashMap<>(fakerFiles.size());
+        for (EnFile fakerFile : fakerFiles) {
+            String className = 
normalizeClassName(fakerFile.getFile().substring(0, 
fakerFile.getFile().indexOf('.')));
+            try {
+                possibleFakerTypeMap.put(className, 
Class.forName(PACKAGE_PREFIX + '.' + className));
+            } catch (Exception e) {
+                // Ignore, these are the ones we want to filter out
+            }
+        }
+
+        // Filter on no-arg methods that return a String, these should be the 
methods the user can use to generate data
+        Faker tempFaker = new Faker();
+        List<AllowableValue> fakerDatatypeValueList = new ArrayList<>();
+        for (Map.Entry<String, Class<?>> entry : 
possibleFakerTypeMap.entrySet()) {
+            List<Method> fakerMethods = 
Arrays.stream(entry.getValue().getDeclaredMethods()).filter((method) ->
+                            Modifier.isPublic(method.getModifiers())
+                                    && method.getParameterCount() == 0
+                                    && method.getReturnType() == String.class)
+                    .collect(Collectors.toList());
+            try {
+                final Object methodObject = 
tempFaker.getClass().getDeclaredMethod(normalizeMethodName(entry.getKey())).invoke(tempFaker);
+                for (Method method : fakerMethods) {
+                    final String allowableValueName = 
normalizeClassName(entry.getKey()) + "." + method.getName();
+                    final String allowableValueDisplayName = 
normalizeDisplayName(entry.getKey()) + " - " + 
normalizeDisplayName(method.getName());
+                    datatypeFunctionMap.put(allowableValueName, new 
FakerMethodHolder(allowableValueName, methodObject, method));
+                    fakerDatatypeValueList.add(new 
AllowableValue(allowableValueName, allowableValueDisplayName, 
allowableValueDisplayName));
+                }
+            } catch (Exception e) {
+                // Ignore, this should indicate a Faker method that we're not 
interested in
+            }
+        }
+
+        // Add types manually for those Faker methods that generate data 
rather than getting it from a resource file
+        fakerDatatypeValueList.add(FT_FUTURE_DATE);
+        fakerDatatypeValueList.add(FT_PAST_DATE);
+        fakerDatatypeValueList.add(FT_BIRTHDAY);
+        fakerDatatypeValueList.add(FT_NUMBER);
+        fakerDatatypeValueList.add(FT_MD5);
+        fakerDatatypeValueList.add(FT_SHA1);

Review Comment:
   `MD5` and `SHA-1` are not consider secure for cryptographic operations, and 
the automated builds fail on Java 11 with the following error:
   
   ```
   'Hash Algorithm' validated against 'SHA-1' is invalid because Given value 
not found in allowed set 'SHA3-512, SHA-384, SHA, SHA3-384, SHA-224, 
SHA-512/256, SHA-256, MD2, SHA-512/224, SHA3-256, SHA-512, MD5, SHA3-224'
   ```
   
   These options should be removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to