You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@beam.apache.org by Joshua Bassett <he...@joshbassett.info> on 2020/04/17 10:43:45 UTC
Reading BigQuery table containing a repeated field into POJOs
Hi there
I'm trying to read rows from a BigQuery table that contains a repeated field into POJOs. Unfortunately, I'm running into issues and I can't figure it out.
I have something like this:
@DefaultSchema(JavaFieldSchema.class)
class Article implements Serializable {
public Long id;
public String title;
@SchemaFieldName("author_ids")
public Long[] authorIds;
}
PCollection<Article> articles = pipeline
.apply(
BigQueryIO
.readTableRowsWithSchema()
.from("myproject:data_warehouse.articles")
)
.apply(Convert.to(Article.class));
The schema looks like this:
[
{
"mode": "NULLABLE",
"name": "id",
"type": "INTEGER"
},
{
"mode": "NULLABLE",
"name": "title",
"type": "STRING"
},
{
"mode": "REPEATED",
"name": "author_ids",
"type": "INTEGER"
}
]
When I run the pipeline, I end up with the following exception:
java.lang.ClassCastException: java.lang.String cannot be cast to java.util.Map
Should this be possible? Strangely, when I remove the repeated field from the schema/POJO it works perfectly.
I'm using Beam SDK 2.19.0 with the direct runner. Any help would be much appreciated.
Josh
Re: Reading BigQuery table containing a repeated field into POJOs
Posted by Joshua Bassett <he...@joshbassett.info>.
Thanks Cham, I'll keep an eye on that issue.
Let me know if you want me to test anything out.
Josh
On Tue, 21 Apr 2020, at 7:41 AM, Chamikara Jayalath wrote:
> Thanks. This does sound like a bug and this code path was added recently. Created https://issues.apache.org/jira/browse/BEAM-9790.
>
> Thanks,
> Cham
>
> On Fri, Apr 17, 2020 at 3:03 PM Joshua Bassett <he...@joshbassett.info> wrote:
>> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.ClassCastException: java.lang.String cannot be cast to java.util.Map
>> at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish (DirectRunner.java:348)
>> at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish (DirectRunner.java:318)
>> at org.apache.beam.runners.direct.DirectRunner.run (DirectRunner.java:213)
>> at org.apache.beam.runners.direct.DirectRunner.run (DirectRunner.java:67)
>> at org.apache.beam.sdk.Pipeline.run (Pipeline.java:317)
>> at org.apache.beam.sdk.Pipeline.run (Pipeline.java:303)
>> at com.theconversation.data.TopArticlesEnriched.main (TopArticlesEnriched.java:181)
>> at sun.reflect.NativeMethodAccessorImpl.invoke0 (Native Method)
>> at sun.reflect.NativeMethodAccessorImpl.invoke (NativeMethodAccessorImpl.java:62)
>> at sun.reflect.DelegatingMethodAccessorImpl.invoke (DelegatingMethodAccessorImpl.java:43)
>> at java.lang.reflect.Method.invoke (Method.java:498)
>> at org.codehaus.mojo.exec.ExecJavaMojo$1.run (ExecJavaMojo.java:282)
>> at java.lang.Thread.run (Thread.java:748)
>> Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to java.util.Map
>> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$toBeamValue$12 (BigQueryUtils.java:528)
>> at java.util.stream.ReferencePipeline$3$1.accept (ReferencePipeline.java:193)
>> at java.util.ArrayList$ArrayListSpliterator.forEachRemaining (ArrayList.java:1382)
>> at java.util.stream.AbstractPipeline.copyInto (AbstractPipeline.java:482)
>> at java.util.stream.AbstractPipeline.wrapAndCopyInto (AbstractPipeline.java:472)
>> at java.util.stream.ReduceOps$ReduceOp.evaluateSequential (ReduceOps.java:708)
>> at java.util.stream.AbstractPipeline.evaluate (AbstractPipeline.java:234)
>> at java.util.stream.ReferencePipeline.collect (ReferencePipeline.java:566)
>> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamValue (BigQueryUtils.java:530)
>> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamRowFieldValue (BigQueryUtils.java:491)
>> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$toBeamRow$6 (BigQueryUtils.java:477)
>> at java.util.stream.ReferencePipeline$3$1.accept (ReferencePipeline.java:193)
>> at java.util.ArrayList$ArrayListSpliterator.forEachRemaining (ArrayList.java:1382)
>> at java.util.stream.AbstractPipeline.copyInto (AbstractPipeline.java:482)
>> at java.util.stream.AbstractPipeline.wrapAndCopyInto (AbstractPipeline.java:472)
>> at java.util.stream.ReduceOps$ReduceOp.evaluateSequential (ReduceOps.java:708)
>> at java.util.stream.AbstractPipeline.evaluate (AbstractPipeline.java:234)
>> at java.util.stream.ReferencePipeline.collect (ReferencePipeline.java:566)
>> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamRow (BigQueryUtils.java:478)
>> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$static$9bc3d4b2$1 (BigQueryUtils.java:341)
>> at org.apache.beam.sdk.schemas.SchemaCoder.encode (SchemaCoder.java:166)
>> at org.apache.beam.sdk.coders.Coder.encode (Coder.java:136)
>> at org.apache.beam.sdk.util.CoderUtils.encodeToSafeStream (CoderUtils.java:82)
>> at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray (CoderUtils.java:66)
>> at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray (CoderUtils.java:51)
>> at org.apache.beam.sdk.util.CoderUtils.clone (CoderUtils.java:141)
>> at org.apache.beam.sdk.util.MutationDetectors$CodedValueMutationDetector.<init> (MutationDetectors.java:115)
>> at org.apache.beam.sdk.util.MutationDetectors.forValueWithCoder (MutationDetectors.java:46)
>> at org.apache.beam.runners.direct.ImmutabilityCheckingBundleFactory$ImmutabilityEnforcingBundle.add (ImmutabilityCheckingBundleFactory.java:112)
>> at org.apache.beam.runners.direct.ParDoEvaluator$BundleOutputManager.output (ParDoEvaluator.java:299)
>> at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.outputWindowedValue (SimpleDoFnRunner.java:258)
>> at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.access$700 (SimpleDoFnRunner.java:78)
>> at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner$DoFnProcessContext.output (SimpleDoFnRunner.java:627)
>> at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner$DoFnProcessContext.output (SimpleDoFnRunner.java:615)
>> at org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup$IdentityFn.processElement (PassThroughThenCleanup.java:83)
>>
>> On Sat, 18 Apr 2020, at 12:59 AM, Chamikara Jayalath wrote:
>> > Do you have the full stack trace ?
>> > Also, does readTableRows() work for you (without using schemas) ?
>> >
>> > On Fri, Apr 17, 2020 at 3:44 AM Joshua Bassett <he...@joshbassett.info> wrote:
>> >> Hi there
>> >>
>> >> I'm trying to read rows from a BigQuery table that contains a repeated field into POJOs. Unfortunately, I'm running into issues and I can't figure it out.
>> >>
>> >> I have something like this:
>> >>
>> >> @DefaultSchema(JavaFieldSchema.class)
>> >> class Article implements Serializable {
>> >> public Long id;
>> >> public String title;
>> >> @SchemaFieldName("author_ids")
>> >> public Long[] authorIds;
>> >> }
>> >>
>> >> PCollection<Article> articles = pipeline
>> >> .apply(
>> >> BigQueryIO
>> >> .readTableRowsWithSchema()
>> >> .from("myproject:data_warehouse.articles")
>> >> )
>> >> .apply(Convert.to(Article.class));
>> >>
>> >> The schema looks like this:
>> >>
>> >> [
>> >> {
>> >> "mode": "NULLABLE",
>> >> "name": "id",
>> >> "type": "INTEGER"
>> >> },
>> >> {
>> >> "mode": "NULLABLE",
>> >> "name": "title",
>> >> "type": "STRING"
>> >> },
>> >> {
>> >> "mode": "REPEATED",
>> >> "name": "author_ids",
>> >> "type": "INTEGER"
>> >> }
>> >> ]
>> >>
>> >> When I run the pipeline, I end up with the following exception:
>> >>
>> >> java.lang.ClassCastException: java.lang.String cannot be cast to java.util.Map
>> >>
>> >> Should this be possible? Strangely, when I remove the repeated field from the schema/POJO it works perfectly.
>> >>
>> >> I'm using Beam SDK 2.19.0 with the direct runner. Any help would be much appreciated.
>> >>
>> >> Josh
Kind regards
Josh
Re: Reading BigQuery table containing a repeated field into POJOs
Posted by Chamikara Jayalath <ch...@google.com>.
Thanks. This does sound like a bug and this code path was added recently.
Created https://issues.apache.org/jira/browse/BEAM-9790.
Thanks,
Cham
On Fri, Apr 17, 2020 at 3:03 PM Joshua Bassett <he...@joshbassett.info>
wrote:
> org.apache.beam.sdk.Pipeline$PipelineExecutionException:
> java.lang.ClassCastException: java.lang.String cannot be cast to
> java.util.Map
> at
> org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish
> (DirectRunner.java:348)
> at
> org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish
> (DirectRunner.java:318)
> at org.apache.beam.runners.direct.DirectRunner.run (DirectRunner.java:213)
> at org.apache.beam.runners.direct.DirectRunner.run (DirectRunner.java:67)
> at org.apache.beam.sdk.Pipeline.run (Pipeline.java:317)
> at org.apache.beam.sdk.Pipeline.run (Pipeline.java:303)
> at com.theconversation.data.TopArticlesEnriched.main
> (TopArticlesEnriched.java:181)
> at sun.reflect.NativeMethodAccessorImpl.invoke0 (Native Method)
> at sun.reflect.NativeMethodAccessorImpl.invoke
> (NativeMethodAccessorImpl.java:62)
> at sun.reflect.DelegatingMethodAccessorImpl.invoke
> (DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke (Method.java:498)
> at org.codehaus.mojo.exec.ExecJavaMojo$1.run (ExecJavaMojo.java:282)
> at java.lang.Thread.run (Thread.java:748)
> Caused by: java.lang.ClassCastException: java.lang.String cannot be cast
> to java.util.Map
> at
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$toBeamValue$12
> (BigQueryUtils.java:528)
> at java.util.stream.ReferencePipeline$3$1.accept
> (ReferencePipeline.java:193)
> at java.util.ArrayList$ArrayListSpliterator.forEachRemaining
> (ArrayList.java:1382)
> at java.util.stream.AbstractPipeline.copyInto (AbstractPipeline.java:482)
> at java.util.stream.AbstractPipeline.wrapAndCopyInto
> (AbstractPipeline.java:472)
> at java.util.stream.ReduceOps$ReduceOp.evaluateSequential
> (ReduceOps.java:708)
> at java.util.stream.AbstractPipeline.evaluate (AbstractPipeline.java:234)
> at java.util.stream.ReferencePipeline.collect (ReferencePipeline.java:566)
> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamValue
> (BigQueryUtils.java:530)
> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamRowFieldValue
> (BigQueryUtils.java:491)
> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$toBeamRow$6
> (BigQueryUtils.java:477)
> at java.util.stream.ReferencePipeline$3$1.accept
> (ReferencePipeline.java:193)
> at java.util.ArrayList$ArrayListSpliterator.forEachRemaining
> (ArrayList.java:1382)
> at java.util.stream.AbstractPipeline.copyInto (AbstractPipeline.java:482)
> at java.util.stream.AbstractPipeline.wrapAndCopyInto
> (AbstractPipeline.java:472)
> at java.util.stream.ReduceOps$ReduceOp.evaluateSequential
> (ReduceOps.java:708)
> at java.util.stream.AbstractPipeline.evaluate (AbstractPipeline.java:234)
> at java.util.stream.ReferencePipeline.collect (ReferencePipeline.java:566)
> at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamRow
> (BigQueryUtils.java:478)
> at
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$static$9bc3d4b2$1
> (BigQueryUtils.java:341)
> at org.apache.beam.sdk.schemas.SchemaCoder.encode (SchemaCoder.java:166)
> at org.apache.beam.sdk.coders.Coder.encode (Coder.java:136)
> at org.apache.beam.sdk.util.CoderUtils.encodeToSafeStream
> (CoderUtils.java:82)
> at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray
> (CoderUtils.java:66)
> at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray
> (CoderUtils.java:51)
> at org.apache.beam.sdk.util.CoderUtils.clone (CoderUtils.java:141)
> at
> org.apache.beam.sdk.util.MutationDetectors$CodedValueMutationDetector.<init>
> (MutationDetectors.java:115)
> at org.apache.beam.sdk.util.MutationDetectors.forValueWithCoder
> (MutationDetectors.java:46)
> at
> org.apache.beam.runners.direct.ImmutabilityCheckingBundleFactory$ImmutabilityEnforcingBundle.add
> (ImmutabilityCheckingBundleFactory.java:112)
> at
> org.apache.beam.runners.direct.ParDoEvaluator$BundleOutputManager.output
> (ParDoEvaluator.java:299)
> at
> org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.outputWindowedValue
> (SimpleDoFnRunner.java:258)
> at
> org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.access$700
> (SimpleDoFnRunner.java:78)
> at
> org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner$DoFnProcessContext.output
> (SimpleDoFnRunner.java:627)
> at
> org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner$DoFnProcessContext.output
> (SimpleDoFnRunner.java:615)
> at
> org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup$IdentityFn.processElement
> (PassThroughThenCleanup.java:83)
>
> On Sat, 18 Apr 2020, at 12:59 AM, Chamikara Jayalath wrote:
> > Do you have the full stack trace ?
> > Also, does readTableRows() work for you (without using schemas) ?
> >
> > On Fri, Apr 17, 2020 at 3:44 AM Joshua Bassett <he...@joshbassett.info>
> wrote:
> >> Hi there
> >>
> >> I'm trying to read rows from a BigQuery table that contains a repeated
> field into POJOs. Unfortunately, I'm running into issues and I can't figure
> it out.
> >>
> >> I have something like this:
> >>
> >> @DefaultSchema(JavaFieldSchema.class)
> >> class Article implements Serializable {
> >> public Long id;
> >> public String title;
> >> @SchemaFieldName("author_ids")
> >> public Long[] authorIds;
> >> }
> >>
> >> PCollection<Article> articles = pipeline
> >> .apply(
> >> BigQueryIO
> >> .readTableRowsWithSchema()
> >> .from("myproject:data_warehouse.articles")
> >> )
> >> .apply(Convert.to(Article.class));
> >>
> >> The schema looks like this:
> >>
> >> [
> >> {
> >> "mode": "NULLABLE",
> >> "name": "id",
> >> "type": "INTEGER"
> >> },
> >> {
> >> "mode": "NULLABLE",
> >> "name": "title",
> >> "type": "STRING"
> >> },
> >> {
> >> "mode": "REPEATED",
> >> "name": "author_ids",
> >> "type": "INTEGER"
> >> }
> >> ]
> >>
> >> When I run the pipeline, I end up with the following exception:
> >>
> >> java.lang.ClassCastException: java.lang.String cannot be cast to
> java.util.Map
> >>
> >> Should this be possible? Strangely, when I remove the repeated field
> from the schema/POJO it works perfectly.
> >>
> >> I'm using Beam SDK 2.19.0 with the direct runner. Any help would be
> much appreciated.
> >>
> >> Josh
>
Re: Reading BigQuery table containing a repeated field into POJOs
Posted by Joshua Bassett <he...@joshbassett.info>.
org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.ClassCastException: java.lang.String cannot be cast to java.util.Map
at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish (DirectRunner.java:348)
at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish (DirectRunner.java:318)
at org.apache.beam.runners.direct.DirectRunner.run (DirectRunner.java:213)
at org.apache.beam.runners.direct.DirectRunner.run (DirectRunner.java:67)
at org.apache.beam.sdk.Pipeline.run (Pipeline.java:317)
at org.apache.beam.sdk.Pipeline.run (Pipeline.java:303)
at com.theconversation.data.TopArticlesEnriched.main (TopArticlesEnriched.java:181)
at sun.reflect.NativeMethodAccessorImpl.invoke0 (Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke (NativeMethodAccessorImpl.java:62)
at sun.reflect.DelegatingMethodAccessorImpl.invoke (DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke (Method.java:498)
at org.codehaus.mojo.exec.ExecJavaMojo$1.run (ExecJavaMojo.java:282)
at java.lang.Thread.run (Thread.java:748)
Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to java.util.Map
at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$toBeamValue$12 (BigQueryUtils.java:528)
at java.util.stream.ReferencePipeline$3$1.accept (ReferencePipeline.java:193)
at java.util.ArrayList$ArrayListSpliterator.forEachRemaining (ArrayList.java:1382)
at java.util.stream.AbstractPipeline.copyInto (AbstractPipeline.java:482)
at java.util.stream.AbstractPipeline.wrapAndCopyInto (AbstractPipeline.java:472)
at java.util.stream.ReduceOps$ReduceOp.evaluateSequential (ReduceOps.java:708)
at java.util.stream.AbstractPipeline.evaluate (AbstractPipeline.java:234)
at java.util.stream.ReferencePipeline.collect (ReferencePipeline.java:566)
at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamValue (BigQueryUtils.java:530)
at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamRowFieldValue (BigQueryUtils.java:491)
at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$toBeamRow$6 (BigQueryUtils.java:477)
at java.util.stream.ReferencePipeline$3$1.accept (ReferencePipeline.java:193)
at java.util.ArrayList$ArrayListSpliterator.forEachRemaining (ArrayList.java:1382)
at java.util.stream.AbstractPipeline.copyInto (AbstractPipeline.java:482)
at java.util.stream.AbstractPipeline.wrapAndCopyInto (AbstractPipeline.java:472)
at java.util.stream.ReduceOps$ReduceOp.evaluateSequential (ReduceOps.java:708)
at java.util.stream.AbstractPipeline.evaluate (AbstractPipeline.java:234)
at java.util.stream.ReferencePipeline.collect (ReferencePipeline.java:566)
at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamRow (BigQueryUtils.java:478)
at org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$static$9bc3d4b2$1 (BigQueryUtils.java:341)
at org.apache.beam.sdk.schemas.SchemaCoder.encode (SchemaCoder.java:166)
at org.apache.beam.sdk.coders.Coder.encode (Coder.java:136)
at org.apache.beam.sdk.util.CoderUtils.encodeToSafeStream (CoderUtils.java:82)
at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray (CoderUtils.java:66)
at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray (CoderUtils.java:51)
at org.apache.beam.sdk.util.CoderUtils.clone (CoderUtils.java:141)
at org.apache.beam.sdk.util.MutationDetectors$CodedValueMutationDetector.<init> (MutationDetectors.java:115)
at org.apache.beam.sdk.util.MutationDetectors.forValueWithCoder (MutationDetectors.java:46)
at org.apache.beam.runners.direct.ImmutabilityCheckingBundleFactory$ImmutabilityEnforcingBundle.add (ImmutabilityCheckingBundleFactory.java:112)
at org.apache.beam.runners.direct.ParDoEvaluator$BundleOutputManager.output (ParDoEvaluator.java:299)
at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.outputWindowedValue (SimpleDoFnRunner.java:258)
at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.access$700 (SimpleDoFnRunner.java:78)
at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner$DoFnProcessContext.output (SimpleDoFnRunner.java:627)
at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner$DoFnProcessContext.output (SimpleDoFnRunner.java:615)
at org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup$IdentityFn.processElement (PassThroughThenCleanup.java:83)
On Sat, 18 Apr 2020, at 12:59 AM, Chamikara Jayalath wrote:
> Do you have the full stack trace ?
> Also, does readTableRows() work for you (without using schemas) ?
>
> On Fri, Apr 17, 2020 at 3:44 AM Joshua Bassett <he...@joshbassett.info> wrote:
>> Hi there
>>
>> I'm trying to read rows from a BigQuery table that contains a repeated field into POJOs. Unfortunately, I'm running into issues and I can't figure it out.
>>
>> I have something like this:
>>
>> @DefaultSchema(JavaFieldSchema.class)
>> class Article implements Serializable {
>> public Long id;
>> public String title;
>> @SchemaFieldName("author_ids")
>> public Long[] authorIds;
>> }
>>
>> PCollection<Article> articles = pipeline
>> .apply(
>> BigQueryIO
>> .readTableRowsWithSchema()
>> .from("myproject:data_warehouse.articles")
>> )
>> .apply(Convert.to(Article.class));
>>
>> The schema looks like this:
>>
>> [
>> {
>> "mode": "NULLABLE",
>> "name": "id",
>> "type": "INTEGER"
>> },
>> {
>> "mode": "NULLABLE",
>> "name": "title",
>> "type": "STRING"
>> },
>> {
>> "mode": "REPEATED",
>> "name": "author_ids",
>> "type": "INTEGER"
>> }
>> ]
>>
>> When I run the pipeline, I end up with the following exception:
>>
>> java.lang.ClassCastException: java.lang.String cannot be cast to java.util.Map
>>
>> Should this be possible? Strangely, when I remove the repeated field from the schema/POJO it works perfectly.
>>
>> I'm using Beam SDK 2.19.0 with the direct runner. Any help would be much appreciated.
>>
>> Josh
Re: Reading BigQuery table containing a repeated field into POJOs
Posted by Chamikara Jayalath <ch...@google.com>.
Do you have the full stack trace ?
Also, does readTableRows() work for you (without using schemas) ?
On Fri, Apr 17, 2020 at 3:44 AM Joshua Bassett <he...@joshbassett.info>
wrote:
> Hi there
>
> I'm trying to read rows from a BigQuery table that contains a repeated
> field into POJOs. Unfortunately, I'm running into issues and I can't figure
> it out.
>
> I have something like this:
>
> @DefaultSchema(JavaFieldSchema.class)
> class Article implements Serializable {
> public Long id;
> public String title;
> @SchemaFieldName("author_ids")
> public Long[] authorIds;
> }
>
> PCollection<Article> articles = pipeline
> .apply(
> BigQueryIO
> .readTableRowsWithSchema()
> .from("myproject:data_warehouse.articles")
> )
> .apply(Convert.to(Article.class));
>
> The schema looks like this:
>
> [
> {
> "mode": "NULLABLE",
> "name": "id",
> "type": "INTEGER"
> },
> {
> "mode": "NULLABLE",
> "name": "title",
> "type": "STRING"
> },
> {
> "mode": "REPEATED",
> "name": "author_ids",
> "type": "INTEGER"
> }
> ]
>
> When I run the pipeline, I end up with the following exception:
>
> java.lang.ClassCastException: java.lang.String cannot be cast to
> java.util.Map
>
> Should this be possible? Strangely, when I remove the repeated field from
> the schema/POJO it works perfectly.
>
> I'm using Beam SDK 2.19.0 with the direct runner. Any help would be much
> appreciated.
>
> Josh
>