-
Notifications
You must be signed in to change notification settings - Fork 13.9k
[FLINK-39605][Flink-Formats] Add avro variant confluent format #28115
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
Open
swapna267
wants to merge
3
commits into
apache:master
Choose a base branch
from
swapna267:FLINK-39605
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from 1 commit
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
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
115 changes: 115 additions & 0 deletions
115
...ache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroVariantFormatFactory.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,115 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you 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 org.apache.flink.formats.avro.registry.confluent; | ||
|
|
||
| import org.apache.flink.annotation.Internal; | ||
| import org.apache.flink.api.common.serialization.DeserializationSchema; | ||
| import org.apache.flink.configuration.ConfigOption; | ||
| import org.apache.flink.configuration.ReadableConfig; | ||
| import org.apache.flink.formats.avro.AvroVariantDecodingFormat; | ||
| import org.apache.flink.table.connector.format.DecodingFormat; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.factories.DeserializationFormatFactory; | ||
| import org.apache.flink.table.factories.DynamicTableFactory; | ||
| import org.apache.flink.table.factories.FactoryUtil; | ||
|
|
||
| import java.util.HashSet; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
|
|
||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE; | ||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO; | ||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_CREDENTIALS_SOURCE; | ||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_TOKEN; | ||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.PROPERTIES; | ||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_LOCATION; | ||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_PASSWORD; | ||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_LOCATION; | ||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_PASSWORD; | ||
| import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.URL; | ||
|
|
||
| /** | ||
| * Table format factory for providing configured instances of Confluent Schema Registry Avro to | ||
| * Variant {@link DeserializationSchema}. Deserialization only — no serialization support. | ||
| */ | ||
| @Internal | ||
| public class ConfluentRegistryAvroVariantFormatFactory implements DeserializationFormatFactory { | ||
|
swapna267 marked this conversation as resolved.
Outdated
|
||
|
|
||
| public static final String IDENTIFIER = "avro-variant-confluent"; | ||
|
|
||
| private static final int SCHEMA_CACHE_CAPACITY = 1000; | ||
|
|
||
| @Override | ||
| public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat( | ||
| DynamicTableFactory.Context context, ReadableConfig formatOptions) { | ||
| FactoryUtil.validateFactoryOptions(this, formatOptions); | ||
|
|
||
| String schemaRegistryURL = formatOptions.get(URL); | ||
| Map<String, String> optionalPropertiesMap = | ||
| RegistryAvroFormatFactory.buildOptionalPropertiesMap(formatOptions); | ||
|
|
||
| return new AvroVariantDecodingFormat( | ||
| new CachedSchemaCoderProvider( | ||
| null, schemaRegistryURL, SCHEMA_CACHE_CAPACITY, optionalPropertiesMap), | ||
| SCHEMA_CACHE_CAPACITY); | ||
| } | ||
|
|
||
| @Override | ||
| public String factoryIdentifier() { | ||
| return IDENTIFIER; | ||
| } | ||
|
|
||
| @Override | ||
| public Set<ConfigOption<?>> requiredOptions() { | ||
| Set<ConfigOption<?>> options = new HashSet<>(); | ||
| options.add(URL); | ||
| return options; | ||
| } | ||
|
|
||
| @Override | ||
| public Set<ConfigOption<?>> optionalOptions() { | ||
| Set<ConfigOption<?>> options = new HashSet<>(); | ||
| options.add(PROPERTIES); | ||
| options.add(SSL_KEYSTORE_LOCATION); | ||
| options.add(SSL_KEYSTORE_PASSWORD); | ||
| options.add(SSL_TRUSTSTORE_LOCATION); | ||
| options.add(SSL_TRUSTSTORE_PASSWORD); | ||
| options.add(BASIC_AUTH_CREDENTIALS_SOURCE); | ||
| options.add(BASIC_AUTH_USER_INFO); | ||
| options.add(BEARER_AUTH_CREDENTIALS_SOURCE); | ||
| options.add(BEARER_AUTH_TOKEN); | ||
| return options; | ||
| } | ||
|
|
||
| @Override | ||
| public Set<ConfigOption<?>> forwardOptions() { | ||
| Set<ConfigOption<?>> options = new HashSet<>(); | ||
| options.add(URL); | ||
| options.add(PROPERTIES); | ||
| options.add(SSL_KEYSTORE_LOCATION); | ||
| options.add(SSL_KEYSTORE_PASSWORD); | ||
| options.add(SSL_TRUSTSTORE_LOCATION); | ||
| options.add(SSL_TRUSTSTORE_PASSWORD); | ||
| options.add(BASIC_AUTH_CREDENTIALS_SOURCE); | ||
| options.add(BASIC_AUTH_USER_INFO); | ||
| options.add(BEARER_AUTH_CREDENTIALS_SOURCE); | ||
| options.add(BEARER_AUTH_TOKEN); | ||
| return options; | ||
| } | ||
| } | ||
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
177 changes: 177 additions & 0 deletions
177
.../flink/formats/avro/registry/confluent/ConfluentRegistryAvroVariantFormatFactoryTest.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,177 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you 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 org.apache.flink.formats.avro.registry.confluent; | ||
|
|
||
| import org.apache.flink.api.common.serialization.DeserializationSchema; | ||
| import org.apache.flink.formats.avro.AvroVariantDeserializationSchema; | ||
| import org.apache.flink.formats.avro.RegistryWriterAvroDeserializationSchema; | ||
| import org.apache.flink.table.api.DataTypes; | ||
| import org.apache.flink.table.api.ValidationException; | ||
| import org.apache.flink.table.catalog.Column; | ||
| import org.apache.flink.table.catalog.ResolvedSchema; | ||
| import org.apache.flink.table.connector.source.DynamicTableSource; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.factories.TestDynamicTableFactory; | ||
| import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; | ||
| import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; | ||
| import org.apache.flink.table.types.logical.RowType; | ||
| import org.apache.flink.table.types.logical.VarCharType; | ||
| import org.apache.flink.table.types.logical.VariantType; | ||
|
|
||
| import org.junit.jupiter.api.Test; | ||
|
|
||
| import java.util.Collections; | ||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
|
|
||
| import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; | ||
| import static org.assertj.core.api.Assertions.assertThat; | ||
| import static org.assertj.core.api.Assertions.assertThatThrownBy; | ||
|
|
||
| class ConfluentRegistryAvroVariantFormatFactoryTest { | ||
|
|
||
| private static final ResolvedSchema SCHEMA = | ||
| ResolvedSchema.of(Column.physical("data", DataTypes.VARIANT())); | ||
|
|
||
| private static final String REGISTRY_URL = "http://localhost:8081"; | ||
| private static final int SCHEMA_CACHE_CAPACITY = 1000; | ||
|
|
||
| @Test | ||
| void testDecodingFormat() { | ||
| final AvroVariantDeserializationSchema expectedDeser = | ||
| new AvroVariantDeserializationSchema( | ||
| new RegistryWriterAvroDeserializationSchema( | ||
| new CachedSchemaCoderProvider( | ||
| null, REGISTRY_URL, SCHEMA_CACHE_CAPACITY, null)), | ||
| false, | ||
| SCHEMA_CACHE_CAPACITY, | ||
| InternalTypeInfo.of(RowType.of(new VariantType()))); | ||
|
|
||
| final DynamicTableSource actualSource = createTableSource(SCHEMA, getDefaultOptions()); | ||
| assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class); | ||
| TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = | ||
| (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; | ||
|
|
||
| DeserializationSchema<RowData> actualDeser = | ||
| scanSourceMock.valueFormat.createRuntimeDecoder( | ||
| ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); | ||
|
|
||
| assertThat(actualDeser).isEqualTo(expectedDeser); | ||
| } | ||
|
|
||
| @Test | ||
| void testDecodingFormatWithOptionalProperties() { | ||
| Map<String, String> expectedProperties = getExpectedProperties(); | ||
|
|
||
| final AvroVariantDeserializationSchema expectedDeser = | ||
| new AvroVariantDeserializationSchema( | ||
| new RegistryWriterAvroDeserializationSchema( | ||
| new CachedSchemaCoderProvider( | ||
| null, | ||
| REGISTRY_URL, | ||
| SCHEMA_CACHE_CAPACITY, | ||
| expectedProperties)), | ||
| false, | ||
| SCHEMA_CACHE_CAPACITY, | ||
| InternalTypeInfo.of(RowType.of(new VariantType()))); | ||
|
|
||
| final DynamicTableSource actualSource = | ||
| createTableSource(SCHEMA, getOptionsWithProperties()); | ||
| TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = | ||
| (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; | ||
|
|
||
| DeserializationSchema<RowData> actualDeser = | ||
| scanSourceMock.valueFormat.createRuntimeDecoder( | ||
| ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); | ||
|
|
||
| assertThat(actualDeser).isEqualTo(expectedDeser); | ||
| } | ||
|
|
||
| @Test | ||
| void testMissingUrl() { | ||
| final Map<String, String> options = getDefaultOptions(); | ||
| options.remove("avro-variant-confluent.url"); | ||
|
|
||
| assertThatThrownBy(() -> createTableSource(SCHEMA, options)) | ||
| .isInstanceOf(ValidationException.class); | ||
| } | ||
|
|
||
| @Test | ||
| void testDecodingFormatWithSchemaMetadata() { | ||
| final AvroVariantDeserializationSchema expectedDeser = | ||
| new AvroVariantDeserializationSchema( | ||
| new RegistryWriterAvroDeserializationSchema( | ||
| new CachedSchemaCoderProvider( | ||
| null, REGISTRY_URL, SCHEMA_CACHE_CAPACITY, null)), | ||
| true, | ||
| SCHEMA_CACHE_CAPACITY, | ||
| InternalTypeInfo.of( | ||
| RowType.of( | ||
| new VariantType(), | ||
| new VarCharType(true, VarCharType.MAX_LENGTH)))); | ||
|
|
||
| final DynamicTableSource actualSource = createTableSource(SCHEMA, getDefaultOptions()); | ||
| TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = | ||
| (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; | ||
|
|
||
| scanSourceMock.valueFormat.applyReadableMetadata(Collections.singletonList("schema")); | ||
|
|
||
| DeserializationSchema<RowData> actualDeser = | ||
| scanSourceMock.valueFormat.createRuntimeDecoder( | ||
| ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); | ||
|
|
||
| assertThat(actualDeser).isEqualTo(expectedDeser); | ||
| } | ||
|
|
||
| private Map<String, String> getDefaultOptions() { | ||
| final Map<String, String> options = new HashMap<>(); | ||
| options.put("connector", TestDynamicTableFactory.IDENTIFIER); | ||
| options.put("target", "MyTarget"); | ||
| options.put("buffer-size", "1000"); | ||
|
|
||
| options.put("format", ConfluentRegistryAvroVariantFormatFactory.IDENTIFIER); | ||
| options.put("avro-variant-confluent.url", REGISTRY_URL); | ||
| return options; | ||
| } | ||
|
|
||
| private Map<String, String> getOptionsWithProperties() { | ||
| final Map<String, String> options = getDefaultOptions(); | ||
| String prefix = "avro-variant-confluent."; | ||
| options.put(prefix + "ssl.keystore.location", "/test-keystore.jks"); | ||
| options.put(prefix + "ssl.keystore.password", "123456"); | ||
| options.put(prefix + "ssl.truststore.location", "/test-keystore.jks"); | ||
| options.put(prefix + "ssl.truststore.password", "123456"); | ||
| options.put(prefix + "basic-auth.credentials-source", "USER_INFO"); | ||
| options.put(prefix + "basic-auth.user-info", "user:pwd"); | ||
| options.put(prefix + "properties.bearer.auth.token", "CUSTOM"); | ||
| return options; | ||
| } | ||
|
|
||
| private Map<String, String> getExpectedProperties() { | ||
| Map<String, String> expectedProperties = new HashMap<>(); | ||
| expectedProperties.put("schema.registry.ssl.keystore.location", "/test-keystore.jks"); | ||
| expectedProperties.put("schema.registry.ssl.keystore.password", "123456"); | ||
| expectedProperties.put("schema.registry.ssl.truststore.location", "/test-keystore.jks"); | ||
| expectedProperties.put("schema.registry.ssl.truststore.password", "123456"); | ||
| expectedProperties.put("basic.auth.credentials.source", "USER_INFO"); | ||
| expectedProperties.put("basic.auth.user.info", "user:pwd"); | ||
| expectedProperties.put("bearer.auth.token", "CUSTOM"); | ||
| return expectedProperties; | ||
| } | ||
| } |
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.