On the intermediate representation: not necessarily needed, and likely a
performance hindrance to do so. Consider converting from a CSV to a flat
json object. This can be done by streaming through the values, and likely
only needing a single input character in memory at a time.
On Mar 22, 2016 11:07 PM, "Dmitry Goldenberg" <dgoldenberg...@gmail.com>
wrote:

> It seems to me that for starters it's great to have the processors which
> convert from various input formats to FlowFile, and from FlowFile to
> various output formats.  That covers all the cases and it gives the users a
> chance to run some extra processors in between which is often handy, and
> sometimes necessary.
>
> ConvertFormat sounds cool but I'd agree that it may grow to be "hairy"
> with the number of conversions, each with its own set of configuration
> options.  From that perspective, might it be easier to deal with 2 * N
> specific converters, and keep adding them as needed, rather than try to
> maintain a large "Swiss knife"?
>
> Would ConvertFormat really be able to avoid having to use some kind of
> intermediary in-memory format as the conversion is going on?  If not, why
> not let this intermediary format be FlowFile, and if it is FlowFile, then
> why not just roll with the ConvertFrom / ConvertTo processors?  That way,
> implementing a direct converter is simply a matter of dropping the two
> converters next to each other into your dataflow (plus a few in-between
> transformations, if necessary).
>
> Furthermore, a combination of a ConvertFrom and a subsequent ConvertTo
> could be saved as a sub-template for reuse, left as an exercise for the
> user, driven by the user's specific use-cases.
>
> I just wrote a Dataflow which converts some input XML to Avro, and I
> suspect that making such a converter work through a common ConvertFormat
> would take quite a few options.  Between the start and the finish, I ended
> up with: SplitXml, EvaluateXPath, UpdateAttributes, AttributesToJSON,
> ConvertJSONToAvro, MergeContent (after that I have a SetAvroFileExtension
> and WriteToHdfs).  Too many options to expose for the XMl-to-Avro use-case,
> IMHO, for the common ConvertFormat, even if perhaps my Dataflow can be
> optimized to avoid a step or two.
>
> Regards,
> - Dmitry
>
>
>
> On Tue, Mar 22, 2016 at 10:25 PM, Matt Burgess <mattyb...@gmail.com>
> wrote:
>
>> I am +1 for the ConvertFormat processor, the  user experience is so much
>> enhanced by the hands-off conversion. Such a capability might be contingent
>> on the "dependent properties" concept (in Jira somewhere).
>>
>> Also this guy could get pretty big in terms of footprint, I'd imagine the
>> forthcoming Registry might be a good place for it.
>>
>> In general a format translator would probably make for a great Apache
>> project :) Martin Fowler has blogged about some ideas like this (w.r.t.
>> abstracting translation logic), Tika has done some of this but AFAIK its
>> focus is on extraction not transformation. In any case, we could certainly
>> capture the idea in NiFi.
>>
>> Regards,
>> Matt
>>
>> On Mar 22, 2016, at 9:52 PM, Edmon Begoli <ebeg...@gmail.com> wrote:
>>
>> Good point.
>>
>> I just think that Parquet and ORC are important targets, just as
>> relational/JDBC stores are.
>>
>> On Tuesday, March 22, 2016, Tony Kurc <trk...@gmail.com> wrote:
>>
>>> Interesting question. A couple discussion points: If we start doing a
>>> processor for each of these conversions, it may become unwieldy (P(x,2)
>>> processors, where x is number of data formats?) I'd say maybe a more
>>> general ConvertFormat processor may be appropriate, but then configuration
>>> and code complexity may suffer. If there is a canonical internal data form
>>> and a bunch (2*x) of convertXtocanonical, and convertcanonicaltoX
>>> processors, the flow could get complex and the extra transform could be
>>> expensive.
>>> On Mar 21, 2016 9:39 PM, "Dmitry Goldenberg" <dgoldenberg...@gmail.com>
>>> wrote:
>>>
>>>> Since NiFi has ConvertJsonToAvro and ConvertCsvToAvro processors, would
>>>> it make sense to add a feature request for a ConvertJsonToParquet processor
>>>> and a ConvertCsvToParquet processor?
>>>>
>>>> - Dmitry
>>>>
>>>> On Mon, Mar 21, 2016 at 9:23 PM, Matt Burgess <mattyb...@gmail.com>
>>>> wrote:
>>>>
>>>>> Edmon,
>>>>>
>>>>> NIFI-1663 [1] was created to add ORC support to NiFi. If you have a
>>>>> target dataset that has been created with Parquet format, I think you can
>>>>> use ConvertCSVtoAvro then StoreInKiteDataset to get flow files in Parquet
>>>>> format into Hive, HDFS, etc. Others in the community know a lot more about
>>>>> the StoreInKiteDataset processor than I do.
>>>>>
>>>>> Regards,
>>>>> Matt
>>>>>
>>>>> [1] https://issues.apache.org/jira/browse/NIFI-1663
>>>>>
>>>>> On Mon, Mar 21, 2016 at 8:25 PM, Edmon Begoli <ebeg...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>>
>>>>>> Is there a way to do straight CSV(PSV) to Parquet or ORC conversion
>>>>>> via Nifi, or do I always need to push the data through some of the
>>>>>> "data engines" - Drill, Spark, Hive, etc.?
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>

Reply via email to