Class HybridSource<T>
java.lang.Object
org.apache.flink.connector.base.source.hybrid.HybridSource<T>
- All Implemented Interfaces:
Serializable,org.apache.flink.api.connector.source.Source<T,,HybridSourceSplit, HybridSourceEnumeratorState> org.apache.flink.api.connector.source.SourceReaderFactory<T,HybridSourceSplit>
@PublicEvolving
public class HybridSource<T>
extends Object
implements org.apache.flink.api.connector.source.Source<T,HybridSourceSplit,HybridSourceEnumeratorState>
Hybrid source that switches underlying sources based on configured source chain.
A simple example with FileSource and KafkaSource with fixed Kafka start position:
FileSource<String> fileSource =
FileSource.forRecordStreamFormat(new TextLineInputFormat(), Path.fromLocalFile(testDir)).build();
KafkaSource<String> kafkaSource =
KafkaSource.<String>builder()
.setBootstrapServers("localhost:9092")
.setGroupId("MyGroup")
.setTopics(Arrays.asList("quickstart-events"))
.setDeserializer(
KafkaRecordDeserializer.valueOnly(StringDeserializer.class))
.setStartingOffsets(OffsetsInitializer.earliest())
.build();
HybridSource<String> hybridSource =
HybridSource.builder(fileSource)
.addSource(kafkaSource)
.build();
A more complex example with Kafka start position derived from previous source:
HybridSource<String> hybridSource =
HybridSource.<String, StaticFileSplitEnumerator>builder(fileSource)
.addSource(
switchContext -> {
StaticFileSplitEnumerator previousEnumerator =
switchContext.getPreviousEnumerator();
// how to get timestamp depends on specific enumerator
long timestamp = previousEnumerator.getEndTimestamp();
OffsetsInitializer offsets =
OffsetsInitializer.timestamp(timestamp);
KafkaSource<String> kafkaSource =
KafkaSource.<String>builder()
.setBootstrapServers("localhost:9092")
.setGroupId("MyGroup")
.setTopics(Arrays.asList("quickstart-events"))
.setDeserializer(
KafkaRecordDeserializer.valueOnly(StringDeserializer.class))
.setStartingOffsets(offsets)
.build();
return kafkaSource;
},
Boundedness.CONTINUOUS_UNBOUNDED)
.build();
- See Also:
-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionstatic classHybridSource.HybridSourceBuilder<T,EnumT extends org.apache.flink.api.connector.source.SplitEnumerator> Builder for HybridSource.static interfaceHybridSource.SourceFactory<T,SourceT extends org.apache.flink.api.connector.source.Source<T, ?, ?>, FromEnumT extends org.apache.flink.api.connector.source.SplitEnumerator> Factory for underlying sources ofHybridSource.static interfaceContext provided to source factory. -
Constructor Summary
ConstructorsModifierConstructorDescriptionprotectedHybridSource(List<org.apache.flink.connector.base.source.hybrid.HybridSource.SourceListEntry> sources) Protected for subclass, usebuilder(Source)to construct source. -
Method Summary
Modifier and TypeMethodDescriptionstatic <T,EnumT extends org.apache.flink.api.connector.source.SplitEnumerator>
HybridSource.HybridSourceBuilder<T,EnumT> builder(org.apache.flink.api.connector.source.Source<T, ?, ?> firstSource) Builder forHybridSource.org.apache.flink.api.connector.source.SplitEnumerator<HybridSourceSplit,HybridSourceEnumeratorState> createEnumerator(org.apache.flink.api.connector.source.SplitEnumeratorContext<HybridSourceSplit> enumContext) org.apache.flink.api.connector.source.SourceReader<T,HybridSourceSplit> createReader(org.apache.flink.api.connector.source.SourceReaderContext readerContext) org.apache.flink.api.connector.source.Boundednessorg.apache.flink.core.io.SimpleVersionedSerializer<HybridSourceEnumeratorState>org.apache.flink.core.io.SimpleVersionedSerializer<HybridSourceSplit>org.apache.flink.api.connector.source.SplitEnumerator<HybridSourceSplit,HybridSourceEnumeratorState> restoreEnumerator(org.apache.flink.api.connector.source.SplitEnumeratorContext<HybridSourceSplit> enumContext, HybridSourceEnumeratorState checkpoint) Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, waitMethods inherited from interface org.apache.flink.api.connector.source.Source
declareWatermarks
-
Constructor Details
-
HybridSource
protected HybridSource(List<org.apache.flink.connector.base.source.hybrid.HybridSource.SourceListEntry> sources) Protected for subclass, usebuilder(Source)to construct source.
-
-
Method Details
-
builder
public static <T,EnumT extends org.apache.flink.api.connector.source.SplitEnumerator> HybridSource.HybridSourceBuilder<T,EnumT> builder(org.apache.flink.api.connector.source.Source<T, ?, ?> firstSource) Builder forHybridSource. -
getBoundedness
public org.apache.flink.api.connector.source.Boundedness getBoundedness()- Specified by:
getBoundednessin interfaceorg.apache.flink.api.connector.source.Source<T,HybridSourceSplit, HybridSourceEnumeratorState>
-
createReader
@Internal public org.apache.flink.api.connector.source.SourceReader<T,HybridSourceSplit> createReader(org.apache.flink.api.connector.source.SourceReaderContext readerContext) throws Exception - Specified by:
createReaderin interfaceorg.apache.flink.api.connector.source.SourceReaderFactory<T,HybridSourceSplit> - Throws:
Exception
-
createEnumerator
@Internal public org.apache.flink.api.connector.source.SplitEnumerator<HybridSourceSplit,HybridSourceEnumeratorState> createEnumerator(org.apache.flink.api.connector.source.SplitEnumeratorContext<HybridSourceSplit> enumContext) - Specified by:
createEnumeratorin interfaceorg.apache.flink.api.connector.source.Source<T,HybridSourceSplit, HybridSourceEnumeratorState>
-
restoreEnumerator
@Internal public org.apache.flink.api.connector.source.SplitEnumerator<HybridSourceSplit,HybridSourceEnumeratorState> restoreEnumerator(org.apache.flink.api.connector.source.SplitEnumeratorContext<HybridSourceSplit> enumContext, HybridSourceEnumeratorState checkpoint) throws Exception - Specified by:
restoreEnumeratorin interfaceorg.apache.flink.api.connector.source.Source<T,HybridSourceSplit, HybridSourceEnumeratorState> - Throws:
Exception
-
getSplitSerializer
@Internal public org.apache.flink.core.io.SimpleVersionedSerializer<HybridSourceSplit> getSplitSerializer()- Specified by:
getSplitSerializerin interfaceorg.apache.flink.api.connector.source.Source<T,HybridSourceSplit, HybridSourceEnumeratorState>
-
getEnumeratorCheckpointSerializer
@Internal public org.apache.flink.core.io.SimpleVersionedSerializer<HybridSourceEnumeratorState> getEnumeratorCheckpointSerializer()- Specified by:
getEnumeratorCheckpointSerializerin interfaceorg.apache.flink.api.connector.source.Source<T,HybridSourceSplit, HybridSourceEnumeratorState>
-