From a4a4e5bb1daeec7089b937e19cab248a35fa8ff0 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Wed, 11 May 2022 09:59:36 +0200 Subject: [PATCH 01/12] DataGeneratorSource that is composed of NumberSequenceSource. --- .../source/lib/DataGeneratorSource.java | 304 ++++++++++++++++++ 1 file changed, 304 insertions(+) create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java new file mode 100644 index 0000000000000..142cffa28519f --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java @@ -0,0 +1,304 @@ +/* + * 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.api.connector.source.lib; + +import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +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.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.lib.DataGeneratorSource.GeneratorSequenceSplit; +import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.NumberSequenceIterator; + +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A data source that produces generators N data points in parallel. This source is useful for + * testing and for cases that just need a stream of N events of any kind. + * + *

The source splits the sequence into as many parallel sub-sequences as there are parallel + * source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is + * limited to one, this will produce one sequence in order. + * + *

This source is always bounded. For very long sequences (for example over the entire domain of + * long integer values), user may want to consider executing the application in a streaming manner, + * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. + */ +@Public +public class DataGeneratorSource + implements Source< + OUT, GeneratorSequenceSplit, Collection>>, + ResultTypeQueryable { + + private static final long serialVersionUID = 1L; + + private final TypeInformation typeInfo; + + public final MapFunction generatorFunction; + + /** The end Generator in the sequence, inclusive. */ + private final NumberSequenceSource numberSource; + + /** + * Creates a new {@code DataGeneratorSource} that produces count records in + * parallel. + * + * @param typeInfo the type info + * @param generatorFunction the generator function + * @param count The count + */ + public DataGeneratorSource( + MapFunction generatorFunction, long count, TypeInformation typeInfo) { + this.typeInfo = checkNotNull(typeInfo); + this.generatorFunction = checkNotNull(generatorFunction); + this.numberSource = new NumberSequenceSource(0, count); + } + + public long getCount() { + return numberSource.getTo(); + } + + // ------------------------------------------------------------------------ + // source methods + // ------------------------------------------------------------------------ + + @Override + public TypeInformation getProducedType() { + return typeInfo; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader> createReader( + SourceReaderContext readerContext) { + return new IteratorSourceReader<>(readerContext); + } + + @Override + public SplitEnumerator, Collection>> + createEnumerator( + final SplitEnumeratorContext> enumContext) { + + final List splits = + numberSource.splitNumberRange(0, getCount(), enumContext.currentParallelism()); + return new IteratorSourceEnumerator<>(enumContext, wrapSplits(splits, generatorFunction)); + } + + @Override + public SplitEnumerator, Collection>> + restoreEnumerator( + final SplitEnumeratorContext> enumContext, + Collection> checkpoint) { + return new IteratorSourceEnumerator<>(enumContext, checkpoint); + } + + @Override + public SimpleVersionedSerializer> getSplitSerializer() { + return new SplitSerializer<>(numberSource.getSplitSerializer(), generatorFunction); + } + + @Override + public SimpleVersionedSerializer>> + getEnumeratorCheckpointSerializer() { + return new CheckpointSerializer<>( + numberSource.getEnumeratorCheckpointSerializer(), generatorFunction); + } + + // ------------------------------------------------------------------------ + // splits & checkpoint + // ------------------------------------------------------------------------ + public static class GeneratorSequenceIterator implements Iterator { + + private final MapFunction generatorFunction; + private final NumberSequenceIterator numSeqIterator; + + public GeneratorSequenceIterator( + NumberSequenceIterator numSeqIterator, MapFunction generatorFunction) { + this.generatorFunction = generatorFunction; + this.numSeqIterator = numSeqIterator; + } + + @Override + public boolean hasNext() { + return numSeqIterator.hasNext(); + } + + public long getCurrent() { + return numSeqIterator.getCurrent(); + } + + public long getTo() { + return numSeqIterator.getTo(); + } + + @Override + public T next() { + try { + return generatorFunction.map(numSeqIterator.next()); + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format( + "Exception while generating element %d", + numSeqIterator.getCurrent()), + e); + } + } + } + + /** A split of the source, representing a Generator sub-sequence. */ + public static class GeneratorSequenceSplit + implements IteratorSourceSplit> { + + public GeneratorSequenceSplit( + NumberSequenceSplit numberSequenceSplit, MapFunction generatorFunction) { + this.numberSequenceSplit = numberSequenceSplit; + this.generatorFunction = generatorFunction; + } + + private final NumberSequenceSplit numberSequenceSplit; + + private final MapFunction generatorFunction; + + public GeneratorSequenceIterator getIterator() { + return new GeneratorSequenceIterator<>( + numberSequenceSplit.getIterator(), generatorFunction); + } + + @Override + public String splitId() { + return numberSequenceSplit.splitId(); + } + + @Override + public IteratorSourceSplit> getUpdatedSplitForIterator( + final GeneratorSequenceIterator iterator) { + return new GeneratorSequenceSplit<>( + (NumberSequenceSplit) + numberSequenceSplit.getUpdatedSplitForIterator(iterator.numSeqIterator), + generatorFunction); + } + + @Override + public String toString() { + return String.format( + "GeneratorSequenceSplit [%d, %d] (%s)", + numberSequenceSplit.from(), + numberSequenceSplit.to(), + numberSequenceSplit.splitId()); + } + } + + private static final class SplitSerializer + implements SimpleVersionedSerializer> { + + private final SimpleVersionedSerializer numberSplitSerializer; + private final MapFunction generatorFunction; + + private SplitSerializer( + SimpleVersionedSerializer numberSplitSerializer, + MapFunction generatorFunction) { + this.numberSplitSerializer = numberSplitSerializer; + this.generatorFunction = generatorFunction; + } + + @Override + public int getVersion() { + return numberSplitSerializer.getVersion(); + } + + @Override + public byte[] serialize(GeneratorSequenceSplit split) throws IOException { + return numberSplitSerializer.serialize(split.numberSequenceSplit); + } + + @Override + public GeneratorSequenceSplit deserialize(int version, byte[] serialized) + throws IOException { + return new GeneratorSequenceSplit<>( + numberSplitSerializer.deserialize(version, serialized), generatorFunction); + } + } + + private static final class CheckpointSerializer + implements SimpleVersionedSerializer>> { + + private final SimpleVersionedSerializer> + numberCheckpointSerializer; + private final MapFunction generatorFunction; + + public CheckpointSerializer( + SimpleVersionedSerializer> + numberCheckpointSerializer, + MapFunction generatorFunction) { + this.numberCheckpointSerializer = numberCheckpointSerializer; + this.generatorFunction = generatorFunction; + } + + @Override + public int getVersion() { + return numberCheckpointSerializer.getVersion(); + } + + @Override + public byte[] serialize(Collection> checkpoint) + throws IOException { + return numberCheckpointSerializer.serialize( + checkpoint.stream() + .map(split -> split.numberSequenceSplit) + .collect(Collectors.toList())); + } + + @Override + public Collection> deserialize(int version, byte[] serialized) + throws IOException { + Collection numberSequenceSplits = + numberCheckpointSerializer.deserialize(version, serialized); + return wrapSplits(numberSequenceSplits, generatorFunction); + } + } + + private static List> wrapSplits( + Collection numberSequenceSplits, + MapFunction generatorFunction) { + return numberSequenceSplits.stream() + .map(split -> new GeneratorSequenceSplit<>(split, generatorFunction)) + .collect(Collectors.toList()); + } +} From 357c036a2fc931aa5ecec706573b07832273e7ea Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 10 Jun 2022 13:06:46 +0200 Subject: [PATCH 02/12] GeneratorSourceCheck --- .../wordcount/GeneratorSourceCheck.java | 85 +++++++++++++++++++ 1 file changed, 85 insertions(+) create mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java new file mode 100644 index 0000000000000..6c91dcfb41dc6 --- /dev/null +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java @@ -0,0 +1,85 @@ +/* + * 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.streaming.examples.wordcount; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.source.lib.DataGeneratorSource; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import java.time.Duration; + +public class GeneratorSourceCheck { + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + env.setParallelism(2); + + // MapFunction generator = value -> ">>> " + value; + // DataGeneratorSource source = new DataGeneratorSource<>(generator, 10, + // Types.STRING); + // DataStreamSource watermarked = + // env.fromSource(source, WatermarkStrategy.noWatermarks(), "watermarked"); + // watermarked.print(); + + MapFunction generator = index -> ">>> " + index; + DataGeneratorSource source = new DataGeneratorSource<>(generator, 10, Types.STRING); + DataStreamSource watermarked = + env.fromSource( + source, + WatermarkStrategy.forBoundedOutOfOrderness(Duration.ofSeconds(1)), + "watermarked"); + watermarked.print(); + + env.fromSequence(1, 10).print(); + + // DataStreamSource ds = env.fromGenerator(generator, 10, Types.STRING); + // ds.print(); + + /* MapFunction generator = value -> ">>> " + value; + GeneratorSource from = GeneratorSource.from(generator, 10, Types.STRING); + DataStreamSource watermarked = + env.fromSource(from, WatermarkStrategy.noWatermarks(), "watermarked"); + watermarked.print();*/ + + // DataStreamSource ds = env.fromGenerator(generator, 10, Types.STRING); + // ds.print(); + + // DataStreamSource longDataStreamSource = env.fromSequence(0, 10); + // longDataStreamSource.print(); + + // --- + // MapFunction generator2 = value -> ">>>>>> " + value; + // SingleOutputStreamOperator ds2 = env.fromFunction(generator2, 10); + // ds2.print(); + + // Apache Flink applications are composed lazily. Calling execute + // submits the Job and begins processing. + env.execute("WordCount"); + } +} From 42f1a2a5bf4f6ed2548beb705eccd65795333b26 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Thu, 30 Jun 2022 20:41:03 +0200 Subject: [PATCH 03/12] WIP: RateLimiter --- .../source/lib/DataGeneratorSource.java | 75 +++++++++++++---- .../source/lib/util/NoOpRateLimiter.java | 33 ++++++++ .../source/lib/util/RateLimiter.java | 34 ++++++++ .../source/lib/util/SimpleRateLimiter.java | 80 +++++++++++++++++++ .../wordcount/GeneratorSourceCheck.java | 11 ++- 5 files changed, 215 insertions(+), 18 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/NoOpRateLimiter.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimiter.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/SimpleRateLimiter.java diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java index 142cffa28519f..696d838e6d8e8 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java @@ -32,11 +32,16 @@ import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; +import org.apache.flink.api.connector.source.lib.util.NoOpRateLimiter; +import org.apache.flink.api.connector.source.lib.util.RateLimiter; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.NumberSequenceIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.util.Collection; import java.util.Iterator; @@ -63,15 +68,19 @@ public class DataGeneratorSource OUT, GeneratorSequenceSplit, Collection>>, ResultTypeQueryable { + private static final Logger LOG = LoggerFactory.getLogger(DataGeneratorSource.class); + private static final long serialVersionUID = 1L; private final TypeInformation typeInfo; - public final MapFunction generatorFunction; + private final MapFunction generatorFunction; /** The end Generator in the sequence, inclusive. */ private final NumberSequenceSource numberSource; + private RateLimiter rateLimiter = new NoOpRateLimiter(); + /** * Creates a new {@code DataGeneratorSource} that produces count records in * parallel. @@ -87,6 +96,18 @@ public DataGeneratorSource( this.numberSource = new NumberSequenceSource(0, count); } + public DataGeneratorSource( + MapFunction generatorFunction, + long count, + RateLimiter rateLimiter, + TypeInformation typeInfo) { + // TODO: checkArgument(maxPerSecond > 0, "maxPerSeconds has to be a positive number"); + this.typeInfo = checkNotNull(typeInfo); + this.generatorFunction = checkNotNull(generatorFunction); + this.numberSource = new NumberSequenceSource(0, count); + this.rateLimiter = rateLimiter; + } + public long getCount() { return numberSource.getTo(); } @@ -116,9 +137,12 @@ public SourceReader> createReader( createEnumerator( final SplitEnumeratorContext> enumContext) { + final int parallelism = enumContext.currentParallelism(); final List splits = - numberSource.splitNumberRange(0, getCount(), enumContext.currentParallelism()); - return new IteratorSourceEnumerator<>(enumContext, wrapSplits(splits, generatorFunction)); + numberSource.splitNumberRange(0, getCount(), parallelism); + + return new IteratorSourceEnumerator<>( + enumContext, wrapSplits(splits, generatorFunction, rateLimiter)); } @Override @@ -131,14 +155,15 @@ public SourceReader> createReader( @Override public SimpleVersionedSerializer> getSplitSerializer() { - return new SplitSerializer<>(numberSource.getSplitSerializer(), generatorFunction); + return new SplitSerializer<>( + numberSource.getSplitSerializer(), generatorFunction, rateLimiter); } @Override public SimpleVersionedSerializer>> getEnumeratorCheckpointSerializer() { return new CheckpointSerializer<>( - numberSource.getEnumeratorCheckpointSerializer(), generatorFunction); + numberSource.getEnumeratorCheckpointSerializer(), generatorFunction, rateLimiter); } // ------------------------------------------------------------------------ @@ -148,11 +173,15 @@ public static class GeneratorSequenceIterator implements Iterator { private final MapFunction generatorFunction; private final NumberSequenceIterator numSeqIterator; + private final RateLimiter rateLimiter; public GeneratorSequenceIterator( - NumberSequenceIterator numSeqIterator, MapFunction generatorFunction) { + NumberSequenceIterator numSeqIterator, + MapFunction generatorFunction, + RateLimiter rateLimiter) { this.generatorFunction = generatorFunction; this.numSeqIterator = numSeqIterator; + this.rateLimiter = rateLimiter; } @Override @@ -171,6 +200,8 @@ public long getTo() { @Override public T next() { try { + LOG.error("NEXT!"); + rateLimiter.acquire(); return generatorFunction.map(numSeqIterator.next()); } catch (Exception e) { throw new FlinkRuntimeException( @@ -187,18 +218,22 @@ public static class GeneratorSequenceSplit implements IteratorSourceSplit> { public GeneratorSequenceSplit( - NumberSequenceSplit numberSequenceSplit, MapFunction generatorFunction) { + NumberSequenceSplit numberSequenceSplit, + MapFunction generatorFunction, + RateLimiter rateLimiter) { this.numberSequenceSplit = numberSequenceSplit; this.generatorFunction = generatorFunction; + this.rateLimiter = rateLimiter; } private final NumberSequenceSplit numberSequenceSplit; private final MapFunction generatorFunction; + private final RateLimiter rateLimiter; public GeneratorSequenceIterator getIterator() { return new GeneratorSequenceIterator<>( - numberSequenceSplit.getIterator(), generatorFunction); + numberSequenceSplit.getIterator(), generatorFunction, rateLimiter); } @Override @@ -212,7 +247,8 @@ public IteratorSourceSplit> getUpdatedSplitForIt return new GeneratorSequenceSplit<>( (NumberSequenceSplit) numberSequenceSplit.getUpdatedSplitForIterator(iterator.numSeqIterator), - generatorFunction); + generatorFunction, + rateLimiter); } @Override @@ -230,12 +266,15 @@ private static final class SplitSerializer private final SimpleVersionedSerializer numberSplitSerializer; private final MapFunction generatorFunction; + private final RateLimiter rateLimiter; private SplitSerializer( SimpleVersionedSerializer numberSplitSerializer, - MapFunction generatorFunction) { + MapFunction generatorFunction, + RateLimiter rateLimiter) { this.numberSplitSerializer = numberSplitSerializer; this.generatorFunction = generatorFunction; + this.rateLimiter = rateLimiter; } @Override @@ -252,7 +291,9 @@ public byte[] serialize(GeneratorSequenceSplit split) throws IOException { public GeneratorSequenceSplit deserialize(int version, byte[] serialized) throws IOException { return new GeneratorSequenceSplit<>( - numberSplitSerializer.deserialize(version, serialized), generatorFunction); + numberSplitSerializer.deserialize(version, serialized), + generatorFunction, + rateLimiter); } } @@ -262,13 +303,16 @@ private static final class CheckpointSerializer private final SimpleVersionedSerializer> numberCheckpointSerializer; private final MapFunction generatorFunction; + private final RateLimiter throttler; public CheckpointSerializer( SimpleVersionedSerializer> numberCheckpointSerializer, - MapFunction generatorFunction) { + MapFunction generatorFunction, + RateLimiter throttler) { this.numberCheckpointSerializer = numberCheckpointSerializer; this.generatorFunction = generatorFunction; + this.throttler = throttler; } @Override @@ -290,15 +334,16 @@ public Collection> deserialize(int version, byte[] ser throws IOException { Collection numberSequenceSplits = numberCheckpointSerializer.deserialize(version, serialized); - return wrapSplits(numberSequenceSplits, generatorFunction); + return wrapSplits(numberSequenceSplits, generatorFunction, throttler); } } private static List> wrapSplits( Collection numberSequenceSplits, - MapFunction generatorFunction) { + MapFunction generatorFunction, + RateLimiter throttler) { return numberSequenceSplits.stream() - .map(split -> new GeneratorSequenceSplit<>(split, generatorFunction)) + .map(split -> new GeneratorSequenceSplit<>(split, generatorFunction, throttler)) .collect(Collectors.toList()); } } diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/NoOpRateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/NoOpRateLimiter.java new file mode 100644 index 0000000000000..9ba4f69e44f8b --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/NoOpRateLimiter.java @@ -0,0 +1,33 @@ +/* + * 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.api.connector.source.lib.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class NoOpRateLimiter implements RateLimiter { + + private static final Logger LOG = LoggerFactory.getLogger(NoOpRateLimiter.class); + + @Override + public int acquire() throws InterruptedException { + LOG.error("!!!THROTTLE!"); + return 0; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimiter.java new file mode 100644 index 0000000000000..dcd11b0d4c932 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimiter.java @@ -0,0 +1,34 @@ +/* + * 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.api.connector.source.lib.util; + +import java.io.Serializable; + +/** The interface that can be used to throttle execution of methods. */ +public interface RateLimiter extends Serializable { + + /** + * Acquire method is a blocking call that is intended to be used in places where it is required + * to limit the rate at which results are produced or other functions are called. + * + * @return The number of milliseconds this call blocked its caller. + * @throws InterruptedException The interrupted exception. + */ + int acquire() throws InterruptedException; +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/SimpleRateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/SimpleRateLimiter.java new file mode 100644 index 0000000000000..8dfc905af12d2 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/SimpleRateLimiter.java @@ -0,0 +1,80 @@ +/* + * 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.api.connector.source.lib.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** Utility to throttle a thread to a given number of executions (records) per second. */ +public final class SimpleRateLimiter implements RateLimiter { + + private static final Logger LOG = LoggerFactory.getLogger(SimpleRateLimiter.class); + + private long maxPerBucket; + private long nanosInBucket; + + private long endOfCurrentBucketNanos; + private int inCurrentBucket; + + private static final int DEFAULT_BUCKETS_PER_SECOND = 10; + private static final long NANOS_IN_ONE_SECOND = 1_000_000_000L; + + public SimpleRateLimiter(long maxPerSecond, int numParallelExecutors) { + this(maxPerSecond, numParallelExecutors, DEFAULT_BUCKETS_PER_SECOND); + } + + public SimpleRateLimiter(long maxPerSecond, int numParallelExecutors, int bucketsPerSecond) { + checkArgument(maxPerSecond > 0, "maxPerSecond must be a positive number"); + checkArgument(numParallelExecutors > 0, "numParallelExecutors must be greater than 0"); + + final float maxPerSecondPerSubtask = (float) maxPerSecond / numParallelExecutors; + + maxPerBucket = ((int) (maxPerSecondPerSubtask / bucketsPerSecond)) + 1; + nanosInBucket = ((int) (NANOS_IN_ONE_SECOND / maxPerSecondPerSubtask)) * maxPerBucket; + + this.endOfCurrentBucketNanos = System.nanoTime() + nanosInBucket; + this.inCurrentBucket = 0; + } + + // TODO: JavaDoc, returns number of seconds idling on this call. + public int acquire() throws InterruptedException { + LOG.error("THROTTLE!"); + if (++inCurrentBucket != maxPerBucket) { + return 0; + } + // The current bucket is "full". Wait until the next bucket. + final long now = System.nanoTime(); + final int millisRemaining = (int) ((endOfCurrentBucketNanos - now) / 1_000_000); + inCurrentBucket = 0; + + if (millisRemaining > 0) { + endOfCurrentBucketNanos += nanosInBucket; + Thread.sleep(millisRemaining); + return millisRemaining; + } else { + // Throttle was not called often enough so that the bucket's capacity was not exhausted + // "in time". We need to push the bucket's "end time" further to compensate and avoid + // bursts in case polling behaviour catches up. + endOfCurrentBucketNanos = now + nanosInBucket; + return 0; + } + } +} diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java index 6c91dcfb41dc6..e191494fb8aee 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java @@ -23,6 +23,8 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.connector.source.lib.DataGeneratorSource; +import org.apache.flink.api.connector.source.lib.util.RateLimiter; +import org.apache.flink.api.connector.source.lib.util.SimpleRateLimiter; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -37,8 +39,9 @@ public class GeneratorSourceCheck { public static void main(String[] args) throws Exception { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + int parallelism = 2; env.setRuntimeMode(RuntimeExecutionMode.STREAMING); - env.setParallelism(2); + env.setParallelism(parallelism); // MapFunction generator = value -> ">>> " + value; // DataGeneratorSource source = new DataGeneratorSource<>(generator, 10, @@ -48,7 +51,9 @@ public static void main(String[] args) throws Exception { // watermarked.print(); MapFunction generator = index -> ">>> " + index; - DataGeneratorSource source = new DataGeneratorSource<>(generator, 10, Types.STRING); + RateLimiter throttler = new SimpleRateLimiter(1, parallelism); + DataGeneratorSource source = + new DataGeneratorSource<>(generator, 100, throttler, Types.STRING); DataStreamSource watermarked = env.fromSource( source, @@ -56,7 +61,7 @@ public static void main(String[] args) throws Exception { "watermarked"); watermarked.print(); - env.fromSequence(1, 10).print(); + // env.fromSequence(1, 10).print(); // DataStreamSource ds = env.fromGenerator(generator, 10, Types.STRING); // ds.print(); From 71ac36d649b1ff184e3336baaf7d645de78a07bf Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 1 Jul 2022 01:03:02 +0200 Subject: [PATCH 04/12] WIP: parallelism in Splits --- .../source/lib/DataGeneratorSource.java | 181 ++++++++++++------ 1 file changed, 125 insertions(+), 56 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java index 696d838e6d8e8..18bac125d4895 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java @@ -32,10 +32,14 @@ import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; -import org.apache.flink.api.connector.source.lib.util.NoOpRateLimiter; import org.apache.flink.api.connector.source.lib.util.RateLimiter; +import org.apache.flink.api.connector.source.lib.util.SimpleRateLimiter; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.NumberSequenceIterator; @@ -43,11 +47,14 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -79,7 +86,7 @@ public class DataGeneratorSource /** The end Generator in the sequence, inclusive. */ private final NumberSequenceSource numberSource; - private RateLimiter rateLimiter = new NoOpRateLimiter(); + private long maxPerSecond = -1; /** * Creates a new {@code DataGeneratorSource} that produces count records in @@ -99,13 +106,13 @@ public DataGeneratorSource( public DataGeneratorSource( MapFunction generatorFunction, long count, - RateLimiter rateLimiter, + long maxPerSecond, TypeInformation typeInfo) { - // TODO: checkArgument(maxPerSecond > 0, "maxPerSeconds has to be a positive number"); + checkArgument(maxPerSecond > 0, "maxPerSeconds has to be a positive number"); this.typeInfo = checkNotNull(typeInfo); this.generatorFunction = checkNotNull(generatorFunction); this.numberSource = new NumberSequenceSource(0, count); - this.rateLimiter = rateLimiter; + this.maxPerSecond = maxPerSecond; } public long getCount() { @@ -142,7 +149,7 @@ public SourceReader> createReader( numberSource.splitNumberRange(0, getCount(), parallelism); return new IteratorSourceEnumerator<>( - enumContext, wrapSplits(splits, generatorFunction, rateLimiter)); + enumContext, wrapSplits(splits, generatorFunction, maxPerSecond, parallelism)); } @Override @@ -155,15 +162,13 @@ public SourceReader> createReader( @Override public SimpleVersionedSerializer> getSplitSerializer() { - return new SplitSerializer<>( - numberSource.getSplitSerializer(), generatorFunction, rateLimiter); + return new SplitSerializer<>(generatorFunction, maxPerSecond); } @Override public SimpleVersionedSerializer>> getEnumeratorCheckpointSerializer() { - return new CheckpointSerializer<>( - numberSource.getEnumeratorCheckpointSerializer(), generatorFunction, rateLimiter); + return new CheckpointSerializer<>(generatorFunction, maxPerSecond); } // ------------------------------------------------------------------------ @@ -178,10 +183,11 @@ public static class GeneratorSequenceIterator implements Iterator { public GeneratorSequenceIterator( NumberSequenceIterator numSeqIterator, MapFunction generatorFunction, - RateLimiter rateLimiter) { + long maxPerSecond, + int parallelism) { this.generatorFunction = generatorFunction; this.numSeqIterator = numSeqIterator; - this.rateLimiter = rateLimiter; + this.rateLimiter = new SimpleRateLimiter(maxPerSecond, parallelism); } @Override @@ -200,7 +206,6 @@ public long getTo() { @Override public T next() { try { - LOG.error("NEXT!"); rateLimiter.acquire(); return generatorFunction.map(numSeqIterator.next()); } catch (Exception e) { @@ -220,20 +225,25 @@ public static class GeneratorSequenceSplit public GeneratorSequenceSplit( NumberSequenceSplit numberSequenceSplit, MapFunction generatorFunction, - RateLimiter rateLimiter) { + long maxPerSecond, + int parallelism) { this.numberSequenceSplit = numberSequenceSplit; this.generatorFunction = generatorFunction; - this.rateLimiter = rateLimiter; + this.maxPerSecond = maxPerSecond; + this.parallelism = parallelism; } private final NumberSequenceSplit numberSequenceSplit; - private final MapFunction generatorFunction; - private final RateLimiter rateLimiter; + private final long maxPerSecond; + private final int parallelism; public GeneratorSequenceIterator getIterator() { return new GeneratorSequenceIterator<>( - numberSequenceSplit.getIterator(), generatorFunction, rateLimiter); + numberSequenceSplit.getIterator(), + generatorFunction, + maxPerSecond, + parallelism); } @Override @@ -248,7 +258,8 @@ public IteratorSourceSplit> getUpdatedSplitForIt (NumberSequenceSplit) numberSequenceSplit.getUpdatedSplitForIterator(iterator.numSeqIterator), generatorFunction, - rateLimiter); + maxPerSecond, + parallelism); } @Override @@ -264,86 +275,144 @@ public String toString() { private static final class SplitSerializer implements SimpleVersionedSerializer> { - private final SimpleVersionedSerializer numberSplitSerializer; + private static final int CURRENT_VERSION = 1; + private final MapFunction generatorFunction; - private final RateLimiter rateLimiter; + private final long maxPerSecond; - private SplitSerializer( - SimpleVersionedSerializer numberSplitSerializer, - MapFunction generatorFunction, - RateLimiter rateLimiter) { - this.numberSplitSerializer = numberSplitSerializer; + private SplitSerializer(MapFunction generatorFunction, long maxPerSecond) { this.generatorFunction = generatorFunction; - this.rateLimiter = rateLimiter; + this.maxPerSecond = maxPerSecond; } @Override public int getVersion() { - return numberSplitSerializer.getVersion(); + return CURRENT_VERSION; } @Override public byte[] serialize(GeneratorSequenceSplit split) throws IOException { - return numberSplitSerializer.serialize(split.numberSequenceSplit); + checkArgument( + split.getClass() == GeneratorSequenceSplit.class, + "cannot serialize subclasses"); + + // We will serialize 2 longs (16 bytes) plus the UTF representation of the string (2 + + // length) + final DataOutputSerializer out = + new DataOutputSerializer(split.splitId().length() + 18); + serializeV1(out, split); + return out.getCopyOfBuffer(); } @Override public GeneratorSequenceSplit deserialize(int version, byte[] serialized) throws IOException { + if (version != CURRENT_VERSION) { + throw new IOException("Unrecognized version: " + version); + } + final DataInputDeserializer in = new DataInputDeserializer(serialized); + return deserializeV1(in, generatorFunction, maxPerSecond); + } + + static void serializeV1(DataOutputView out, GeneratorSequenceSplit split) + throws IOException { + serializeNumberSequenceSplit(out, split.numberSequenceSplit); + out.writeInt(split.parallelism); + } + + static void serializeNumberSequenceSplit( + DataOutputView out, NumberSequenceSplit numberSequenceSplit) throws IOException { + out.writeUTF(numberSequenceSplit.splitId()); + out.writeLong(numberSequenceSplit.from()); + out.writeLong(numberSequenceSplit.to()); + } + + static GeneratorSequenceSplit deserializeV1( + DataInputView in, MapFunction generatorFunction, long maxPerSecond) + throws IOException { + NumberSequenceSplit numberSequenceSplit = deserializeNumberSequenceSplit(in); + int parallelism = in.readInt(); return new GeneratorSequenceSplit<>( - numberSplitSerializer.deserialize(version, serialized), - generatorFunction, - rateLimiter); + numberSequenceSplit, generatorFunction, maxPerSecond, parallelism); + } + + private static NumberSequenceSplit deserializeNumberSequenceSplit(DataInputView in) + throws IOException { + return new NumberSequenceSplit(in.readUTF(), in.readLong(), in.readLong()); } } private static final class CheckpointSerializer implements SimpleVersionedSerializer>> { - private final SimpleVersionedSerializer> - numberCheckpointSerializer; - private final MapFunction generatorFunction; - private final RateLimiter throttler; - - public CheckpointSerializer( - SimpleVersionedSerializer> - numberCheckpointSerializer, - MapFunction generatorFunction, - RateLimiter throttler) { - this.numberCheckpointSerializer = numberCheckpointSerializer; - this.generatorFunction = generatorFunction; - this.throttler = throttler; - } + private static final int CURRENT_VERSION = 1; @Override public int getVersion() { - return numberCheckpointSerializer.getVersion(); + return CURRENT_VERSION; + } + + private final MapFunction generatorFunction; + private final long maxPerSecond; + + public CheckpointSerializer(MapFunction generatorFunction, long maxPerSecond) { + this.generatorFunction = generatorFunction; + this.maxPerSecond = maxPerSecond; } @Override public byte[] serialize(Collection> checkpoint) throws IOException { - return numberCheckpointSerializer.serialize( - checkpoint.stream() - .map(split -> split.numberSequenceSplit) - .collect(Collectors.toList())); + // Each split needs 2 longs (16 bytes) plus the UTG representation of the string (2 + + // length). + // Assuming at most 4 digit split IDs, 22 bytes per split avoids any intermediate array + // resizing. + // Plus four bytes for the length field. + // Plus four bytes for the parallelism. + final DataOutputSerializer out = + new DataOutputSerializer(checkpoint.size() * 22 + 4 + 4); + out.writeInt(checkpoint.size()); + for (GeneratorSequenceSplit split : checkpoint) { + DataGeneratorSource.SplitSerializer.serializeNumberSequenceSplit( + out, split.numberSequenceSplit); + } + + final Optional> aSplit = checkpoint.stream().findFirst(); + if (aSplit.isPresent()) { + int parallelism = aSplit.get().parallelism; + out.writeInt(parallelism); + } + + return out.getCopyOfBuffer(); } @Override public Collection> deserialize(int version, byte[] serialized) throws IOException { - Collection numberSequenceSplits = - numberCheckpointSerializer.deserialize(version, serialized); - return wrapSplits(numberSequenceSplits, generatorFunction, throttler); + if (version != CURRENT_VERSION) { + throw new IOException("Unrecognized version: " + version); + } + final DataInputDeserializer in = new DataInputDeserializer(serialized); + final int num = in.readInt(); + final List result = new ArrayList<>(num); + for (int remaining = num; remaining > 0; remaining--) { + result.add(SplitSerializer.deserializeNumberSequenceSplit(in)); + } + final int parallelism = in.readInt(); + return wrapSplits(result, generatorFunction, maxPerSecond, parallelism); } } private static List> wrapSplits( Collection numberSequenceSplits, MapFunction generatorFunction, - RateLimiter throttler) { + long maxPerSecond, + int parallelism) { return numberSequenceSplits.stream() - .map(split -> new GeneratorSequenceSplit<>(split, generatorFunction, throttler)) + .map( + split -> + new GeneratorSequenceSplit<>( + split, generatorFunction, maxPerSecond, parallelism)) .collect(Collectors.toList()); } } From f5e84f9cd14ad518a051c77370dd167ce09aa29f Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 1 Jul 2022 17:02:39 +0200 Subject: [PATCH 05/12] WIP: Move ratelimiters --- .../ratelimiting/BucketingRateLimiter.java} | 18 +++------ .../io/ratelimiting/GuavaRateLimiter.java | 37 +++++++++++++++++++ .../io/ratelimiting}/NoOpRateLimiter.java | 8 +--- .../io/ratelimiting}/RateLimiter.java | 12 +++--- .../source/lib/DataGeneratorSource.java | 11 ++++-- .../wordcount/GeneratorSourceCheck.java | 10 +++-- 6 files changed, 63 insertions(+), 33 deletions(-) rename flink-core/src/main/java/org/apache/flink/api/{connector/source/lib/util/SimpleRateLimiter.java => common/io/ratelimiting/BucketingRateLimiter.java} (84%) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/GuavaRateLimiter.java rename flink-core/src/main/java/org/apache/flink/api/{connector/source/lib/util => common/io/ratelimiting}/NoOpRateLimiter.java (80%) rename flink-core/src/main/java/org/apache/flink/api/{connector/source/lib/util => common/io/ratelimiting}/RateLimiter.java (73%) diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/SimpleRateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/BucketingRateLimiter.java similarity index 84% rename from flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/SimpleRateLimiter.java rename to flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/BucketingRateLimiter.java index 8dfc905af12d2..8ff619c2bf30f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/SimpleRateLimiter.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/BucketingRateLimiter.java @@ -16,20 +16,15 @@ * limitations under the License. */ -package org.apache.flink.api.connector.source.lib.util; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +package org.apache.flink.api.common.io.ratelimiting; import static org.apache.flink.util.Preconditions.checkArgument; /** Utility to throttle a thread to a given number of executions (records) per second. */ -public final class SimpleRateLimiter implements RateLimiter { - - private static final Logger LOG = LoggerFactory.getLogger(SimpleRateLimiter.class); +public final class BucketingRateLimiter implements RateLimiter { - private long maxPerBucket; - private long nanosInBucket; + private final long maxPerBucket; + private final long nanosInBucket; private long endOfCurrentBucketNanos; private int inCurrentBucket; @@ -37,11 +32,11 @@ public final class SimpleRateLimiter implements RateLimiter { private static final int DEFAULT_BUCKETS_PER_SECOND = 10; private static final long NANOS_IN_ONE_SECOND = 1_000_000_000L; - public SimpleRateLimiter(long maxPerSecond, int numParallelExecutors) { + public BucketingRateLimiter(long maxPerSecond, int numParallelExecutors) { this(maxPerSecond, numParallelExecutors, DEFAULT_BUCKETS_PER_SECOND); } - public SimpleRateLimiter(long maxPerSecond, int numParallelExecutors, int bucketsPerSecond) { + public BucketingRateLimiter(long maxPerSecond, int numParallelExecutors, int bucketsPerSecond) { checkArgument(maxPerSecond > 0, "maxPerSecond must be a positive number"); checkArgument(numParallelExecutors > 0, "numParallelExecutors must be greater than 0"); @@ -56,7 +51,6 @@ public SimpleRateLimiter(long maxPerSecond, int numParallelExecutors, int bucket // TODO: JavaDoc, returns number of seconds idling on this call. public int acquire() throws InterruptedException { - LOG.error("THROTTLE!"); if (++inCurrentBucket != maxPerBucket) { return 0; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/GuavaRateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/GuavaRateLimiter.java new file mode 100644 index 0000000000000..264e8fff1d0ab --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/GuavaRateLimiter.java @@ -0,0 +1,37 @@ +/* + * 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.api.common.io.ratelimiting; + +import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.RateLimiter; + +/** An implementation of {@link RateLimiter} based on Guava's RateLimiter. */ +public class GuavaRateLimiter implements org.apache.flink.api.common.io.ratelimiting.RateLimiter { + + private final RateLimiter rateLimiter; + + public GuavaRateLimiter(long maxPerSecond, int numParallelExecutors) { + final float maxPerSecondPerSubtask = (float) maxPerSecond / numParallelExecutors; + this.rateLimiter = RateLimiter.create(maxPerSecondPerSubtask); + } + + @Override + public int acquire() { + return (int) rateLimiter.acquire(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/NoOpRateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/NoOpRateLimiter.java similarity index 80% rename from flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/NoOpRateLimiter.java rename to flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/NoOpRateLimiter.java index 9ba4f69e44f8b..10d55c4211ca8 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/NoOpRateLimiter.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/NoOpRateLimiter.java @@ -16,18 +16,12 @@ * limitations under the License. */ -package org.apache.flink.api.connector.source.lib.util; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +package org.apache.flink.api.common.io.ratelimiting; public class NoOpRateLimiter implements RateLimiter { - private static final Logger LOG = LoggerFactory.getLogger(NoOpRateLimiter.class); - @Override public int acquire() throws InterruptedException { - LOG.error("!!!THROTTLE!"); return 0; } } diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/RateLimiter.java similarity index 73% rename from flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimiter.java rename to flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/RateLimiter.java index dcd11b0d4c932..577897f41ae51 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimiter.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/RateLimiter.java @@ -9,14 +9,14 @@ * * 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. + * 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.api.connector.source.lib.util; +package org.apache.flink.api.common.io.ratelimiting; import java.io.Serializable; diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java index 18bac125d4895..224a144d9acc7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java @@ -20,6 +20,9 @@ import org.apache.flink.annotation.Public; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.io.ratelimiting.GuavaRateLimiter; +import org.apache.flink.api.common.io.ratelimiting.NoOpRateLimiter; +import org.apache.flink.api.common.io.ratelimiting.RateLimiter; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; @@ -32,8 +35,6 @@ import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; -import org.apache.flink.api.connector.source.lib.util.RateLimiter; -import org.apache.flink.api.connector.source.lib.util.SimpleRateLimiter; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.core.memory.DataInputDeserializer; @@ -178,7 +179,7 @@ public static class GeneratorSequenceIterator implements Iterator { private final MapFunction generatorFunction; private final NumberSequenceIterator numSeqIterator; - private final RateLimiter rateLimiter; + private RateLimiter rateLimiter = new NoOpRateLimiter(); public GeneratorSequenceIterator( NumberSequenceIterator numSeqIterator, @@ -187,7 +188,9 @@ public GeneratorSequenceIterator( int parallelism) { this.generatorFunction = generatorFunction; this.numSeqIterator = numSeqIterator; - this.rateLimiter = new SimpleRateLimiter(maxPerSecond, parallelism); + if (maxPerSecond > 0) { + this.rateLimiter = new GuavaRateLimiter(maxPerSecond, parallelism); + } } @Override diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java index e191494fb8aee..d4b4b1907b283 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java @@ -21,10 +21,10 @@ import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.io.ratelimiting.BucketingRateLimiter; +import org.apache.flink.api.common.io.ratelimiting.RateLimiter; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.connector.source.lib.DataGeneratorSource; -import org.apache.flink.api.connector.source.lib.util.RateLimiter; -import org.apache.flink.api.connector.source.lib.util.SimpleRateLimiter; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -51,9 +51,11 @@ public static void main(String[] args) throws Exception { // watermarked.print(); MapFunction generator = index -> ">>> " + index; - RateLimiter throttler = new SimpleRateLimiter(1, parallelism); + RateLimiter rateLimiter = new BucketingRateLimiter(1, parallelism); + // DataGeneratorSource source = + // new DataGeneratorSource<>(generator, 100, rateLimiter, Types.STRING); DataGeneratorSource source = - new DataGeneratorSource<>(generator, 100, throttler, Types.STRING); + new DataGeneratorSource<>(generator, 10, 2, Types.STRING); DataStreamSource watermarked = env.fromSource( source, From 78f441da34b16e40ee23fa3a01a9b184192c94d2 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Mon, 4 Jul 2022 13:44:35 +0200 Subject: [PATCH 06/12] WIP: parallelism in SourceReaderContext --- .../connector/file/src/FileSourceHeavyThroughputTest.java | 5 +++++ .../flink/api/connector/source/SourceReaderContext.java | 3 +++ .../api/connector/source/lib/NumberSequenceSourceTest.java | 5 +++++ .../testutils/source/reader/TestingReaderContext.java | 5 +++++ 4 files changed, 18 insertions(+) diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java index 3c175a80c8d58..bb44462ab051e 100644 --- a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java +++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java @@ -225,6 +225,11 @@ public void sendSourceEventToCoordinator(SourceEvent sourceEvent) {} public UserCodeClassLoader getUserCodeClassLoader() { return SimpleUserCodeClassLoader.create(getClass().getClassLoader()); } + + @Override + public int currentParallelism() { + return 1; + } } private static final class NoOpReaderOutput implements ReaderOutput { diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java index 304afddf90a83..d5763cc41ae70 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java @@ -63,4 +63,7 @@ public interface SourceReaderContext { * @see UserCodeClassLoader */ UserCodeClassLoader getUserCodeClassLoader(); + + // TODO: add javadoc + int currentParallelism(); } diff --git a/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java b/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java index 5944201d1ad7d..ce97ff7126911 100644 --- a/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java @@ -143,6 +143,11 @@ public void sendSourceEventToCoordinator(SourceEvent sourceEvent) {} public UserCodeClassLoader getUserCodeClassLoader() { return SimpleUserCodeClassLoader.create(getClass().getClassLoader()); } + + @Override + public int currentParallelism() { + return 1; + } } private static final class TestingReaderOutput implements ReaderOutput { diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java index d83f8978684d9..4855ced967b34 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java @@ -86,6 +86,11 @@ public UserCodeClassLoader getUserCodeClassLoader() { return SimpleUserCodeClassLoader.create(getClass().getClassLoader()); } + @Override + public int currentParallelism() { + return 1; + } + // ------------------------------------------------------------------------ public int getNumSplitRequests() { From 460a61aa50b44ccd9c283ed11fb4ef6918279953 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Mon, 4 Jul 2022 17:10:58 +0200 Subject: [PATCH 07/12] WIP: working V2 --- .../src/FileSourceHeavyThroughputTest.java | 6 + .../connector/source/SourceReaderContext.java | 14 +- .../source/lib/DataGeneratorSourceV2.java | 324 ++++++++++++++++++ .../RateLimitedIteratorSourceReaderNew.java | 98 ++++++ .../source/lib/NumberSequenceSourceTest.java | 6 + .../wordcount/GeneratorSourceCheck.java | 11 +- .../api/operators/SourceOperator.java | 12 + 7 files changed, 465 insertions(+), 6 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReaderNew.java diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java index bb44462ab051e..4528b221c1efa 100644 --- a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java +++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java @@ -19,6 +19,7 @@ package org.apache.flink.connector.file.src; import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.ReaderOutput; @@ -230,6 +231,11 @@ public UserCodeClassLoader getUserCodeClassLoader() { public int currentParallelism() { return 1; } + + @Override + public RuntimeContext getRuntimeContext() { + return null; + } } private static final class NoOpReaderOutput implements ReaderOutput { diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java index d5763cc41ae70..bbce2687af46e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java @@ -19,6 +19,7 @@ package org.apache.flink.api.connector.source; import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.groups.SourceReaderMetricGroup; import org.apache.flink.util.UserCodeClassLoader; @@ -27,7 +28,9 @@ @Public public interface SourceReaderContext { - /** @return The metric group this source belongs to. */ + /** + * @return The metric group this source belongs to. + */ SourceReaderMetricGroup metricGroup(); /** Gets the configuration with which Flink was started. */ @@ -39,7 +42,9 @@ public interface SourceReaderContext { */ String getLocalHostName(); - /** @return The index of this subtask. */ + /** + * @return The index of this subtask. + */ int getIndexOfSubtask(); /** @@ -64,6 +69,9 @@ public interface SourceReaderContext { */ UserCodeClassLoader getUserCodeClassLoader(); - // TODO: add javadoc + // TODO: add JavaDoc int currentParallelism(); + + // TODO: add JavaDoc + RuntimeContext getRuntimeContext(); } diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java new file mode 100644 index 0000000000000..894f9908db842 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java @@ -0,0 +1,324 @@ +/* + * 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.api.connector.source.lib; + +import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.io.ratelimiting.GuavaRateLimiter; +import org.apache.flink.api.common.io.ratelimiting.RateLimiter; +import org.apache.flink.api.common.typeinfo.TypeInformation; +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.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; +import org.apache.flink.api.connector.source.lib.util.RateLimitedIteratorSourceReaderNew; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.NumberSequenceIterator; + +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A data source that produces generators N data points in parallel. This source is useful for + * testing and for cases that just need a stream of N events of any kind. + * + *

The source splits the sequence into as many parallel sub-sequences as there are parallel + * source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is + * limited to one, this will produce one sequence in order. + * + *

This source is always bounded. For very long sequences (for example over the entire domain of + * long integer values), user may want to consider executing the application in a streaming manner, + * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. + */ +@Public +public class DataGeneratorSourceV2 + implements Source< + OUT, + DataGeneratorSourceV2.GeneratorSequenceSplit, + Collection>>, + ResultTypeQueryable { + + private static final long serialVersionUID = 1L; + + private final TypeInformation typeInfo; + + public final MapFunction generatorFunction; + + /** The end Generator in the sequence, inclusive. */ + private final NumberSequenceSource numberSource; + + private long maxPerSecond = -1; + + /** + * Creates a new {@code DataGeneratorSource} that produces count records in + * parallel. + * + * @param typeInfo the type info + * @param generatorFunction the generator function + * @param count The count + */ + public DataGeneratorSourceV2( + MapFunction generatorFunction, long count, TypeInformation typeInfo) { + this.typeInfo = checkNotNull(typeInfo); + this.generatorFunction = checkNotNull(generatorFunction); + this.numberSource = new NumberSequenceSource(0, count); + } + + public DataGeneratorSourceV2( + MapFunction generatorFunction, + long count, + long maxPerSecond, + TypeInformation typeInfo) { + checkArgument(maxPerSecond > 0, "maxPerSeconds has to be a positive number"); + this.typeInfo = checkNotNull(typeInfo); + this.generatorFunction = checkNotNull(generatorFunction); + this.numberSource = new NumberSequenceSource(0, count); + this.maxPerSecond = maxPerSecond; + } + + public long getCount() { + return numberSource.getTo(); + } + + // ------------------------------------------------------------------------ + // source methods + // ------------------------------------------------------------------------ + + @Override + public TypeInformation getProducedType() { + return typeInfo; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader> createReader( + SourceReaderContext readerContext) { + RateLimiter rateLimiter = + new GuavaRateLimiter(maxPerSecond, readerContext.currentParallelism()); + return new RateLimitedIteratorSourceReaderNew<>(readerContext, rateLimiter); + } + + @Override + public SplitEnumerator, Collection>> + createEnumerator( + final SplitEnumeratorContext> enumContext) { + + final List splits = + numberSource.splitNumberRange(0, getCount(), enumContext.currentParallelism()); + return new IteratorSourceEnumerator<>(enumContext, wrapSplits(splits, generatorFunction)); + } + + @Override + public SplitEnumerator, Collection>> + restoreEnumerator( + final SplitEnumeratorContext> enumContext, + Collection> checkpoint) { + return new IteratorSourceEnumerator<>(enumContext, checkpoint); + } + + @Override + public SimpleVersionedSerializer> getSplitSerializer() { + return new SplitSerializer<>(numberSource.getSplitSerializer(), generatorFunction); + } + + @Override + public SimpleVersionedSerializer>> + getEnumeratorCheckpointSerializer() { + return new CheckpointSerializer<>( + numberSource.getEnumeratorCheckpointSerializer(), generatorFunction); + } + + // ------------------------------------------------------------------------ + // splits & checkpoint + // ------------------------------------------------------------------------ + public static class GeneratorSequenceIterator implements Iterator { + + private final MapFunction generatorFunction; + private final NumberSequenceIterator numSeqIterator; + + public GeneratorSequenceIterator( + NumberSequenceIterator numSeqIterator, MapFunction generatorFunction) { + this.generatorFunction = generatorFunction; + this.numSeqIterator = numSeqIterator; + } + + @Override + public boolean hasNext() { + return numSeqIterator.hasNext(); + } + + public long getCurrent() { + return numSeqIterator.getCurrent(); + } + + public long getTo() { + return numSeqIterator.getTo(); + } + + @Override + public T next() { + try { + return generatorFunction.map(numSeqIterator.next()); + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format( + "Exception while generating element %d", + numSeqIterator.getCurrent()), + e); + } + } + } + + /** A split of the source, representing a Generator sub-sequence. */ + public static class GeneratorSequenceSplit + implements IteratorSourceSplit> { + + public GeneratorSequenceSplit( + NumberSequenceSplit numberSequenceSplit, MapFunction generatorFunction) { + this.numberSequenceSplit = numberSequenceSplit; + this.generatorFunction = generatorFunction; + } + + private final NumberSequenceSplit numberSequenceSplit; + + private final MapFunction generatorFunction; + + public GeneratorSequenceIterator getIterator() { + return new GeneratorSequenceIterator<>( + numberSequenceSplit.getIterator(), generatorFunction); + } + + @Override + public String splitId() { + return numberSequenceSplit.splitId(); + } + + @Override + public IteratorSourceSplit> getUpdatedSplitForIterator( + final GeneratorSequenceIterator iterator) { + return new GeneratorSequenceSplit<>( + (NumberSequenceSplit) + numberSequenceSplit.getUpdatedSplitForIterator(iterator.numSeqIterator), + generatorFunction); + } + + @Override + public String toString() { + return String.format( + "GeneratorSequenceSplit [%d, %d] (%s)", + numberSequenceSplit.from(), + numberSequenceSplit.to(), + numberSequenceSplit.splitId()); + } + } + + private static final class SplitSerializer + implements SimpleVersionedSerializer> { + + private final SimpleVersionedSerializer numberSplitSerializer; + private final MapFunction generatorFunction; + + private SplitSerializer( + SimpleVersionedSerializer numberSplitSerializer, + MapFunction generatorFunction) { + this.numberSplitSerializer = numberSplitSerializer; + this.generatorFunction = generatorFunction; + } + + @Override + public int getVersion() { + return numberSplitSerializer.getVersion(); + } + + @Override + public byte[] serialize(GeneratorSequenceSplit split) throws IOException { + return numberSplitSerializer.serialize(split.numberSequenceSplit); + } + + @Override + public GeneratorSequenceSplit deserialize(int version, byte[] serialized) + throws IOException { + return new GeneratorSequenceSplit<>( + numberSplitSerializer.deserialize(version, serialized), generatorFunction); + } + } + + private static final class CheckpointSerializer + implements SimpleVersionedSerializer>> { + + private final SimpleVersionedSerializer> + numberCheckpointSerializer; + private final MapFunction generatorFunction; + + public CheckpointSerializer( + SimpleVersionedSerializer> + numberCheckpointSerializer, + MapFunction generatorFunction) { + this.numberCheckpointSerializer = numberCheckpointSerializer; + this.generatorFunction = generatorFunction; + } + + @Override + public int getVersion() { + return numberCheckpointSerializer.getVersion(); + } + + @Override + public byte[] serialize(Collection> checkpoint) + throws IOException { + return numberCheckpointSerializer.serialize( + checkpoint.stream() + .map(split -> split.numberSequenceSplit) + .collect(Collectors.toList())); + } + + @Override + public Collection> deserialize(int version, byte[] serialized) + throws IOException { + Collection numberSequenceSplits = + numberCheckpointSerializer.deserialize(version, serialized); + return wrapSplits(numberSequenceSplits, generatorFunction); + } + } + + private static List> wrapSplits( + Collection numberSequenceSplits, + MapFunction generatorFunction) { + return numberSequenceSplits.stream() + .map(split -> new GeneratorSequenceSplit<>(split, generatorFunction)) + .collect(Collectors.toList()); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReaderNew.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReaderNew.java new file mode 100644 index 0000000000000..1741b796a5502 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReaderNew.java @@ -0,0 +1,98 @@ +/* + * 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.api.connector.source.lib.util; + +import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.io.ratelimiting.RateLimiter; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; + +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link SourceReader} that returns the values of an iterator, supplied via an {@link + * IteratorSourceSplit}. + * + *

The {@code IteratorSourceSplit} is also responsible for taking the current iterator and + * turning it back into a split for checkpointing. + * + * @param The type of events returned by the reader. + * @param The type of the iterator that produces the events. This type exists to make the + * conversion between iterator and {@code IteratorSourceSplit} type safe. + * @param The concrete type of the {@code IteratorSourceSplit} that creates and converts + * the iterator that produces this reader's elements. + */ +@Public +public class RateLimitedIteratorSourceReaderNew< + E, IterT extends Iterator, SplitT extends IteratorSourceSplit> + implements SourceReader { + + private final IteratorSourceReader iteratorSourceReader; + private final RateLimiter rateLimiter; + + public RateLimitedIteratorSourceReaderNew( + SourceReaderContext readerContext, RateLimiter rateLimiter) { + checkNotNull(readerContext); + iteratorSourceReader = new IteratorSourceReader<>(readerContext); + this.rateLimiter = rateLimiter; + } + + // ------------------------------------------------------------------------ + @Override + public void start() { + iteratorSourceReader.start(); + } + + @Override + public InputStatus pollNext(ReaderOutput output) throws InterruptedException { + rateLimiter.acquire(); + return iteratorSourceReader.pollNext(output); + } + + @Override + public CompletableFuture isAvailable() { + return iteratorSourceReader.isAvailable(); + } + + @Override + public void addSplits(List splits) { + iteratorSourceReader.addSplits(splits); + } + + @Override + public void notifyNoMoreSplits() { + iteratorSourceReader.notifyNoMoreSplits(); + } + + @Override + public List snapshotState(long checkpointId) { + return iteratorSourceReader.snapshotState(checkpointId); + } + + @Override + public void close() throws Exception { + iteratorSourceReader.close(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java b/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java index ce97ff7126911..daee302729223 100644 --- a/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java @@ -19,6 +19,7 @@ package org.apache.flink.api.connector.source.lib; import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SourceOutput; @@ -148,6 +149,11 @@ public UserCodeClassLoader getUserCodeClassLoader() { public int currentParallelism() { return 1; } + + @Override + public RuntimeContext getRuntimeContext() { + return null; + } } private static final class TestingReaderOutput implements ReaderOutput { diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java index d4b4b1907b283..8c227bc1a8fff 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java @@ -24,7 +24,7 @@ import org.apache.flink.api.common.io.ratelimiting.BucketingRateLimiter; import org.apache.flink.api.common.io.ratelimiting.RateLimiter; import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.connector.source.lib.DataGeneratorSource; +import org.apache.flink.api.connector.source.lib.DataGeneratorSourceV2; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -54,8 +54,13 @@ public static void main(String[] args) throws Exception { RateLimiter rateLimiter = new BucketingRateLimiter(1, parallelism); // DataGeneratorSource source = // new DataGeneratorSource<>(generator, 100, rateLimiter, Types.STRING); - DataGeneratorSource source = - new DataGeneratorSource<>(generator, 10, 2, Types.STRING); + + // DataGeneratorSource source = + // new DataGeneratorSource<>(generator, 10, 2, Types.STRING); + + DataGeneratorSourceV2 source = + new DataGeneratorSourceV2<>(generator, 1000, 100, Types.STRING); + DataStreamSource watermarked = env.fromSource( source, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java index 53621bd1b6e0b..aa56a7f55dd37 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.eventtime.WatermarkAlignmentParams; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; @@ -230,6 +231,7 @@ public void initReader() throws Exception { } final int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + final RuntimeContext runtimeContext = getRuntimeContext(); final SourceReaderContext context = new SourceReaderContext() { @@ -281,6 +283,16 @@ public void registerReleaseHookIfAbsent( } }; } + + @Override + public int currentParallelism() { + return getRuntimeContext().getNumberOfParallelSubtasks(); + } + + @Override + public RuntimeContext getRuntimeContext() { + return runtimeContext; + } }; sourceReader = readerFactory.apply(context); From 2da82fabef2b9438eeb8d735fed5ec0896071422 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Mon, 4 Jul 2022 17:48:25 +0200 Subject: [PATCH 08/12] WIP: working V3 --- .../connector/source/SourceReaderContext.java | 15 +- .../source/lib/DataGeneratorSourceV0.java | 305 ++++++++++++++++++ ...Source.java => DataGeneratorSourceV1.java} | 16 +- .../source/lib/DataGeneratorSourceV2.java | 8 +- .../source/lib/DataGeneratorSourceV3.java | 175 ++++++++++ .../source/lib/NumberSequenceSource.java | 2 +- .../source/lib/util/IteratorSourceReader.java | 118 +------ .../lib/util/IteratorSourceReaderBase.java | 164 ++++++++++ .../lib/util/MappingIteratorSourceReader.java | 69 ++++ ...a => RateLimitedIteratorSourceReader.java} | 9 +- .../lib/util/RateLimitedSourceReader.java | 84 +++++ .../wordcount/GeneratorSourceCheck.java | 97 ------ .../wordcount/GeneratorSourcePOC.java | 74 +++++ .../source/reader/TestingReaderContext.java | 6 + 14 files changed, 905 insertions(+), 237 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV0.java rename flink-core/src/main/java/org/apache/flink/api/connector/source/lib/{DataGeneratorSource.java => DataGeneratorSourceV1.java} (98%) create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReaderBase.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/MappingIteratorSourceReader.java rename flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/{RateLimitedIteratorSourceReaderNew.java => RateLimitedIteratorSourceReader.java} (95%) create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedSourceReader.java delete mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java create mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourcePOC.java diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java index bbce2687af46e..c01ac0df0b91e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java @@ -28,9 +28,7 @@ @Public public interface SourceReaderContext { - /** - * @return The metric group this source belongs to. - */ + /** @return The metric group this source belongs to. */ SourceReaderMetricGroup metricGroup(); /** Gets the configuration with which Flink was started. */ @@ -42,9 +40,7 @@ public interface SourceReaderContext { */ String getLocalHostName(); - /** - * @return The index of this subtask. - */ + /** @return The index of this subtask. */ int getIndexOfSubtask(); /** @@ -72,6 +68,11 @@ public interface SourceReaderContext { // TODO: add JavaDoc int currentParallelism(); - // TODO: add JavaDoc + /** + * Gets the context that contains information about the readers runtime, such as the parallelism + * of the source. + * + * @return The runtime context of the source reader. + */ RuntimeContext getRuntimeContext(); } diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV0.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV0.java new file mode 100644 index 0000000000000..9700fc3158545 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV0.java @@ -0,0 +1,305 @@ +/* + * 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.api.connector.source.lib; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +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.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.NumberSequenceIterator; + +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A data source that produces generators N data points in parallel. This source is useful for + * testing and for cases that just need a stream of N events of any kind. + * + *

The source splits the sequence into as many parallel sub-sequences as there are parallel + * source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is + * limited to one, this will produce one sequence in order. + * + *

This source is always bounded. For very long sequences (for example over the entire domain of + * long integer values), user may want to consider executing the application in a streaming manner, + * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. + */ +@Experimental +public class DataGeneratorSourceV0 + implements Source< + OUT, + DataGeneratorSourceV0.GeneratorSequenceSplit, + Collection>>, + ResultTypeQueryable { + + private static final long serialVersionUID = 1L; + + private final TypeInformation typeInfo; + + public final MapFunction generatorFunction; + + /** The end Generator in the sequence, inclusive. */ + private final NumberSequenceSource numberSource; + + /** + * Creates a new {@code DataGeneratorSource} that produces count records in + * parallel. + * + * @param typeInfo the type info + * @param generatorFunction the generator function + * @param count The count + */ + public DataGeneratorSourceV0( + MapFunction generatorFunction, long count, TypeInformation typeInfo) { + this.typeInfo = checkNotNull(typeInfo); + this.generatorFunction = checkNotNull(generatorFunction); + this.numberSource = new NumberSequenceSource(0, count); + } + + public long getCount() { + return numberSource.getTo(); + } + + // ------------------------------------------------------------------------ + // source methods + // ------------------------------------------------------------------------ + + @Override + public TypeInformation getProducedType() { + return typeInfo; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader> createReader( + SourceReaderContext readerContext) { + return new IteratorSourceReader<>(readerContext); + } + + @Override + public SplitEnumerator, Collection>> + createEnumerator( + final SplitEnumeratorContext> enumContext) { + + final List splits = + numberSource.splitNumberRange(0, getCount(), enumContext.currentParallelism()); + return new IteratorSourceEnumerator<>(enumContext, wrapSplits(splits, generatorFunction)); + } + + @Override + public SplitEnumerator, Collection>> + restoreEnumerator( + final SplitEnumeratorContext> enumContext, + Collection> checkpoint) { + return new IteratorSourceEnumerator<>(enumContext, checkpoint); + } + + @Override + public SimpleVersionedSerializer> getSplitSerializer() { + return new SplitSerializer<>(numberSource.getSplitSerializer(), generatorFunction); + } + + @Override + public SimpleVersionedSerializer>> + getEnumeratorCheckpointSerializer() { + return new CheckpointSerializer<>( + numberSource.getEnumeratorCheckpointSerializer(), generatorFunction); + } + + // ------------------------------------------------------------------------ + // splits & checkpoint + // ------------------------------------------------------------------------ + public static class GeneratorSequenceIterator implements Iterator { + + private final MapFunction generatorFunction; + private final NumberSequenceIterator numSeqIterator; + + public GeneratorSequenceIterator( + NumberSequenceIterator numSeqIterator, MapFunction generatorFunction) { + this.generatorFunction = generatorFunction; + this.numSeqIterator = numSeqIterator; + } + + @Override + public boolean hasNext() { + return numSeqIterator.hasNext(); + } + + public long getCurrent() { + return numSeqIterator.getCurrent(); + } + + public long getTo() { + return numSeqIterator.getTo(); + } + + @Override + public T next() { + try { + return generatorFunction.map(numSeqIterator.next()); + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format( + "Exception while generating element %d", + numSeqIterator.getCurrent()), + e); + } + } + } + + /** A split of the source, representing a Generator sub-sequence. */ + public static class GeneratorSequenceSplit + implements IteratorSourceSplit> { + + public GeneratorSequenceSplit( + NumberSequenceSplit numberSequenceSplit, MapFunction generatorFunction) { + this.numberSequenceSplit = numberSequenceSplit; + this.generatorFunction = generatorFunction; + } + + private final NumberSequenceSplit numberSequenceSplit; + + private final MapFunction generatorFunction; + + public GeneratorSequenceIterator getIterator() { + return new GeneratorSequenceIterator<>( + numberSequenceSplit.getIterator(), generatorFunction); + } + + @Override + public String splitId() { + return numberSequenceSplit.splitId(); + } + + @Override + public IteratorSourceSplit> getUpdatedSplitForIterator( + final GeneratorSequenceIterator iterator) { + return new GeneratorSequenceSplit<>( + (NumberSequenceSplit) + numberSequenceSplit.getUpdatedSplitForIterator(iterator.numSeqIterator), + generatorFunction); + } + + @Override + public String toString() { + return String.format( + "GeneratorSequenceSplit [%d, %d] (%s)", + numberSequenceSplit.from(), + numberSequenceSplit.to(), + numberSequenceSplit.splitId()); + } + } + + private static final class SplitSerializer + implements SimpleVersionedSerializer> { + + private final SimpleVersionedSerializer numberSplitSerializer; + private final MapFunction generatorFunction; + + private SplitSerializer( + SimpleVersionedSerializer numberSplitSerializer, + MapFunction generatorFunction) { + this.numberSplitSerializer = numberSplitSerializer; + this.generatorFunction = generatorFunction; + } + + @Override + public int getVersion() { + return numberSplitSerializer.getVersion(); + } + + @Override + public byte[] serialize(GeneratorSequenceSplit split) throws IOException { + return numberSplitSerializer.serialize(split.numberSequenceSplit); + } + + @Override + public GeneratorSequenceSplit deserialize(int version, byte[] serialized) + throws IOException { + return new GeneratorSequenceSplit<>( + numberSplitSerializer.deserialize(version, serialized), generatorFunction); + } + } + + private static final class CheckpointSerializer + implements SimpleVersionedSerializer>> { + + private final SimpleVersionedSerializer> + numberCheckpointSerializer; + private final MapFunction generatorFunction; + + public CheckpointSerializer( + SimpleVersionedSerializer> + numberCheckpointSerializer, + MapFunction generatorFunction) { + this.numberCheckpointSerializer = numberCheckpointSerializer; + this.generatorFunction = generatorFunction; + } + + @Override + public int getVersion() { + return numberCheckpointSerializer.getVersion(); + } + + @Override + public byte[] serialize(Collection> checkpoint) + throws IOException { + return numberCheckpointSerializer.serialize( + checkpoint.stream() + .map(split -> split.numberSequenceSplit) + .collect(Collectors.toList())); + } + + @Override + public Collection> deserialize(int version, byte[] serialized) + throws IOException { + Collection numberSequenceSplits = + numberCheckpointSerializer.deserialize(version, serialized); + return wrapSplits(numberSequenceSplits, generatorFunction); + } + } + + private static List> wrapSplits( + Collection numberSequenceSplits, + MapFunction generatorFunction) { + return numberSequenceSplits.stream() + .map(split -> new GeneratorSequenceSplit<>(split, generatorFunction)) + .collect(Collectors.toList()); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV1.java similarity index 98% rename from flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java rename to flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV1.java index 224a144d9acc7..20c9aef9a6f56 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV1.java @@ -18,7 +18,7 @@ package org.apache.flink.api.connector.source.lib; -import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.io.ratelimiting.GuavaRateLimiter; import org.apache.flink.api.common.io.ratelimiting.NoOpRateLimiter; @@ -30,7 +30,7 @@ import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.connector.source.lib.DataGeneratorSource.GeneratorSequenceSplit; +import org.apache.flink.api.connector.source.lib.DataGeneratorSourceV1.GeneratorSequenceSplit; import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; @@ -70,13 +70,13 @@ * long integer values), user may want to consider executing the application in a streaming manner, * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. */ -@Public -public class DataGeneratorSource +@Experimental +public class DataGeneratorSourceV1 implements Source< OUT, GeneratorSequenceSplit, Collection>>, ResultTypeQueryable { - private static final Logger LOG = LoggerFactory.getLogger(DataGeneratorSource.class); + private static final Logger LOG = LoggerFactory.getLogger(DataGeneratorSourceV1.class); private static final long serialVersionUID = 1L; @@ -97,14 +97,14 @@ public class DataGeneratorSource * @param generatorFunction the generator function * @param count The count */ - public DataGeneratorSource( + public DataGeneratorSourceV1( MapFunction generatorFunction, long count, TypeInformation typeInfo) { this.typeInfo = checkNotNull(typeInfo); this.generatorFunction = checkNotNull(generatorFunction); this.numberSource = new NumberSequenceSource(0, count); } - public DataGeneratorSource( + public DataGeneratorSourceV1( MapFunction generatorFunction, long count, long maxPerSecond, @@ -376,7 +376,7 @@ public byte[] serialize(Collection> checkpoint) new DataOutputSerializer(checkpoint.size() * 22 + 4 + 4); out.writeInt(checkpoint.size()); for (GeneratorSequenceSplit split : checkpoint) { - DataGeneratorSource.SplitSerializer.serializeNumberSequenceSplit( + DataGeneratorSourceV1.SplitSerializer.serializeNumberSequenceSplit( out, split.numberSequenceSplit); } diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java index 894f9908db842..012adc19b0f52 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java @@ -18,7 +18,7 @@ package org.apache.flink.api.connector.source.lib; -import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.io.ratelimiting.GuavaRateLimiter; import org.apache.flink.api.common.io.ratelimiting.RateLimiter; @@ -32,7 +32,7 @@ import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; -import org.apache.flink.api.connector.source.lib.util.RateLimitedIteratorSourceReaderNew; +import org.apache.flink.api.connector.source.lib.util.RateLimitedIteratorSourceReader; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.util.FlinkRuntimeException; @@ -59,7 +59,7 @@ * long integer values), user may want to consider executing the application in a streaming manner, * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. */ -@Public +@Experimental public class DataGeneratorSourceV2 implements Source< OUT, @@ -128,7 +128,7 @@ public SourceReader> createReader( SourceReaderContext readerContext) { RateLimiter rateLimiter = new GuavaRateLimiter(maxPerSecond, readerContext.currentParallelism()); - return new RateLimitedIteratorSourceReaderNew<>(readerContext, rateLimiter); + return new RateLimitedIteratorSourceReader<>(readerContext, rateLimiter); } @Override diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java new file mode 100644 index 0000000000000..aa497bec91a81 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java @@ -0,0 +1,175 @@ +/* + * 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.api.connector.source.lib; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.io.ratelimiting.GuavaRateLimiter; +import org.apache.flink.api.common.io.ratelimiting.RateLimiter; +import org.apache.flink.api.common.typeinfo.TypeInformation; +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.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; +import org.apache.flink.api.connector.source.lib.util.MappingIteratorSourceReader; +import org.apache.flink.api.connector.source.lib.util.RateLimitedSourceReader; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.util.Collection; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A data source that produces generators N data points in parallel. This source is useful for + * testing and for cases that just need a stream of N events of any kind. + * + *

The source splits the sequence into as many parallel sub-sequences as there are parallel + * source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is + * limited to one, this will produce one sequence in order. + * + *

This source is always bounded. For very long sequences (for example over the entire domain of + * long integer values), user may want to consider executing the application in a streaming manner, + * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. + */ +@Experimental +public class DataGeneratorSourceV3 + implements Source< + OUT, + NumberSequenceSource.NumberSequenceSplit, + Collection>, + ResultTypeQueryable { + + private static final long serialVersionUID = 1L; + + private final TypeInformation typeInfo; + + public final MapFunction generatorFunction; + + /** The end Generator in the sequence, inclusive. */ + private final NumberSequenceSource numberSource; + + private long maxPerSecond = -1; + + /** + * Creates a new {@code DataGeneratorSource} that produces count records in + * parallel. + * + * @param generatorFunction The generator function that receives index numbers and translates + * them into events of the output type. + * @param count The number of events to be produced. + * @param typeInfo The type information of the returned events. + */ + public DataGeneratorSourceV3( + MapFunction generatorFunction, long count, TypeInformation typeInfo) { + this.typeInfo = checkNotNull(typeInfo); + this.generatorFunction = checkNotNull(generatorFunction); + this.numberSource = new NumberSequenceSource(0, count); + } + + /** + * Creates a new {@code DataGeneratorSource} that produces count records in + * parallel. + * + * @param generatorFunction The generator function that receives index numbers and translates + * them into events of the output type. + * @param count The number of events to be produced. + * @param sourceRatePerSecond The maximum number of events per seconds that this generator aims + * to produce. This is a target number for the whole source and the individual parallel + * source instances automatically adjust their rate taking based on the {@code + * sourceRatePerSecond} and the source parallelism. + * @param typeInfo The type information of the returned events. + */ + public DataGeneratorSourceV3( + MapFunction generatorFunction, + long count, + long sourceRatePerSecond, + TypeInformation typeInfo) { + checkArgument(sourceRatePerSecond > 0, "maxPerSeconds has to be a positive number"); + this.typeInfo = checkNotNull(typeInfo); + this.generatorFunction = checkNotNull(generatorFunction); + this.numberSource = new NumberSequenceSource(0, count); + this.maxPerSecond = sourceRatePerSecond; + } + + public long getCount() { + return numberSource.getTo(); + } + + // ------------------------------------------------------------------------ + // source methods + // ------------------------------------------------------------------------ + + @Override + public TypeInformation getProducedType() { + return typeInfo; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) + throws Exception { + if (maxPerSecond > 0) { + int parallelism = readerContext.getRuntimeContext().getNumberOfParallelSubtasks(); + RateLimiter rateLimiter = new GuavaRateLimiter(maxPerSecond, parallelism); + return new RateLimitedSourceReader<>( + new MappingIteratorSourceReader<>(readerContext, generatorFunction), + rateLimiter); + } else { + return new MappingIteratorSourceReader<>(readerContext, generatorFunction); + } + } + + @Override + public SplitEnumerator> restoreEnumerator( + SplitEnumeratorContext enumContext, + Collection checkpoint) + throws Exception { + return new IteratorSourceEnumerator<>(enumContext, checkpoint); + } + + @Override + public SplitEnumerator> createEnumerator( + final SplitEnumeratorContext enumContext) { + final List splits = + numberSource.splitNumberRange(0, getCount(), enumContext.currentParallelism()); + return new IteratorSourceEnumerator<>(enumContext, splits); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return numberSource.getSplitSerializer(); + } + + @Override + public SimpleVersionedSerializer> + getEnumeratorCheckpointSerializer() { + return numberSource.getEnumeratorCheckpointSerializer(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/NumberSequenceSource.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/NumberSequenceSource.java index 53e6a2832b7be..aba63670ecbca 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/NumberSequenceSource.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/NumberSequenceSource.java @@ -247,7 +247,7 @@ static NumberSequenceSplit deserializeV1(DataInputView in) throws IOException { } } - private static final class CheckpointSerializer + public static final class CheckpointSerializer implements SimpleVersionedSerializer> { private static final int CURRENT_VERSION = 1; diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReader.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReader.java index d7a63c06a186b..396bc57f4f97d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReader.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReader.java @@ -24,17 +24,7 @@ import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.core.io.InputStatus; -import javax.annotation.Nullable; - -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Collections; import java.util.Iterator; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.CompletableFuture; - -import static org.apache.flink.util.Preconditions.checkNotNull; /** * A {@link SourceReader} that returns the values of an iterator, supplied via an {@link @@ -52,47 +42,14 @@ @Public public class IteratorSourceReader< E, IterT extends Iterator, SplitT extends IteratorSourceSplit> - implements SourceReader { - - /** The context for this reader, to communicate with the enumerator. */ - private final SourceReaderContext context; - - /** The availability future. This reader is available as soon as a split is assigned. */ - private CompletableFuture availability; - - /** - * The iterator producing data. Non-null after a split has been assigned. This field is null or - * non-null always together with the {@link #currentSplit} field. - */ - @Nullable private IterT iterator; - - /** - * The split whose data we return. Non-null after a split has been assigned. This field is null - * or non-null always together with the {@link #iterator} field. - */ - @Nullable private SplitT currentSplit; - - /** The remaining splits that were assigned but not yet processed. */ - private final Queue remainingSplits; - - private boolean noMoreSplits; + extends IteratorSourceReaderBase { public IteratorSourceReader(SourceReaderContext context) { - this.context = checkNotNull(context); - this.availability = new CompletableFuture<>(); - this.remainingSplits = new ArrayDeque<>(); + super(context); } // ------------------------------------------------------------------------ - @Override - public void start() { - // request a split if we don't have one - if (remainingSplits.isEmpty()) { - context.sendSplitRequest(); - } - } - @Override public InputStatus pollNext(ReaderOutput output) { if (iterator != null) { @@ -103,77 +60,6 @@ public InputStatus pollNext(ReaderOutput output) { finishSplit(); } } - return tryMoveToNextSplit(); } - - private void finishSplit() { - iterator = null; - currentSplit = null; - - // request another split if no other is left - // we do this only here in the finishSplit part to avoid requesting a split - // whenever the reader is polled and doesn't currently have a split - if (remainingSplits.isEmpty() && !noMoreSplits) { - context.sendSplitRequest(); - } - } - - private InputStatus tryMoveToNextSplit() { - currentSplit = remainingSplits.poll(); - if (currentSplit != null) { - iterator = currentSplit.getIterator(); - return InputStatus.MORE_AVAILABLE; - } else if (noMoreSplits) { - return InputStatus.END_OF_INPUT; - } else { - // ensure we are not called in a loop by resetting the availability future - if (availability.isDone()) { - availability = new CompletableFuture<>(); - } - - return InputStatus.NOTHING_AVAILABLE; - } - } - - @Override - public CompletableFuture isAvailable() { - return availability; - } - - @Override - public void addSplits(List splits) { - remainingSplits.addAll(splits); - // set availability so that pollNext is actually called - availability.complete(null); - } - - @Override - public void notifyNoMoreSplits() { - noMoreSplits = true; - // set availability so that pollNext is actually called - availability.complete(null); - } - - @Override - public List snapshotState(long checkpointId) { - if (currentSplit == null && remainingSplits.isEmpty()) { - return Collections.emptyList(); - } - - final ArrayList allSplits = new ArrayList<>(1 + remainingSplits.size()); - if (iterator != null && iterator.hasNext()) { - assert currentSplit != null; - - @SuppressWarnings("unchecked") - final SplitT inProgressSplit = - (SplitT) currentSplit.getUpdatedSplitForIterator(iterator); - allSplits.add(inProgressSplit); - } - allSplits.addAll(remainingSplits); - return allSplits; - } - - @Override - public void close() throws Exception {} } diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReaderBase.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReaderBase.java new file mode 100644 index 0000000000000..c8957bc124339 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReaderBase.java @@ -0,0 +1,164 @@ +/* + * 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.api.connector.source.lib.util; + +import org.apache.flink.annotation.Public; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; + +import javax.annotation.Nullable; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link SourceReader} that returns the values of an iterator, supplied via an {@link + * IteratorSourceSplit}. + * + *

The {@code IteratorSourceSplit} is also responsible for taking the current iterator and + * turning it back into a split for checkpointing. + * + * @param The type of events returned by the reader. + * @param The type of the iterator that produces the events. This type exists to make the + * conversion between iterator and {@code IteratorSourceSplit} type safe. + * @param The concrete type of the {@code IteratorSourceSplit} that creates and converts + * the iterator that produces this reader's elements. + */ +@Public +abstract class IteratorSourceReaderBase< + E, O, IterT extends Iterator, SplitT extends IteratorSourceSplit> + implements SourceReader { + + /** The context for this reader, to communicate with the enumerator. */ + protected final SourceReaderContext context; + + /** The availability future. This reader is available as soon as a split is assigned. */ + protected CompletableFuture availability; + + /** + * The iterator producing data. Non-null after a split has been assigned. This field is null or + * non-null always together with the {@link #currentSplit} field. + */ + @Nullable protected IterT iterator; + + /** + * The split whose data we return. Non-null after a split has been assigned. This field is null + * or non-null always together with the {@link #iterator} field. + */ + @Nullable protected SplitT currentSplit; + + /** The remaining splits that were assigned but not yet processed. */ + protected final Queue remainingSplits; + + protected boolean noMoreSplits; + + public IteratorSourceReaderBase(SourceReaderContext context) { + this.context = checkNotNull(context); + this.availability = new CompletableFuture<>(); + this.remainingSplits = new ArrayDeque<>(); + } + + // ------------------------------------------------------------------------ + + @Override + public void start() { + // request a split if we don't have one + if (remainingSplits.isEmpty()) { + context.sendSplitRequest(); + } + } + + protected void finishSplit() { + iterator = null; + currentSplit = null; + + // request another split if no other is left + // we do this only here in the finishSplit part to avoid requesting a split + // whenever the reader is polled and doesn't currently have a split + if (remainingSplits.isEmpty() && !noMoreSplits) { + context.sendSplitRequest(); + } + } + + protected InputStatus tryMoveToNextSplit() { + currentSplit = remainingSplits.poll(); + if (currentSplit != null) { + iterator = currentSplit.getIterator(); + return InputStatus.MORE_AVAILABLE; + } else if (noMoreSplits) { + return InputStatus.END_OF_INPUT; + } else { + // ensure we are not called in a loop by resetting the availability future + if (availability.isDone()) { + availability = new CompletableFuture<>(); + } + + return InputStatus.NOTHING_AVAILABLE; + } + } + + @Override + public CompletableFuture isAvailable() { + return availability; + } + + @Override + public void addSplits(List splits) { + remainingSplits.addAll(splits); + // set availability so that pollNext is actually called + availability.complete(null); + } + + @Override + public void notifyNoMoreSplits() { + noMoreSplits = true; + // set availability so that pollNext is actually called + availability.complete(null); + } + + @Override + public List snapshotState(long checkpointId) { + if (currentSplit == null && remainingSplits.isEmpty()) { + return Collections.emptyList(); + } + + final ArrayList allSplits = new ArrayList<>(1 + remainingSplits.size()); + if (iterator != null && iterator.hasNext()) { + assert currentSplit != null; + + @SuppressWarnings("unchecked") + final SplitT inProgressSplit = + (SplitT) currentSplit.getUpdatedSplitForIterator(iterator); + allSplits.add(inProgressSplit); + } + allSplits.addAll(remainingSplits); + return allSplits; + } + + @Override + public void close() throws Exception {} +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/MappingIteratorSourceReader.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/MappingIteratorSourceReader.java new file mode 100644 index 0000000000000..20680441cdfb6 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/MappingIteratorSourceReader.java @@ -0,0 +1,69 @@ +/* + * 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.api.connector.source.lib.util; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.util.FlinkRuntimeException; + +import java.util.Iterator; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +@Experimental +public class MappingIteratorSourceReader< + E, O, IterT extends Iterator, SplitT extends IteratorSourceSplit> + extends IteratorSourceReaderBase { + + private final MapFunction generatorFunction; + + public MappingIteratorSourceReader( + SourceReaderContext context, MapFunction generatorFunction) { + super(context); + this.generatorFunction = checkNotNull(generatorFunction); + } + + // ------------------------------------------------------------------------ + + @Override + public InputStatus pollNext(ReaderOutput output) { + if (iterator != null) { + if (iterator.hasNext()) { + E next = iterator.next(); + try { + O mapped = generatorFunction.map(next); + output.collect(mapped); + } catch (Exception e) { + String message = + String.format( + "A user-provided generator function threw an exception on this input: %s", + next.toString()); + throw new FlinkRuntimeException(message); + } + return InputStatus.MORE_AVAILABLE; + } else { + finishSplit(); + } + } + return tryMoveToNextSplit(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReaderNew.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReader.java similarity index 95% rename from flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReaderNew.java rename to flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReader.java index 1741b796a5502..ad66676a7c03f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReaderNew.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReader.java @@ -18,7 +18,7 @@ package org.apache.flink.api.connector.source.lib.util; -import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.io.ratelimiting.RateLimiter; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReader; @@ -44,15 +44,15 @@ * @param The concrete type of the {@code IteratorSourceSplit} that creates and converts * the iterator that produces this reader's elements. */ -@Public -public class RateLimitedIteratorSourceReaderNew< +@Experimental +public class RateLimitedIteratorSourceReader< E, IterT extends Iterator, SplitT extends IteratorSourceSplit> implements SourceReader { private final IteratorSourceReader iteratorSourceReader; private final RateLimiter rateLimiter; - public RateLimitedIteratorSourceReaderNew( + public RateLimitedIteratorSourceReader( SourceReaderContext readerContext, RateLimiter rateLimiter) { checkNotNull(readerContext); iteratorSourceReader = new IteratorSourceReader<>(readerContext); @@ -60,6 +60,7 @@ public RateLimitedIteratorSourceReaderNew( } // ------------------------------------------------------------------------ + @Override public void start() { iteratorSourceReader.start(); diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedSourceReader.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedSourceReader.java new file mode 100644 index 0000000000000..084b0a945f98d --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedSourceReader.java @@ -0,0 +1,84 @@ +/* + * 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.api.connector.source.lib.util; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.io.ratelimiting.RateLimiter; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.io.InputStatus; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +@Experimental +public class RateLimitedSourceReader + implements SourceReader { + + private final SourceReader sourceReader; + private final RateLimiter rateLimiter; + + public RateLimitedSourceReader(SourceReader sourceReader, RateLimiter rateLimiter) { + checkNotNull(sourceReader); + checkNotNull(rateLimiter); + this.sourceReader = sourceReader; + this.rateLimiter = rateLimiter; + } + + // ------------------------------------------------------------------------ + + @Override + public void start() { + sourceReader.start(); + } + + @Override + public InputStatus pollNext(ReaderOutput output) throws Exception { + rateLimiter.acquire(); + return sourceReader.pollNext(output); + } + + @Override + public CompletableFuture isAvailable() { + return sourceReader.isAvailable(); + } + + @Override + public void addSplits(List splits) { + sourceReader.addSplits(splits); + } + + @Override + public void notifyNoMoreSplits() { + sourceReader.notifyNoMoreSplits(); + } + + @Override + public List snapshotState(long checkpointId) { + return sourceReader.snapshotState(checkpointId); + } + + @Override + public void close() throws Exception { + sourceReader.close(); + } +} diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java deleted file mode 100644 index 8c227bc1a8fff..0000000000000 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourceCheck.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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.streaming.examples.wordcount; - -import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.io.ratelimiting.BucketingRateLimiter; -import org.apache.flink.api.common.io.ratelimiting.RateLimiter; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.connector.source.lib.DataGeneratorSourceV2; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - -import java.time.Duration; - -public class GeneratorSourceCheck { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - int parallelism = 2; - env.setRuntimeMode(RuntimeExecutionMode.STREAMING); - env.setParallelism(parallelism); - - // MapFunction generator = value -> ">>> " + value; - // DataGeneratorSource source = new DataGeneratorSource<>(generator, 10, - // Types.STRING); - // DataStreamSource watermarked = - // env.fromSource(source, WatermarkStrategy.noWatermarks(), "watermarked"); - // watermarked.print(); - - MapFunction generator = index -> ">>> " + index; - RateLimiter rateLimiter = new BucketingRateLimiter(1, parallelism); - // DataGeneratorSource source = - // new DataGeneratorSource<>(generator, 100, rateLimiter, Types.STRING); - - // DataGeneratorSource source = - // new DataGeneratorSource<>(generator, 10, 2, Types.STRING); - - DataGeneratorSourceV2 source = - new DataGeneratorSourceV2<>(generator, 1000, 100, Types.STRING); - - DataStreamSource watermarked = - env.fromSource( - source, - WatermarkStrategy.forBoundedOutOfOrderness(Duration.ofSeconds(1)), - "watermarked"); - watermarked.print(); - - // env.fromSequence(1, 10).print(); - - // DataStreamSource ds = env.fromGenerator(generator, 10, Types.STRING); - // ds.print(); - - /* MapFunction generator = value -> ">>> " + value; - GeneratorSource from = GeneratorSource.from(generator, 10, Types.STRING); - DataStreamSource watermarked = - env.fromSource(from, WatermarkStrategy.noWatermarks(), "watermarked"); - watermarked.print();*/ - - // DataStreamSource ds = env.fromGenerator(generator, 10, Types.STRING); - // ds.print(); - - // DataStreamSource longDataStreamSource = env.fromSequence(0, 10); - // longDataStreamSource.print(); - - // --- - // MapFunction generator2 = value -> ">>>>>> " + value; - // SingleOutputStreamOperator ds2 = env.fromFunction(generator2, 10); - // ds2.print(); - - // Apache Flink applications are composed lazily. Calling execute - // submits the Job and begins processing. - env.execute("WordCount"); - } -} diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourcePOC.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourcePOC.java new file mode 100644 index 0000000000000..14d3c4c094f4c --- /dev/null +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourcePOC.java @@ -0,0 +1,74 @@ +/* + * 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.streaming.examples.wordcount; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.source.lib.DataGeneratorSourceV3; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import java.time.Duration; + +public class GeneratorSourcePOC { + + public static void main(String[] args) throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + int parallelism = 2; + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + env.setParallelism(parallelism); + + MapFunction generator = index -> ">>> " + index; + + /* V0 */ + /* + DataGeneratorSourceV0 source = + new DataGeneratorSourceV0<>(generator, 1000, Types.STRING); + */ + + /* V1 */ + /* + DataGeneratorSource source = + new DataGeneratorSource<>(generator, 1000, 2, Types.STRING); + */ + + /* V2 */ + /* + + DataGeneratorSourceV2 source = + new DataGeneratorSourceV2<>(generator, 1000, 2, Types.STRING); + */ + + /* V3 */ + DataGeneratorSourceV3 source = + new DataGeneratorSourceV3<>(generator, 1000, 2, Types.STRING); + + DataStreamSource watermarked = + env.fromSource( + source, + WatermarkStrategy.forBoundedOutOfOrderness(Duration.ofSeconds(1)), + "watermarked"); + watermarked.print(); + + env.execute("Generator Source POC"); + } +} diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java index 4855ced967b34..0cbfbdd7c63cc 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.testutils.source.reader; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.configuration.Configuration; @@ -91,6 +92,11 @@ public int currentParallelism() { return 1; } + @Override + public RuntimeContext getRuntimeContext() { + return null; + } + // ------------------------------------------------------------------------ public int getNumSplitRequests() { From 798c106e2e3159c94b7d66d56024f5887ded9c77 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Thu, 7 Jul 2022 00:08:15 +0200 Subject: [PATCH 09/12] WIP: remove ReadOnlyContext, keep currentParallelism() --- .../file/src/FileSourceHeavyThroughputTest.java | 6 ------ .../api/connector/source/SourceReaderContext.java | 11 +++-------- .../connector/source/lib/DataGeneratorSourceV3.java | 2 +- .../source/lib/NumberSequenceSourceTest.java | 6 ------ .../flink/streaming/api/operators/SourceOperator.java | 7 ------- .../testutils/source/reader/TestingReaderContext.java | 6 ------ 6 files changed, 4 insertions(+), 34 deletions(-) diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java index 4528b221c1efa..bb44462ab051e 100644 --- a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java +++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/src/FileSourceHeavyThroughputTest.java @@ -19,7 +19,6 @@ package org.apache.flink.connector.file.src; import org.apache.flink.api.common.eventtime.Watermark; -import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.ReaderOutput; @@ -231,11 +230,6 @@ public UserCodeClassLoader getUserCodeClassLoader() { public int currentParallelism() { return 1; } - - @Override - public RuntimeContext getRuntimeContext() { - return null; - } } private static final class NoOpReaderOutput implements ReaderOutput { diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java index c01ac0df0b91e..f2861758b6865 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java @@ -19,7 +19,6 @@ package org.apache.flink.api.connector.source; import org.apache.flink.annotation.Public; -import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.groups.SourceReaderMetricGroup; import org.apache.flink.util.UserCodeClassLoader; @@ -65,14 +64,10 @@ public interface SourceReaderContext { */ UserCodeClassLoader getUserCodeClassLoader(); - // TODO: add JavaDoc - int currentParallelism(); - /** - * Gets the context that contains information about the readers runtime, such as the parallelism - * of the source. + * Get the current parallelism of this Source. * - * @return The runtime context of the source reader. + * @return the parallelism of the Source. */ - RuntimeContext getRuntimeContext(); + int currentParallelism(); } diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java index aa497bec91a81..5fcb15b5af2f2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java @@ -136,7 +136,7 @@ public Boundedness getBoundedness() { public SourceReader createReader(SourceReaderContext readerContext) throws Exception { if (maxPerSecond > 0) { - int parallelism = readerContext.getRuntimeContext().getNumberOfParallelSubtasks(); + int parallelism = readerContext.currentParallelism(); RateLimiter rateLimiter = new GuavaRateLimiter(maxPerSecond, parallelism); return new RateLimitedSourceReader<>( new MappingIteratorSourceReader<>(readerContext, generatorFunction), diff --git a/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java b/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java index daee302729223..ce97ff7126911 100644 --- a/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceTest.java @@ -19,7 +19,6 @@ package org.apache.flink.api.connector.source.lib; import org.apache.flink.api.common.eventtime.Watermark; -import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SourceOutput; @@ -149,11 +148,6 @@ public UserCodeClassLoader getUserCodeClassLoader() { public int currentParallelism() { return 1; } - - @Override - public RuntimeContext getRuntimeContext() { - return null; - } } private static final class TestingReaderOutput implements ReaderOutput { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java index aa56a7f55dd37..b379653ceab7f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.eventtime.WatermarkAlignmentParams; import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; @@ -231,7 +230,6 @@ public void initReader() throws Exception { } final int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); - final RuntimeContext runtimeContext = getRuntimeContext(); final SourceReaderContext context = new SourceReaderContext() { @@ -288,11 +286,6 @@ public void registerReleaseHookIfAbsent( public int currentParallelism() { return getRuntimeContext().getNumberOfParallelSubtasks(); } - - @Override - public RuntimeContext getRuntimeContext() { - return runtimeContext; - } }; sourceReader = readerFactory.apply(context); diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java index 0cbfbdd7c63cc..4855ced967b34 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testutils/source/reader/TestingReaderContext.java @@ -18,7 +18,6 @@ package org.apache.flink.connector.testutils.source.reader; -import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.configuration.Configuration; @@ -92,11 +91,6 @@ public int currentParallelism() { return 1; } - @Override - public RuntimeContext getRuntimeContext() { - return null; - } - // ------------------------------------------------------------------------ public int getNumSplitRequests() { From 4e576f8c5bf93a11c1c77bcb683420f916a43ef1 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 8 Jul 2022 00:48:11 +0200 Subject: [PATCH 10/12] WIP: comments cleanup --- .../source/lib/DataGeneratorSourceV3.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java index 5fcb15b5af2f2..7f8b34ea82100 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java @@ -43,15 +43,15 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * A data source that produces generators N data points in parallel. This source is useful for - * testing and for cases that just need a stream of N events of any kind. + * A data source that produces N data points in parallel. This source is useful for testing and for + * cases that just need a stream of N events of any kind. * *

The source splits the sequence into as many parallel sub-sequences as there are parallel * source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is * limited to one, this will produce one sequence in order. * *

This source is always bounded. For very long sequences (for example over the entire domain of - * long integer values), user may want to consider executing the application in a streaming manner, + * long integer values), user may want to consider executing the application in a s treaming manner, * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. */ @Experimental @@ -66,16 +66,14 @@ public class DataGeneratorSourceV3 private final TypeInformation typeInfo; - public final MapFunction generatorFunction; + private final MapFunction generatorFunction; - /** The end Generator in the sequence, inclusive. */ private final NumberSequenceSource numberSource; private long maxPerSecond = -1; /** - * Creates a new {@code DataGeneratorSource} that produces count records in - * parallel. + * Creates a new {@code DataGeneratorSource} that produces {@code count} records in parallel. * * @param generatorFunction The generator function that receives index numbers and translates * them into events of the output type. @@ -90,8 +88,7 @@ public DataGeneratorSourceV3( } /** - * Creates a new {@code DataGeneratorSource} that produces count records in - * parallel. + * Creates a new {@code DataGeneratorSource} that produces {@code count} records in parallel. * * @param generatorFunction The generator function that receives index numbers and translates * them into events of the output type. @@ -114,7 +111,9 @@ public DataGeneratorSourceV3( this.maxPerSecond = sourceRatePerSecond; } + /** @return The number of records produced by this source. */ public long getCount() { + return numberSource.getTo(); } From ca4a232bce27197506d40e4c850e8925ce4cac06 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 8 Jul 2022 12:40:54 +0200 Subject: [PATCH 11/12] WIP: cleanup --- .../io/ratelimiting/BucketingRateLimiter.java | 74 --- .../io/ratelimiting/NoOpRateLimiter.java | 27 -- .../source/lib/DataGeneratorSourceV0.java | 305 ------------- .../source/lib/DataGeneratorSourceV1.java | 421 ------------------ .../source/lib/DataGeneratorSourceV2.java | 324 -------------- .../util/RateLimitedIteratorSourceReader.java | 99 ---- 6 files changed, 1250 deletions(-) delete mode 100644 flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/BucketingRateLimiter.java delete mode 100644 flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/NoOpRateLimiter.java delete mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV0.java delete mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV1.java delete mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java delete mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReader.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/BucketingRateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/BucketingRateLimiter.java deleted file mode 100644 index 8ff619c2bf30f..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/BucketingRateLimiter.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.api.common.io.ratelimiting; - -import static org.apache.flink.util.Preconditions.checkArgument; - -/** Utility to throttle a thread to a given number of executions (records) per second. */ -public final class BucketingRateLimiter implements RateLimiter { - - private final long maxPerBucket; - private final long nanosInBucket; - - private long endOfCurrentBucketNanos; - private int inCurrentBucket; - - private static final int DEFAULT_BUCKETS_PER_SECOND = 10; - private static final long NANOS_IN_ONE_SECOND = 1_000_000_000L; - - public BucketingRateLimiter(long maxPerSecond, int numParallelExecutors) { - this(maxPerSecond, numParallelExecutors, DEFAULT_BUCKETS_PER_SECOND); - } - - public BucketingRateLimiter(long maxPerSecond, int numParallelExecutors, int bucketsPerSecond) { - checkArgument(maxPerSecond > 0, "maxPerSecond must be a positive number"); - checkArgument(numParallelExecutors > 0, "numParallelExecutors must be greater than 0"); - - final float maxPerSecondPerSubtask = (float) maxPerSecond / numParallelExecutors; - - maxPerBucket = ((int) (maxPerSecondPerSubtask / bucketsPerSecond)) + 1; - nanosInBucket = ((int) (NANOS_IN_ONE_SECOND / maxPerSecondPerSubtask)) * maxPerBucket; - - this.endOfCurrentBucketNanos = System.nanoTime() + nanosInBucket; - this.inCurrentBucket = 0; - } - - // TODO: JavaDoc, returns number of seconds idling on this call. - public int acquire() throws InterruptedException { - if (++inCurrentBucket != maxPerBucket) { - return 0; - } - // The current bucket is "full". Wait until the next bucket. - final long now = System.nanoTime(); - final int millisRemaining = (int) ((endOfCurrentBucketNanos - now) / 1_000_000); - inCurrentBucket = 0; - - if (millisRemaining > 0) { - endOfCurrentBucketNanos += nanosInBucket; - Thread.sleep(millisRemaining); - return millisRemaining; - } else { - // Throttle was not called often enough so that the bucket's capacity was not exhausted - // "in time". We need to push the bucket's "end time" further to compensate and avoid - // bursts in case polling behaviour catches up. - endOfCurrentBucketNanos = now + nanosInBucket; - return 0; - } - } -} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/NoOpRateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/NoOpRateLimiter.java deleted file mode 100644 index 10d55c4211ca8..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/NoOpRateLimiter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.api.common.io.ratelimiting; - -public class NoOpRateLimiter implements RateLimiter { - - @Override - public int acquire() throws InterruptedException { - return 0; - } -} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV0.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV0.java deleted file mode 100644 index 9700fc3158545..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV0.java +++ /dev/null @@ -1,305 +0,0 @@ -/* - * 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.api.connector.source.lib; - -import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; -import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; -import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; -import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.util.NumberSequenceIterator; - -import java.io.IOException; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * A data source that produces generators N data points in parallel. This source is useful for - * testing and for cases that just need a stream of N events of any kind. - * - *

The source splits the sequence into as many parallel sub-sequences as there are parallel - * source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is - * limited to one, this will produce one sequence in order. - * - *

This source is always bounded. For very long sequences (for example over the entire domain of - * long integer values), user may want to consider executing the application in a streaming manner, - * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. - */ -@Experimental -public class DataGeneratorSourceV0 - implements Source< - OUT, - DataGeneratorSourceV0.GeneratorSequenceSplit, - Collection>>, - ResultTypeQueryable { - - private static final long serialVersionUID = 1L; - - private final TypeInformation typeInfo; - - public final MapFunction generatorFunction; - - /** The end Generator in the sequence, inclusive. */ - private final NumberSequenceSource numberSource; - - /** - * Creates a new {@code DataGeneratorSource} that produces count records in - * parallel. - * - * @param typeInfo the type info - * @param generatorFunction the generator function - * @param count The count - */ - public DataGeneratorSourceV0( - MapFunction generatorFunction, long count, TypeInformation typeInfo) { - this.typeInfo = checkNotNull(typeInfo); - this.generatorFunction = checkNotNull(generatorFunction); - this.numberSource = new NumberSequenceSource(0, count); - } - - public long getCount() { - return numberSource.getTo(); - } - - // ------------------------------------------------------------------------ - // source methods - // ------------------------------------------------------------------------ - - @Override - public TypeInformation getProducedType() { - return typeInfo; - } - - @Override - public Boundedness getBoundedness() { - return Boundedness.BOUNDED; - } - - @Override - public SourceReader> createReader( - SourceReaderContext readerContext) { - return new IteratorSourceReader<>(readerContext); - } - - @Override - public SplitEnumerator, Collection>> - createEnumerator( - final SplitEnumeratorContext> enumContext) { - - final List splits = - numberSource.splitNumberRange(0, getCount(), enumContext.currentParallelism()); - return new IteratorSourceEnumerator<>(enumContext, wrapSplits(splits, generatorFunction)); - } - - @Override - public SplitEnumerator, Collection>> - restoreEnumerator( - final SplitEnumeratorContext> enumContext, - Collection> checkpoint) { - return new IteratorSourceEnumerator<>(enumContext, checkpoint); - } - - @Override - public SimpleVersionedSerializer> getSplitSerializer() { - return new SplitSerializer<>(numberSource.getSplitSerializer(), generatorFunction); - } - - @Override - public SimpleVersionedSerializer>> - getEnumeratorCheckpointSerializer() { - return new CheckpointSerializer<>( - numberSource.getEnumeratorCheckpointSerializer(), generatorFunction); - } - - // ------------------------------------------------------------------------ - // splits & checkpoint - // ------------------------------------------------------------------------ - public static class GeneratorSequenceIterator implements Iterator { - - private final MapFunction generatorFunction; - private final NumberSequenceIterator numSeqIterator; - - public GeneratorSequenceIterator( - NumberSequenceIterator numSeqIterator, MapFunction generatorFunction) { - this.generatorFunction = generatorFunction; - this.numSeqIterator = numSeqIterator; - } - - @Override - public boolean hasNext() { - return numSeqIterator.hasNext(); - } - - public long getCurrent() { - return numSeqIterator.getCurrent(); - } - - public long getTo() { - return numSeqIterator.getTo(); - } - - @Override - public T next() { - try { - return generatorFunction.map(numSeqIterator.next()); - } catch (Exception e) { - throw new FlinkRuntimeException( - String.format( - "Exception while generating element %d", - numSeqIterator.getCurrent()), - e); - } - } - } - - /** A split of the source, representing a Generator sub-sequence. */ - public static class GeneratorSequenceSplit - implements IteratorSourceSplit> { - - public GeneratorSequenceSplit( - NumberSequenceSplit numberSequenceSplit, MapFunction generatorFunction) { - this.numberSequenceSplit = numberSequenceSplit; - this.generatorFunction = generatorFunction; - } - - private final NumberSequenceSplit numberSequenceSplit; - - private final MapFunction generatorFunction; - - public GeneratorSequenceIterator getIterator() { - return new GeneratorSequenceIterator<>( - numberSequenceSplit.getIterator(), generatorFunction); - } - - @Override - public String splitId() { - return numberSequenceSplit.splitId(); - } - - @Override - public IteratorSourceSplit> getUpdatedSplitForIterator( - final GeneratorSequenceIterator iterator) { - return new GeneratorSequenceSplit<>( - (NumberSequenceSplit) - numberSequenceSplit.getUpdatedSplitForIterator(iterator.numSeqIterator), - generatorFunction); - } - - @Override - public String toString() { - return String.format( - "GeneratorSequenceSplit [%d, %d] (%s)", - numberSequenceSplit.from(), - numberSequenceSplit.to(), - numberSequenceSplit.splitId()); - } - } - - private static final class SplitSerializer - implements SimpleVersionedSerializer> { - - private final SimpleVersionedSerializer numberSplitSerializer; - private final MapFunction generatorFunction; - - private SplitSerializer( - SimpleVersionedSerializer numberSplitSerializer, - MapFunction generatorFunction) { - this.numberSplitSerializer = numberSplitSerializer; - this.generatorFunction = generatorFunction; - } - - @Override - public int getVersion() { - return numberSplitSerializer.getVersion(); - } - - @Override - public byte[] serialize(GeneratorSequenceSplit split) throws IOException { - return numberSplitSerializer.serialize(split.numberSequenceSplit); - } - - @Override - public GeneratorSequenceSplit deserialize(int version, byte[] serialized) - throws IOException { - return new GeneratorSequenceSplit<>( - numberSplitSerializer.deserialize(version, serialized), generatorFunction); - } - } - - private static final class CheckpointSerializer - implements SimpleVersionedSerializer>> { - - private final SimpleVersionedSerializer> - numberCheckpointSerializer; - private final MapFunction generatorFunction; - - public CheckpointSerializer( - SimpleVersionedSerializer> - numberCheckpointSerializer, - MapFunction generatorFunction) { - this.numberCheckpointSerializer = numberCheckpointSerializer; - this.generatorFunction = generatorFunction; - } - - @Override - public int getVersion() { - return numberCheckpointSerializer.getVersion(); - } - - @Override - public byte[] serialize(Collection> checkpoint) - throws IOException { - return numberCheckpointSerializer.serialize( - checkpoint.stream() - .map(split -> split.numberSequenceSplit) - .collect(Collectors.toList())); - } - - @Override - public Collection> deserialize(int version, byte[] serialized) - throws IOException { - Collection numberSequenceSplits = - numberCheckpointSerializer.deserialize(version, serialized); - return wrapSplits(numberSequenceSplits, generatorFunction); - } - } - - private static List> wrapSplits( - Collection numberSequenceSplits, - MapFunction generatorFunction) { - return numberSequenceSplits.stream() - .map(split -> new GeneratorSequenceSplit<>(split, generatorFunction)) - .collect(Collectors.toList()); - } -} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV1.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV1.java deleted file mode 100644 index 20c9aef9a6f56..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV1.java +++ /dev/null @@ -1,421 +0,0 @@ -/* - * 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.api.connector.source.lib; - -import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.io.ratelimiting.GuavaRateLimiter; -import org.apache.flink.api.common.io.ratelimiting.NoOpRateLimiter; -import org.apache.flink.api.common.io.ratelimiting.RateLimiter; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.connector.source.lib.DataGeneratorSourceV1.GeneratorSequenceSplit; -import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; -import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; -import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; -import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.util.NumberSequenceIterator; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * A data source that produces generators N data points in parallel. This source is useful for - * testing and for cases that just need a stream of N events of any kind. - * - *

The source splits the sequence into as many parallel sub-sequences as there are parallel - * source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is - * limited to one, this will produce one sequence in order. - * - *

This source is always bounded. For very long sequences (for example over the entire domain of - * long integer values), user may want to consider executing the application in a streaming manner, - * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. - */ -@Experimental -public class DataGeneratorSourceV1 - implements Source< - OUT, GeneratorSequenceSplit, Collection>>, - ResultTypeQueryable { - - private static final Logger LOG = LoggerFactory.getLogger(DataGeneratorSourceV1.class); - - private static final long serialVersionUID = 1L; - - private final TypeInformation typeInfo; - - private final MapFunction generatorFunction; - - /** The end Generator in the sequence, inclusive. */ - private final NumberSequenceSource numberSource; - - private long maxPerSecond = -1; - - /** - * Creates a new {@code DataGeneratorSource} that produces count records in - * parallel. - * - * @param typeInfo the type info - * @param generatorFunction the generator function - * @param count The count - */ - public DataGeneratorSourceV1( - MapFunction generatorFunction, long count, TypeInformation typeInfo) { - this.typeInfo = checkNotNull(typeInfo); - this.generatorFunction = checkNotNull(generatorFunction); - this.numberSource = new NumberSequenceSource(0, count); - } - - public DataGeneratorSourceV1( - MapFunction generatorFunction, - long count, - long maxPerSecond, - TypeInformation typeInfo) { - checkArgument(maxPerSecond > 0, "maxPerSeconds has to be a positive number"); - this.typeInfo = checkNotNull(typeInfo); - this.generatorFunction = checkNotNull(generatorFunction); - this.numberSource = new NumberSequenceSource(0, count); - this.maxPerSecond = maxPerSecond; - } - - public long getCount() { - return numberSource.getTo(); - } - - // ------------------------------------------------------------------------ - // source methods - // ------------------------------------------------------------------------ - - @Override - public TypeInformation getProducedType() { - return typeInfo; - } - - @Override - public Boundedness getBoundedness() { - return Boundedness.BOUNDED; - } - - @Override - public SourceReader> createReader( - SourceReaderContext readerContext) { - return new IteratorSourceReader<>(readerContext); - } - - @Override - public SplitEnumerator, Collection>> - createEnumerator( - final SplitEnumeratorContext> enumContext) { - - final int parallelism = enumContext.currentParallelism(); - final List splits = - numberSource.splitNumberRange(0, getCount(), parallelism); - - return new IteratorSourceEnumerator<>( - enumContext, wrapSplits(splits, generatorFunction, maxPerSecond, parallelism)); - } - - @Override - public SplitEnumerator, Collection>> - restoreEnumerator( - final SplitEnumeratorContext> enumContext, - Collection> checkpoint) { - return new IteratorSourceEnumerator<>(enumContext, checkpoint); - } - - @Override - public SimpleVersionedSerializer> getSplitSerializer() { - return new SplitSerializer<>(generatorFunction, maxPerSecond); - } - - @Override - public SimpleVersionedSerializer>> - getEnumeratorCheckpointSerializer() { - return new CheckpointSerializer<>(generatorFunction, maxPerSecond); - } - - // ------------------------------------------------------------------------ - // splits & checkpoint - // ------------------------------------------------------------------------ - public static class GeneratorSequenceIterator implements Iterator { - - private final MapFunction generatorFunction; - private final NumberSequenceIterator numSeqIterator; - private RateLimiter rateLimiter = new NoOpRateLimiter(); - - public GeneratorSequenceIterator( - NumberSequenceIterator numSeqIterator, - MapFunction generatorFunction, - long maxPerSecond, - int parallelism) { - this.generatorFunction = generatorFunction; - this.numSeqIterator = numSeqIterator; - if (maxPerSecond > 0) { - this.rateLimiter = new GuavaRateLimiter(maxPerSecond, parallelism); - } - } - - @Override - public boolean hasNext() { - return numSeqIterator.hasNext(); - } - - public long getCurrent() { - return numSeqIterator.getCurrent(); - } - - public long getTo() { - return numSeqIterator.getTo(); - } - - @Override - public T next() { - try { - rateLimiter.acquire(); - return generatorFunction.map(numSeqIterator.next()); - } catch (Exception e) { - throw new FlinkRuntimeException( - String.format( - "Exception while generating element %d", - numSeqIterator.getCurrent()), - e); - } - } - } - - /** A split of the source, representing a Generator sub-sequence. */ - public static class GeneratorSequenceSplit - implements IteratorSourceSplit> { - - public GeneratorSequenceSplit( - NumberSequenceSplit numberSequenceSplit, - MapFunction generatorFunction, - long maxPerSecond, - int parallelism) { - this.numberSequenceSplit = numberSequenceSplit; - this.generatorFunction = generatorFunction; - this.maxPerSecond = maxPerSecond; - this.parallelism = parallelism; - } - - private final NumberSequenceSplit numberSequenceSplit; - private final MapFunction generatorFunction; - private final long maxPerSecond; - private final int parallelism; - - public GeneratorSequenceIterator getIterator() { - return new GeneratorSequenceIterator<>( - numberSequenceSplit.getIterator(), - generatorFunction, - maxPerSecond, - parallelism); - } - - @Override - public String splitId() { - return numberSequenceSplit.splitId(); - } - - @Override - public IteratorSourceSplit> getUpdatedSplitForIterator( - final GeneratorSequenceIterator iterator) { - return new GeneratorSequenceSplit<>( - (NumberSequenceSplit) - numberSequenceSplit.getUpdatedSplitForIterator(iterator.numSeqIterator), - generatorFunction, - maxPerSecond, - parallelism); - } - - @Override - public String toString() { - return String.format( - "GeneratorSequenceSplit [%d, %d] (%s)", - numberSequenceSplit.from(), - numberSequenceSplit.to(), - numberSequenceSplit.splitId()); - } - } - - private static final class SplitSerializer - implements SimpleVersionedSerializer> { - - private static final int CURRENT_VERSION = 1; - - private final MapFunction generatorFunction; - private final long maxPerSecond; - - private SplitSerializer(MapFunction generatorFunction, long maxPerSecond) { - this.generatorFunction = generatorFunction; - this.maxPerSecond = maxPerSecond; - } - - @Override - public int getVersion() { - return CURRENT_VERSION; - } - - @Override - public byte[] serialize(GeneratorSequenceSplit split) throws IOException { - checkArgument( - split.getClass() == GeneratorSequenceSplit.class, - "cannot serialize subclasses"); - - // We will serialize 2 longs (16 bytes) plus the UTF representation of the string (2 + - // length) - final DataOutputSerializer out = - new DataOutputSerializer(split.splitId().length() + 18); - serializeV1(out, split); - return out.getCopyOfBuffer(); - } - - @Override - public GeneratorSequenceSplit deserialize(int version, byte[] serialized) - throws IOException { - if (version != CURRENT_VERSION) { - throw new IOException("Unrecognized version: " + version); - } - final DataInputDeserializer in = new DataInputDeserializer(serialized); - return deserializeV1(in, generatorFunction, maxPerSecond); - } - - static void serializeV1(DataOutputView out, GeneratorSequenceSplit split) - throws IOException { - serializeNumberSequenceSplit(out, split.numberSequenceSplit); - out.writeInt(split.parallelism); - } - - static void serializeNumberSequenceSplit( - DataOutputView out, NumberSequenceSplit numberSequenceSplit) throws IOException { - out.writeUTF(numberSequenceSplit.splitId()); - out.writeLong(numberSequenceSplit.from()); - out.writeLong(numberSequenceSplit.to()); - } - - static GeneratorSequenceSplit deserializeV1( - DataInputView in, MapFunction generatorFunction, long maxPerSecond) - throws IOException { - NumberSequenceSplit numberSequenceSplit = deserializeNumberSequenceSplit(in); - int parallelism = in.readInt(); - return new GeneratorSequenceSplit<>( - numberSequenceSplit, generatorFunction, maxPerSecond, parallelism); - } - - private static NumberSequenceSplit deserializeNumberSequenceSplit(DataInputView in) - throws IOException { - return new NumberSequenceSplit(in.readUTF(), in.readLong(), in.readLong()); - } - } - - private static final class CheckpointSerializer - implements SimpleVersionedSerializer>> { - - private static final int CURRENT_VERSION = 1; - - @Override - public int getVersion() { - return CURRENT_VERSION; - } - - private final MapFunction generatorFunction; - private final long maxPerSecond; - - public CheckpointSerializer(MapFunction generatorFunction, long maxPerSecond) { - this.generatorFunction = generatorFunction; - this.maxPerSecond = maxPerSecond; - } - - @Override - public byte[] serialize(Collection> checkpoint) - throws IOException { - // Each split needs 2 longs (16 bytes) plus the UTG representation of the string (2 + - // length). - // Assuming at most 4 digit split IDs, 22 bytes per split avoids any intermediate array - // resizing. - // Plus four bytes for the length field. - // Plus four bytes for the parallelism. - final DataOutputSerializer out = - new DataOutputSerializer(checkpoint.size() * 22 + 4 + 4); - out.writeInt(checkpoint.size()); - for (GeneratorSequenceSplit split : checkpoint) { - DataGeneratorSourceV1.SplitSerializer.serializeNumberSequenceSplit( - out, split.numberSequenceSplit); - } - - final Optional> aSplit = checkpoint.stream().findFirst(); - if (aSplit.isPresent()) { - int parallelism = aSplit.get().parallelism; - out.writeInt(parallelism); - } - - return out.getCopyOfBuffer(); - } - - @Override - public Collection> deserialize(int version, byte[] serialized) - throws IOException { - if (version != CURRENT_VERSION) { - throw new IOException("Unrecognized version: " + version); - } - final DataInputDeserializer in = new DataInputDeserializer(serialized); - final int num = in.readInt(); - final List result = new ArrayList<>(num); - for (int remaining = num; remaining > 0; remaining--) { - result.add(SplitSerializer.deserializeNumberSequenceSplit(in)); - } - final int parallelism = in.readInt(); - return wrapSplits(result, generatorFunction, maxPerSecond, parallelism); - } - } - - private static List> wrapSplits( - Collection numberSequenceSplits, - MapFunction generatorFunction, - long maxPerSecond, - int parallelism) { - return numberSequenceSplits.stream() - .map( - split -> - new GeneratorSequenceSplit<>( - split, generatorFunction, maxPerSecond, parallelism)) - .collect(Collectors.toList()); - } -} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java deleted file mode 100644 index 012adc19b0f52..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV2.java +++ /dev/null @@ -1,324 +0,0 @@ -/* - * 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.api.connector.source.lib; - -import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.io.ratelimiting.GuavaRateLimiter; -import org.apache.flink.api.common.io.ratelimiting.RateLimiter; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; -import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; -import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; -import org.apache.flink.api.connector.source.lib.util.RateLimitedIteratorSourceReader; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.util.NumberSequenceIterator; - -import java.io.IOException; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * A data source that produces generators N data points in parallel. This source is useful for - * testing and for cases that just need a stream of N events of any kind. - * - *

The source splits the sequence into as many parallel sub-sequences as there are parallel - * source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is - * limited to one, this will produce one sequence in order. - * - *

This source is always bounded. For very long sequences (for example over the entire domain of - * long integer values), user may want to consider executing the application in a streaming manner, - * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. - */ -@Experimental -public class DataGeneratorSourceV2 - implements Source< - OUT, - DataGeneratorSourceV2.GeneratorSequenceSplit, - Collection>>, - ResultTypeQueryable { - - private static final long serialVersionUID = 1L; - - private final TypeInformation typeInfo; - - public final MapFunction generatorFunction; - - /** The end Generator in the sequence, inclusive. */ - private final NumberSequenceSource numberSource; - - private long maxPerSecond = -1; - - /** - * Creates a new {@code DataGeneratorSource} that produces count records in - * parallel. - * - * @param typeInfo the type info - * @param generatorFunction the generator function - * @param count The count - */ - public DataGeneratorSourceV2( - MapFunction generatorFunction, long count, TypeInformation typeInfo) { - this.typeInfo = checkNotNull(typeInfo); - this.generatorFunction = checkNotNull(generatorFunction); - this.numberSource = new NumberSequenceSource(0, count); - } - - public DataGeneratorSourceV2( - MapFunction generatorFunction, - long count, - long maxPerSecond, - TypeInformation typeInfo) { - checkArgument(maxPerSecond > 0, "maxPerSeconds has to be a positive number"); - this.typeInfo = checkNotNull(typeInfo); - this.generatorFunction = checkNotNull(generatorFunction); - this.numberSource = new NumberSequenceSource(0, count); - this.maxPerSecond = maxPerSecond; - } - - public long getCount() { - return numberSource.getTo(); - } - - // ------------------------------------------------------------------------ - // source methods - // ------------------------------------------------------------------------ - - @Override - public TypeInformation getProducedType() { - return typeInfo; - } - - @Override - public Boundedness getBoundedness() { - return Boundedness.BOUNDED; - } - - @Override - public SourceReader> createReader( - SourceReaderContext readerContext) { - RateLimiter rateLimiter = - new GuavaRateLimiter(maxPerSecond, readerContext.currentParallelism()); - return new RateLimitedIteratorSourceReader<>(readerContext, rateLimiter); - } - - @Override - public SplitEnumerator, Collection>> - createEnumerator( - final SplitEnumeratorContext> enumContext) { - - final List splits = - numberSource.splitNumberRange(0, getCount(), enumContext.currentParallelism()); - return new IteratorSourceEnumerator<>(enumContext, wrapSplits(splits, generatorFunction)); - } - - @Override - public SplitEnumerator, Collection>> - restoreEnumerator( - final SplitEnumeratorContext> enumContext, - Collection> checkpoint) { - return new IteratorSourceEnumerator<>(enumContext, checkpoint); - } - - @Override - public SimpleVersionedSerializer> getSplitSerializer() { - return new SplitSerializer<>(numberSource.getSplitSerializer(), generatorFunction); - } - - @Override - public SimpleVersionedSerializer>> - getEnumeratorCheckpointSerializer() { - return new CheckpointSerializer<>( - numberSource.getEnumeratorCheckpointSerializer(), generatorFunction); - } - - // ------------------------------------------------------------------------ - // splits & checkpoint - // ------------------------------------------------------------------------ - public static class GeneratorSequenceIterator implements Iterator { - - private final MapFunction generatorFunction; - private final NumberSequenceIterator numSeqIterator; - - public GeneratorSequenceIterator( - NumberSequenceIterator numSeqIterator, MapFunction generatorFunction) { - this.generatorFunction = generatorFunction; - this.numSeqIterator = numSeqIterator; - } - - @Override - public boolean hasNext() { - return numSeqIterator.hasNext(); - } - - public long getCurrent() { - return numSeqIterator.getCurrent(); - } - - public long getTo() { - return numSeqIterator.getTo(); - } - - @Override - public T next() { - try { - return generatorFunction.map(numSeqIterator.next()); - } catch (Exception e) { - throw new FlinkRuntimeException( - String.format( - "Exception while generating element %d", - numSeqIterator.getCurrent()), - e); - } - } - } - - /** A split of the source, representing a Generator sub-sequence. */ - public static class GeneratorSequenceSplit - implements IteratorSourceSplit> { - - public GeneratorSequenceSplit( - NumberSequenceSplit numberSequenceSplit, MapFunction generatorFunction) { - this.numberSequenceSplit = numberSequenceSplit; - this.generatorFunction = generatorFunction; - } - - private final NumberSequenceSplit numberSequenceSplit; - - private final MapFunction generatorFunction; - - public GeneratorSequenceIterator getIterator() { - return new GeneratorSequenceIterator<>( - numberSequenceSplit.getIterator(), generatorFunction); - } - - @Override - public String splitId() { - return numberSequenceSplit.splitId(); - } - - @Override - public IteratorSourceSplit> getUpdatedSplitForIterator( - final GeneratorSequenceIterator iterator) { - return new GeneratorSequenceSplit<>( - (NumberSequenceSplit) - numberSequenceSplit.getUpdatedSplitForIterator(iterator.numSeqIterator), - generatorFunction); - } - - @Override - public String toString() { - return String.format( - "GeneratorSequenceSplit [%d, %d] (%s)", - numberSequenceSplit.from(), - numberSequenceSplit.to(), - numberSequenceSplit.splitId()); - } - } - - private static final class SplitSerializer - implements SimpleVersionedSerializer> { - - private final SimpleVersionedSerializer numberSplitSerializer; - private final MapFunction generatorFunction; - - private SplitSerializer( - SimpleVersionedSerializer numberSplitSerializer, - MapFunction generatorFunction) { - this.numberSplitSerializer = numberSplitSerializer; - this.generatorFunction = generatorFunction; - } - - @Override - public int getVersion() { - return numberSplitSerializer.getVersion(); - } - - @Override - public byte[] serialize(GeneratorSequenceSplit split) throws IOException { - return numberSplitSerializer.serialize(split.numberSequenceSplit); - } - - @Override - public GeneratorSequenceSplit deserialize(int version, byte[] serialized) - throws IOException { - return new GeneratorSequenceSplit<>( - numberSplitSerializer.deserialize(version, serialized), generatorFunction); - } - } - - private static final class CheckpointSerializer - implements SimpleVersionedSerializer>> { - - private final SimpleVersionedSerializer> - numberCheckpointSerializer; - private final MapFunction generatorFunction; - - public CheckpointSerializer( - SimpleVersionedSerializer> - numberCheckpointSerializer, - MapFunction generatorFunction) { - this.numberCheckpointSerializer = numberCheckpointSerializer; - this.generatorFunction = generatorFunction; - } - - @Override - public int getVersion() { - return numberCheckpointSerializer.getVersion(); - } - - @Override - public byte[] serialize(Collection> checkpoint) - throws IOException { - return numberCheckpointSerializer.serialize( - checkpoint.stream() - .map(split -> split.numberSequenceSplit) - .collect(Collectors.toList())); - } - - @Override - public Collection> deserialize(int version, byte[] serialized) - throws IOException { - Collection numberSequenceSplits = - numberCheckpointSerializer.deserialize(version, serialized); - return wrapSplits(numberSequenceSplits, generatorFunction); - } - } - - private static List> wrapSplits( - Collection numberSequenceSplits, - MapFunction generatorFunction) { - return numberSequenceSplits.stream() - .map(split -> new GeneratorSequenceSplit<>(split, generatorFunction)) - .collect(Collectors.toList()); - } -} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReader.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReader.java deleted file mode 100644 index ad66676a7c03f..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedIteratorSourceReader.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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.api.connector.source.lib.util; - -import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.common.io.ratelimiting.RateLimiter; -import org.apache.flink.api.connector.source.ReaderOutput; -import org.apache.flink.api.connector.source.SourceReader; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.core.io.InputStatus; - -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.CompletableFuture; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * A {@link SourceReader} that returns the values of an iterator, supplied via an {@link - * IteratorSourceSplit}. - * - *

The {@code IteratorSourceSplit} is also responsible for taking the current iterator and - * turning it back into a split for checkpointing. - * - * @param The type of events returned by the reader. - * @param The type of the iterator that produces the events. This type exists to make the - * conversion between iterator and {@code IteratorSourceSplit} type safe. - * @param The concrete type of the {@code IteratorSourceSplit} that creates and converts - * the iterator that produces this reader's elements. - */ -@Experimental -public class RateLimitedIteratorSourceReader< - E, IterT extends Iterator, SplitT extends IteratorSourceSplit> - implements SourceReader { - - private final IteratorSourceReader iteratorSourceReader; - private final RateLimiter rateLimiter; - - public RateLimitedIteratorSourceReader( - SourceReaderContext readerContext, RateLimiter rateLimiter) { - checkNotNull(readerContext); - iteratorSourceReader = new IteratorSourceReader<>(readerContext); - this.rateLimiter = rateLimiter; - } - - // ------------------------------------------------------------------------ - - @Override - public void start() { - iteratorSourceReader.start(); - } - - @Override - public InputStatus pollNext(ReaderOutput output) throws InterruptedException { - rateLimiter.acquire(); - return iteratorSourceReader.pollNext(output); - } - - @Override - public CompletableFuture isAvailable() { - return iteratorSourceReader.isAvailable(); - } - - @Override - public void addSplits(List splits) { - iteratorSourceReader.addSplits(splits); - } - - @Override - public void notifyNoMoreSplits() { - iteratorSourceReader.notifyNoMoreSplits(); - } - - @Override - public List snapshotState(long checkpointId) { - return iteratorSourceReader.snapshotState(checkpointId); - } - - @Override - public void close() throws Exception { - iteratorSourceReader.close(); - } -} From 38febefeb686b517910cdab3eb7f84ce352f2ead Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Tue, 19 Jul 2022 01:09:48 +0200 Subject: [PATCH 12/12] WIP: DataGeneratorV4 --- .../io/ratelimiting/GuavaRateLimiter.java | 2 +- .../source/lib/DataGeneratorSourceV3.java | 6 +- .../source/lib/DataGeneratorSourceV4.java | 141 ++++++++++++++++++ .../source/lib/GeneratorFunction.java | 38 +++++ .../lib/GeneratorSourceReaderFactory.java | 55 +++++++ .../source/lib/NumberSequenceSource.java | 2 +- .../source/lib/SourceReaderFactory.java | 29 ++++ ...va => GeneratingIteratorSourceReader.java} | 31 +++- .../wordcount/GeneratorSourcePOC.java | 35 ++++- 9 files changed, 328 insertions(+), 11 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV4.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/GeneratorFunction.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/GeneratorSourceReaderFactory.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/lib/SourceReaderFactory.java rename flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/{MappingIteratorSourceReader.java => GeneratingIteratorSourceReader.java} (73%) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/GuavaRateLimiter.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/GuavaRateLimiter.java index 264e8fff1d0ab..a6c3216d04ee9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/GuavaRateLimiter.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/ratelimiting/GuavaRateLimiter.java @@ -32,6 +32,6 @@ public GuavaRateLimiter(long maxPerSecond, int numParallelExecutors) { @Override public int acquire() { - return (int) rateLimiter.acquire(); + return (int) (1000 * rateLimiter.acquire()); } } diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java index 7f8b34ea82100..21f4297ea2b5c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV3.java @@ -30,8 +30,8 @@ import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; +import org.apache.flink.api.connector.source.lib.util.GeneratingIteratorSourceReader; import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; -import org.apache.flink.api.connector.source.lib.util.MappingIteratorSourceReader; import org.apache.flink.api.connector.source.lib.util.RateLimitedSourceReader; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -138,10 +138,10 @@ public SourceReader createReader(SourceReaderContext r int parallelism = readerContext.currentParallelism(); RateLimiter rateLimiter = new GuavaRateLimiter(maxPerSecond, parallelism); return new RateLimitedSourceReader<>( - new MappingIteratorSourceReader<>(readerContext, generatorFunction), + new GeneratingIteratorSourceReader<>(readerContext, generatorFunction), rateLimiter); } else { - return new MappingIteratorSourceReader<>(readerContext, generatorFunction); + return new GeneratingIteratorSourceReader<>(readerContext, generatorFunction); } } diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV4.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV4.java new file mode 100644 index 0000000000000..21ac37a5e71fc --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSourceV4.java @@ -0,0 +1,141 @@ +/* + * 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.api.connector.source.lib; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.typeinfo.TypeInformation; +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.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.lib.NumberSequenceSource.NumberSequenceSplit; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.util.Collection; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A data source that produces N data points in parallel. This source is useful for testing and for + * cases that just need a stream of N events of any kind. + * + *

The source splits the sequence into as many parallel sub-sequences as there are parallel + * source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is + * limited to one, this will produce one sequence in order. + * + *

This source is always bounded. For very long sequences (for example over the entire domain of + * long integer values), user may want to consider executing the application in a s treaming manner, + * because, despite the fact that the produced stream is bounded, the end bound is pretty far away. + */ +@Experimental +public class DataGeneratorSourceV4 + implements Source>, + ResultTypeQueryable { + + private static final long serialVersionUID = 1L; + + private final SourceReaderFactory sourceReaderFactory; + private final TypeInformation typeInfo; + + private final NumberSequenceSource numberSource; + + public DataGeneratorSourceV4( + SourceReaderFactory sourceReaderFactory, + long count, + TypeInformation typeInfo) { + this.sourceReaderFactory = checkNotNull(sourceReaderFactory); + this.typeInfo = checkNotNull(typeInfo); + this.numberSource = new NumberSequenceSource(0, count); + } + + public DataGeneratorSourceV4( + GeneratorFunction generatorFunction, + long count, + long sourceRatePerSecond, + TypeInformation typeInfo) { + this( + new GeneratorSourceReaderFactory<>(generatorFunction, sourceRatePerSecond), + count, + typeInfo); + } + + public DataGeneratorSourceV4( + GeneratorFunction generatorFunction, + long count, + TypeInformation typeInfo) { + this(generatorFunction, count, -1, typeInfo); + } + + /** @return The number of records produced by this source. */ + public long getCount() { + return numberSource.getTo(); + } + + // ------------------------------------------------------------------------ + // source methods + // ------------------------------------------------------------------------ + + @Override + public TypeInformation getProducedType() { + return typeInfo; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) + throws Exception { + return sourceReaderFactory.newSourceReader(readerContext); + } + + @Override + public SplitEnumerator> restoreEnumerator( + SplitEnumeratorContext enumContext, + Collection checkpoint) + throws Exception { + return new IteratorSourceEnumerator<>(enumContext, checkpoint); + } + + @Override + public SplitEnumerator> createEnumerator( + final SplitEnumeratorContext enumContext) { + final List splits = + numberSource.splitNumberRange(0, getCount(), enumContext.currentParallelism()); + return new IteratorSourceEnumerator<>(enumContext, splits); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return numberSource.getSplitSerializer(); + } + + @Override + public SimpleVersionedSerializer> + getEnumeratorCheckpointSerializer() { + return numberSource.getEnumeratorCheckpointSerializer(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/GeneratorFunction.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/GeneratorFunction.java new file mode 100644 index 0000000000000..ea506b2923156 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/GeneratorFunction.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.api.connector.source.lib; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.connector.source.SourceReaderContext; + +@Experimental +public interface GeneratorFunction extends Function { + + /** + * Initialization method for the function. It is called once before the actual working process + * methods. + */ + default void open(SourceReaderContext readerContext) throws Exception {} + + /** Tear-down method for the function. */ + default void close() throws Exception {} + + O map(T value) throws Exception; +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/GeneratorSourceReaderFactory.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/GeneratorSourceReaderFactory.java new file mode 100644 index 0000000000000..003f058875e61 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/GeneratorSourceReaderFactory.java @@ -0,0 +1,55 @@ +/* + * 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.api.connector.source.lib; + +import org.apache.flink.api.common.io.ratelimiting.GuavaRateLimiter; +import org.apache.flink.api.common.io.ratelimiting.RateLimiter; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.lib.util.GeneratingIteratorSourceReader; +import org.apache.flink.api.connector.source.lib.util.RateLimitedSourceReader; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +public class GeneratorSourceReaderFactory + implements SourceReaderFactory { + + private final GeneratorFunction generatorFunction; + private final long sourceRatePerSecond; + + public GeneratorSourceReaderFactory( + GeneratorFunction generatorFunction, long sourceRatePerSecond) { + this.generatorFunction = checkNotNull(generatorFunction); + this.sourceRatePerSecond = sourceRatePerSecond; + } + + @Override + public SourceReader newSourceReader( + SourceReaderContext readerContext) { + if (sourceRatePerSecond > 0) { + int parallelism = readerContext.currentParallelism(); + RateLimiter rateLimiter = new GuavaRateLimiter(sourceRatePerSecond, parallelism); + return new RateLimitedSourceReader<>( + new GeneratingIteratorSourceReader<>(readerContext, generatorFunction), + rateLimiter); + } else { + return new GeneratingIteratorSourceReader<>(readerContext, generatorFunction); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/NumberSequenceSource.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/NumberSequenceSource.java index aba63670ecbca..f8814d3c72981 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/NumberSequenceSource.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/NumberSequenceSource.java @@ -247,7 +247,7 @@ static NumberSequenceSplit deserializeV1(DataInputView in) throws IOException { } } - public static final class CheckpointSerializer + static final class CheckpointSerializer implements SimpleVersionedSerializer> { private static final int CURRENT_VERSION = 1; diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/SourceReaderFactory.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/SourceReaderFactory.java new file mode 100644 index 0000000000000..5b2ce7f95c7c4 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/SourceReaderFactory.java @@ -0,0 +1,29 @@ +/* + * 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.api.connector.source.lib; + +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 java.io.Serializable; + +public interface SourceReaderFactory extends Serializable { + SourceReader newSourceReader(SourceReaderContext readerContext); +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/MappingIteratorSourceReader.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/GeneratingIteratorSourceReader.java similarity index 73% rename from flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/MappingIteratorSourceReader.java rename to flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/GeneratingIteratorSourceReader.java index 20680441cdfb6..8f702724ba384 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/MappingIteratorSourceReader.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/GeneratingIteratorSourceReader.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.lib.GeneratorFunction; import org.apache.flink.core.io.InputStatus; import org.apache.flink.util.FlinkRuntimeException; @@ -30,15 +31,23 @@ import static org.apache.flink.util.Preconditions.checkNotNull; @Experimental -public class MappingIteratorSourceReader< +public class GeneratingIteratorSourceReader< E, O, IterT extends Iterator, SplitT extends IteratorSourceSplit> extends IteratorSourceReaderBase { - private final MapFunction generatorFunction; + private final GeneratorFunction generatorFunction; - public MappingIteratorSourceReader( + // TODO: [tmp] kept for compatibility with V3. To remove if V4 is accepted. + public GeneratingIteratorSourceReader( SourceReaderContext context, MapFunction generatorFunction) { super(context); + checkNotNull(generatorFunction); + this.generatorFunction = generatorFunction::map; + } + + public GeneratingIteratorSourceReader( + SourceReaderContext context, GeneratorFunction generatorFunction) { + super(context); this.generatorFunction = checkNotNull(generatorFunction); } @@ -66,4 +75,20 @@ public InputStatus pollNext(ReaderOutput output) { } return tryMoveToNextSplit(); } + + @Override + public void close() throws Exception { + super.close(); + generatorFunction.close(); + } + + @Override + public void start() { + super.start(); + try { + generatorFunction.open(context); + } catch (Exception e) { + throw new FlinkRuntimeException("Failed to open the GeneratorFunction", e); + } + } } diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourcePOC.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourcePOC.java index 14d3c4c094f4c..4c568cb2a50cf 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourcePOC.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/GeneratorSourcePOC.java @@ -22,7 +22,10 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.connector.source.lib.DataGeneratorSourceV3; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.lib.DataGeneratorSourceV4; +import org.apache.flink.api.connector.source.lib.GeneratorFunction; +import org.apache.flink.metrics.groups.SourceReaderMetricGroup; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -59,8 +62,34 @@ public static void main(String[] args) throws Exception { */ /* V3 */ - DataGeneratorSourceV3 source = - new DataGeneratorSourceV3<>(generator, 1000, 2, Types.STRING); + /* + DataGeneratorSourceV3 source = + new DataGeneratorSourceV3<>(generator, 1000, 2, Types.STRING); + */ + + GeneratorFunction generatorFunction = + new GeneratorFunction() { + + transient SourceReaderMetricGroup sourceReaderMetricGroup; + + @Override + public void open(SourceReaderContext readerContext) { + sourceReaderMetricGroup = readerContext.metricGroup(); + } + + @Override + public String map(Long value) { + return "Generated: >> " + + value.toString() + + "; local metric group: " + + sourceReaderMetricGroup.hashCode(); + } + }; + + GeneratorFunction generatorFunctionStateless = index -> ">>> " + index; + + DataGeneratorSourceV4 source = + new DataGeneratorSourceV4<>(generatorFunction, 1000, 2, Types.STRING); DataStreamSource watermarked = env.fromSource(