Skip to content
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

[BEAM-9364] Refactor KafkaIO to use DeserializerProviders #10947

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
[BEAM-9364] Refactor KafkaIO to use DeserializerProviders
  • Loading branch information
iemejia committed Feb 25, 2020
commit 00cf8217846c277fba8a56ad75c690d756859344
1 change: 1 addition & 0 deletions sdks/java/io/kafka/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ dependencies {
testCompile project(path: ":sdks:java:testing:test-utils", configuration: "testRuntime")
// For testing Cross-language transforms
testCompile project(":runners:core-construction-java")
testCompile library.java.avro
testCompile library.java.hamcrest_core
testCompile library.java.hamcrest_library
testCompile library.java.junit
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/*
* 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
*
* https://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.beam.sdk.io.kafka;

import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;

import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.SchemaMetadata;
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig;
import io.confluent.kafka.serializers.KafkaAvroDeserializer;
import java.io.IOException;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.coders.AvroCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderRegistry;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.kafka.common.serialization.Deserializer;

/**
* A {@link DeserializerProvider} that uses <a
* href="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/confluentinc/schema-registry">Confluent Schema Registry</a> to resolve a
* {@link Deserializer}s and {@link Coder} given a subject.
*/
@Experimental(Kind.SOURCE_SINK)
public class ConfluentSchemaRegistryDeserializerProvider<T> implements DeserializerProvider<T> {
private final SerializableFunction<Void, SchemaRegistryClient> schemaRegistryClientProviderFn;
private final String schemaRegistryUrl;
private final String subject;

@VisibleForTesting
ConfluentSchemaRegistryDeserializerProvider(
SerializableFunction<Void, SchemaRegistryClient> schemaRegistryClientProviderFn,
String schemaRegistryUrl,
String subject) {
checkArgument(
schemaRegistryClientProviderFn != null,
"You should provide a schemaRegistryClientProviderFn.");
checkArgument(schemaRegistryUrl != null, "You should provide a schemaRegistryUrl.");
checkArgument(subject != null, "You should provide a subject to fetch the schema from.");
this.schemaRegistryClientProviderFn = schemaRegistryClientProviderFn;
this.schemaRegistryUrl = schemaRegistryUrl;
this.subject = subject;
}

public static <T> ConfluentSchemaRegistryDeserializerProvider<T> of(
String schemaRegistryUrl, String subject) {
return new ConfluentSchemaRegistryDeserializerProvider(
(SerializableFunction<Void, SchemaRegistryClient>)
input -> new CachedSchemaRegistryClient(schemaRegistryUrl, Integer.MAX_VALUE),
schemaRegistryUrl,
subject);
}

@Override
public Deserializer<T> getDeserializer(Map<String, ?> configs, boolean isKey) {
ImmutableMap<String, Object> csrConfig =
ImmutableMap.<String, Object>builder()
.putAll(configs)
.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl)
.build();
Deserializer<T> deserializer =
(Deserializer<T>) new KafkaAvroDeserializer(getSchemaRegistryClient());
deserializer.configure(csrConfig, isKey);
return deserializer;
}

@Override
public Coder<T> getCoder(CoderRegistry coderRegistry) {
final Schema avroSchema = new Schema.Parser().parse(getSchemaMetadata().getSchema());
return (Coder<T>) AvroCoder.of(avroSchema);
}

private SchemaMetadata getSchemaMetadata() {
try {
return getSchemaRegistryClient().getLatestSchemaMetadata(subject);
} catch (IOException | RestClientException e) {
throw new RuntimeException("Unable to get latest schema metadata for subject: " + subject, e);
}
}

private SchemaRegistryClient getSchemaRegistryClient() {
return this.schemaRegistryClientProviderFn.apply(null);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,23 +17,19 @@
*/
package org.apache.beam.sdk.io.kafka;

import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
import java.io.Serializable;
import java.util.Map;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderRegistry;
import org.apache.kafka.common.serialization.Deserializer;

/**
* Provides instances of Confluent Schema Registry client, schema registry url, key and value
* subjects.
*
* <p>Please note, that any instance of {@link CSRClientProvider} must be {@link Serializable} to
* ensure it can be sent to worker machines.
*/
public interface CSRClientProvider extends Serializable {

SchemaRegistryClient getCSRClient();

String getSchemaRegistryUrl();
/** Provides a configured {@link Deserializer} instance and its associated {@link Coder}. */
@Experimental(Kind.SOURCE_SINK)
interface DeserializerProvider<T> extends Serializable {

String getKeySchemaSubject();
Deserializer<T> getDeserializer(Map<String, ?> configs, boolean isKey);

String getValueSchemaSubject();
Coder<T> getCoder(CoderRegistry coderRegistry);
}
Loading