SteNicholas commented on a change in pull request #15924: URL: https://github.com/apache/flink/pull/15924#discussion_r641153345
########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSource.java ########## @@ -0,0 +1,163 @@ +/* + * 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.base.source.hybrid; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; + +/** Hybrid source that switches underlying sources based on configured source chain. */ +@PublicEvolving +public class HybridSource<T> implements Source<T, HybridSourceSplit, HybridSourceEnumeratorState> { + + private final SourceChain<T, ? extends SourceSplit, ?> sourceChain; + + public HybridSource(SourceChain<T, ? extends SourceSplit, ?> sourceChain) { + Preconditions.checkArgument(!sourceChain.sources.isEmpty()); + for (int i = 0; i < sourceChain.sources.size() - 1; i++) { + Preconditions.checkArgument( + Boundedness.BOUNDED.equals(sourceChain.sources.get(i).f0.getBoundedness()), + "All sources except the final source need to be bounded."); + } + this.sourceChain = sourceChain; + } + + @Override + public Boundedness getBoundedness() { + return sourceChain.sources.get(sourceChain.sources.size() - 1).f0.getBoundedness(); + } + + @Override + public SourceReader<T, HybridSourceSplit> createReader(SourceReaderContext readerContext) + throws Exception { + List<SourceReader<T, ? extends SourceSplit>> readers = new ArrayList<>(); + for (Tuple2<Source<T, ? extends SourceSplit, ?>, ?> source : sourceChain.sources) { + readers.add(source.f0.createReader(readerContext)); + } + return new HybridSourceReader(readerContext, readers); + } + + @Override + public SplitEnumerator<HybridSourceSplit, HybridSourceEnumeratorState> createEnumerator( + SplitEnumeratorContext<HybridSourceSplit> enumContext) { + return new HybridSourceSplitEnumerator(enumContext, sourceChain); + } + + @Override + public SplitEnumerator<HybridSourceSplit, HybridSourceEnumeratorState> restoreEnumerator( + SplitEnumeratorContext<HybridSourceSplit> enumContext, + HybridSourceEnumeratorState checkpoint) + throws Exception { + return new HybridSourceSplitEnumerator( + enumContext, sourceChain, checkpoint.getCurrentSourceIndex()); + } + + @Override + public SimpleVersionedSerializer<HybridSourceSplit> getSplitSerializer() { + List<SimpleVersionedSerializer<SourceSplit>> serializers = new ArrayList<>(); + sourceChain.sources.forEach( + t -> serializers.add(castSerializer(t.f0.getSplitSerializer()))); + return new HybridSourceSplitSerializer(serializers); + } + + @Override + public SimpleVersionedSerializer<HybridSourceEnumeratorState> + getEnumeratorCheckpointSerializer() { + List<SimpleVersionedSerializer<Object>> serializers = new ArrayList<>(); + sourceChain.sources.forEach( + t -> serializers.add(castSerializer(t.f0.getEnumeratorCheckpointSerializer()))); + return new HybridSourceEnumeratorStateSerializer(serializers); + } + + private static <T> SimpleVersionedSerializer<T> castSerializer( + SimpleVersionedSerializer<? extends T> s) { + @SuppressWarnings("rawtypes") + SimpleVersionedSerializer s1 = s; + return s1; + } + + /** + * Converts checkpoint between sources to transfer end position to next source's start position. + * Only required for dynamic position transfer at time of switching, otherwise source can be + * preconfigured with a start position during job submission. + */ + public interface CheckpointConverter<InCheckpointT, OutCheckpointT> + extends Function<InCheckpointT, OutCheckpointT>, Serializable {} + + /** Chain of sources with option to convert start position at switch-time. */ + public static class SourceChain<T, SplitT extends SourceSplit, EnumChkT> + implements Serializable { + final List<Tuple2<Source<T, ? extends SourceSplit, ?>, CheckpointConverter<?, ?>>> sources; + + public SourceChain(Source<T, SplitT, EnumChkT> initialSource) { + this(concat(Collections.emptyList(), Tuple2.of(initialSource, null))); + } + + public static <T, SplitT extends SourceSplit, EnumChkT> SourceChain<T, SplitT, EnumChkT> of( Review comment: Does the `of` method need the `CheckpointConverter`? ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceReader.java ########## @@ -0,0 +1,213 @@ +/* + * 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.base.source.hybrid; + +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * Hybrid source reader that delegates to the actual source reader. + * + * <p>This reader is setup with a sequence of underlying source readers. At a given point in time, + * one of these readers is active. Underlying readers are opened and closed on demand as determined + * by the enumerator, which selects the active reader via {@link SwitchSourceEvent}. + * + * <p>When the underlying reader has consumed all input, {@link HybridSourceReader} sends {@link + * SourceReaderFinishedEvent} to the coordinator and waits for the {@link SwitchSourceEvent}. + */ +public class HybridSourceReader<T> implements SourceReader<T, HybridSourceSplit<?>> { + private static final Logger LOG = LoggerFactory.getLogger(HybridSourceReader.class); + private SourceReaderContext readerContext; + private List<SourceReader<T, ? extends SourceSplit>> realReaders; + private int currentSourceIndex = -1; + private long lastCheckpointId = -1; + private SourceReader<T, ? extends SourceSplit> currentReader; + // track last availability to resume reader after source switch + private CompletableFuture<Void> availabilityFuture; + + public HybridSourceReader( + SourceReaderContext readerContext, + List<SourceReader<T, ? extends SourceSplit>> readers) { + this.readerContext = readerContext; + this.realReaders = readers; + } + + @Override + public void start() { + setCurrentReader(0); + } + + @Override + public InputStatus pollNext(ReaderOutput output) throws Exception { + InputStatus status = currentReader.pollNext(output); + if (status == InputStatus.END_OF_INPUT) { + // trap END_OF_INPUT if this wasn't the final reader + LOG.debug( + "End of input subtask={} sourceIndex={} {}", + readerContext.getIndexOfSubtask(), + currentSourceIndex, + currentReader); + if (currentSourceIndex + 1 < realReaders.size()) { + // Signal the coordinator that the current reader has consumed all input and the + // next source can potentially be activated (after all readers are ready). + readerContext.sendSourceEventToCoordinator( + new SourceReaderFinishedEvent(currentSourceIndex, lastCheckpointId)); + // More data will be available from the next reader. + // InputStatus.NOTHING_AVAILABLE requires us to complete the availability + // future after source switch to resume poll. + return InputStatus.NOTHING_AVAILABLE; + } + } + return status; + } + + @Override + public List<HybridSourceSplit<?>> snapshotState(long checkpointId) { + this.lastCheckpointId = checkpointId; + List<? extends SourceSplit> state = currentReader.snapshotState(checkpointId); + return wrappedSplits(currentSourceIndex, state); + } + + public static List<HybridSourceSplit<?>> wrappedSplits( + int readerIndex, List<? extends SourceSplit> state) { + List<HybridSourceSplit<?>> wrappedSplits = new ArrayList<>(state.size()); + for (SourceSplit split : state) { + wrappedSplits.add(new HybridSourceSplit<>(readerIndex, split)); + } + return wrappedSplits; + } + + public static <SplitT extends SourceSplit> List<SplitT> unwrappedSplits( + List<HybridSourceSplit<SplitT>> splits) { + List<SplitT> unwrappedSplits = new ArrayList<>(splits.size()); + for (HybridSourceSplit<SplitT> split : splits) { + unwrappedSplits.add(split.getWrappedSplit()); + } + return unwrappedSplits; + } + + @Override + public CompletableFuture<Void> isAvailable() { + return availabilityFuture = currentReader.isAvailable(); Review comment: Why not directly return `currentReader.isAvailable()`? ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceReader.java ########## @@ -0,0 +1,213 @@ +/* + * 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.base.source.hybrid; + +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * Hybrid source reader that delegates to the actual source reader. + * + * <p>This reader is setup with a sequence of underlying source readers. At a given point in time, + * one of these readers is active. Underlying readers are opened and closed on demand as determined + * by the enumerator, which selects the active reader via {@link SwitchSourceEvent}. + * + * <p>When the underlying reader has consumed all input, {@link HybridSourceReader} sends {@link + * SourceReaderFinishedEvent} to the coordinator and waits for the {@link SwitchSourceEvent}. + */ +public class HybridSourceReader<T> implements SourceReader<T, HybridSourceSplit<?>> { + private static final Logger LOG = LoggerFactory.getLogger(HybridSourceReader.class); + private SourceReaderContext readerContext; + private List<SourceReader<T, ? extends SourceSplit>> realReaders; + private int currentSourceIndex = -1; + private long lastCheckpointId = -1; + private SourceReader<T, ? extends SourceSplit> currentReader; + // track last availability to resume reader after source switch + private CompletableFuture<Void> availabilityFuture; + + public HybridSourceReader( + SourceReaderContext readerContext, + List<SourceReader<T, ? extends SourceSplit>> readers) { + this.readerContext = readerContext; + this.realReaders = readers; + } + + @Override + public void start() { + setCurrentReader(0); + } + + @Override + public InputStatus pollNext(ReaderOutput output) throws Exception { + InputStatus status = currentReader.pollNext(output); + if (status == InputStatus.END_OF_INPUT) { + // trap END_OF_INPUT if this wasn't the final reader + LOG.debug( + "End of input subtask={} sourceIndex={} {}", + readerContext.getIndexOfSubtask(), + currentSourceIndex, + currentReader); + if (currentSourceIndex + 1 < realReaders.size()) { + // Signal the coordinator that the current reader has consumed all input and the + // next source can potentially be activated (after all readers are ready). + readerContext.sendSourceEventToCoordinator( + new SourceReaderFinishedEvent(currentSourceIndex, lastCheckpointId)); + // More data will be available from the next reader. + // InputStatus.NOTHING_AVAILABLE requires us to complete the availability + // future after source switch to resume poll. + return InputStatus.NOTHING_AVAILABLE; + } + } + return status; + } + + @Override + public List<HybridSourceSplit<?>> snapshotState(long checkpointId) { + this.lastCheckpointId = checkpointId; + List<? extends SourceSplit> state = currentReader.snapshotState(checkpointId); + return wrappedSplits(currentSourceIndex, state); + } + + public static List<HybridSourceSplit<?>> wrappedSplits( Review comment: Does the `wrappedSplits` update to `wrapSplits`? ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceReader.java ########## @@ -0,0 +1,213 @@ +/* + * 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.base.source.hybrid; + +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * Hybrid source reader that delegates to the actual source reader. + * + * <p>This reader is setup with a sequence of underlying source readers. At a given point in time, + * one of these readers is active. Underlying readers are opened and closed on demand as determined + * by the enumerator, which selects the active reader via {@link SwitchSourceEvent}. + * + * <p>When the underlying reader has consumed all input, {@link HybridSourceReader} sends {@link + * SourceReaderFinishedEvent} to the coordinator and waits for the {@link SwitchSourceEvent}. + */ +public class HybridSourceReader<T> implements SourceReader<T, HybridSourceSplit<?>> { + private static final Logger LOG = LoggerFactory.getLogger(HybridSourceReader.class); + private SourceReaderContext readerContext; + private List<SourceReader<T, ? extends SourceSplit>> realReaders; + private int currentSourceIndex = -1; + private long lastCheckpointId = -1; + private SourceReader<T, ? extends SourceSplit> currentReader; + // track last availability to resume reader after source switch + private CompletableFuture<Void> availabilityFuture; + + public HybridSourceReader( + SourceReaderContext readerContext, + List<SourceReader<T, ? extends SourceSplit>> readers) { + this.readerContext = readerContext; + this.realReaders = readers; + } + + @Override + public void start() { + setCurrentReader(0); + } + + @Override + public InputStatus pollNext(ReaderOutput output) throws Exception { + InputStatus status = currentReader.pollNext(output); + if (status == InputStatus.END_OF_INPUT) { + // trap END_OF_INPUT if this wasn't the final reader + LOG.debug( + "End of input subtask={} sourceIndex={} {}", + readerContext.getIndexOfSubtask(), + currentSourceIndex, + currentReader); + if (currentSourceIndex + 1 < realReaders.size()) { + // Signal the coordinator that the current reader has consumed all input and the + // next source can potentially be activated (after all readers are ready). + readerContext.sendSourceEventToCoordinator( + new SourceReaderFinishedEvent(currentSourceIndex, lastCheckpointId)); + // More data will be available from the next reader. + // InputStatus.NOTHING_AVAILABLE requires us to complete the availability + // future after source switch to resume poll. + return InputStatus.NOTHING_AVAILABLE; + } + } + return status; + } + + @Override + public List<HybridSourceSplit<?>> snapshotState(long checkpointId) { + this.lastCheckpointId = checkpointId; + List<? extends SourceSplit> state = currentReader.snapshotState(checkpointId); + return wrappedSplits(currentSourceIndex, state); + } + + public static List<HybridSourceSplit<?>> wrappedSplits( + int readerIndex, List<? extends SourceSplit> state) { + List<HybridSourceSplit<?>> wrappedSplits = new ArrayList<>(state.size()); + for (SourceSplit split : state) { + wrappedSplits.add(new HybridSourceSplit<>(readerIndex, split)); + } + return wrappedSplits; + } + + public static <SplitT extends SourceSplit> List<SplitT> unwrappedSplits( + List<HybridSourceSplit<SplitT>> splits) { + List<SplitT> unwrappedSplits = new ArrayList<>(splits.size()); + for (HybridSourceSplit<SplitT> split : splits) { + unwrappedSplits.add(split.getWrappedSplit()); + } + return unwrappedSplits; + } + + @Override + public CompletableFuture<Void> isAvailable() { + return availabilityFuture = currentReader.isAvailable(); + } + + @Override + public void addSplits(List<HybridSourceSplit<?>> splits) { + LOG.info( + "Adding splits subtask={} sourceIndex={} {} {}", + readerContext.getIndexOfSubtask(), + currentSourceIndex, + currentReader, + splits); + List<SourceSplit> realSplits = new ArrayList<>(splits.size()); + for (HybridSourceSplit<?> split : splits) { + Preconditions.checkState( + split.sourceIndex() == currentSourceIndex, + "Split %s while current source is %s", + split, + currentSourceIndex); + realSplits.add(split.getWrappedSplit()); + } + currentReader.addSplits((List) realSplits); + } + + @Override + public void notifyNoMoreSplits() { + currentReader.notifyNoMoreSplits(); + LOG.debug( + "No more splits for reader subtask={} sourceIndex={} {}", + readerContext.getIndexOfSubtask(), + currentSourceIndex, + currentReader); + } + + @Override + public void handleSourceEvents(SourceEvent sourceEvent) { + if (sourceEvent instanceof SwitchSourceEvent) { + SwitchSourceEvent sse = (SwitchSourceEvent) sourceEvent; + LOG.debug( + "Switch source event: subtask={} sourceIndex={}", + readerContext.getIndexOfSubtask(), + sse.sourceIndex()); + setCurrentReader(sse.sourceIndex()); + if (availabilityFuture != null && !availabilityFuture.isDone()) { + // continue polling + availabilityFuture.complete(null); + } + } else { + currentReader.handleSourceEvents(sourceEvent); + } + } + + @Override + public void close() throws Exception { + currentReader.close(); + LOG.debug( + "Reader closed: subtask={} sourceIndex={} {}", + readerContext.getIndexOfSubtask(), + currentSourceIndex, + currentReader); + } + + private void setCurrentReader(int index) { + Preconditions.checkState( + currentSourceIndex <= index, "reader index monotonically increasing"); + Preconditions.checkState(index < realReaders.size(), "invalid reader index: %s", index); + if (currentSourceIndex == index) { + LOG.debug( + "Reader already set to process source: subtask={} sourceIndex={} {}", + readerContext.getIndexOfSubtask(), + currentSourceIndex); Review comment: Is this lack of `currentReader`? ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceEnumeratorState.java ########## @@ -0,0 +1,38 @@ +/* + * 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.base.source.hybrid; + +/** The state of hybrid source enumerator. */ +public class HybridSourceEnumeratorState { + private final int currentSourceIndex; + private final Object wrappedState; Review comment: Could the `wrappedState` update to `enumeratorState`? ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitSerializer.java ########## @@ -0,0 +1,86 @@ +/* + * 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.base.source.hybrid; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.Preconditions; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.List; + +/** Serializes splits by delegating to the source-indexed underlying split serializer. */ +public class HybridSourceSplitSerializer implements SimpleVersionedSerializer<HybridSourceSplit> { + + final List<? extends SimpleVersionedSerializer<SourceSplit>> serializers; + + public HybridSourceSplitSerializer( + List<? extends SimpleVersionedSerializer<SourceSplit>> serializers) { + this.serializers = serializers; + } + + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(HybridSourceSplit split) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + out.writeInt(split.sourceIndex()); + out.writeInt(serializerOf(split.sourceIndex()).getVersion()); + byte[] serializedSplit = + serializerOf(split.sourceIndex()).serialize(split.getWrappedSplit()); + out.writeInt(serializedSplit.length); + out.write(serializedSplit); + return baos.toByteArray(); + } + } + + @Override + public HybridSourceSplit deserialize(int version, byte[] serialized) throws IOException { + if (version == 0) { + return deserializeV0(version, serialized); + } + throw new AssertionError(String.format("Invalid version %d", version)); + } + + private HybridSourceSplit deserializeV0(int version, byte[] serialized) throws IOException { Review comment: Should the `deserializeV0` method check the version? ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceReader.java ########## @@ -0,0 +1,213 @@ +/* + * 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.base.source.hybrid; + +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * Hybrid source reader that delegates to the actual source reader. + * + * <p>This reader is setup with a sequence of underlying source readers. At a given point in time, + * one of these readers is active. Underlying readers are opened and closed on demand as determined + * by the enumerator, which selects the active reader via {@link SwitchSourceEvent}. + * + * <p>When the underlying reader has consumed all input, {@link HybridSourceReader} sends {@link + * SourceReaderFinishedEvent} to the coordinator and waits for the {@link SwitchSourceEvent}. + */ +public class HybridSourceReader<T> implements SourceReader<T, HybridSourceSplit<?>> { + private static final Logger LOG = LoggerFactory.getLogger(HybridSourceReader.class); + private SourceReaderContext readerContext; + private List<SourceReader<T, ? extends SourceSplit>> realReaders; + private int currentSourceIndex = -1; + private long lastCheckpointId = -1; + private SourceReader<T, ? extends SourceSplit> currentReader; + // track last availability to resume reader after source switch + private CompletableFuture<Void> availabilityFuture; + + public HybridSourceReader( + SourceReaderContext readerContext, + List<SourceReader<T, ? extends SourceSplit>> readers) { + this.readerContext = readerContext; + this.realReaders = readers; + } + + @Override + public void start() { + setCurrentReader(0); + } + + @Override + public InputStatus pollNext(ReaderOutput output) throws Exception { + InputStatus status = currentReader.pollNext(output); + if (status == InputStatus.END_OF_INPUT) { + // trap END_OF_INPUT if this wasn't the final reader + LOG.debug( + "End of input subtask={} sourceIndex={} {}", + readerContext.getIndexOfSubtask(), + currentSourceIndex, + currentReader); + if (currentSourceIndex + 1 < realReaders.size()) { + // Signal the coordinator that the current reader has consumed all input and the + // next source can potentially be activated (after all readers are ready). + readerContext.sendSourceEventToCoordinator( + new SourceReaderFinishedEvent(currentSourceIndex, lastCheckpointId)); + // More data will be available from the next reader. + // InputStatus.NOTHING_AVAILABLE requires us to complete the availability + // future after source switch to resume poll. + return InputStatus.NOTHING_AVAILABLE; + } + } + return status; + } + + @Override + public List<HybridSourceSplit<?>> snapshotState(long checkpointId) { + this.lastCheckpointId = checkpointId; + List<? extends SourceSplit> state = currentReader.snapshotState(checkpointId); + return wrappedSplits(currentSourceIndex, state); + } + + public static List<HybridSourceSplit<?>> wrappedSplits( + int readerIndex, List<? extends SourceSplit> state) { + List<HybridSourceSplit<?>> wrappedSplits = new ArrayList<>(state.size()); + for (SourceSplit split : state) { + wrappedSplits.add(new HybridSourceSplit<>(readerIndex, split)); + } + return wrappedSplits; + } + + public static <SplitT extends SourceSplit> List<SplitT> unwrappedSplits( + List<HybridSourceSplit<SplitT>> splits) { + List<SplitT> unwrappedSplits = new ArrayList<>(splits.size()); + for (HybridSourceSplit<SplitT> split : splits) { + unwrappedSplits.add(split.getWrappedSplit()); + } + return unwrappedSplits; + } + + @Override + public CompletableFuture<Void> isAvailable() { + return availabilityFuture = currentReader.isAvailable(); + } + + @Override + public void addSplits(List<HybridSourceSplit<?>> splits) { + LOG.info( + "Adding splits subtask={} sourceIndex={} {} {}", + readerContext.getIndexOfSubtask(), + currentSourceIndex, + currentReader, + splits); + List<SourceSplit> realSplits = new ArrayList<>(splits.size()); + for (HybridSourceSplit<?> split : splits) { + Preconditions.checkState( + split.sourceIndex() == currentSourceIndex, + "Split %s while current source is %s", + split, + currentSourceIndex); + realSplits.add(split.getWrappedSplit()); + } + currentReader.addSplits((List) realSplits); + } + + @Override + public void notifyNoMoreSplits() { + currentReader.notifyNoMoreSplits(); + LOG.debug( + "No more splits for reader subtask={} sourceIndex={} {}", Review comment: Could the log message be `No more splits for reader subtask={} sourceIndex={} currentReader={}`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
