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


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/faker/FakerUtils.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.faker;
+
+import com.github.javafaker.Faker;
+import com.github.javafaker.service.files.EnFile;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class FakerUtils {
+
+    private static final int RANDOM_DATE_DAYS = 365;
+
+    public static final String DEFAULT_DATE_PROPERTY_NAME = 
"DateAndTime.pastDate";
+
+
+    protected static final Map<String, FakerMethodHolder> datatypeFunctionMap 
= new HashMap<>();
+
+    // 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(DEFAULT_DATE_PROPERTY_NAME, "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_NUMBER = new 
AllowableValue("Number.Integer", "Number - Integer", "A integer number");
+    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");
+
+    private static final String PACKAGE_PREFIX = "com.github.javafaker";
+
+    public static AllowableValue[] createFakerPropertyList() {
+        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();

Review Comment:
   This can be renamed:
   ```suggestion
           Faker faker = new Faker();
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/faker/FakerUtils.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.faker;
+
+import com.github.javafaker.Faker;
+import com.github.javafaker.service.files.EnFile;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class FakerUtils {
+
+    private static final int RANDOM_DATE_DAYS = 365;
+
+    public static final String DEFAULT_DATE_PROPERTY_NAME = 
"DateAndTime.pastDate";
+
+
+    protected static final Map<String, FakerMethodHolder> datatypeFunctionMap 
= new HashMap<>();
+
+    // 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(DEFAULT_DATE_PROPERTY_NAME, "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_NUMBER = new 
AllowableValue("Number.Integer", "Number - Integer", "A integer number");
+    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");
+
+    private static final String PACKAGE_PREFIX = "com.github.javafaker";
+
+    public static AllowableValue[] createFakerPropertyList() {
+        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_SHA256);
+        fakerDatatypeValueList.add(FT_SHA512);
+        return fakerDatatypeValueList.toArray(new AllowableValue[]{});
+    }
+
+    // This method identifies "segments" by splitting the given name on 
underscores, then capitalizes each segment and removes the underscores. Ex: 
'game_of_thrones' = 'GameOfThrones'
+    private static String normalizeClassName(String name) {
+        String[] segments = name.split("_");
+        String newName = Arrays.stream(segments).map((s) -> s.substring(0, 
1).toUpperCase() + s.substring(1)).collect(Collectors.joining());

Review Comment:
   Recommend reformatting for clarity.
   ```suggestion
           String newName = Arrays.stream(segments)
               .map(s -> s.substring(0, 1).toUpperCase() + s.substring(1))
               .collect(Collectors.joining());
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateRecord.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * 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 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.processors.standard.faker.FakerUtils;
+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.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.math.BigInteger;
+import java.time.LocalDate;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static 
org.apache.nifi.processors.standard.faker.FakerUtils.DEFAULT_DATE_PROPERTY_NAME;
+
+@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. GenerateRecord 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 {
+
+    private static final AllowableValue[] fakerDatatypeValues = 
FakerUtils.createFakerPropertyList();
+
+    // Fake keys when generating a map
+    private static final String KEY1 = "key1";
+    private static final String KEY2 = "key2";
+    private static final String KEY3 = "key3";
+    private static final String KEY4 = "key4";
+
+
+    static final PropertyDescriptor SCHEMA_TEXT = new 
PropertyDescriptor.Builder()
+            .name("schema-text")
+            .displayName("Schema Text")
+            .description("The text of an Avro-formatted Schema used to 
generate record data. If this property is set, any user-defined properties are 
ignored.")
+            .addValidator(new AvroSchemaValidator())
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false)
+            .build();
+    static final PropertyDescriptor RECORD_WRITER = new 
PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing 
out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor NUM_RECORDS = new 
PropertyDescriptor.Builder()
+            .name("-num-records")
+            .displayName("Number of Records")
+            .description("Specifies how many records will be generated for 
each outgoing FlowFile.")
+            .required(true)
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("100")
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor NULLABLE_FIELDS = new 
PropertyDescriptor.Builder()
+            .name("nullable-fields")
+            .displayName("Nullable Fields")
+            .description("Whether the generated fields will be nullable. Note 
that this property is ignored if Schema Text is set. Also it only affects the 
schema of the generated data, " +
+                    "not whether any values will be null. If this property is 
true, see 'Null Value Percentage' to set the probability that any generated 
field will be null.")
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+    static final PropertyDescriptor NULL_PERCENTAGE = new 
PropertyDescriptor.Builder()
+            .name("null-pct")
+            .displayName("Null Value Percentage")
+            .description("The percent probability (0-100%) that a generated 
value for any nullable field will be null. Set this property to zero to have no 
null values, or 100 to have all " +
+                    "null values.")
+            .addValidator(StandardValidators.createLongValidator(0L, 100L, 
true))
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .defaultValue("0")
+            .dependsOn(NULLABLE_FIELDS, "true")
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully created will be 
routed to this relationship")
+            .build();
+
+    static final Set<Relationship> RELATIONSHIPS = 
Collections.singleton(REL_SUCCESS);
+
+    private volatile Faker faker = new Faker();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(SCHEMA_TEXT);
+        properties.add(RECORD_WRITER);
+        properties.add(NUM_RECORDS);
+        properties.add(NULLABLE_FIELDS);
+        properties.add(NULL_PERCENTAGE);

Review Comment:
   ```suggestion
           properties.add(RECORD_WRITER);
           properties.add(NUM_RECORDS);
           properties.add(NULLABLE_FIELDS);
           properties.add(NULL_PERCENTAGE);
           properties.add(SCHEMA_TEXT);
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateRecord.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * 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 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.processors.standard.faker.FakerUtils;
+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.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.math.BigInteger;
+import java.time.LocalDate;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static 
org.apache.nifi.processors.standard.faker.FakerUtils.DEFAULT_DATE_PROPERTY_NAME;
+
+@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. GenerateRecord 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 {
+
+    private static final AllowableValue[] fakerDatatypeValues = 
FakerUtils.createFakerPropertyList();
+
+    // Fake keys when generating a map
+    private static final String KEY1 = "key1";
+    private static final String KEY2 = "key2";
+    private static final String KEY3 = "key3";
+    private static final String KEY4 = "key4";
+
+
+    static final PropertyDescriptor SCHEMA_TEXT = new 
PropertyDescriptor.Builder()
+            .name("schema-text")
+            .displayName("Schema Text")
+            .description("The text of an Avro-formatted Schema used to 
generate record data. If this property is set, any user-defined properties are 
ignored.")
+            .addValidator(new AvroSchemaValidator())
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false)
+            .build();
+    static final PropertyDescriptor RECORD_WRITER = new 
PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing 
out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor NUM_RECORDS = new 
PropertyDescriptor.Builder()
+            .name("-num-records")

Review Comment:
   ```suggestion
               .name("number-of-records")
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/faker/FakerUtils.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.faker;
+
+import com.github.javafaker.Faker;
+import com.github.javafaker.service.files.EnFile;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class FakerUtils {
+
+    private static final int RANDOM_DATE_DAYS = 365;
+
+    public static final String DEFAULT_DATE_PROPERTY_NAME = 
"DateAndTime.pastDate";
+
+
+    protected static final Map<String, FakerMethodHolder> datatypeFunctionMap 
= new HashMap<>();
+
+    // 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(DEFAULT_DATE_PROPERTY_NAME, "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_NUMBER = new 
AllowableValue("Number.Integer", "Number - Integer", "A integer number");
+    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");
+
+    private static final String PACKAGE_PREFIX = "com.github.javafaker";
+
+    public static AllowableValue[] createFakerPropertyList() {
+        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_SHA256);
+        fakerDatatypeValueList.add(FT_SHA512);
+        return fakerDatatypeValueList.toArray(new AllowableValue[]{});
+    }
+
+    // This method identifies "segments" by splitting the given name on 
underscores, then capitalizes each segment and removes the underscores. Ex: 
'game_of_thrones' = 'GameOfThrones'
+    private static String normalizeClassName(String name) {
+        String[] segments = name.split("_");
+        String newName = Arrays.stream(segments).map((s) -> s.substring(0, 
1).toUpperCase() + s.substring(1)).collect(Collectors.joining());
+        return newName;
+    }
+
+    // This method lowercases the first letter of the given name in order to 
match the name to a Faker method
+    private static String normalizeMethodName(String name) {
+
+        String newName = name.substring(0, 1).toLowerCase() + 
name.substring(1);
+        return newName;
+    }
+
+    // This method splits the given name on uppercase letters, ensures the 
first letter is capitalized, then joins the segments using a space. Ex. 
'gameOfThrones' = 'Game Of Thrones'
+    private static String normalizeDisplayName(String name) {
+        // Split when the next letter is uppercase
+        String[] upperCaseSegments = name.split("(?=\\p{Upper})");
+
+        return Arrays.stream(upperCaseSegments).map(
+                (upperCaseSegment) -> upperCaseSegment.substring(0, 
1).toUpperCase() + upperCaseSegment.substring(1)).collect(Collectors.joining(" 
"));

Review Comment:
   ```suggestion
                   upperCaseSegment -> upperCaseSegment.substring(0, 
1).toUpperCase() + upperCaseSegment.substring(1))
                   .collect(Collectors.joining(" "));
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateRecord.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * 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 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.processors.standard.faker.FakerUtils;
+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.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.math.BigInteger;
+import java.time.LocalDate;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static 
org.apache.nifi.processors.standard.faker.FakerUtils.DEFAULT_DATE_PROPERTY_NAME;
+
+@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. GenerateRecord 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 {
+
+    private static final AllowableValue[] fakerDatatypeValues = 
FakerUtils.createFakerPropertyList();
+
+    // Fake keys when generating a map
+    private static final String KEY1 = "key1";
+    private static final String KEY2 = "key2";
+    private static final String KEY3 = "key3";
+    private static final String KEY4 = "key4";
+
+
+    static final PropertyDescriptor SCHEMA_TEXT = new 
PropertyDescriptor.Builder()
+            .name("schema-text")
+            .displayName("Schema Text")
+            .description("The text of an Avro-formatted Schema used to 
generate record data. If this property is set, any user-defined properties are 
ignored.")
+            .addValidator(new AvroSchemaValidator())
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false)
+            .build();

Review Comment:
   Recommend moving this optional property after the required properties.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/faker/FakerUtils.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.faker;
+
+import com.github.javafaker.Faker;
+import com.github.javafaker.service.files.EnFile;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class FakerUtils {
+
+    private static final int RANDOM_DATE_DAYS = 365;
+
+    public static final String DEFAULT_DATE_PROPERTY_NAME = 
"DateAndTime.pastDate";
+
+
+    protected static final Map<String, FakerMethodHolder> datatypeFunctionMap 
= new HashMap<>();

Review Comment:
   Recommend reordering according to visibility and using `LinkedHashMap`:
   ```suggestion
       public static final String DEFAULT_DATE_PROPERTY_NAME = 
"DateAndTime.pastDate";
   
       private static final int RANDOM_DATE_DAYS = 365;
   
       private static final Map<String, FakerMethodHolder> datatypeFunctionMap 
= new LinkedHashMap<>();
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/faker/FakerUtils.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.faker;
+
+import com.github.javafaker.Faker;
+import com.github.javafaker.service.files.EnFile;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class FakerUtils {
+
+    private static final int RANDOM_DATE_DAYS = 365;
+
+    public static final String DEFAULT_DATE_PROPERTY_NAME = 
"DateAndTime.pastDate";
+
+
+    protected static final Map<String, FakerMethodHolder> datatypeFunctionMap 
= new HashMap<>();
+
+    // 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(DEFAULT_DATE_PROPERTY_NAME, "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_NUMBER = new 
AllowableValue("Number.Integer", "Number - Integer", "A integer number");
+    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");
+
+    private static final String PACKAGE_PREFIX = "com.github.javafaker";
+
+    public static AllowableValue[] createFakerPropertyList() {
+        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<>();

Review Comment:
   ```suggestion
           List<AllowableValue> supportedDataTypes = new ArrayList<>();
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/faker/FakerUtils.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.faker;
+
+import com.github.javafaker.Faker;
+import com.github.javafaker.service.files.EnFile;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class FakerUtils {
+
+    private static final int RANDOM_DATE_DAYS = 365;
+
+    public static final String DEFAULT_DATE_PROPERTY_NAME = 
"DateAndTime.pastDate";
+
+
+    protected static final Map<String, FakerMethodHolder> datatypeFunctionMap 
= new HashMap<>();
+
+    // 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(DEFAULT_DATE_PROPERTY_NAME, "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_NUMBER = new 
AllowableValue("Number.Integer", "Number - Integer", "A integer number");
+    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");
+
+    private static final String PACKAGE_PREFIX = "com.github.javafaker";
+
+    public static AllowableValue[] createFakerPropertyList() {
+        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_SHA256);
+        fakerDatatypeValueList.add(FT_SHA512);
+        return fakerDatatypeValueList.toArray(new AllowableValue[]{});
+    }
+
+    // This method identifies "segments" by splitting the given name on 
underscores, then capitalizes each segment and removes the underscores. Ex: 
'game_of_thrones' = 'GameOfThrones'
+    private static String normalizeClassName(String name) {
+        String[] segments = name.split("_");
+        String newName = Arrays.stream(segments).map((s) -> s.substring(0, 
1).toUpperCase() + s.substring(1)).collect(Collectors.joining());
+        return newName;
+    }
+
+    // This method lowercases the first letter of the given name in order to 
match the name to a Faker method
+    private static String normalizeMethodName(String name) {
+
+        String newName = name.substring(0, 1).toLowerCase() + 
name.substring(1);
+        return newName;
+    }
+
+    // This method splits the given name on uppercase letters, ensures the 
first letter is capitalized, then joins the segments using a space. Ex. 
'gameOfThrones' = 'Game Of Thrones'
+    private static String normalizeDisplayName(String name) {
+        // Split when the next letter is uppercase
+        String[] upperCaseSegments = name.split("(?=\\p{Upper})");
+
+        return Arrays.stream(upperCaseSegments).map(
+                (upperCaseSegment) -> upperCaseSegment.substring(0, 
1).toUpperCase() + upperCaseSegment.substring(1)).collect(Collectors.joining(" 
"));
+    }

Review Comment:
   These private methods should be moved after all public methods.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/faker/FakerUtils.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.faker;
+
+import com.github.javafaker.Faker;
+import com.github.javafaker.service.files.EnFile;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class FakerUtils {
+
+    private static final int RANDOM_DATE_DAYS = 365;
+
+    public static final String DEFAULT_DATE_PROPERTY_NAME = 
"DateAndTime.pastDate";
+
+
+    protected static final Map<String, FakerMethodHolder> datatypeFunctionMap 
= new HashMap<>();
+
+    // 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(DEFAULT_DATE_PROPERTY_NAME, "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_NUMBER = new 
AllowableValue("Number.Integer", "Number - Integer", "A integer number");
+    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");
+
+    private static final String PACKAGE_PREFIX = "com.github.javafaker";
+
+    public static AllowableValue[] createFakerPropertyList() {
+        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_SHA256);
+        fakerDatatypeValueList.add(FT_SHA512);
+        return fakerDatatypeValueList.toArray(new AllowableValue[]{});

Review Comment:
   In light of the large number of values, it would be very helpful to sort the 
final list using the display name.



-- 
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