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

[FLINK-36177] Deprecate KafkaShuffle and more #121

Merged
merged 2 commits into from
Sep 27, 2024
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
@@ -0,0 +1,50 @@
/*
* 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.connector.kafka.sink;

import org.apache.flink.annotation.PublicEvolving;

import java.io.Serializable;

/**
* A {@code KafkaPartitioner} wraps logic on how to partition records across partitions of multiple
* Kafka topics.
*/
@PublicEvolving
public interface KafkaPartitioner<T> extends Serializable {
/**
* Initializer for the partitioner. This is called once on each parallel sink instance of the
* Flink Kafka producer. This method should be overridden if necessary.
*
* @param parallelInstanceId 0-indexed id of the parallel sink instance in Flink
* @param parallelInstances the total number of parallel instances
*/
default void open(int parallelInstanceId, int parallelInstances) {}

/**
* Determine the id of the partition that the record should be written to.
*
* @param record the record value
* @param key serialized key of the record
* @param value serialized value of the record
* @param targetTopic target topic for the record
* @param partitions found partitions for the target topic
* @return the id of the target partition
*/
int partition(T record, byte[] key, byte[] value, String targetTopic, int[] partitions);
}
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ public class KafkaRecordSerializationSchemaBuilder<IN> {

@Nullable private Function<? super IN, String> topicSelector;
@Nullable private SerializationSchema<? super IN> valueSerializationSchema;
@Nullable private FlinkKafkaPartitioner<? super IN> partitioner;
@Nullable private KafkaPartitioner<? super IN> partitioner;
@Nullable private SerializationSchema<? super IN> keySerializationSchema;
@Nullable private HeaderProvider<? super IN> headerProvider;

Expand All @@ -91,6 +91,7 @@ public class KafkaRecordSerializationSchemaBuilder<IN> {
*
* @param partitioner
* @return {@code this}
* @deprecated use {@link #setPartitioner(KafkaPartitioner)}
*/
public <T extends IN> KafkaRecordSerializationSchemaBuilder<T> setPartitioner(
FlinkKafkaPartitioner<? super T> partitioner) {
Expand All @@ -99,6 +100,19 @@ public <T extends IN> KafkaRecordSerializationSchemaBuilder<T> setPartitioner(
return self;
}

/**
* Sets a custom partitioner determining the target partition of the target topic.
*
* @param partitioner
* @return {@code this}
*/
public <T extends IN> KafkaRecordSerializationSchemaBuilder<T> setPartitioner(
KafkaPartitioner<? super T> partitioner) {
KafkaRecordSerializationSchemaBuilder<T> self = self();
self.partitioner = checkNotNull(partitioner);
return self;
}

/**
* Sets a fixed topic which used as destination for all records.
*
Expand Down Expand Up @@ -295,15 +309,15 @@ private static class KafkaRecordSerializationSchemaWrapper<IN>
implements KafkaRecordSerializationSchema<IN> {
private final SerializationSchema<? super IN> valueSerializationSchema;
private final Function<? super IN, String> topicSelector;
private final FlinkKafkaPartitioner<? super IN> partitioner;
private final KafkaPartitioner<? super IN> partitioner;
private final SerializationSchema<? super IN> keySerializationSchema;
private final HeaderProvider<? super IN> headerProvider;

KafkaRecordSerializationSchemaWrapper(
Function<? super IN, String> topicSelector,
SerializationSchema<? super IN> valueSerializationSchema,
@Nullable SerializationSchema<? super IN> keySerializationSchema,
@Nullable FlinkKafkaPartitioner<? super IN> partitioner,
@Nullable KafkaPartitioner<? super IN> partitioner,
@Nullable HeaderProvider<? super IN> headerProvider) {
this.topicSelector = checkNotNull(topicSelector);
this.valueSerializationSchema = checkNotNull(valueSerializationSchema);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,9 @@
* ConsumerRecord ConsumerRecords}.
*
* @param <T> the type of the deserialized records.
* @deprecated Remove with @{@link KafkaDeserializationSchema}
*/
@Deprecated
class KafkaDeserializationSchemaWrapper<T> implements KafkaRecordDeserializationSchema<T> {
private static final long serialVersionUID = 1L;
private final KafkaDeserializationSchema<T> kafkaDeserializationSchema;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,9 @@ default void open(DeserializationSchema.InitializationContext context) throws Ex
* @param <V> the return type of the deserialized record.
* @return A {@link KafkaRecordDeserializationSchema} that uses the given {@link
* KafkaDeserializationSchema} to deserialize the {@link ConsumerRecord ConsumerRecords}.
* @deprecated Will be removed with {@link KafkaDeserializationSchema}.
*/
@Deprecated
static <V> KafkaRecordDeserializationSchema<V> of(
KafkaDeserializationSchema<V> kafkaDeserializationSchema) {
return new KafkaDeserializationSchemaWrapper<>(kafkaDeserializationSchema);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@
* @param <T> The type of records produced by this data source
*/
@Internal
@Deprecated
public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFunction<T>
implements CheckpointListener, ResultTypeQueryable<T>, CheckpointedFunction {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,14 @@

import org.apache.flink.annotation.PublicEvolving;

/** Error codes used in {@link FlinkKafkaException}. */
/**
* Error codes used in {@link FlinkKafkaException}.
*
* @deprecated Will be removed with {@link FlinkKafkaProducer} and {@link
* org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle}.
*/
@PublicEvolving
@Deprecated
public enum FlinkKafkaErrorCode {
PRODUCERS_POOL_EMPTY,
EXTERNAL_ERROR
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,14 @@
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.util.FlinkException;

/** Exception used by {@link FlinkKafkaProducer} and {@link FlinkKafkaConsumer}. */
/**
* Exception used by {@link FlinkKafkaProducer} and {@link FlinkKafkaConsumer}.
*
* @deprecated Will be removed with {@link FlinkKafkaProducer} and {@link
* org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle}.
*/
@PublicEvolving
@Deprecated
public class FlinkKafkaException extends FlinkException {

private static final long serialVersionUID = 920269130311214200L;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,11 @@
*
* <p>You only need to override the methods for the information that you need. However, {@link
* #getTargetTopic(Object)} is required because it is used to determine the available partitions.
*
* @deprecated Will be turned into internal API when {@link FlinkKafkaProducer} is removed.
*/
@PublicEvolving
@Deprecated
public interface KafkaContextAware<T> {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,10 @@
* (Java/Scala objects) that are processed by Flink.
*
* @param <T> The type created by the keyed deserialization schema.
* @deprecated Will be turned into internal API when {@link FlinkKafkaConsumer} is removed.
*/
@PublicEvolving
@Deprecated
public interface KafkaDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,8 +35,10 @@
* which the Kafka Producer is running.
*
* @param <T> the type of values being serialized
* @deprecated Will be turned into internal API when {@link FlinkKafkaProducer} is removed.
*/
@PublicEvolving
@Deprecated
public interface KafkaSerializationSchema<T> extends Serializable {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
* <p>The exact value of this is determined at runtime in the consumer subtasks.
*/
@Internal
@Deprecated
public enum OffsetCommitMode {

/** Completely disable offset committing. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

/** Utilities for {@link OffsetCommitMode}. */
@Internal
@Deprecated
public class OffsetCommitModes {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
* @param <KPH> The type of topic/partition identifier used by Kafka in the specific version.
*/
@Internal
@Deprecated
public abstract class AbstractFetcher<T, KPH> {

private static final int NO_TIMESTAMPS_WATERMARKS = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
* allows the discoverer to be interrupted during a {@link #discoverPartitions()} call.
*/
@Internal
@Deprecated
public abstract class AbstractPartitionDiscoverer {

/** Describes whether we are discovering partitions for fixed topics or a topic pattern. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
* @param <E> The type of elements in the queue.
*/
@Internal
@Deprecated
public class ClosableBlockingQueue<E> {

/** The lock used to make queue accesses and open checks atomic. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
* }</pre>
*/
@Internal
@Deprecated
public class ExceptionProxy {

/** The thread that should be interrupted when an exception occurs. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@

/** Internal flink kafka producer. */
@PublicEvolving
@Deprecated
public class FlinkKafkaInternalProducer<K, V> implements Producer<K, V> {
private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
*/
@ThreadSafe
@Internal
@Deprecated
public final class Handover implements Closeable {

private final Object lock = new Object();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
* commit request completes, which should normally be triggered from checkpoint complete event.
*/
@Internal
@Deprecated
public interface KafkaCommitCallback {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
* an indirection to the KafkaConsumer calls that change signature.
*/
@Internal
@Deprecated
public class KafkaConsumerThread<T> extends Thread {

/** Logger for this consumer. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
* @param <T> The type created by the deserialization schema.
*/
@Internal
@Deprecated
public class KafkaDeserializationSchemaWrapper<T> implements KafkaDeserializationSchema<T> {

private static final long serialVersionUID = 2651665280744549932L;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
* @param <T> The type of elements produced by the fetcher.
*/
@Internal
@Deprecated
public class KafkaFetcher<T> extends AbstractFetcher<T, TopicPartition> {

private static final Logger LOG = LoggerFactory.getLogger(KafkaFetcher.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
* brokers via the Kafka high-level consumer API.
*/
@Internal
@Deprecated
public class KafkaPartitionDiscoverer extends AbstractPartitionDiscoverer {

private final Properties kafkaProperties;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
* KafkaSerializationSchema}.
*/
@Internal
@Deprecated
public class KafkaSerializationSchemaWrapper<T>
implements KafkaSerializationSchema<T>, KafkaContextAware<T> {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@

/** Fetch data from Kafka for Kafka Shuffle. */
@Internal
@Deprecated
public class KafkaShuffleFetcher<T> extends KafkaFetcher<T> {
/** The handler to check and generate watermarks from fetched records. * */
private final WatermarkHandler watermarkHandler;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,13 @@
*
* <p>Note: This class must not change in its structure, because it would change the serialization
* format and make previous savepoints unreadable.
*
* @deprecated Will be turned into internal class when {@link
* org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer} is removed. Replace with
* {@link org.apache.kafka.common.TopicPartition}.
*/
@PublicEvolving
@Deprecated
public final class KafkaTopicPartition implements Serializable {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

/** Utility for assigning Kafka partitions to consumer subtasks. */
@Internal
@Deprecated
public class KafkaTopicPartitionAssigner {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
* Serializable Topic Partition info with leader Node information. This class is used at runtime.
*/
@Internal
@Deprecated
public class KafkaTopicPartitionLeader implements Serializable {

private static final long serialVersionUID = 9145855900303748582L;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
* @param <KPH> The type of the Kafka partition descriptor, which varies across Kafka versions.
*/
@Internal
@Deprecated
public class KafkaTopicPartitionState<T, KPH> {

// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
* @param <KPH> The type of the Kafka partition descriptor, which varies across Kafka versions.
*/
@Internal
@Deprecated
public final class KafkaTopicPartitionStateWithWatermarkGenerator<T, KPH>
extends KafkaTopicPartitionState<T, KPH> {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
* list of topics, or a topic pattern.
*/
@Internal
@Deprecated
public class KafkaTopicsDescriptor implements Serializable {

private static final long serialVersionUID = -3807227764764900975L;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
* @param <T> The type to serialize
*/
@Internal
@Deprecated
public class KeyedSerializationSchemaWrapper<T> implements KeyedSerializationSchema<T> {

private static final long serialVersionUID = 1351665280744549933L;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
* A {@link org.apache.flink.api.common.eventtime.WatermarkOutput} that forwards calls to a {@link
* org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext}.
*/
@Deprecated
public class SourceContextWatermarkOutputAdapter<T> implements WatermarkOutput {
private final SourceContext<T> sourceContext;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
* subtask.
*/
@Internal
@Deprecated
public class TransactionalIdsGenerator {
private final String prefix;
private final int subtaskIndex;
Expand Down
Loading
Loading