You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by MikeThomsen <gi...@git.apache.org> on 2018/06/26 00:55:30 UTC

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

GitHub user MikeThomsen opened a pull request:

    https://github.com/apache/nifi/pull/2813

    NIFI-5084 Added GenerateRecord processor.

    Thank you for submitting a contribution to Apache NiFi.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [ ] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    
    - [ ] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    - [ ] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [ ] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
    - [ ] Have you written or updated unit tests to verify your changes?
    - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
    - [ ] If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly?
    - [ ] If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly?
    - [ ] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
    
    ### For documentation related changes:
    - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
    
    ### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/MikeThomsen/nifi NIFI-5084

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/2813.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2813
    
----
commit 292e91938ce6f460d8735b33159f6b5a90842e82
Author: Mike Thomsen <mi...@...>
Date:   2018-06-23T22:52:27Z

    NIFI-5084 Added GenerateRecord processor.

----


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    Also, the space issue is becoming a serious concern for our binary distributions. So now things that aren't really important for core have to be weighed against that consideration if you want normal users to have an easy time getting to them.


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    @ottobackwards 
    
    1. I wanted to keep it simple. You can't really wire up the writer without going through most of that, and I wanted to have a simple configuration option for just dumping a schema in for less experienced users who just want to get stuff done.
    
    2. I think that would make a good follow on ticket. Some thought would be need to determine how to control the size of a record set. I could see that causing issues for processors.


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199576036
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    --- End diff --
    
    I appreciate that this library has a DSL that is a modified version of an Avro schema definition, as they may need the actual Avro schema defined in the writer as well. Does the writer support "Inherit Record Schema" so it can just get the one generated by the library without having to specify it?
    
    Also I appreciate the flexibility of the DSL to be able to generate data of different types, lengths, patterns, etc. On the downside it appears to be more about generating data in the desired structure rather than generating desired data and putting it into the structure, meaning there is no direct support for generating emails, SSNs, phone numbers, etc. Those would have to be done by regex and/or using provided values.
    
    Also the library generates Avro which we are currently converting in every case, which seems like an unnecessary step. At the least we may want to call getMimeType() on the writer, if it is Avro and we are inheriting the schema (versus defining it explicitly in the writer) we might be able to skip the "conversion" and write directly to the flow file. Not sure how much of that is available via the API, I'm just saying it's a bummer to have to convert the generated records. What kind of throughput are you seeing when it runs at full speed?
    
    Did you vet other Java libraries for data generation? [avro-mocker](https://github.com/speedment/avro-mocker) uses the actual output schema, then the CLI asks questions about the strategy for each field, I wonder if we could leverage that via a separate DSL or user-defined properties. [JFairy](https://github.com/Devskiller/jfairy) is more concerned with semantic datatypes, but does not provide a DSL so we would likely have to do something similar to this library in terms of an Avro-schema-based DSL or something even simpler (if possible and prudent). All the libraries I looked at had similar pros/cons, so if we stick with this one I'm fine with that. Would be nice to have more examples in the additional details though, for email addresses, IPs, phone numbers, etc.



---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199637841
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    --- End diff --
    
    I'll be honest, I didn't deep dive into too many options. So I didn't know about avro-mocker, for instance. Might be worth checking out.
    
    I did check out JFairy and Faker (Java version), and neither of them have particularly good performance. I am kinda thinking about working on that library from Confluent to let it integrate with them optionally.
    
    FWIW, you can use the regexes to generate ip strings, etc. AFAIK, this would do IPv4 IPs:
    
    `[\d]{,3}\.[\d]{,3}\.[\d]{,3}\.[\d]{,3}`


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r200322961
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("An Avro schema to use for generating records.")
    +        .required(false)
    +        .addValidator(Validator.VALID)
    +        .build();
    +
    +    static final PropertyDescriptor LIMIT = new PropertyDescriptor.Builder()
    +        .name("generate-record-limit")
    +        .displayName("Limit")
    +        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +        .description("")
    +        .defaultValue("25")
    +        .required(false)
    +        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor FIXED_SIZE = new PropertyDescriptor.Builder()
    +        .name("generate-record-fixed-size")
    +        .displayName("Fixed Size")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("")
    +        .defaultValue("true")
    +        .allowableValues("true", "false")
    +        .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +        .build();
    +
    +    static final Relationship REL_FAILURE = new Relationship.Builder()
    +        .name("failure")
    +        .description("If there is an error building a record set, the input will go here.")
    +        .build();
    +    static final Relationship REL_SUCCESS = new Relationship.Builder()
    +        .name("success")
    +        .description("Generated results go to this relationship.")
    +        .build();
    +    static final Relationship REL_ORIGINAL = new Relationship.Builder()
    +        .name("original")
    +        .description("If input is provided, the input flowfile gets routed to this relationship.")
    +        .build();
    +
    +    static final List<PropertyDescriptor> PROPS;
    +    static final Set<Relationship> RELS;
    +
    +    static {
    +        List<PropertyDescriptor> _temp = new ArrayList<>();
    +        _temp.add(WRITER);
    +        _temp.add(SCHEMA);
    +        _temp.add(LIMIT);
    +        _temp.add(FIXED_SIZE);
    +
    +        Set<Relationship> _rels = new HashSet<>();
    +        _rels.add(REL_SUCCESS);
    +        _rels.add(REL_FAILURE);
    +        _rels.add(REL_ORIGINAL);
    +
    +        PROPS = Collections.unmodifiableList(_temp);
    +        RELS  = Collections.unmodifiableSet(_rels);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPS;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELS;
    +    }
    +
    +    private volatile RecordSetWriterFactory writerFactory;
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        writerFactory = context.getProperty(WRITER).asControllerService(RecordSetWriterFactory.class);
    +    }
    +
    +    @Override
    +    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
    +        FlowFile input = null;
    +        if (context.hasIncomingConnection()) {
    +            input = session.get();
    +
    +            if (input == null && context.hasNonLoopConnection()) {
    +                return;
    +            }
    +        }
    +
    +        final int limit;
    +        final boolean fixed = context.getProperty(FIXED_SIZE).asBoolean();
    +        if (fixed) {
    +            limit = context.getProperty(LIMIT).evaluateAttributeExpressions(input).asInteger();
    +        } else {
    +            int ceiling = context.getProperty(LIMIT).evaluateAttributeExpressions(input).asInteger();
    +            limit = new Random().nextInt(ceiling);
    --- End diff --
    
    Fixed.


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199562590
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    --- End diff --
    
    Any reason not to allow EL here?


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    Not sure.  But we should just organize a proper effort to get an extension registry in play.  It will take some time but with us not being able to make the download larger we should be properly motivated to sort that out.


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199564451
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("An Avro schema to use for generating records.")
    +        .required(false)
    +        .addValidator(Validator.VALID)
    +        .build();
    +
    +    static final PropertyDescriptor LIMIT = new PropertyDescriptor.Builder()
    +        .name("generate-record-limit")
    +        .displayName("Limit")
    +        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +        .description("")
    +        .defaultValue("25")
    +        .required(false)
    +        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor FIXED_SIZE = new PropertyDescriptor.Builder()
    +        .name("generate-record-fixed-size")
    +        .displayName("Fixed Size")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("")
    --- End diff --
    
    This should be populated with the description of the property, looks like it is used to determine whether the limit is the number of records produced, or (if false) some random number less than or equal to the limit? If so, is it easier for the user to have two properties, or to document a "Number of Records" property with an example of using the `random()` function in Expression Language? 


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199565057
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("An Avro schema to use for generating records.")
    +        .required(false)
    +        .addValidator(Validator.VALID)
    +        .build();
    +
    +    static final PropertyDescriptor LIMIT = new PropertyDescriptor.Builder()
    +        .name("generate-record-limit")
    +        .displayName("Limit")
    +        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +        .description("")
    +        .defaultValue("25")
    +        .required(false)
    +        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor FIXED_SIZE = new PropertyDescriptor.Builder()
    +        .name("generate-record-fixed-size")
    +        .displayName("Fixed Size")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("")
    +        .defaultValue("true")
    +        .allowableValues("true", "false")
    +        .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +        .build();
    +
    +    static final Relationship REL_FAILURE = new Relationship.Builder()
    +        .name("failure")
    +        .description("If there is an error building a record set, the input will go here.")
    +        .build();
    +    static final Relationship REL_SUCCESS = new Relationship.Builder()
    +        .name("success")
    +        .description("Generated results go to this relationship.")
    +        .build();
    +    static final Relationship REL_ORIGINAL = new Relationship.Builder()
    +        .name("original")
    +        .description("If input is provided, the input flowfile gets routed to this relationship.")
    +        .build();
    +
    +    static final List<PropertyDescriptor> PROPS;
    +    static final Set<Relationship> RELS;
    +
    +    static {
    +        List<PropertyDescriptor> _temp = new ArrayList<>();
    +        _temp.add(WRITER);
    +        _temp.add(SCHEMA);
    +        _temp.add(LIMIT);
    +        _temp.add(FIXED_SIZE);
    +
    +        Set<Relationship> _rels = new HashSet<>();
    +        _rels.add(REL_SUCCESS);
    +        _rels.add(REL_FAILURE);
    +        _rels.add(REL_ORIGINAL);
    +
    +        PROPS = Collections.unmodifiableList(_temp);
    +        RELS  = Collections.unmodifiableSet(_rels);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPS;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELS;
    +    }
    +
    +    private volatile RecordSetWriterFactory writerFactory;
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        writerFactory = context.getProperty(WRITER).asControllerService(RecordSetWriterFactory.class);
    +    }
    +
    +    @Override
    +    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
    +        FlowFile input = null;
    +        if (context.hasIncomingConnection()) {
    +            input = session.get();
    +
    +            if (input == null && context.hasNonLoopConnection()) {
    +                return;
    +            }
    +        }
    +
    +        final int limit;
    +        final boolean fixed = context.getProperty(FIXED_SIZE).asBoolean();
    +        if (fixed) {
    +            limit = context.getProperty(LIMIT).evaluateAttributeExpressions(input).asInteger();
    +        } else {
    +            int ceiling = context.getProperty(LIMIT).evaluateAttributeExpressions(input).asInteger();
    +            limit = new Random().nextInt(ceiling);
    --- End diff --
    
    This makes it possible to generate 0-record files and impossible to generate "LIMIT" number of files. Should it be +1?


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    I don't think it's compatible with your use case. What I built, based on how I read the ticket, was a processor that uses a Confluent library to generate random data based on an Avro schema. So maybe I'm missing something, but I think that's different from what you need.


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by zenfenan <gi...@git.apache.org>.
Github user zenfenan commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    Sure @MikeThomsen I'll take a look


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199578541
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("An Avro schema to use for generating records.")
    +        .required(false)
    +        .addValidator(Validator.VALID)
    +        .build();
    +
    +    static final PropertyDescriptor LIMIT = new PropertyDescriptor.Builder()
    +        .name("generate-record-limit")
    +        .displayName("Limit")
    +        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +        .description("")
    +        .defaultValue("25")
    +        .required(false)
    +        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor FIXED_SIZE = new PropertyDescriptor.Builder()
    +        .name("generate-record-fixed-size")
    +        .displayName("Fixed Size")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("")
    +        .defaultValue("true")
    +        .allowableValues("true", "false")
    +        .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +        .build();
    +
    +    static final Relationship REL_FAILURE = new Relationship.Builder()
    +        .name("failure")
    +        .description("If there is an error building a record set, the input will go here.")
    +        .build();
    +    static final Relationship REL_SUCCESS = new Relationship.Builder()
    +        .name("success")
    +        .description("Generated results go to this relationship.")
    +        .build();
    +    static final Relationship REL_ORIGINAL = new Relationship.Builder()
    +        .name("original")
    +        .description("If input is provided, the input flowfile gets routed to this relationship.")
    +        .build();
    +
    +    static final List<PropertyDescriptor> PROPS;
    +    static final Set<Relationship> RELS;
    +
    +    static {
    +        List<PropertyDescriptor> _temp = new ArrayList<>();
    +        _temp.add(WRITER);
    +        _temp.add(SCHEMA);
    +        _temp.add(LIMIT);
    +        _temp.add(FIXED_SIZE);
    +
    +        Set<Relationship> _rels = new HashSet<>();
    +        _rels.add(REL_SUCCESS);
    +        _rels.add(REL_FAILURE);
    +        _rels.add(REL_ORIGINAL);
    +
    +        PROPS = Collections.unmodifiableList(_temp);
    +        RELS  = Collections.unmodifiableSet(_rels);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPS;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELS;
    +    }
    +
    +    private volatile RecordSetWriterFactory writerFactory;
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        writerFactory = context.getProperty(WRITER).asControllerService(RecordSetWriterFactory.class);
    +    }
    +
    +    @Override
    +    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
    +        FlowFile input = null;
    +        if (context.hasIncomingConnection()) {
    +            input = session.get();
    +
    +            if (input == null && context.hasNonLoopConnection()) {
    +                return;
    +            }
    +        }
    +
    +        final int limit;
    +        final boolean fixed = context.getProperty(FIXED_SIZE).asBoolean();
    +        if (fixed) {
    +            limit = context.getProperty(LIMIT).evaluateAttributeExpressions(input).asInteger();
    +        } else {
    +            int ceiling = context.getProperty(LIMIT).evaluateAttributeExpressions(input).asInteger();
    +            limit = new Random().nextInt(ceiling);
    +        }
    +
    +        final Generator generator;
    +        final RecordSchema schema;
    +
    +        try {
    +            if (!context.getProperty(SCHEMA).isSet() && input != null) {
    +                schema = writerFactory.getSchema(input.getAttributes(), null);
    +                String text = schema.getSchemaText().get();
    +                generator = new Generator(text, new Random());
    +            } else if (!context.getProperty(SCHEMA).isSet() && input == null) {
    +                throw new ProcessException("When there is no incoming connection, a avro schema must be set " +
    +                        "in the Schema configuration property for this processor.");
    +            } else {
    +                Schema parsed = new Schema.Parser().parse(context.getProperty(SCHEMA).getValue());
    +                schema = AvroTypeUtil.createSchema(parsed);
    +                generator = new Generator(parsed, new Random());
    +            }
    +
    +            List<Record> records = new ArrayList<>();
    +            for (int x = 0; x < limit; x++) {
    +                GenericData.Record o = (GenericData.Record) generator.generate();
    +                Map y = AvroTypeUtil.convertAvroRecordToMap(o, schema);
    +                records.add(new MapRecord(schema, y));
    +            }
    +
    +            FlowFile out = session.create(input);
    +            out = session.write(out, outputStream -> {
    +                try {
    +                    RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, outputStream);
    +                    writer.beginRecordSet();
    --- End diff --
    
    If you create a `ListRecordSet` from `records`, you can do a single `writer.write()` and the default implementation will do the loop for you.


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by ottobackwards <gi...@git.apache.org>.
Github user ottobackwards commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    @MikeThomsen this is really great.  A couple of questions:
    
    - Why wouldn't you want to have the same schema selection capabilities for both the processor (virtual reader) and the writer?
    - Did you consider creating a Generating Reader Service that could be re-used by this processor and possibly many others at the same time?



---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    NP.


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen closed the pull request at:

    https://github.com/apache/nifi/pull/2813


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r200323522
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("An Avro schema to use for generating records.")
    +        .required(false)
    +        .addValidator(Validator.VALID)
    +        .build();
    +
    +    static final PropertyDescriptor LIMIT = new PropertyDescriptor.Builder()
    +        .name("generate-record-limit")
    +        .displayName("Limit")
    +        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +        .description("")
    +        .defaultValue("25")
    +        .required(false)
    +        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor FIXED_SIZE = new PropertyDescriptor.Builder()
    +        .name("generate-record-fixed-size")
    +        .displayName("Fixed Size")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("")
    --- End diff --
    
    I'd rather keep it simple and let users explicitly control it.


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    Mike - i'll look when able.  In the mean time anything present in the resulting Nar(s) needs to be accounted for in the L&N of that nar.  The nar should probably not be added to the assembly for now since we're space constrained and it is for test purposes and someone could easily add it when needed.  Therefore, no need to update the nifi-assembly/L&N.
    
    thanks


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    @JPercivall 
    
    Here you go:
    
    http://apache-nifi-developer-list.39713.n7.nabble.com/Should-I-withdraw-this-PR-td19475.html
    
    I'm fine with reopening this if you want to take over the code review. Folks have been pretty busy and it was starting to get to stale, so I pulled the trigger to include it in a larger bundle of testing tools for users.


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    Ah yup, totally different lol. I was thinking GenerateFlowFile but with just a writer + dynamic properties to set fields. Sorry for the confusion!


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199637157
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("An Avro schema to use for generating records.")
    +        .required(false)
    +        .addValidator(Validator.VALID)
    +        .build();
    +
    --- End diff --
    
    The library it uses claims to provide that capability baked in, both as a file reference and as an inline JSON list of acceptable values. I should expand the scope of testing for that.


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199567777
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("An Avro schema to use for generating records.")
    +        .required(false)
    +        .addValidator(Validator.VALID)
    +        .build();
    +
    --- End diff --
    
    It would be cool to have a property for "Additional schema resources" or something, so they can refer to files containing words such as in their adjectives-list.json example. Not sure if this is possible as it appears to expect the file locations to be relative to the current working directory or something, but worth a look.


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    @joewitt so here's a thought... do you think it would be feasible to write a deployment process that pushes the extra nars (ex atlas, hive3 and this) to our GitHub repo's "releases" list? Or are there restrictions on space, etc.?


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    @zenfenan can you review?


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    @joewitt can you review the L&N? I added the new stuff, but wasn't sure if the existing dependencies that are covered elsewhere (Ex. Jackson and Bouncy Castle) had to also get added to the L&N here.


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    @MikeThomsen I'm looking for the discussion on the mailing list on why this was decided to not be merged in and I can't find it. I only see the "How would we handle this?" thread[1]. Could you link to it?
    
    [1] http://apache-nifi-developer-list.39713.n7.nabble.com/How-would-we-handle-this-td19151.html


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199636926
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    --- End diff --
    
    I'd rather start small and expand out because there is a lot that can go wrong here.


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    Per the discussion on the mailing list, I am moving the code [here](https://github.com/MikeThomsen/nifi-datageneration-bundle) and closing this PR.


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    As mentioned in the docs, the Confluent lib that I use for this uses extensions to Avro syntax to define the rules.


---

[GitHub] nifi pull request #2813: NIFI-5084 Added GenerateRecord processor.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2813#discussion_r199578085
  
    --- Diff: nifi-nar-bundles/nifi-data-generation-bundle/nifi-data-generation-processors/src/main/java/org/apache/nifi/processors/generation/GenerateRecord.java ---
    @@ -0,0 +1,209 @@
    +/*
    + * 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.generation;
    +
    +import io.confluent.avro.random.generator.Generator;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.avro.AvroTypeUtil;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +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.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.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.Set;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class GenerateRecord extends AbstractProcessor {
    +    static final PropertyDescriptor WRITER = new PropertyDescriptor.Builder()
    +        .name("generate-record-writer")
    +        .displayName("Record Writer")
    +        .identifiesControllerService(RecordSetWriterFactory.class)
    +        .description("The record writer to use for serializing generated records.")
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
    +        .name("generate-record-schema")
    +        .displayName("Schema")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("An Avro schema to use for generating records.")
    +        .required(false)
    +        .addValidator(Validator.VALID)
    +        .build();
    +
    +    static final PropertyDescriptor LIMIT = new PropertyDescriptor.Builder()
    +        .name("generate-record-limit")
    +        .displayName("Limit")
    +        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +        .description("")
    +        .defaultValue("25")
    +        .required(false)
    +        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor FIXED_SIZE = new PropertyDescriptor.Builder()
    +        .name("generate-record-fixed-size")
    +        .displayName("Fixed Size")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .description("")
    +        .defaultValue("true")
    +        .allowableValues("true", "false")
    +        .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +        .build();
    +
    +    static final Relationship REL_FAILURE = new Relationship.Builder()
    +        .name("failure")
    +        .description("If there is an error building a record set, the input will go here.")
    +        .build();
    +    static final Relationship REL_SUCCESS = new Relationship.Builder()
    +        .name("success")
    +        .description("Generated results go to this relationship.")
    +        .build();
    +    static final Relationship REL_ORIGINAL = new Relationship.Builder()
    +        .name("original")
    +        .description("If input is provided, the input flowfile gets routed to this relationship.")
    +        .build();
    +
    +    static final List<PropertyDescriptor> PROPS;
    +    static final Set<Relationship> RELS;
    +
    +    static {
    +        List<PropertyDescriptor> _temp = new ArrayList<>();
    +        _temp.add(WRITER);
    +        _temp.add(SCHEMA);
    +        _temp.add(LIMIT);
    +        _temp.add(FIXED_SIZE);
    +
    +        Set<Relationship> _rels = new HashSet<>();
    +        _rels.add(REL_SUCCESS);
    +        _rels.add(REL_FAILURE);
    +        _rels.add(REL_ORIGINAL);
    +
    +        PROPS = Collections.unmodifiableList(_temp);
    +        RELS  = Collections.unmodifiableSet(_rels);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPS;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELS;
    +    }
    +
    +    private volatile RecordSetWriterFactory writerFactory;
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        writerFactory = context.getProperty(WRITER).asControllerService(RecordSetWriterFactory.class);
    +    }
    +
    +    @Override
    +    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
    +        FlowFile input = null;
    +        if (context.hasIncomingConnection()) {
    +            input = session.get();
    +
    +            if (input == null && context.hasNonLoopConnection()) {
    +                return;
    +            }
    +        }
    +
    +        final int limit;
    +        final boolean fixed = context.getProperty(FIXED_SIZE).asBoolean();
    +        if (fixed) {
    +            limit = context.getProperty(LIMIT).evaluateAttributeExpressions(input).asInteger();
    +        } else {
    +            int ceiling = context.getProperty(LIMIT).evaluateAttributeExpressions(input).asInteger();
    +            limit = new Random().nextInt(ceiling);
    +        }
    +
    +        final Generator generator;
    +        final RecordSchema schema;
    +
    +        try {
    +            if (!context.getProperty(SCHEMA).isSet() && input != null) {
    +                schema = writerFactory.getSchema(input.getAttributes(), null);
    +                String text = schema.getSchemaText().get();
    +                generator = new Generator(text, new Random());
    +            } else if (!context.getProperty(SCHEMA).isSet() && input == null) {
    +                throw new ProcessException("When there is no incoming connection, a avro schema must be set " +
    +                        "in the Schema configuration property for this processor.");
    +            } else {
    +                Schema parsed = new Schema.Parser().parse(context.getProperty(SCHEMA).getValue());
    +                schema = AvroTypeUtil.createSchema(parsed);
    +                generator = new Generator(parsed, new Random());
    +            }
    +
    +            List<Record> records = new ArrayList<>();
    +            for (int x = 0; x < limit; x++) {
    +                GenericData.Record o = (GenericData.Record) generator.generate();
    +                Map y = AvroTypeUtil.convertAvroRecordToMap(o, schema);
    +                records.add(new MapRecord(schema, y));
    +            }
    +
    +            FlowFile out = session.create(input);
    +            out = session.write(out, outputStream -> {
    +                try {
    +                    RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, outputStream);
    +                    writer.beginRecordSet();
    +                    for (int x = 0; x < records.size(); x++) {
    +                        writer.write(records.get(x));
    +                    }
    +                    writer.finishRecordSet();
    --- End diff --
    
    As a record-based processor, the WriteResult returned from `writer.finishRecordSet()` should be used to store an attribute called "record.count", and `writer.getMimeType()` should be used to set the "mime.type" attribute (and these should be added as WritesAttribute annotations at the processor class definition at the top).


---

[GitHub] nifi issue #2813: NIFI-5084 Added GenerateRecord processor.

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the issue:

    https://github.com/apache/nifi/pull/2813
  
    Is there a reason this processor wasn't just a part of the normal record bundle instead of being thought of as part of a larger reprocessing bundle? If it's just another file in that same nar, it wouldn't be a lot of space.
    
    For reference, I have the use-case where I have attributes on a FF and the content is a binary file. After putting the file to disk, I want to create a nested json object to continue processing. I essentially want to create a new record just from EL (same as UpdateRecord but with no reader). I believe GenerateRecord would fill this need?


---