s,
Uwe
Gesendet: Mittwoch, 23. März 2016 um 03:25 Uhr
Von: "Matt Burgess"
An: users@nifi.apache.org
Betreff: Re: CSV/delimited to Parquet conversion via Nifi
I am +1 for the ConvertFormat processor, the user experience is so much enhanced by the hands-off conversion. Such a
Agreed, but probably not the case with XML to Avro. Perhaps ConvertFormat would
be for a set of the more straightforward conversions.
> On Mar 22, 2016, at 11:30 PM, Tony Kurc wrote:
>
> On the intermediate representation: not necessarily needed, and likely a
> performance hindrance to do so.
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:0
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
sometime
I actually can’t agree more. . .
IMHO and especially the recent Spring integration effort it kind of brings an
idea where a Processor may have it’s own context-based extension mechanism. For
example, here the context is ‘transformation':
public interface Transformer {
O transform(I value);
}
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 R
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 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)
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 comp
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 wrote:
> Edmon,
>
> NIFI-1663 [1] was created to add OR
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
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.?
11 matches
Mail list logo