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

Upgrade Kafka read/write transforms without upgrading the pipeline #29362

Merged
merged 2 commits into from
Nov 17, 2023
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
Original file line number Diff line number Diff line change
Expand Up @@ -25,22 +25,26 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.ServiceLoader;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.beam.model.expansion.v1.ExpansionApi;
import org.apache.beam.model.pipeline.v1.Endpoints;
import org.apache.beam.model.pipeline.v1.ExternalTransforms;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
import org.apache.beam.model.pipeline.v1.SchemaApi;
import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transformservice.launcher.TransformServiceLauncher;
import org.apache.beam.sdk.values.PInput;
import org.apache.beam.sdk.values.POutput;
import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
import org.checkerframework.checker.nullness.qual.Nullable;

/**
* A utility class that allows upgrading transforms of a given pipeline using the Beam Transform
Expand Down Expand Up @@ -145,7 +149,7 @@ RunnerApi.Pipeline updateTransformViaTransformService(
String transformId,
Endpoints.ApiServiceDescriptor transformServiceEndpoint)
throws IOException {
PTransform transformToUpgrade =
RunnerApi.PTransform transformToUpgrade =
runnerAPIpipeline.getComponents().getTransformsMap().get(transformId);
if (transformToUpgrade == null) {
throw new IllegalArgumentException("Could not find a transform with the ID " + transformId);
Expand Down Expand Up @@ -252,7 +256,7 @@ RunnerApi.Pipeline updateTransformViaTransformService(
recursivelyFindSubTransforms(
transformId, runnerAPIpipeline.getComponents(), transformsToRemove);

Map<String, PTransform> updatedExpandedTransformMap =
Map<String, RunnerApi.PTransform> updatedExpandedTransformMap =
expandedComponents.getTransformsMap().entrySet().stream()
.filter(
entry -> {
Expand All @@ -265,7 +269,7 @@ RunnerApi.Pipeline updateTransformViaTransformService(
entry -> {
// Fix inputs
Map<String, String> inputsMap = entry.getValue().getInputsMap();
PTransform.Builder transformBuilder = entry.getValue().toBuilder();
RunnerApi.PTransform.Builder transformBuilder = entry.getValue().toBuilder();
if (!Collections.disjoint(inputsMap.values(), inputReplacements.keySet())) {
Map<String, String> updatedInputsMap = new HashMap<>();
for (Map.Entry<String, String> inputEntry : inputsMap.entrySet()) {
Expand Down Expand Up @@ -297,7 +301,7 @@ RunnerApi.Pipeline updateTransformViaTransformService(
private static void recursivelyFindSubTransforms(
String transformId, RunnerApi.Components components, List<String> results) {
results.add(transformId);
PTransform transform = components.getTransformsMap().get(transformId);
RunnerApi.PTransform transform = components.getTransformsMap().get(transformId);
if (transform == null) {
throw new IllegalArgumentException("Could not find a transform with id " + transformId);
}
Expand Down Expand Up @@ -328,4 +332,39 @@ private static int findAvailablePort() throws IOException {
public void close() throws Exception {
clientFactory.close();
}

/**
* A utility to find the registered URN for a given transform.
*
* <p>This URN can be used to upgrade this transform to a new Beam version without upgrading the
* rest of the pipeline. Please see <a
* href="https://beam.apache.org/documentation/programming-guide/#transform-service">Beam
* Transform Service documentation</a> for more details.
*
* <p>For this lookup to work, the a {@link TransformPayloadTranslatorRegistrar} for the transform
* has to be available in the classpath.
*
* @param transform transform to lookup.
* @return a URN if discovered. Returns {@code null} otherwise.
*/
@SuppressWarnings({
"rawtypes",
"EqualsIncompatibleType",
})
public static @Nullable String findUpgradeURN(PTransform transform) {
for (TransformPayloadTranslatorRegistrar registrar :
ServiceLoader.load(TransformPayloadTranslatorRegistrar.class)) {

for (Entry<
? extends Class<? extends org.apache.beam.sdk.transforms.PTransform>,
? extends TransformPayloadTranslator>
entry : registrar.getTransformPayloadTranslators().entrySet()) {
if (entry.getKey().equals(transform.getClass())) {
return entry.getValue().getUrn();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we care if there are multiple potential matches?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The contract is to upgrade all transforms that match the URN (and TransformUpgrader already handles this).

}
}
}

return null;
}
}
2 changes: 2 additions & 0 deletions sdks/java/io/expansion-service/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@ dependencies {
permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761
implementation project(":sdks:java:io:kafka")
permitUnusedDeclared project(":sdks:java:io:kafka") // BEAM-11761
implementation project(":sdks:java:io:kafka:upgrade")
permitUnusedDeclared project(":sdks:java:io:kafka:upgrade") // BEAM-11761
runtimeOnly library.java.kafka_clients
runtimeOnly library.java.slf4j_jdk14
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -626,66 +626,70 @@ public static <K, V> WriteRecords<K, V> writeRecords() {
@AutoValue.CopyAnnotations
public abstract static class Read<K, V>
extends PTransform<PBegin, PCollection<KafkaRecord<K, V>>> {

public static final Class<AutoValue_KafkaIO_Read> AUTOVALUE_CLASS =
AutoValue_KafkaIO_Read.class;

@Pure
abstract Map<String, Object> getConsumerConfig();
public abstract Map<String, Object> getConsumerConfig();

@Pure
abstract @Nullable List<String> getTopics();
public abstract @Nullable List<String> getTopics();

@Pure
abstract @Nullable List<TopicPartition> getTopicPartitions();
public abstract @Nullable List<TopicPartition> getTopicPartitions();

@Pure
abstract @Nullable Pattern getTopicPattern();
public abstract @Nullable Pattern getTopicPattern();

@Pure
abstract @Nullable Coder<K> getKeyCoder();
public abstract @Nullable Coder<K> getKeyCoder();

@Pure
abstract @Nullable Coder<V> getValueCoder();
public abstract @Nullable Coder<V> getValueCoder();

@Pure
abstract SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>>
public abstract SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>>
getConsumerFactoryFn();

@Pure
abstract @Nullable SerializableFunction<KafkaRecord<K, V>, Instant> getWatermarkFn();
public abstract @Nullable SerializableFunction<KafkaRecord<K, V>, Instant> getWatermarkFn();

@Pure
abstract long getMaxNumRecords();
public abstract long getMaxNumRecords();

@Pure
abstract @Nullable Duration getMaxReadTime();
public abstract @Nullable Duration getMaxReadTime();

@Pure
abstract @Nullable Instant getStartReadTime();
public abstract @Nullable Instant getStartReadTime();

@Pure
abstract @Nullable Instant getStopReadTime();
public abstract @Nullable Instant getStopReadTime();

@Pure
abstract boolean isCommitOffsetsInFinalizeEnabled();
public abstract boolean isCommitOffsetsInFinalizeEnabled();

@Pure
abstract boolean isDynamicRead();
public abstract boolean isDynamicRead();

@Pure
abstract @Nullable Duration getWatchTopicPartitionDuration();
public abstract @Nullable Duration getWatchTopicPartitionDuration();

@Pure
abstract TimestampPolicyFactory<K, V> getTimestampPolicyFactory();
public abstract TimestampPolicyFactory<K, V> getTimestampPolicyFactory();

@Pure
abstract @Nullable Map<String, Object> getOffsetConsumerConfig();
public abstract @Nullable Map<String, Object> getOffsetConsumerConfig();

@Pure
abstract @Nullable DeserializerProvider<K> getKeyDeserializerProvider();
public abstract @Nullable DeserializerProvider<K> getKeyDeserializerProvider();

@Pure
abstract @Nullable DeserializerProvider<V> getValueDeserializerProvider();
public abstract @Nullable DeserializerProvider<V> getValueDeserializerProvider();

@Pure
abstract @Nullable CheckStopReadingFn getCheckStopReadingFn();
public abstract @Nullable CheckStopReadingFn getCheckStopReadingFn();

abstract Builder<K, V> toBuilder();

Expand Down Expand Up @@ -996,6 +1000,14 @@ public Read<K, V> withKeyDeserializer(DeserializerProvider<K> deserializerProvid
return toBuilder().setKeyDeserializerProvider(deserializerProvider).build();
}

public Read<K, V> withKeyDeserializerProviderAndCoder(
DeserializerProvider<K> deserializerProvider, Coder<K> keyCoder) {
return toBuilder()
.setKeyDeserializerProvider(deserializerProvider)
.setKeyCoder(keyCoder)
.build();
}

/**
* Sets a Kafka {@link Deserializer} to interpret value bytes read from Kafka.
*
Expand Down Expand Up @@ -1024,6 +1036,14 @@ public Read<K, V> withValueDeserializer(DeserializerProvider<V> deserializerProv
return toBuilder().setValueDeserializerProvider(deserializerProvider).build();
}

public Read<K, V> withValueDeserializerProviderAndCoder(
DeserializerProvider<V> deserializerProvider, Coder<V> valueCoder) {
return toBuilder()
.setValueDeserializerProvider(deserializerProvider)
.setValueCoder(valueCoder)
.build();
}

/**
* A factory to create Kafka {@link Consumer} from consumer configuration. This is useful for
* supporting another version of Kafka consumer. Default is {@link KafkaConsumer}.
Expand Down Expand Up @@ -2485,37 +2505,37 @@ public abstract static class WriteRecords<K, V>
// {@link WriteRecords}. See example at {@link PubsubIO.Write}.

@Pure
abstract @Nullable String getTopic();
public abstract @Nullable String getTopic();

@Pure
abstract Map<String, Object> getProducerConfig();
public abstract Map<String, Object> getProducerConfig();

@Pure
abstract @Nullable SerializableFunction<Map<String, Object>, Producer<K, V>>
public abstract @Nullable SerializableFunction<Map<String, Object>, Producer<K, V>>
getProducerFactoryFn();

@Pure
abstract @Nullable Class<? extends Serializer<K>> getKeySerializer();
public abstract @Nullable Class<? extends Serializer<K>> getKeySerializer();

@Pure
abstract @Nullable Class<? extends Serializer<V>> getValueSerializer();
public abstract @Nullable Class<? extends Serializer<V>> getValueSerializer();

@Pure
abstract @Nullable KafkaPublishTimestampFunction<ProducerRecord<K, V>>
public abstract @Nullable KafkaPublishTimestampFunction<ProducerRecord<K, V>>
getPublishTimestampFunction();

// Configuration for EOS sink
@Pure
abstract boolean isEOS();
public abstract boolean isEOS();

@Pure
abstract @Nullable String getSinkGroupId();
public abstract @Nullable String getSinkGroupId();

@Pure
abstract int getNumShards();
public abstract int getNumShards();

@Pure
abstract @Nullable SerializableFunction<Map<String, Object>, ? extends Consumer<?, ?>>
public abstract @Nullable SerializableFunction<Map<String, Object>, ? extends Consumer<?, ?>>
getConsumerFactoryFn();

abstract Builder<K, V> toBuilder();
Expand Down Expand Up @@ -2777,9 +2797,12 @@ public abstract static class Write<K, V> extends PTransform<PCollection<KV<K, V>
// we shouldn't have to duplicate the same API for similar transforms like {@link Write} and
// {@link WriteRecords}. See example at {@link PubsubIO.Write}.

public static final Class<AutoValue_KafkaIO_Write> AUTOVALUE_CLASS =
AutoValue_KafkaIO_Write.class;

abstract @Nullable String getTopic();

abstract WriteRecords<K, V> getWriteRecordsTransform();
public abstract WriteRecords<K, V> getWriteRecordsTransform();

abstract Builder<K, V> toBuilder();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
* Common utility functions and default configurations for {@link KafkaIO.Read} and {@link
* KafkaIO.ReadSourceDescriptors}.
*/
final class KafkaIOUtils {
public final class KafkaIOUtils {
// A set of config defaults.
static final Map<String, Object> DEFAULT_CONSUMER_PROPERTIES =
ImmutableMap.of(
Expand All @@ -61,7 +61,7 @@ final class KafkaIOUtils {
false);

// A set of properties that are not required or don't make sense for our consumer.
static final Map<String, String> DISALLOWED_CONSUMER_PROPERTIES =
public static final Map<String, String> DISALLOWED_CONSUMER_PROPERTIES =
ImmutableMap.of(
ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyDeserializer instead",
ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "Set valueDeserializer instead"
Expand Down
46 changes: 46 additions & 0 deletions sdks/java/io/kafka/upgrade/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.
*/

import java.util.stream.Collectors

plugins { id 'org.apache.beam.module' }
applyJavaNature(
automaticModuleName: 'org.apache.beam.sdk.io.kafka.upgrade',
)

description = "Apache Beam :: SDKs :: Java :: IO :: Kafka :: Upgrade"
ext.summary = "Library to support upgrading Kafka transforms without upgrading the pipeline."

dependencies {
implementation project(path: ":model:pipeline", configuration: "shadow")
implementation project(":sdks:java:io:kafka")
implementation library.java.vendored_grpc_1_54_0
implementation project(path: ":sdks:java:core", configuration: "shadow")

implementation library.java.vendored_guava_32_1_2_jre
implementation project(":runners:core-construction-java")
implementation project(":sdks:java:expansion-service")
permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761
permitUnusedDeclared project(":model:pipeline")
implementation library.java.joda_time
// Get back to "provided" since 2.14
provided library.java.kafka_clients
testImplementation library.java.junit
testImplementation library.java.kafka_clients

}
Loading
Loading