-
Notifications
You must be signed in to change notification settings - Fork 3.2k
Do not require Confluent libraries for Kafka connector if Protobuf features are not being used #17858
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
Merged
hashhar
merged 7 commits into
trinodb:master
from
hashhar:hashhar/fix-protobuf-eager-loading
Jun 15, 2023
Merged
Do not require Confluent libraries for Kafka connector if Protobuf features are not being used #17858
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
c25887a
Remove ProtobufSchema from method signatures
hashhar dae19c5
Lazily load Confluent licensed ProtobufSchemaParser
hashhar 0373446
Fix comment in ConfluentModule
hashhar b36d60c
Rename Kafka Protobuf test classes to match Avro test classes
hashhar 417fa0b
Refactor KafkaCatalog in Kafka product tests
hashhar d952b2c
Convert KafkaCatalog to record
hashhar 12eee83
Add product tests for Kafka connector with Confluent licensed libraries
hashhar File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
30 changes: 30 additions & 0 deletions
30
...no-kafka/src/main/java/io/trino/plugin/kafka/schema/confluent/ForwardingSchemaParser.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,30 @@ | ||
/* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package io.trino.plugin.kafka.schema.confluent; | ||
|
||
import io.confluent.kafka.schemaregistry.ParsedSchema; | ||
import io.trino.plugin.kafka.KafkaTopicFieldGroup; | ||
import io.trino.spi.connector.ConnectorSession; | ||
|
||
public abstract class ForwardingSchemaParser | ||
implements SchemaParser | ||
{ | ||
protected abstract SchemaParser delegate(); | ||
|
||
@Override | ||
public KafkaTopicFieldGroup parse(ConnectorSession session, String subject, ParsedSchema parsedSchema) | ||
{ | ||
return delegate().parse(session, subject, parsedSchema); | ||
} | ||
} |
27 changes: 27 additions & 0 deletions
27
...afka/src/test/java/io/trino/plugin/kafka/schema/confluent/TestForwardingSchemaParser.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,27 @@ | ||
/* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package io.trino.plugin.kafka.schema.confluent; | ||
|
||
import org.testng.annotations.Test; | ||
|
||
import static io.trino.spi.testing.InterfaceTestUtils.assertAllMethodsOverridden; | ||
|
||
public class TestForwardingSchemaParser | ||
{ | ||
@Test | ||
public void testAllMethodsOverridden() | ||
{ | ||
assertAllMethodsOverridden(SchemaParser.class, ForwardingSchemaParser.class); | ||
} | ||
} |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
76 changes: 76 additions & 0 deletions
76
...main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeConfluentKafka.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,76 @@ | ||
/* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package io.trino.tests.product.launcher.env.environment; | ||
|
||
import com.google.common.collect.ImmutableList; | ||
import com.google.inject.Inject; | ||
import io.trino.tests.product.launcher.docker.DockerFiles; | ||
import io.trino.tests.product.launcher.docker.DockerFiles.ResourceProvider; | ||
import io.trino.tests.product.launcher.env.Environment; | ||
import io.trino.tests.product.launcher.env.EnvironmentProvider; | ||
import io.trino.tests.product.launcher.env.common.Kafka; | ||
import io.trino.tests.product.launcher.env.common.StandardMultinode; | ||
import io.trino.tests.product.launcher.env.common.TestsEnvironment; | ||
|
||
import java.io.File; | ||
|
||
import static io.trino.testing.TestingProperties.getConfluentVersion; | ||
import static io.trino.tests.product.launcher.env.EnvironmentContainers.configureTempto; | ||
import static io.trino.tests.product.launcher.env.EnvironmentContainers.isTrinoContainer; | ||
import static io.trino.tests.product.launcher.env.common.Standard.CONTAINER_TRINO_ETC; | ||
import static java.util.Objects.requireNonNull; | ||
import static org.testcontainers.utility.MountableFile.forClasspathResource; | ||
import static org.testcontainers.utility.MountableFile.forHostPath; | ||
|
||
/** | ||
* {@link EnvMultinodeConfluentKafka} is intended to be the only Kafka product test environment which copies the non-free Confluent License libraries to the Kafka connector | ||
* classpath to test functionality which requires those classes. | ||
* The other {@link Kafka} environments MUST NOT copy these jars otherwise it's not possible to verify the out of box Trino setup which doesn't ship with the Confluent licensed | ||
* libraries. | ||
*/ | ||
@TestsEnvironment | ||
public final class EnvMultinodeConfluentKafka | ||
extends EnvironmentProvider | ||
{ | ||
private static final File KAFKA_PROTOBUF_PROVIDER = new File("testing/trino-product-tests-launcher/target/kafka-protobuf-provider-" + getConfluentVersion() + ".jar"); | ||
private static final File KAFKA_PROTOBUF_TYPES = new File("testing/trino-product-tests-launcher/target/kafka-protobuf-types-" + getConfluentVersion() + ".jar"); | ||
|
||
private final ResourceProvider configDir; | ||
|
||
@Inject | ||
public EnvMultinodeConfluentKafka(Kafka kafka, StandardMultinode standardMultinode, DockerFiles dockerFiles) | ||
{ | ||
super(ImmutableList.of(standardMultinode, kafka)); | ||
requireNonNull(dockerFiles, "dockerFiles is null"); | ||
configDir = dockerFiles.getDockerFilesHostDirectory("conf/environment/multinode-kafka-confluent-license/"); | ||
} | ||
|
||
@Override | ||
public void extendEnvironment(Environment.Builder builder) | ||
{ | ||
builder.configureContainers(container -> { | ||
if (isTrinoContainer(container.getLogicalName())) { | ||
builder.addConnector("kafka", forHostPath(configDir.getPath("kafka.properties")), CONTAINER_TRINO_ETC + "/catalog/kafka.properties"); | ||
builder.addConnector("kafka", forHostPath(configDir.getPath("kafka_schema_registry.properties")), CONTAINER_TRINO_ETC + "/catalog/kafka_schema_registry.properties"); | ||
container | ||
.withCopyFileToContainer(forHostPath(KAFKA_PROTOBUF_PROVIDER.getAbsolutePath()), "/docker/kafka-protobuf-provider/kafka-protobuf-provider.jar") | ||
.withCopyFileToContainer(forHostPath(KAFKA_PROTOBUF_TYPES.getAbsolutePath()), "/docker/kafka-protobuf-provider/kafka-protobuf-types.jar") | ||
.withCopyFileToContainer(forClasspathResource("install-kafka-protobuf-provider.sh", 0755), "/docker/presto-init.d/install-kafka-protobuf-provider.sh"); | ||
} | ||
}); | ||
|
||
configureTempto(builder, configDir); | ||
} | ||
} |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
23 changes: 23 additions & 0 deletions
23
.../presto-product-tests/conf/environment/multinode-kafka-confluent-license/kafka.properties
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,23 @@ | ||
connector.name=kafka | ||
kafka.table-names=product_tests.read_simple_key_and_value,\ | ||
product_tests.read_all_datatypes_raw,\ | ||
product_tests.read_all_datatypes_csv,\ | ||
product_tests.read_all_datatypes_json,\ | ||
product_tests.read_all_datatypes_avro,\ | ||
product_tests.read_all_null_avro,\ | ||
product_tests.read_structural_datatype_avro,\ | ||
product_tests.write_simple_key_and_value,\ | ||
product_tests.write_all_datatypes_raw,\ | ||
product_tests.write_all_datatypes_csv,\ | ||
product_tests.write_all_datatypes_json,\ | ||
product_tests.write_all_datatypes_avro,\ | ||
product_tests.write_structural_datatype_avro,\ | ||
product_tests.pushdown_partition,\ | ||
product_tests.pushdown_offset,\ | ||
product_tests.pushdown_create_time,\ | ||
product_tests.all_datatypes_protobuf,\ | ||
product_tests.structural_datatype_protobuf,\ | ||
product_tests.read_basic_datatypes_protobuf,\ | ||
product_tests.read_basic_structural_datatypes_protobuf | ||
kafka.nodes=kafka:9092 | ||
kafka.table-description-dir=/docker/presto-product-tests/conf/presto/etc/catalog/kafka |
5 changes: 5 additions & 0 deletions
5
...tests/conf/environment/multinode-kafka-confluent-license/kafka_schema_registry.properties
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,5 @@ | ||
connector.name=kafka | ||
kafka.nodes=kafka:9092 | ||
kafka.table-description-supplier=confluent | ||
kafka.confluent-schema-registry-url=http://schema-registry:8081 | ||
kafka.default-schema=product_tests |
2 changes: 2 additions & 0 deletions
2
...roduct-tests/conf/environment/multinode-kafka-confluent-license/tempto-configuration.yaml
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,2 @@ | ||
schema-registry: | ||
url: http://schema-registry:8081 |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.