|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.connector.kafka.source; |
| 20 | + |
| 21 | +import org.apache.flink.api.connector.source.Boundedness; |
| 22 | +import org.apache.flink.api.connector.source.Source; |
| 23 | +import org.apache.flink.api.connector.source.SourceReader; |
| 24 | +import org.apache.flink.api.connector.source.SourceReaderContext; |
| 25 | +import org.apache.flink.api.connector.source.SplitEnumerator; |
| 26 | +import org.apache.flink.api.connector.source.SplitEnumeratorContext; |
| 27 | +import org.apache.flink.api.java.tuple.Tuple3; |
| 28 | +import org.apache.flink.configuration.Configuration; |
| 29 | +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; |
| 30 | +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; |
| 31 | +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; |
| 32 | +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumStateSerializer; |
| 33 | +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator; |
| 34 | +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; |
| 35 | +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; |
| 36 | +import org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader; |
| 37 | +import org.apache.flink.connector.kafka.source.reader.KafkaRecordEmitter; |
| 38 | +import org.apache.flink.connector.kafka.source.reader.KafkaSourceReader; |
| 39 | +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializer; |
| 40 | +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; |
| 41 | +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; |
| 42 | +import org.apache.flink.core.io.SimpleVersionedSerializer; |
| 43 | + |
| 44 | +import javax.annotation.Nullable; |
| 45 | + |
| 46 | +import java.io.IOException; |
| 47 | +import java.util.Properties; |
| 48 | +import java.util.function.Supplier; |
| 49 | + |
| 50 | +/** |
| 51 | + * The Source implementation of Kafka. Please use a {@link KafkaSourceBuilder} to construct a {@link KafkaSource}. |
| 52 | + * The following example shows how to create a KafkaSource emitting records of <code>String</code> type. |
| 53 | + * |
| 54 | + * <pre>{@code |
| 55 | + * KafkaSource<String> source = KafkaSource |
| 56 | + * .<String>builder() |
| 57 | + * .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings) |
| 58 | + * .setGroupId("MyGroup") |
| 59 | + * .setTopics(Arrays.asList(TOPIC1, TOPIC2)) |
| 60 | + * .setDeserializer(new TestingKafkaRecordDeserializer()) |
| 61 | + * .setStartingOffsetInitializer(OffsetsInitializer.earliest()) |
| 62 | + * .build(); |
| 63 | + * }</pre> |
| 64 | + * |
| 65 | + * <p>See {@link KafkaSourceBuilder} for more details. |
| 66 | + * |
| 67 | + * @param <OUT> the output type of the source. |
| 68 | + */ |
| 69 | +public class KafkaSource<OUT> implements Source<OUT, KafkaPartitionSplit, KafkaSourceEnumState> { |
| 70 | + private static final long serialVersionUID = -8755372893283732098L; |
| 71 | + // Users can choose only one of the following ways to specify the topics to consume from. |
| 72 | + private final KafkaSubscriber subscriber; |
| 73 | + // Users can specify the starting / stopping offset initializer. |
| 74 | + private final OffsetsInitializer startingOffsetsInitializer; |
| 75 | + private final OffsetsInitializer stoppingOffsetsInitializer; |
| 76 | + // Boundedness |
| 77 | + private final Boundedness boundedness; |
| 78 | + private final KafkaRecordDeserializer<OUT> deserializationSchema; |
| 79 | + // The configurations. |
| 80 | + private final Properties props; |
| 81 | + |
| 82 | + KafkaSource( |
| 83 | + KafkaSubscriber subscriber, |
| 84 | + OffsetsInitializer startingOffsetsInitializer, |
| 85 | + @Nullable OffsetsInitializer stoppingOffsetsInitializer, |
| 86 | + Boundedness boundedness, |
| 87 | + KafkaRecordDeserializer<OUT> deserializationSchema, |
| 88 | + Properties props) { |
| 89 | + this.subscriber = subscriber; |
| 90 | + this.startingOffsetsInitializer = startingOffsetsInitializer; |
| 91 | + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; |
| 92 | + this.boundedness = boundedness; |
| 93 | + this.deserializationSchema = deserializationSchema; |
| 94 | + this.props = props; |
| 95 | + } |
| 96 | + |
| 97 | + /** |
| 98 | + * Get a kafkaSourceBuilder to build a {@link KafkaSource}. |
| 99 | + * |
| 100 | + * @return a Kafka source builder. |
| 101 | + */ |
| 102 | + public static <OUT> KafkaSourceBuilder<OUT> builder() { |
| 103 | + return new KafkaSourceBuilder<>(); |
| 104 | + } |
| 105 | + |
| 106 | + @Override |
| 107 | + public Boundedness getBoundedness() { |
| 108 | + return this.boundedness; |
| 109 | + } |
| 110 | + |
| 111 | + @Override |
| 112 | + public SourceReader<OUT, KafkaPartitionSplit> createReader(SourceReaderContext readerContext) { |
| 113 | + FutureCompletingBlockingQueue<RecordsWithSplitIds<Tuple3<OUT, Long, Long>>> elementsQueue = |
| 114 | + new FutureCompletingBlockingQueue<>(); |
| 115 | + Supplier<KafkaPartitionSplitReader<OUT>> splitReaderSupplier = |
| 116 | + () -> new KafkaPartitionSplitReader<>( |
| 117 | + props, |
| 118 | + deserializationSchema, |
| 119 | + readerContext.getIndexOfSubtask()); |
| 120 | + KafkaRecordEmitter<OUT> recordEmitter = new KafkaRecordEmitter<>(); |
| 121 | + |
| 122 | + return new KafkaSourceReader<>( |
| 123 | + elementsQueue, |
| 124 | + splitReaderSupplier, |
| 125 | + recordEmitter, |
| 126 | + toConfiguration(props), |
| 127 | + readerContext); |
| 128 | + } |
| 129 | + |
| 130 | + @Override |
| 131 | + public SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState> createEnumerator( |
| 132 | + SplitEnumeratorContext<KafkaPartitionSplit> enumContext) { |
| 133 | + return new KafkaSourceEnumerator( |
| 134 | + subscriber, |
| 135 | + startingOffsetsInitializer, |
| 136 | + stoppingOffsetsInitializer, |
| 137 | + props, |
| 138 | + enumContext); |
| 139 | + } |
| 140 | + |
| 141 | + @Override |
| 142 | + public SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState> restoreEnumerator( |
| 143 | + SplitEnumeratorContext<KafkaPartitionSplit> enumContext, |
| 144 | + KafkaSourceEnumState checkpoint) throws IOException { |
| 145 | + return new KafkaSourceEnumerator( |
| 146 | + subscriber, |
| 147 | + startingOffsetsInitializer, |
| 148 | + stoppingOffsetsInitializer, |
| 149 | + props, |
| 150 | + enumContext, |
| 151 | + checkpoint.getCurrentAssignment()); |
| 152 | + } |
| 153 | + |
| 154 | + @Override |
| 155 | + public SimpleVersionedSerializer<KafkaPartitionSplit> getSplitSerializer() { |
| 156 | + return new KafkaPartitionSplitSerializer(); |
| 157 | + } |
| 158 | + |
| 159 | + @Override |
| 160 | + public SimpleVersionedSerializer<KafkaSourceEnumState> getEnumeratorCheckpointSerializer() { |
| 161 | + return new KafkaSourceEnumStateSerializer(); |
| 162 | + } |
| 163 | + |
| 164 | + // ----------- private helper methods --------------- |
| 165 | + |
| 166 | + private Configuration toConfiguration(Properties props) { |
| 167 | + Configuration config = new Configuration(); |
| 168 | + props.stringPropertyNames().forEach(key -> config.setString(key, props.getProperty(key))); |
| 169 | + return config; |
| 170 | + } |
| 171 | +} |
0 commit comments