[
https://issues.apache.org/jira/browse/FLINK-30093?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654485#comment-17654485
]
Tomoyuki NAKAMURA edited comment on FLINK-30093 at 1/4/23 2:29 PM:
-------------------------------------------------------------------
Unfortunately, Flink 1.16 can no longer handle the google.protobuf.Timestamp
type. I have added a test case for the google.protobuf.Timestamp type to
confirm this.
https://github.com/laughingman7743/flink/pull/1/files
{code:java}
org.apache.flink.formats.protobuf.PbCodegenException:
org.apache.flink.api.common.InvalidProgramException: Program cannot be
compiled. This is a bug. Please file an issue.
at
org.apache.flink.formats.protobuf.deserialize.ProtoToRowConverter.<init>(ProtoToRowConverter.java:126)
at
org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema.open(PbRowDataDeserializationSchema.java:64)
at
org.apache.flink.formats.protobuf.ProtobufTestHelper.pbBytesToRow(ProtobufTestHelper.java:118)
at
org.apache.flink.formats.protobuf.ProtobufTestHelper.pbBytesToRow(ProtobufTestHelper.java:102)
at
org.apache.flink.formats.protobuf.ProtobufTestHelper.pbBytesToRow(ProtobufTestHelper.java:97)
at
org.apache.flink.formats.protobuf.TimestampToRowTest.testSimple(TimestampToRowTest.java:22)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:566)
at
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
at
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
at
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
at
org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
at
com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69)
at
com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38)
at
com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11)
at
com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35)
at
com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:235)
at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54)
Caused by: org.apache.flink.api.common.InvalidProgramException: Program cannot
be compiled. This is a bug. Please file an issue.
at
org.apache.flink.formats.protobuf.util.PbCodegenUtils.compileClass(PbCodegenUtils.java:265)
at
org.apache.flink.formats.protobuf.deserialize.ProtoToRowConverter.<init>(ProtoToRowConverter.java:118)
... 32 more
Caused by: org.codehaus.commons.compiler.CompileException: Line 19, Column 5:
Cannot determine simple type name "org"
at
org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12211)
at
org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6833)
at
org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6594)
at
org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
at
org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
at
org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
at
org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
at
org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
at
org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6573)
at org.codehaus.janino.UnitCompiler.access$13900(UnitCompiler.java:215)
at
org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6481)
at
org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6476)
at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3928)
at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6476)
at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6469)
at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3927)
at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469)
at
org.codehaus.janino.UnitCompiler.getLocalVariable(UnitCompiler.java:2616)
at
org.codehaus.janino.UnitCompiler.buildLocalVariableMap(UnitCompiler.java:3662)
at org.codehaus.janino.UnitCompiler.access$5800(UnitCompiler.java:215)
at
org.codehaus.janino.UnitCompiler$12.visitLocalVariableDeclarationStatement(UnitCompiler.java:3545)
at
org.codehaus.janino.UnitCompiler$12.visitLocalVariableDeclarationStatement(UnitCompiler.java:3513)
at
org.codehaus.janino.Java$LocalVariableDeclarationStatement.accept(Java.java:3522)
at
org.codehaus.janino.UnitCompiler.buildLocalVariableMap(UnitCompiler.java:3512)
at
org.codehaus.janino.UnitCompiler.buildLocalVariableMap(UnitCompiler.java:3501)
at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3322)
at
org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1357)
at
org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1330)
at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:822)
at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:432)
at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:215)
at
org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:411)
at
org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:406)
at
org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1414)
at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:406)
at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:378)
at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:237)
at
org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:465)
at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:216)
at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:207)
at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:80)
at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:75)
at
org.apache.flink.formats.protobuf.util.PbCodegenUtils.compileClass(PbCodegenUtils.java:262)
... 33 more
{code}
I will try to check the difference of the implementation in Flink 1.16 from the
earlier version.
was (Author: laughingman7743):
Unfortunately, Flink 1.16 can no longer handle the google.protobuf.Timestamp
type. I have added a test case for the google.protobuf.Timestamp type to
confirm this.
https://github.com/laughingman7743/flink/pull/1/files
I will try to check the difference of the implementation in Flink 1.16 from the
earlier version.
> [Flink SQL][Protobuf] CompileException when querying Kafka topic using
> google.protobuf.Timestamp
> -------------------------------------------------------------------------------------------------
>
> Key: FLINK-30093
> URL: https://issues.apache.org/jira/browse/FLINK-30093
> Project: Flink
> Issue Type: Bug
> Components: Formats (JSON, Avro, Parquet, ORC, SequenceFile), Table
> SQL / Ecosystem
> Affects Versions: 1.16.0
> Environment: Mac OS Ventura
> Reporter: James Mcguire
> Assignee: hubert dulay
> Priority: Major
> Labels: pull-request-available
> Attachments: taskmanager_172.22.0 (1).4_46291-40eec2_log
>
>
> I am encountering an issue when trying to use Flink SQL to query a Kafka
> topic that uses {{{}google.protobuf.Timestamp{}}}.
>
> When attempting to use Flink SQL to query a protobuf serialized Kafka topic
> that uses {{{}google.protobuf.Timestamp{}}}, a
> {{org.codehaus.commons.compiler.CompileException: Line 23, Column 5: Cannot
> determine simple type name "com" }}error occurs when trying to query the
> table.
>
> *Replication steps:*
> 1. Use a protobuf definition that contains a
> {{{}google.protobuf.Timestamp{}}}:
> {noformat}
> syntax = "proto3";
> package example.message;
> import "google/protobuf/timestamp.proto";
> option java_package = "com.example.message";
> option java_multiple_files = true;
> message Test {
> int64 id = 1;
> google.protobuf.Timestamp created_at = 5;
> }{noformat}
> 2. Use protobuf definition to produce message to topic
> 3. Confirm message is deserializable by protoc:
> {code:java}
> kcat -C -t development.example.message -b localhost:9092 -o -1 -e -q -D "" |
> protoc --decode=example.message.Test
> --proto_path=/Users/jamesmcguire/repos/flink-proto-example/schemas/
> example/message/test.proto
> id: 123
> created_at {
> seconds: 456
> nanos: 789
> }{code}
> 4. Create table in Flink SQL using kafka connector and protobuf format
> {code:java}
> CREATE TABLE tests (
> id BIGINT,
> created_at row<seconds BIGINT, nanos INT>
> )
> COMMENT ''
> WITH (
> 'connector' = 'kafka',
> 'format' = 'protobuf',
> 'protobuf.message-class-name' = 'com.example.message.Test',
> 'properties.auto.offset.reset' = 'earliest',
> 'properties.bootstrap.servers' = 'host.docker.internal:9092',
> 'properties.group.id' = 'test-1',
> 'topic' = 'development.example.message'
> );{code}
> 5. Run query in Flink SQL and encounter error:
> {code:java}
> Flink SQL> select * from tests;
> [ERROR] Could not execute SQL statement. Reason:
> org.codehaus.commons.compiler.CompileException: Line 23, Column 5: Cannot
> determine simple type name "com" {code}
> {*}NOTE{*}: If you repeat steps 4-5 without {{created_at row<seconds BIGINT,
> nanos INT>}} in the table, step 5 will complete successfully.
> 6. Observe in attached log file, Flink appears to be using the incorrect
> namespace (should be {{google.protobuf.Timestamp):}}
> {code:java}
> com.example.message.Timestamp message3 = message0.getCreatedAt(); {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)