Hi Tao,

In ZetaSQL all "integers" are 64 bits. So if your integers in column 1 and
2 are 32-bit it won't work. In terms of Beam schema it corresponds to INT64
type.

Best,
Robin

On Thu, Mar 4, 2021 at 6:07 PM Brian Hulette <[email protected]> wrote:

> Ah, I suspect this is because our ZetaSQL planner only supports 64 bit
> integers (see
> https://beam.apache.org/documentation/dsls/sql/zetasql/data-types/#integer-type
> ). +Robin Qiu <[email protected]> maybe we should have a better error
> message for this?
>
> On Thu, Mar 4, 2021 at 5:24 PM Tao Li <[email protected]> wrote:
>
>> Brian the schema is really simple. Just 3 primitive type columns:
>>
>>
>>
>> root
>>
>> |-- column_1: integer (nullable = true)
>>
>> |-- column_2: integer (nullable = true)
>>
>> |-- column_3: string (nullable = true)
>>
>>
>>
>>
>>
>> *From: *Brian Hulette <[email protected]>
>> *Date: *Thursday, March 4, 2021 at 2:29 PM
>> *To: *Tao Li <[email protected]>
>> *Cc: *"[email protected]" <[email protected]>
>> *Subject: *Re: A problem with ZetaSQL
>>
>>
>>
>> Thanks, It would also be helpful to know what avroSchema is, or at least
>> the types of its fields, so we can understand what the schema of the
>> PCollection is.
>>
>>
>>
>> On Tue, Mar 2, 2021 at 11:00 AM Tao Li <[email protected]> wrote:
>>
>> Hi Brian,
>>
>>
>>
>> Here is my code to create the PCollection<Row>.
>>
>>
>>
>> PCollection<FileIO.ReadableFile> files = pipeline
>>
>>                 .apply(FileIO.match().filepattern(path))
>>
>>                 .apply(FileIO.readMatches());
>>
>>
>>
>> PCollection<Row> input =  files
>>
>>                 .apply(ParquetIO.readFiles(avroSchema))
>>
>>                 .apply(MapElements
>>
>>                         .into(TypeDescriptors.rows())
>>
>>
>> .via(AvroUtils.getGenericRecordToRowFunction(AvroUtils.toBeamSchema(avroSchema))))
>>
>>
>> .setCoder(RowCoder.of(AvroUtils.toBeamSchema(avroSchema)));
>>
>>
>>
>>
>>
>> *From: *Brian Hulette <[email protected]>
>> *Reply-To: *"[email protected]" <[email protected]>
>> *Date: *Tuesday, March 2, 2021 at 10:31 AM
>> *To: *user <[email protected]>
>> *Subject: *Re: A problem with ZetaSQL
>>
>>
>>
>> Thanks for reporting this Tao - could you share what the type of your
>> input PCollection is?
>>
>>
>>
>> On Tue, Mar 2, 2021 at 9:33 AM Tao Li <[email protected]> wrote:
>>
>> Hi all,
>>
>>
>>
>> I was following the instructions from this doc to play with ZetaSQL
>> https://beam.apache.org/documentation/dsls/sql/overview/
>> <https://nam11.safelinks.protection.outlook.com/?url=https%3A%2F%2Fbeam.apache.org%2Fdocumentation%2Fdsls%2Fsql%2Foverview%2F&data=04%7C01%7Ctaol%40zillow.com%7C44e3c1a43333455172a108d8df5d0428%7C033464830d1840e7a5883784ac50e16f%7C0%7C0%7C637504937882864479%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=RAXCN9Fbze5N41n35EkgY%2BkNn7pvN1Exib6%2BUr7Df3k%3D&reserved=0>
>>
>>
>>
>> The query is really simple:
>>
>>
>>
>> options.as
>> <https://nam11.safelinks.protection.outlook.com/?url=http%3A%2F%2Foptions.as%2F&data=04%7C01%7Ctaol%40zillow.com%7C44e3c1a43333455172a108d8df5d0428%7C033464830d1840e7a5883784ac50e16f%7C0%7C0%7C637504937882864479%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=UdcvMpWl%2FfmUhxlIu7igK1yTRMDWgIpA7bV2yKYlInU%3D&reserved=0>
>> (BeamSqlPipelineOptions.class).setPlannerName("org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner")
>>
>> input.apply(SqlTransform.query("SELECT * from PCOLLECTION"))
>>
>>
>>
>> I am seeing this error with ZetaSQL  :
>>
>>
>>
>> Exception in thread "main" java.lang.UnsupportedOperationException:
>> Unknown Calcite type: INTEGER
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlCalciteTranslationUtils.toZetaSqlType(ZetaSqlCalciteTranslationUtils.java:114)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.zetasql.SqlAnalyzer.addFieldsToTable(SqlAnalyzer.java:359)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.zetasql.SqlAnalyzer.addTableToLeafCatalog(SqlAnalyzer.java:350)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.zetasql.SqlAnalyzer.lambda$createPopulatedCatalog$1(SqlAnalyzer.java:225)
>>
>>                 at
>> com.google.common.collect.ImmutableList.forEach(ImmutableList.java:406)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.zetasql.SqlAnalyzer.createPopulatedCatalog(SqlAnalyzer.java:225)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLPlannerImpl.rel(ZetaSQLPlannerImpl.java:102)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRelInternal(ZetaSQLQueryPlanner.java:180)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:168)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv.parseQuery(BeamSqlEnv.java:114)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:140)
>>
>>                 at
>> org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:86)
>>
>>
>>
>> This query works fine when using Calcite (by just removing setPlannerName
>> call). Am I missing anything here? For example I am specifying
>> 'com.google.guava:guava:23.0' as the dependency.
>>
>>
>>
>> Thanks!
>>
>>
>>
>>
>>
>>

Reply via email to