[ 
https://issues.apache.org/jira/browse/BEAM-6566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16795765#comment-16795765
 ] 

Xuefeng Zhang commented on BEAM-6566:
-------------------------------------

Thank you all for looking into this. After remove line of 
.withCoder(RowCoder.of(schema))); I got below errors, so I guess coder must be 
defined when we create input. Could you please advise or provide working code 
example? [~reuvenlax] [~kenn]

java.util.ServiceConfigurationError: 
org.apache.beam.sdk.extensions.sql.impl.udf.BeamBuiltinFunctionProvider: 
Provider org.apache.beam.sdk.extensions.sql.impl.udf.BuiltinStringFunctions 
could not be instantiated

at java.util.ServiceLoader.fail(ServiceLoader.java:232)
 at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
 at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
 at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
 at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
 at 
org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv.loadBeamBuiltinFunctions(BeamSqlEnv.java:128)
 at org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:94)
 at org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:76)
 at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:537)
 at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:471)
 at org.apache.beam.sdk.values.PCollection.apply(PCollection.java:357)
 at 
com.gs.imd.crp.beam.transform.SelectColumnsTest.testSelectColumnsBasic(SelectColumnsTest.java:42)
 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:497)
 at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
 at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
 at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
 at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
 at org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:319)
 at org.junit.rules.RunRules.evaluate(RunRules.java:20)
 at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
 at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
 at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
 at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
 at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
 at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
 at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
 at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
 at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
 at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
 at 
com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
 at 
com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:51)
 at 
com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:237)
 at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70)
Caused by: java.lang.NoClassDefFoundError: 
org/apache/commons/codec/DecoderException
 at java.lang.Class.getDeclaredConstructors0(Native Method)
 at java.lang.Class.privateGetDeclaredConstructors(Class.java:2671)
 at java.lang.Class.getConstructor0(Class.java:3075)
 at java.lang.Class.newInstance(Class.java:412)
 at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:380)
 ... 33 more
Caused by: java.lang.ClassNotFoundException: 
org.apache.commons.codec.DecoderException
 at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
 at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
 ... 38 more

> SqlTransform does not work for beam version above 2.6.0 if RowCoder 
> explicitly chosen
> -------------------------------------------------------------------------------------
>
>                 Key: BEAM-6566
>                 URL: https://issues.apache.org/jira/browse/BEAM-6566
>             Project: Beam
>          Issue Type: Bug
>          Components: dsl-sql
>    Affects Versions: 2.7.0, 2.8.0, 2.9.0, 2.10.0
>            Reporter: Xuefeng Zhang
>            Assignee: Reuven Lax
>            Priority: Critical
>              Labels: triaged
>          Time Spent: 1h
>  Remaining Estimate: 0h
>
> *Issue*:
> Beam versions above 2.6.0 do not work for SqlTransform. By looking at the 
> code, those versions use PCollection.getSchema, this function never works 
> even for 2.6.0
> *Details:*
> Beam 2.6.0, class BeamPcollectionTable which is used by SqlTransform:
> public BeamPCollectionTable(PCollection<Row> upstream) \{   super(((RowCoder) 
> upstream.getCoder()).getSchema());   this.upstream = upstream; }
> But for Beam 2.7.0 and 2.8.0, it is changed to : 
> https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamPCollectionTable.java
> So that got below errors after upgrading beam version from 2.6.0 to 2.9.0
> java.lang.IllegalStateException: Cannot call getSchema when there is no schema
> at org.apache.beam.sdk.values.PCollection.getSchema(PCollection.java:328)
>  at 
> org.apache.beam.sdk.extensions.sql.impl.schema.BeamPCollectionTable.<init>(BeamPCollectionTable.java:34)
>  at 
> org.apache.beam.sdk.extensions.sql.SqlTransform.toTableMap(SqlTransform.java:111)
>  at 
> org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:91)
>  at 
> org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:76)
>  at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:537)
>  at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:471)
>  at org.apache.beam.sdk.values.PCollection.apply(PCollection.java:357)
> *Codes:*
> Schema schema = Schema.builder()
> .addStringField("weightMarketValue")
> .addStringField("ticker")
> .addStringField("ratingLongTermFitchRaw")
> .build();
> Row row = Row.withSchema(schema)
> .addValues("weightMarketValue 1", "ticker 1", "ratingLongTermFitchRaw 1")
> .build();
> Version 1:
>  
> {color:#FF0000}PCollection<Row> input = p.apply(Create.of(row){color}
> {color:#FF0000} .withSchema(schema, SerializableFunctions.identity(), 
> SerializableFunctions.identity()){color}
> {color:#FF0000} .withCoder(RowCoder.of(schema)));{color}
> PCollection<Row> output  = input.apply(SqlTransform.query("select * from 
> PCOLLECTION"));
> Version 2:
> {color:#FF0000}PCollection<Row> input = p.apply(Create.of(row){color}
> {color:#FF0000}.withRowSchema(schema){color}
> {color:#FF0000}.withCoder(RowCoder.of(schema)));{color}
> PCollection<Row> output  = input.apply(SqlTransform.query("select * from 
> PCOLLECTION"));
>  
>  
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to