I created a jira issue for what you're asking in case you want to follow it
or contribute: https://issues.apache.org/jira/browse/BEAM-4812.

Best regards,
Łukasz Gajowy

wt., 17 lip 2018 o 19:24 Anton Kedin <[email protected]> napisał(a):

> I think Beam schemas will solve what you describe, eventually. At this
> moment though the schemas support in Beam is still in the early stages and
> is under active development, so things like automatic schema conversions
> between Beam/Avro/Pojos are not fully there yet. You can watch dev@ list
> for the announcements/discussions of these features.
>
> If in the mean time you end up building your own utilities for this,
> please send a pull request, contributions like this will help with the
> effort of building better schemas support.
>
> Regards,
> Anton
>
> On Tue, Jul 17, 2018 at 6:38 AM Akanksha Sharma B <
> [email protected]> wrote:
>
>> Hi,
>>
>>
>> Reading/Writing to various sources/sinks is very easy in Spark. It will
>> be the best if Beam can provide something similar.
>>
>> If not possible, then it will also help if you eliminate avro schema - by
>> using Beam schema instead or just accepting the typename (and creating
>> POJOs instead of GenericRecords). It will be so good if we can avoid seeing
>> intermediate step of dealing with avro while reading/writing to Parquet.
>>
>>
>> Regards,
>>
>> Akanksha
>> ------------------------------
>> *From:* Łukasz Gajowy <[email protected]>
>> *Sent:* Tuesday, July 17, 2018 2:29:22 PM
>> *To:* [email protected]
>> *Subject:* Re: Schema class in 2.5 ?
>>
>> I think what you're asking should be doable but requires modifications in
>> the ParquetIO code. It uses schema in 2 places:
>>  - read: to setCoder on the PCollection [1]. As long as there already is
>> a way to set the coder in a different way that does not require the
>> Avro Schema we're good to go there (at the time of developing ParquetIO I
>> don't think there was). From the doc mentioned above, I suspect that
>> SchemaCoder may be the best fit for that.
>>  - write: avro schema is used by AvroParquetWriter.builder() it
>> explicitly requires the Avro schema. [2] I think we could accept the Beam's
>> schema as long as there's a way to transform it to Avro Schema. I think
>> it's doable but we would need (for example) to transform Beam's schema to
>> json and then pass it to  Avro's new Schema.Parser().parse() method to get
>> Avro's schema for the builder.
>>
>> [1]
>> https://github.com/apache/beam/blob/641678d242fc76229329a2ac5e61072faf93aceb/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java#L195
>> [2]
>> https://github.com/apache/beam/blob/641678d242fc76229329a2ac5e61072faf93aceb/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java#L285
>>
>> Best regards,
>> Łukasz
>>
>> wt., 17 lip 2018 o 09:52 Akanksha Sharma B <
>> [email protected]> napisał(a):
>>
>> Hi,
>>
>>
>> ParquetIO needs avro Schema(org.apache.avro.Schema) to read and write.
>>
>> Will it also be possible not to use any avro Schema at all or use Beams
>> Schema (org.apache.beam.sdk.schemas.Schema)
>>
>> Regards,
>> Akanksha
>>
>> ------------------------------
>> *From:* Akanksha Sharma B
>> *Sent:* Thursday, July 12, 2018 1:13:14 PM
>> *To:* [email protected]
>> *Subject:* Re: Schema class in 2.5 ?
>>
>>
>> Hi,
>>
>>
>> Created https://issues.apache.org/jira/browse/BEAM-4771
>>
>>
>> Regards,
>>
>> Akanksha
>> ------------------------------
>> *From:* Alexey Romanenko <[email protected]>
>> *Sent:* Thursday, July 12, 2018 12:31:02 PM
>> *To:* [email protected]
>> *Subject:* Re: Schema class in 2.5 ?
>>
>> Good catch, Akanksha!
>> Yes, RowType was renamed to Schema a while ago and BeamSQL doc seems was
>> not updated.
>> Could you create a Jira issue for that?
>>
>> On 12 Jul 2018, at 11:10, Akanksha Sharma B <
>> [email protected]> wrote:
>>
>> Hi,
>>
>> As I see, in 2.5 BeamSQL had been changed to work with Schema.
>> The sample code provided in
>> https://beam.apache.org/documentation/dsls/sql/walkthrough/ does not
>> compile with Beam 2.5, and needs to be updated.
>>
>>  Row
>>                   .withRowType(appType)
>>
>> The above mentioned line needs to be adapted to use schema.
>> Beam SQL: Walkthrough
>> <https://beam.apache.org/documentation/dsls/sql/walkthrough/>
>> beam.apache.org
>> Apache Beam is an open source, unified model and set of language-specific
>> SDKs for defining and executing data processing workflows, and also data
>> ingestion and integration flows, supporting Enterprise Integration Patterns
>> (EIPs) and Domain Specific Languages (DSLs).
>> Regards,
>>
>> Akanksha
>> ------------------------------
>> *From:* Akanksha Sharma B
>> *Sent:* Wednesday, July 11, 2018 11:02:37 AM
>> *To:* [email protected]
>> *Subject:* Re: Schema class in 2.5 ?
>>
>> Thanks a lot!!!
>> ------------------------------
>> *From:* Alexey Romanenko <[email protected]>
>> *Sent:* Wednesday, July 11, 2018 11:01:05 AM
>> *To:* [email protected]
>> *Subject:* Re: Schema class in 2.5 ?
>>
>> Hi Akanksha,
>>
>> I believe this design document can be helpful for you:
>>
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc
>>
>> On 11 Jul 2018, at 10:38, Akanksha Sharma B <
>> [email protected]> wrote:
>>
>> Hi,
>>
>> Can you please share some documentation about ongoing changes related to
>> Schema class.
>> I am looking to understand why is it being introduced and how can I use
>> it.
>> I was looking for something like RDD in Beam, i.e. Beam understands
>> schema of data internally and thus can handle some conversions itself, e.g.
>> to SqlRow, ParquetFile etc.
>>
>> Regards,
>> Akanksha
>>
>>
>>

Reply via email to