Thanks @Ismaël Mejía for your detailed answers. Also thanks for creating
BEAM-11650 to track my feature request. I do understand that reading parquet
files without providing a schema may have some problems in the schema evolution
scenarios, but this could also be very handy in the cases when we just want to
read whatever schema exists in the parquet files.
Regarding withBeamSchemas() method, I did see it from AvroIO, but not from
ParquetIO. It's good to know this handy API to enable schemas transforms though.
Regarding this usage "mycollection.setCoder(AvroUtils.schemaCoder(schema))" to
enable the schemas transforms, thanks for sharing that and that's similar to
what I have been using in my beam app.
Regarding the conversion of PCollection<Row> to PCollection<GenericRecord>,
thanks for sharing the new converter feature from beam 2.28. Below is the code
I have been using for this purpose.
MapElements
.into(new TypeDescriptor<GenericRecord>() {})
.via(AvroUtils.getRowToGenericRecordFunction(avroSchema)))
.setCoder(AvroCoder.of(GenericRecord.class, avroSchema))
On 1/18/21, 3:19 AM, "Ismaël Mejía" <[email protected]> wrote:
Catching up on this thread sorry if late to the party :) and my excuses
because
this is going to be loooong but worth.
> It does look like BEAM-11460 could work for you. Note that relies on a
dynamic
> object which won't work with schema-aware transforms and SqlTransform.
It's
> likely this isn't a problem for you, I just wanted to point it out.
We may be missing in this discussion the existence of the
`withBeamSchemas(true)` method on the IOs that produce Avro objects. This
method
sets up a Schema-based coder for the output of the PCollection generated by
the
read. This allows both SQL and Schema-based transforms just afterwards by
auto-infering the Beam Row schema and auto-transforming everything into Rows
when needed.
PCollection<GenericRecord> input =
p.apply(
ParquetIO.read(SCHEMA)
.from(path)
.withBeamSchemas(true));
Now input can be used by SQL/Schema-based PTransforms.
> @Kobe Feng thank you so much for the insights. Agree that it may be a good
> practice to read all sorts of file formats (e.g. parquet, avro etc) into a
> PCollection<Row> and then perform the schema aware transforms that you are
> referring to.
This is not the case at the moment because most IOs precede the schema-based
APIs, but more and more PTransforms are supporting it. Notice that for
dynamic
objects or Schema-aware PCollection you don't even need them to produce
PCollection<Row>. You can take a PCollection<GenericRecord> (like above) and
connect directly to schema-aware transformations as if it was a
PCollection<Row>
the transformation is done automatically for the user because of the
Schema-based coder.
You can do this manually if you have a non-schema PCollection of
GenericRecords
by setting explicitly a Schema-based coder for the PCollection:
mycollection.setCoder(AvroUtils.schemaCoder(schema));
Beam also includes the schema-based `Convert` transform to convert different
types from/to Rows so this could be handy for cases when you need to
transform
in both directions and it is not supported. Beam 2.28.0 introduces an
improvement that allows to Convert from any Schema-based PCollection (Rows
or
others) into GenericRecords. This is really useful because Avro/Parquet
based
writes expect a PCollection<GenericRecord> not one of rows, and now you can
just
transform a schema-based PCollection (e.g. PCollection<Row> or of other
objects)
into a PCollection<GenericRecord> like this:
myrowcollection.apply(Convert.to(GenericRecord.class)).apply(AnAvroBasedSinkIO.write(...))
https://nam11.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FBEAM-11571&data=04%7C01%7Ctaol%40zillow.com%7C7cc9c01c692c4c00b8b108d8bba2ea2a%7C033464830d1840e7a5883784ac50e16f%7C0%7C0%7C637465655684451869%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=HdRLVZmo0TFw%2BgnwRjHNIfMw1sTTfAIzv3qhz%2BMpES0%3D&reserved=0
So now the full scenario is covered for reads via .withBeamSchemas(true) or
by
setting manually an AvroCoder for schemas and for writes by preceding the
Sinks
with `Convert.to`. That's the beauty of Beam's bidirectional Schema coders.
Note that this probably can be better documented in the programming guide
or in
the javadocs so contributions welcome!
And now back to the initial question:
> Quick question about ParquetIO. Is there a way to avoid specifying the
avro
> schema when reading parquet files?
No, you cannot at the moment. BEAM-11460 allows you to parametrize the
transformation from a GenericRecord (with a schema you expect in advance
even if
you don't specify it) into your own type of objects.
In Parquet/Avro the schema you use to write can differ from the schema you
use
to read, this is done to support schema evolution, so the most general use
case
is to allow users to read from specific versions of the Schema provided into
their objects. That's probably one of the reasons why this is not supported.
Since the Schema is part of the Parquet file metadata I suppose we could
somehow
use it and produce the Schema for the output collection, notice however
that if
the schema differs on the files this will break in runtime.
Filled
https://nam11.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FBEAM-11650&data=04%7C01%7Ctaol%40zillow.com%7C7cc9c01c692c4c00b8b108d8bba2ea2a%7C033464830d1840e7a5883784ac50e16f%7C0%7C0%7C637465655684451869%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=Uim3XsNtFa9ynXWixl1Vmm9gRu6%2F4LwrAAPLIha2oAQ%3D&reserved=0
to track this.
On Wed, Jan 13, 2021 at 7:42 PM Tao Li <[email protected]> wrote:
>
> @Kobe Feng thank you so much for the insights. Agree that it may be a
good practice to read all sorts of file formats (e.g. parquet, avro etc) into a
PCollection<Row> and then perform the schema aware transforms that you are
referring to.
>
>
>
> The new dataframe APIs for Python SDK sound pretty cool and I can imagine
it will save a lot of hassles during a beam app development. Hopefully it will
be added to Java SDK as well.
>
>
>
> From: Kobe Feng <[email protected]>
> Reply-To: "[email protected]" <[email protected]>
> Date: Friday, January 8, 2021 at 11:39 AM
> To: "[email protected]" <[email protected]>
> Subject: Re: Quick question regarding ParquetIO
>
>
>
> Tao,
> I'm not an expert, and good intuition, all you want is schema awareness
transformations or let's say schema based transformation in Beam not only for
IO but also for other DoFn, etc, and possibly have schema revolution in future
as well.
>
>
> This is how I try to understand and explain in other places before: Not
like spark, flink to leverage internal/built-in types (e.g, catalyst struct
type) for built-in operators as more as possible to infer the schema when IOs
could convert to, beam is trying to have capable to handle any type during
transforms for people to migrate existing ones to beam (Do spark map partition
func with own type, Encoder can't be avoided as well, right). Also yes, we
could leverage beam own type "Row" to do all transformations and converting all
in/out types like parquet, avro, orc, etc at IO side, and then do schema
inferring in built-in operators base on row type when we know they will operate
on internal types, that's how to avoid the coder or explicit schema there, more
further, provide IO for schema registry capability and then transform will
lookup when necessary for the revolution. I saw beam put schema base
transformation in goals last year which will be convenient for people (since
normally people would rather use builtin types instead of providing their own
types' coder for following operators until we have to), that's why dataframe
APIs for python SDK here I think.
>
> Kobe
>
>
>
>
> On Fri, Jan 8, 2021 at 9:34 AM Tao Li <[email protected]> wrote:
>
> Thanks Alexey for your explanation. That’s also what I was thinking.
Parquet files already have the schema built in, so it might be feasible to
infer a coder automatically (like spark parquet reader). It would be great if
we have some experts chime in here. @Brian Hulette already mentioned that the
community is working on new DataFrame APIs in Python SDK, which are based on
the pandas methods and use those methods at construction time to determine the
schema. I think this is very close to the schema inference we have been
discussing. Not sure it will be available to Java SDK though.
>
>
>
> Regarding BEAM-11460, looks like it may not totally solve my problem. As
@Alexey Romanenko mentioned, we may still need to know the avro or beam schema
for following operations after the parquet read. A dumb question is, with
BEAM-11460, after we get a PCollection<GenericRecord> from parquet read
(without the need to specify avro schema), is it possible to get the attached
avro schema from a GenericRecord element of this PCollection<GenericRecord>?
>
>
>
> Really appreciate it if you can help clarify my questions. Thanks!
>
>
>
>
>
> From: Alexey Romanenko <[email protected]>
> Reply-To: "[email protected]" <[email protected]>
> Date: Friday, January 8, 2021 at 4:48 AM
> To: "[email protected]" <[email protected]>
> Subject: Re: Quick question regarding ParquetIO
>
>
>
> Well, this is how I see it, let me explain.
>
>
>
> Since every PCollection is required to have a Coder to materialize the
intermediate data, we need to have a coder for "PCollection<GenericRecord>" as
well. If I’m not mistaken, for “GenericRecord" we used to set AvroCoder that is
based on Avro (or Beam too?) schema.
>
>
>
> Actually, currently it will throw an exception if you will try to use
“parseGenericRecords()” with a PCollection<GenericRecord> as output pcollection
since it can’t infer a Coder based on provided “parseFn”. I guess it was done
intentially in this way and I doubt that we can have a proper coder for
PCollection<GenericRecord> without knowing a schema. Maybe some Avro experts
here can add more on this if we can somehow overcome it.
>
>
>
> On 7 Jan 2021, at 19:44, Tao Li <[email protected]> wrote:
>
>
>
> Alexey,
>
>
>
> Why do I need to set AvroCoder? I assume with BEAM-11460 we don’t need to
specify a schema when reading parquet files to get aPCollection<GenericRecord>.
Is my understanding correct? Am I missing anything here?
>
>
>
> Thanks!
>
>
>
> From: Alexey Romanenko <[email protected]>
> Reply-To: "[email protected]" <[email protected]>
> Date: Thursday, January 7, 2021 at 9:56 AM
> To: "[email protected]" <[email protected]>
> Subject: Re: Quick question regarding ParquetIO
>
>
>
> If you want to get just a PCollection<GenericRecord> as output then you
would still need to set AvroCoder, but which schema to use in this case?
>
>
>
> On 6 Jan 2021, at 19:53, Tao Li <[email protected]> wrote:
>
>
>
> Hi Alexey,
>
>
>
> Thank you so much for this info. I will definitely give it a try once
2.28 is released.
>
>
>
> Regarding this feature, it’s basically mimicking the feature from
AvroIO:https://nam11.safelinks.protection.outlook.com/?url=https%3A%2F%2Fbeam.apache.org%2Freleases%2Fjavadoc%2F2.26.0%2Forg%2Fapache%2Fbeam%2Fsdk%2Fio%2FAvroIO.html&data=04%7C01%7Ctaol%40zillow.com%7C7cc9c01c692c4c00b8b108d8bba2ea2a%7C033464830d1840e7a5883784ac50e16f%7C0%7C0%7C637465655684451869%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=Ryw8WY39cDMFDehjxLDLYiIEdsBXZ8Iz8EyLw66eNVM%3D&reserved=0
>
>
>
> I have one more quick question regarding the “reading records of an
unknown schema” scenario. In the sample code a PCollection<Foo> is being
returned and the parseGenericRecords requires a parsing logic. What if I just
want to get a PCollection<GenericRecord> instead of a specific class (e.g. Foo
in the example)? I guess I can just skip the ParquetIO.parseGenericRecords
transform? So do I still have to specify the dummy parsing logic like below?
Thanks!
>
>
>
> p.apply(AvroIO.parseGenericRecords(new
SerializableFunction<GenericRecord, GenericRecord >() {
>
> public Foo apply(GenericRecord record) {
>
> return record;
>
> }
>
>
>
> From: Alexey Romanenko <[email protected]>
> Reply-To: "[email protected]" <[email protected]>
> Date: Wednesday, January 6, 2021 at 10:13 AM
> To: "[email protected]" <[email protected]>
> Subject: Re: Quick question regarding ParquetIO
>
>
>
> Hi Tao,
>
>
>
> This jira [1] looks exactly what you are asking but it was merged
recently (thanks to Anant Damle for working on this!) and it should be
available only in Beam 2.28.0.
>
>
>
> [1]
https://nam11.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FBEAM-11460&data=04%7C01%7Ctaol%40zillow.com%7C7cc9c01c692c4c00b8b108d8bba2ea2a%7C033464830d1840e7a5883784ac50e16f%7C0%7C0%7C637465655684451869%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=mzIG8CoXtcU87jWUZoc3oaU8auNyg7i2e8VdarRV6HY%3D&reserved=0
>
>
>
> Regards,
>
> Alexey
>
>
>
>
> On 6 Jan 2021, at 18:57, Tao Li <[email protected]> wrote:
>
>
>
> Hi beam community,
>
>
>
> Quick question about ParquetIO. Is there a way to avoid specifying the
avro schema when reading parquet files? The reason is that we may not know the
parquet schema until we read the files. In comparison, spark parquet reader
does not require such a schema specification.
>
>
>
> Please advise. Thanks a lot!
>
>
>
>
>
>
> --
>
> Yours Sincerely
> Kobe Feng