-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Improve error message in Kafka connector when parsing recursive protobuf schemas #15724
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
31ac84a
3110104
ba41e56
97e6ee3
24d3033
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -339,6 +339,46 @@ | |
| </ignoredResourcePatterns> | ||
| </configuration> | ||
| </plugin> | ||
| <!-- TODO: Instead of using this plugin to generate classes we should invoke the parser directly (https://github.com/trinodb/trino/issues/16039) --> | ||
| <plugin> | ||
| <groupId>com.github.os72</groupId> | ||
| <artifactId>protoc-jar-maven-plugin</artifactId> | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's use a specific version of this plugin - #16043
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thank you. |
||
| <executions> | ||
| <execution> | ||
| <id>generate-test-sources</id> | ||
| <phase>generate-test-sources</phase> | ||
| <goals> | ||
| <goal>run</goal> | ||
| </goals> | ||
| <configuration> | ||
| <protocVersion>${dep.protobuf.version}</protocVersion> | ||
| <addSources>none</addSources> | ||
| <inputDirectories> | ||
| <include>src/test/resources/protobuf-sources</include> | ||
| </inputDirectories> | ||
| <outputDirectory>target/generated-test-sources/</outputDirectory> | ||
| </configuration> | ||
| </execution> | ||
| </executions> | ||
| </plugin> | ||
| <plugin> | ||
| <groupId>org.codehaus.mojo</groupId> | ||
| <artifactId>build-helper-maven-plugin</artifactId> | ||
| <executions> | ||
| <execution> | ||
| <id>add-test-sources</id> | ||
| <phase>generate-test-sources</phase> | ||
| <goals> | ||
| <goal>add-test-source</goal> | ||
| </goals> | ||
| <configuration> | ||
| <sources> | ||
| <source>${basedir}/target/generated-test-sources</source> | ||
| </sources> | ||
| </configuration> | ||
| </execution> | ||
| </executions> | ||
| </plugin> | ||
| </plugins> | ||
| </build> | ||
| </project> | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,19 +18,22 @@ | |
| import com.google.protobuf.Descriptors.Descriptor; | ||
| import com.google.protobuf.Descriptors.FieldDescriptor; | ||
| import com.google.protobuf.DynamicMessage; | ||
| import com.google.protobuf.Timestamp; | ||
| import dev.failsafe.Failsafe; | ||
| import dev.failsafe.RetryPolicy; | ||
| import io.confluent.kafka.serializers.protobuf.KafkaProtobufSerializer; | ||
| import io.confluent.kafka.serializers.subject.RecordNameStrategy; | ||
| import io.confluent.kafka.serializers.subject.TopicRecordNameStrategy; | ||
| import io.trino.plugin.kafka.schema.confluent.KafkaWithConfluentSchemaRegistryQueryRunner; | ||
| import io.trino.spi.type.SqlTimestamp; | ||
| import io.trino.testing.AbstractTestQueryFramework; | ||
| import io.trino.testing.QueryRunner; | ||
| import io.trino.testing.kafka.TestingKafka; | ||
| import org.apache.kafka.clients.producer.ProducerRecord; | ||
| import org.testng.annotations.Test; | ||
|
|
||
| import java.time.Duration; | ||
| import java.time.LocalDateTime; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
|
|
||
|
|
@@ -41,12 +44,19 @@ | |
| import static io.trino.decoder.protobuf.ProtobufRowDecoderFactory.DEFAULT_MESSAGE; | ||
| import static io.trino.decoder.protobuf.ProtobufUtils.getFileDescriptor; | ||
| import static io.trino.decoder.protobuf.ProtobufUtils.getProtoFile; | ||
| import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_SECOND; | ||
| import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MICROSECOND; | ||
| import static io.trino.testing.DateTimeTestingUtils.sqlTimestampOf; | ||
| import static java.lang.Math.floorDiv; | ||
| import static java.lang.Math.multiplyExact; | ||
| import static java.lang.StrictMath.floorMod; | ||
| import static java.lang.String.format; | ||
| import static java.nio.charset.StandardCharsets.UTF_8; | ||
| import static java.util.Locale.ENGLISH; | ||
| import static java.util.Objects.requireNonNull; | ||
| import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; | ||
| import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; | ||
| import static org.assertj.core.api.Assertions.assertThat; | ||
| import static org.testng.Assert.assertTrue; | ||
|
|
||
| @Test(singleThreaded = true) | ||
|
|
@@ -140,6 +150,113 @@ public void testBasicTopicForInsert() | |
| "Insert is not supported for schema registry based tables"); | ||
| } | ||
|
|
||
| @Test | ||
| public void testUnsupportedRecursiveDataTypes() | ||
| throws Exception | ||
| { | ||
| String topic = "topic-unsupported-recursive"; | ||
| assertNotExists(topic); | ||
|
|
||
| UnsupportedRecursiveTypes.schema message = UnsupportedRecursiveTypes.schema.newBuilder() | ||
| .setRecursiveValueOne(UnsupportedRecursiveTypes.RecursiveValue.newBuilder().setStringValue("Value1").build()) | ||
| .build(); | ||
|
|
||
| ImmutableList.Builder<ProducerRecord<DynamicMessage, UnsupportedRecursiveTypes.schema>> producerRecordBuilder = ImmutableList.builder(); | ||
| producerRecordBuilder.add(new ProducerRecord<>(topic, createKeySchema(0, getKeySchema()), message)); | ||
| List<ProducerRecord<DynamicMessage, UnsupportedRecursiveTypes.schema>> messages = producerRecordBuilder.build(); | ||
| testingKafka.sendMessages( | ||
| messages.stream(), | ||
| producerProperties()); | ||
|
|
||
| waitUntilTableExists(topic); | ||
| assertQueryFails("SELECT * FROM " + toDoubleQuoted(topic), | ||
| "Protobuf schema containing fields with self-reference are not supported because they cannot be mapped to a Trino type: " + | ||
| "io.trino.protobuf.schema.recursive_value_one: io.trino.protobuf.RecursiveValue > " + | ||
| "io.trino.protobuf.RecursiveValue.struct_value: io.trino.protobuf.RecursiveStruct > " + | ||
| "io.trino.protobuf.RecursiveStruct.fields: io.trino.protobuf.RecursiveStruct.FieldsEntry > " + | ||
| "io.trino.protobuf.RecursiveStruct.FieldsEntry.value: io.trino.protobuf.RecursiveValue"); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This error message goes to user, they are not much aware of trino java packages. Should use field names from protobuf schema?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The field names are from the protobuf schema, see https://github.com/trinodb/trino/pull/15724/files#diff-8ca538d4195acbb011666dca6daee504059c8090ffbedda1cf1fb8389c1e4827R3. |
||
| } | ||
|
|
||
| @Test | ||
| public void testSchemaWithImportDataTypes() | ||
| throws Exception | ||
| { | ||
| String topic = "topic-schema-with-import"; | ||
| assertNotExists(topic); | ||
|
|
||
| Descriptor descriptor = getDescriptor("structural_datatypes.proto"); | ||
|
|
||
| Timestamp timestamp = getTimestamp(sqlTimestampOf(3, LocalDateTime.parse("2020-12-12T15:35:45.923"))); | ||
| DynamicMessage message = buildDynamicMessage( | ||
| descriptor, | ||
| ImmutableMap.<String, Object>builder() | ||
| .put("list", ImmutableList.of("Search")) | ||
| .put("map", ImmutableList.of(buildDynamicMessage( | ||
| descriptor.findFieldByName("map").getMessageType(), | ||
| ImmutableMap.of("key", "Key1", "value", "Value1")))) | ||
| .put("row", ImmutableMap.<String, Object>builder() | ||
| .put("string_column", "Trino") | ||
| .put("integer_column", 1) | ||
| .put("long_column", 493857959588286460L) | ||
| .put("double_column", 3.14159265358979323846) | ||
| .put("float_column", 3.14f) | ||
| .put("boolean_column", true) | ||
| .put("number_column", descriptor.findEnumTypeByName("Number").findValueByName("ONE")) | ||
| .put("timestamp_column", timestamp) | ||
| .put("bytes_column", "Trino".getBytes(UTF_8)) | ||
| .buildOrThrow()) | ||
| .buildOrThrow()); | ||
|
|
||
| ImmutableList.Builder<ProducerRecord<DynamicMessage, DynamicMessage>> producerRecordBuilder = ImmutableList.builder(); | ||
| producerRecordBuilder.add(new ProducerRecord<>(topic, createKeySchema(0, getKeySchema()), message)); | ||
| List<ProducerRecord<DynamicMessage, DynamicMessage>> messages = producerRecordBuilder.build(); | ||
| testingKafka.sendMessages( | ||
| messages.stream(), | ||
| producerProperties()); | ||
| waitUntilTableExists(topic); | ||
|
|
||
| assertThat(query(format("SELECT list, map, row FROM %s", toDoubleQuoted(topic)))) | ||
| .matches(""" | ||
| VALUES ( | ||
| ARRAY[CAST('Search' AS VARCHAR)], | ||
| MAP(CAST(ARRAY['Key1'] AS ARRAY(VARCHAR)), CAST(ARRAY['Value1'] AS ARRAY(VARCHAR))), | ||
| CAST(ROW('Trino', 1, 493857959588286460, 3.14159265358979323846, 3.14, True, 'ONE', TIMESTAMP '2020-12-12 15:35:45.923', to_utf8('Trino')) | ||
| AS ROW( | ||
| string_column VARCHAR, | ||
| integer_column INTEGER, | ||
| long_column BIGINT, | ||
| double_column DOUBLE, | ||
| float_column REAL, | ||
| boolean_column BOOLEAN, | ||
| number_column VARCHAR, | ||
| timestamp_column TIMESTAMP(6), | ||
| bytes_column VARBINARY)))"""); | ||
| } | ||
|
|
||
| private DynamicMessage buildDynamicMessage(Descriptor descriptor, Map<String, Object> data) | ||
| { | ||
| DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor); | ||
| for (Map.Entry<String, Object> entry : data.entrySet()) { | ||
| FieldDescriptor fieldDescriptor = descriptor.findFieldByName(entry.getKey()); | ||
| if (entry.getValue() instanceof Map<?, ?>) { | ||
| builder.setField(fieldDescriptor, buildDynamicMessage(fieldDescriptor.getMessageType(), (Map<String, Object>) entry.getValue())); | ||
| } | ||
| else { | ||
| builder.setField(fieldDescriptor, entry.getValue()); | ||
| } | ||
| } | ||
|
|
||
| return builder.build(); | ||
| } | ||
|
|
||
| protected static Timestamp getTimestamp(SqlTimestamp sqlTimestamp) | ||
| { | ||
| return Timestamp.newBuilder() | ||
| .setSeconds(floorDiv(sqlTimestamp.getEpochMicros(), MICROSECONDS_PER_SECOND)) | ||
| .setNanos(floorMod(sqlTimestamp.getEpochMicros(), MICROSECONDS_PER_SECOND) * NANOSECONDS_PER_MICROSECOND) | ||
| .build(); | ||
| } | ||
|
|
||
| private Map<String, String> producerProperties() | ||
| { | ||
| return ImmutableMap.of( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,24 @@ | ||
| syntax = "proto3"; | ||
|
|
||
| package io.trino.protobuf; | ||
|
|
||
| option java_package = "io.trino.plugin.kafka.protobuf"; | ||
| option java_outer_classname = "UnsupportedRecursiveTypes"; | ||
|
|
||
| message schema { | ||
| RecursiveValue recursive_value_one = 1; | ||
| } | ||
|
|
||
| message RecursiveStruct { | ||
| map<string, RecursiveValue> fields = 1; | ||
| } | ||
|
|
||
| message RecursiveValue { | ||
| string string_value = 1; | ||
| RecursiveStruct struct_value = 2; | ||
| RecursiveListValue list_value = 3; | ||
| } | ||
|
|
||
| message RecursiveListValue { | ||
| repeated RecursiveValue values = 1; | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: Can we add a issue to remove this workaround on generating the code ? The way we invoke parser should support it out of the box, I have a rough fix for it and will be working on a PR for the same.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, I'll create an issue + add TODO.
cc: @adamjshook regarding some schema parser improvements Praveen plans to make which might be relevant for you.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yup, that'd be relevant as I am using the same plugin. I'll keep an eye out for it.