zentol commented on code in PR #20757:
URL: https://github.com/apache/flink/pull/20757#discussion_r963448684


##########
flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java:
##########
@@ -63,4 +63,11 @@ public interface SourceReaderContext {
      * @see UserCodeClassLoader
      */
     UserCodeClassLoader getUserCodeClassLoader();
+
+    /**
+     * Get the current parallelism of this Source.
+     *
+     * @return the parallelism of the Source.
+     */
+    int currentParallelism();

Review Comment:
   This breaks source compatibility. Either justify an exclusion or add a 
default implementation.



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.ExecutionConfig;
+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.GeneratorSourceReaderFactory;
+import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator;
+import org.apache.flink.api.java.ClosureCleaner;
+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.
+ *
+ * <p>The source splits the sequence into as many parallel sub-sequences as 
there are parallel
+ * source readers.
+ *
+ * <p>Users can supply a {@code GeneratorFunction} for mapping the 
(sub-)sequences of Long values
+ * into the generated events. For instance, the following code will produce 
the sequence of
+ * ["Number: 0", "Number: 2", ... , "Number: 999"] elements.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, String> generatorFunction = index -> "Number: " + 
index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunction, 1000, Types.STRING);
+ *
+ * DataStreamSource<String> stream =
+ *         env.fromSource(source,
+ *         WatermarkStrategy.noWatermarks(),
+ *         "Generator Source");
+ * }</pre>
+ *
+ * <p>The order of elements depends on the parallelism. Each sub-sequence will 
be produced in order.
+ * Consequently, if the parallelism is limited to one, this will produce one 
sequence in order from
+ * "Number: 0" to "Number: 999".
+ *
+ * <p>Note that this approach also makes it possible to produce deterministic 
watermarks at the
+ * source based on the generated events and a custom {@code WatermarkStrategy}.
+ *
+ * <p>This source has built-in support for rate limiting. The following code 
will produce an
+ * effectively unbounded (Long.MAX_VALUE from practical perspective will never 
be reached) stream of
+ * Long values at the overall source rate (across all source subtasks) of 100 
events per second.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, Long> generatorFunction = index -> index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunctionStateless, 
Long.MAX_VALUE, 100, Types.STRING);
+ * }</pre>
+ *
+ * <p>For more sophisticates use cases, users can take full control of the 
low-level data generation
+ * details by supplying a custom {@code SourceReaderFactory}. The instantiated 
{@code SourceReader}s
+ * are expected to produce data based on processing {@code 
NumberSequenceSplit}s. A customized
+ * generator could, for instance, synchronize the data release process with 
checkpointing by making
+ * use of ({@link SourceReader#notifyCheckpointComplete(long)}). Such 
functionality could be
+ * helpful, for instance, for testing sinks that are expected to create 
specific metadata upon the
+ * arrival of a checkpoint barrier and other similar use cases.
+ *
+ * <p>This source is always bounded. For very long sequences (for example when 
the {@code count} is
+ * set to Long.MAX_VALUE), users 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 DataGeneratorSource<OUT>
+        implements Source<OUT, NumberSequenceSplit, 
Collection<NumberSequenceSplit>>,
+                ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SourceReaderFactory<OUT, NumberSequenceSplit> 
sourceReaderFactory;
+    private final TypeInformation<OUT> typeInfo;
+
+    private final NumberSequenceSource numberSource;
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            TypeInformation<OUT> typeInfo) {
+        this(generatorFunction, count, -1, typeInfo);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param sourceRatePerSecond The overall source rate per second (across 
all source subtasks).
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            double sourceRatePerSecond,
+            TypeInformation<OUT> typeInfo) {
+        this(
+                new GeneratorSourceReaderFactory<>(generatorFunction, 
sourceRatePerSecond),
+                count,
+                typeInfo);
+        ClosureCleaner.clean(
+                generatorFunction, 
ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}. This constructor allows 
users can take
+     * control of the low-level data generation details by supplying a custom 
{@code
+     * SourceReaderFactory}. The instantiated {@code SourceReader}s are 
expected to produce data
+     * based on processing {@code NumberSequenceSplit}s.
+     *
+     * @param sourceReaderFactory The {@link SourceReader} factory.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            SourceReaderFactory<OUT, NumberSequenceSplit> sourceReaderFactory,

Review Comment:
   What use-case do you have in mind for this? What would this allow you to do 
that you couldn't via a generator function?



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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;

Review Comment:
   How did you arrive at that package name? It's rather non-descriptive.



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.ExecutionConfig;
+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.GeneratorSourceReaderFactory;
+import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator;
+import org.apache.flink.api.java.ClosureCleaner;
+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.
+ *
+ * <p>The source splits the sequence into as many parallel sub-sequences as 
there are parallel
+ * source readers.
+ *
+ * <p>Users can supply a {@code GeneratorFunction} for mapping the 
(sub-)sequences of Long values
+ * into the generated events. For instance, the following code will produce 
the sequence of
+ * ["Number: 0", "Number: 2", ... , "Number: 999"] elements.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, String> generatorFunction = index -> "Number: " + 
index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunction, 1000, Types.STRING);
+ *
+ * DataStreamSource<String> stream =
+ *         env.fromSource(source,
+ *         WatermarkStrategy.noWatermarks(),
+ *         "Generator Source");
+ * }</pre>
+ *
+ * <p>The order of elements depends on the parallelism. Each sub-sequence will 
be produced in order.
+ * Consequently, if the parallelism is limited to one, this will produce one 
sequence in order from
+ * "Number: 0" to "Number: 999".
+ *
+ * <p>Note that this approach also makes it possible to produce deterministic 
watermarks at the
+ * source based on the generated events and a custom {@code WatermarkStrategy}.
+ *
+ * <p>This source has built-in support for rate limiting. The following code 
will produce an
+ * effectively unbounded (Long.MAX_VALUE from practical perspective will never 
be reached) stream of
+ * Long values at the overall source rate (across all source subtasks) of 100 
events per second.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, Long> generatorFunction = index -> index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunctionStateless, 
Long.MAX_VALUE, 100, Types.STRING);
+ * }</pre>
+ *
+ * <p>For more sophisticates use cases, users can take full control of the 
low-level data generation
+ * details by supplying a custom {@code SourceReaderFactory}. The instantiated 
{@code SourceReader}s
+ * are expected to produce data based on processing {@code 
NumberSequenceSplit}s. A customized
+ * generator could, for instance, synchronize the data release process with 
checkpointing by making
+ * use of ({@link SourceReader#notifyCheckpointComplete(long)}). Such 
functionality could be
+ * helpful, for instance, for testing sinks that are expected to create 
specific metadata upon the
+ * arrival of a checkpoint barrier and other similar use cases.
+ *
+ * <p>This source is always bounded. For very long sequences (for example when 
the {@code count} is
+ * set to Long.MAX_VALUE), users 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 DataGeneratorSource<OUT>
+        implements Source<OUT, NumberSequenceSplit, 
Collection<NumberSequenceSplit>>,
+                ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SourceReaderFactory<OUT, NumberSequenceSplit> 
sourceReaderFactory;
+    private final TypeInformation<OUT> typeInfo;
+
+    private final NumberSequenceSource numberSource;
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            TypeInformation<OUT> typeInfo) {
+        this(generatorFunction, count, -1, typeInfo);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param sourceRatePerSecond The overall source rate per second (across 
all source subtasks).
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            double sourceRatePerSecond,
+            TypeInformation<OUT> typeInfo) {
+        this(
+                new GeneratorSourceReaderFactory<>(generatorFunction, 
sourceRatePerSecond),
+                count,
+                typeInfo);
+        ClosureCleaner.clean(
+                generatorFunction, 
ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}. This constructor allows 
users can take
+     * control of the low-level data generation details by supplying a custom 
{@code
+     * SourceReaderFactory}. The instantiated {@code SourceReader}s are 
expected to produce data
+     * based on processing {@code NumberSequenceSplit}s.
+     *
+     * @param sourceReaderFactory The {@link SourceReader} factory.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            SourceReaderFactory<OUT, NumberSequenceSplit> sourceReaderFactory,
+            long count,
+            TypeInformation<OUT> typeInfo) {
+        this.sourceReaderFactory = checkNotNull(sourceReaderFactory);
+        this.typeInfo = checkNotNull(typeInfo);
+        this.numberSource = new NumberSequenceSource(0, count - 1);
+    }
+
+    /** @return The number of records produced by this source. */
+    public long getCount() {
+        return numberSource.getTo();
+    }
+
+    // ------------------------------------------------------------------------
+    //  source methods
+    // ------------------------------------------------------------------------
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return typeInfo;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.BOUNDED;
+    }
+
+    @Override
+    public SourceReader<OUT, NumberSequenceSplit> 
createReader(SourceReaderContext readerContext) {
+        return sourceReaderFactory.newSourceReader(readerContext);
+    }
+
+    @Override
+    public SplitEnumerator<NumberSequenceSplit, 
Collection<NumberSequenceSplit>> restoreEnumerator(
+            SplitEnumeratorContext<NumberSequenceSplit> enumContext,
+            Collection<NumberSequenceSplit> checkpoint) {
+        return new IteratorSourceEnumerator<>(enumContext, checkpoint);
+    }
+
+    @Override
+    public SplitEnumerator<NumberSequenceSplit, 
Collection<NumberSequenceSplit>> createEnumerator(
+            final SplitEnumeratorContext<NumberSequenceSplit> enumContext) {
+        final List<NumberSequenceSplit> splits =
+                numberSource.splitNumberRange(0, getCount(), 
enumContext.currentParallelism());
+        return new IteratorSourceEnumerator<>(enumContext, splits);

Review Comment:
   same as above



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/GeneratingIteratorSourceReader.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.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;
+
+import java.util.Iterator;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@code SourceReader} that takes the values of an iterator, supplied via 
an {@link
+ * IteratorSourceSplit}, and applies a {@link GeneratorFunction} to them to 
perform arbitrary
+ * transformations.
+ */
+@Experimental
+public class GeneratingIteratorSourceReader<
+                E, O, IterT extends Iterator<E>, SplitT extends 
IteratorSourceSplit<E, IterT>>
+        extends IteratorSourceReaderBase<E, O, IterT, SplitT> {
+
+    private final GeneratorFunction<E, O> generatorFunction;
+
+    public GeneratingIteratorSourceReader(
+            SourceReaderContext context, GeneratorFunction<E, O> 
generatorFunction) {
+        super(context);
+        this.generatorFunction = checkNotNull(generatorFunction);
+    }
+
+    // ------------------------------------------------------------------------
+
+    @Override
+    public InputStatus pollNext(ReaderOutput<O> 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, e);
+                }
+                return InputStatus.MORE_AVAILABLE;

Review Comment:
   maybe add a comment similar to the one in the SourceReaderBase that this is 
technically incorrect (because we don't actually there that there is another 
record) but cheaper than the alternative.



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/GeneratorSourceReaderFactory.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.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.lib.GeneratorFunction;
+import org.apache.flink.api.connector.source.lib.NumberSequenceSource;
+import org.apache.flink.api.connector.source.lib.SourceReaderFactory;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A factory for instantiating source readers that produce elements by 
applying a user-supplied
+ * {@link GeneratorFunction}. This implementation also implicitly supports 
throttling the data rate
+ * by using a default rate limiter.

Review Comment:
   rate limiting is explicitly supported.



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/GuavaRateLimiter.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.util;
+
+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

Review Comment:
   Use `GuavaFlinkConnectorRateLimiter` instead (or unify them). We shouldn't 
have 2 different rate limiters within flink-core for connectors.



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedSourceReader.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.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;
+
+/** Wraps the actual {@link SourceReader} and rate limits its data emission. */
+@Experimental
+public class RateLimitedSourceReader<E, SplitT extends SourceSplit>
+        implements SourceReader<E, SplitT> {
+
+    private final SourceReader<E, SplitT> sourceReader;
+    private final RateLimiter rateLimiter;
+
+    /**
+     * Instantiates a new rate-limited source reader.
+     *
+     * @param sourceReader The actual source reader.
+     * @param rateLimiter The rate limiter.
+     */
+    public RateLimitedSourceReader(SourceReader<E, SplitT> sourceReader, 
RateLimiter rateLimiter) {
+        checkNotNull(sourceReader);
+        checkNotNull(rateLimiter);
+        this.sourceReader = sourceReader;
+        this.rateLimiter = rateLimiter;
+    }
+
+    // ------------------------------------------------------------------------
+
+    @Override
+    public void start() {
+        sourceReader.start();
+    }
+
+    @Override
+    public InputStatus pollNext(ReaderOutput<E> output) throws Exception {
+        rateLimiter.acquire();

Review Comment:
   is the reader thread allowed to block? The javadocs state this method must 
be non-blocking.
   
   How would a non-blocking source communicate that data is available at a 
later time + that it has become available?



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.ExecutionConfig;
+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.GeneratorSourceReaderFactory;
+import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator;
+import org.apache.flink.api.java.ClosureCleaner;
+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.
+ *
+ * <p>The source splits the sequence into as many parallel sub-sequences as 
there are parallel
+ * source readers.
+ *
+ * <p>Users can supply a {@code GeneratorFunction} for mapping the 
(sub-)sequences of Long values
+ * into the generated events. For instance, the following code will produce 
the sequence of
+ * ["Number: 0", "Number: 2", ... , "Number: 999"] elements.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, String> generatorFunction = index -> "Number: " + 
index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunction, 1000, Types.STRING);
+ *
+ * DataStreamSource<String> stream =
+ *         env.fromSource(source,
+ *         WatermarkStrategy.noWatermarks(),
+ *         "Generator Source");
+ * }</pre>
+ *
+ * <p>The order of elements depends on the parallelism. Each sub-sequence will 
be produced in order.
+ * Consequently, if the parallelism is limited to one, this will produce one 
sequence in order from
+ * "Number: 0" to "Number: 999".
+ *
+ * <p>Note that this approach also makes it possible to produce deterministic 
watermarks at the
+ * source based on the generated events and a custom {@code WatermarkStrategy}.
+ *
+ * <p>This source has built-in support for rate limiting. The following code 
will produce an
+ * effectively unbounded (Long.MAX_VALUE from practical perspective will never 
be reached) stream of
+ * Long values at the overall source rate (across all source subtasks) of 100 
events per second.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, Long> generatorFunction = index -> index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunctionStateless, 
Long.MAX_VALUE, 100, Types.STRING);
+ * }</pre>
+ *
+ * <p>For more sophisticates use cases, users can take full control of the 
low-level data generation
+ * details by supplying a custom {@code SourceReaderFactory}. The instantiated 
{@code SourceReader}s
+ * are expected to produce data based on processing {@code 
NumberSequenceSplit}s. A customized
+ * generator could, for instance, synchronize the data release process with 
checkpointing by making
+ * use of ({@link SourceReader#notifyCheckpointComplete(long)}). Such 
functionality could be
+ * helpful, for instance, for testing sinks that are expected to create 
specific metadata upon the
+ * arrival of a checkpoint barrier and other similar use cases.
+ *
+ * <p>This source is always bounded. For very long sequences (for example when 
the {@code count} is
+ * set to Long.MAX_VALUE), users 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 DataGeneratorSource<OUT>
+        implements Source<OUT, NumberSequenceSplit, 
Collection<NumberSequenceSplit>>,
+                ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SourceReaderFactory<OUT, NumberSequenceSplit> 
sourceReaderFactory;
+    private final TypeInformation<OUT> typeInfo;
+
+    private final NumberSequenceSource numberSource;
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            TypeInformation<OUT> typeInfo) {
+        this(generatorFunction, count, -1, typeInfo);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param sourceRatePerSecond The overall source rate per second (across 
all source subtasks).
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            double sourceRatePerSecond,
+            TypeInformation<OUT> typeInfo) {
+        this(
+                new GeneratorSourceReaderFactory<>(generatorFunction, 
sourceRatePerSecond),
+                count,
+                typeInfo);
+        ClosureCleaner.clean(
+                generatorFunction, 
ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}. This constructor allows 
users can take
+     * control of the low-level data generation details by supplying a custom 
{@code
+     * SourceReaderFactory}. The instantiated {@code SourceReader}s are 
expected to produce data
+     * based on processing {@code NumberSequenceSplit}s.
+     *
+     * @param sourceReaderFactory The {@link SourceReader} factory.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            SourceReaderFactory<OUT, NumberSequenceSplit> sourceReaderFactory,
+            long count,
+            TypeInformation<OUT> typeInfo) {
+        this.sourceReaderFactory = checkNotNull(sourceReaderFactory);
+        this.typeInfo = checkNotNull(typeInfo);
+        this.numberSource = new NumberSequenceSource(0, count - 1);
+    }
+
+    /** @return The number of records produced by this source. */
+    public long getCount() {
+        return numberSource.getTo();
+    }
+
+    // ------------------------------------------------------------------------
+    //  source methods
+    // ------------------------------------------------------------------------
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return typeInfo;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.BOUNDED;
+    }
+
+    @Override
+    public SourceReader<OUT, NumberSequenceSplit> 
createReader(SourceReaderContext readerContext) {
+        return sourceReaderFactory.newSourceReader(readerContext);
+    }
+
+    @Override
+    public SplitEnumerator<NumberSequenceSplit, 
Collection<NumberSequenceSplit>> restoreEnumerator(
+            SplitEnumeratorContext<NumberSequenceSplit> enumContext,
+            Collection<NumberSequenceSplit> checkpoint) {
+        return new IteratorSourceEnumerator<>(enumContext, checkpoint);

Review Comment:
   why are this not re-using the implementation from the NumberSequenceSource?



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.ExecutionConfig;
+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.GeneratorSourceReaderFactory;
+import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator;
+import org.apache.flink.api.java.ClosureCleaner;
+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.
+ *
+ * <p>The source splits the sequence into as many parallel sub-sequences as 
there are parallel
+ * source readers.
+ *
+ * <p>Users can supply a {@code GeneratorFunction} for mapping the 
(sub-)sequences of Long values
+ * into the generated events. For instance, the following code will produce 
the sequence of
+ * ["Number: 0", "Number: 2", ... , "Number: 999"] elements.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, String> generatorFunction = index -> "Number: " + 
index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunction, 1000, Types.STRING);
+ *
+ * DataStreamSource<String> stream =
+ *         env.fromSource(source,
+ *         WatermarkStrategy.noWatermarks(),
+ *         "Generator Source");
+ * }</pre>
+ *
+ * <p>The order of elements depends on the parallelism. Each sub-sequence will 
be produced in order.
+ * Consequently, if the parallelism is limited to one, this will produce one 
sequence in order from
+ * "Number: 0" to "Number: 999".
+ *
+ * <p>Note that this approach also makes it possible to produce deterministic 
watermarks at the
+ * source based on the generated events and a custom {@code WatermarkStrategy}.
+ *
+ * <p>This source has built-in support for rate limiting. The following code 
will produce an
+ * effectively unbounded (Long.MAX_VALUE from practical perspective will never 
be reached) stream of
+ * Long values at the overall source rate (across all source subtasks) of 100 
events per second.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, Long> generatorFunction = index -> index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunctionStateless, 
Long.MAX_VALUE, 100, Types.STRING);
+ * }</pre>
+ *
+ * <p>For more sophisticates use cases, users can take full control of the 
low-level data generation
+ * details by supplying a custom {@code SourceReaderFactory}. The instantiated 
{@code SourceReader}s
+ * are expected to produce data based on processing {@code 
NumberSequenceSplit}s. A customized
+ * generator could, for instance, synchronize the data release process with 
checkpointing by making
+ * use of ({@link SourceReader#notifyCheckpointComplete(long)}). Such 
functionality could be
+ * helpful, for instance, for testing sinks that are expected to create 
specific metadata upon the
+ * arrival of a checkpoint barrier and other similar use cases.
+ *
+ * <p>This source is always bounded. For very long sequences (for example when 
the {@code count} is
+ * set to Long.MAX_VALUE), users 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 DataGeneratorSource<OUT>
+        implements Source<OUT, NumberSequenceSplit, 
Collection<NumberSequenceSplit>>,
+                ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SourceReaderFactory<OUT, NumberSequenceSplit> 
sourceReaderFactory;
+    private final TypeInformation<OUT> typeInfo;
+
+    private final NumberSequenceSource numberSource;
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            TypeInformation<OUT> typeInfo) {

Review Comment:
   you may be able to extract the OUT type via the type extractor.



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/GeneratorFunction.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.annotation.Experimental;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+
+/**
+ * Base interface for data generator functions. Data generator functions take 
elements and transform
+ * them, element-wise. They are the core building block of the {@link 
DataGeneratorSource} that
+ * drives the data generation process by supplying "index" values of type 
Long. It makes it possible
+ * to produce specific elements at concrete positions of the generated data 
stream.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, String> generatorFunction = index -> "Number: " + 
index;
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunction, 1000, Types.STRING);
+ * }</pre>
+ *
+ * @param <T> Type of the input elements.
+ * @param <O> Type of the returned elements.
+ */
+@Experimental
+public interface GeneratorFunction<T, O> extends Function {
+
+    /**
+     * Initialization method for the function. It is called once before the 
actual data mapping
+     * methods.
+     */
+    default void open(SourceReaderContext readerContext) throws Exception {}

Review Comment:
   What do we envision what the readerContext will be used for? It's a 
rather...large interface.



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/SourceReaderFactory.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+/**
+ * A factory for creating source reader instances.
+ *
+ * @param <OUT> The type of the output elements.
+ */
+public interface SourceReaderFactory<OUT, SplitT extends SourceSplit> extends 
Serializable {

Review Comment:
   Would it make sense to have the Source interface extend this interface to 
keep the signatures the same?



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.ExecutionConfig;
+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.GeneratorSourceReaderFactory;
+import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator;
+import org.apache.flink.api.java.ClosureCleaner;
+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.
+ *
+ * <p>The source splits the sequence into as many parallel sub-sequences as 
there are parallel
+ * source readers.
+ *
+ * <p>Users can supply a {@code GeneratorFunction} for mapping the 
(sub-)sequences of Long values
+ * into the generated events. For instance, the following code will produce 
the sequence of
+ * ["Number: 0", "Number: 2", ... , "Number: 999"] elements.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, String> generatorFunction = index -> "Number: " + 
index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunction, 1000, Types.STRING);
+ *
+ * DataStreamSource<String> stream =
+ *         env.fromSource(source,
+ *         WatermarkStrategy.noWatermarks(),
+ *         "Generator Source");
+ * }</pre>
+ *
+ * <p>The order of elements depends on the parallelism. Each sub-sequence will 
be produced in order.
+ * Consequently, if the parallelism is limited to one, this will produce one 
sequence in order from
+ * "Number: 0" to "Number: 999".
+ *
+ * <p>Note that this approach also makes it possible to produce deterministic 
watermarks at the
+ * source based on the generated events and a custom {@code WatermarkStrategy}.
+ *
+ * <p>This source has built-in support for rate limiting. The following code 
will produce an
+ * effectively unbounded (Long.MAX_VALUE from practical perspective will never 
be reached) stream of
+ * Long values at the overall source rate (across all source subtasks) of 100 
events per second.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, Long> generatorFunction = index -> index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunctionStateless, 
Long.MAX_VALUE, 100, Types.STRING);
+ * }</pre>
+ *
+ * <p>For more sophisticates use cases, users can take full control of the 
low-level data generation
+ * details by supplying a custom {@code SourceReaderFactory}. The instantiated 
{@code SourceReader}s
+ * are expected to produce data based on processing {@code 
NumberSequenceSplit}s. A customized
+ * generator could, for instance, synchronize the data release process with 
checkpointing by making
+ * use of ({@link SourceReader#notifyCheckpointComplete(long)}). Such 
functionality could be
+ * helpful, for instance, for testing sinks that are expected to create 
specific metadata upon the
+ * arrival of a checkpoint barrier and other similar use cases.
+ *
+ * <p>This source is always bounded. For very long sequences (for example when 
the {@code count} is
+ * set to Long.MAX_VALUE), users 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 DataGeneratorSource<OUT>
+        implements Source<OUT, NumberSequenceSplit, 
Collection<NumberSequenceSplit>>,
+                ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SourceReaderFactory<OUT, NumberSequenceSplit> 
sourceReaderFactory;
+    private final TypeInformation<OUT> typeInfo;
+
+    private final NumberSequenceSource numberSource;
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            TypeInformation<OUT> typeInfo) {
+        this(generatorFunction, count, -1, typeInfo);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param sourceRatePerSecond The overall source rate per second (across 
all source subtasks).
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            double sourceRatePerSecond,
+            TypeInformation<OUT> typeInfo) {
+        this(
+                new GeneratorSourceReaderFactory<>(generatorFunction, 
sourceRatePerSecond),
+                count,
+                typeInfo);
+        ClosureCleaner.clean(
+                generatorFunction, 
ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}. This constructor allows 
users can take
+     * control of the low-level data generation details by supplying a custom 
{@code
+     * SourceReaderFactory}. The instantiated {@code SourceReader}s are 
expected to produce data
+     * based on processing {@code NumberSequenceSplit}s.
+     *
+     * @param sourceReaderFactory The {@link SourceReader} factory.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            SourceReaderFactory<OUT, NumberSequenceSplit> sourceReaderFactory,
+            long count,
+            TypeInformation<OUT> typeInfo) {
+        this.sourceReaderFactory = checkNotNull(sourceReaderFactory);

Review Comment:
   missing ClosureCleaner



##########
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
+    static final class CheckpointSerializer

Review Comment:
   Is this change actually necessary?



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/DataGeneratorSource.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.ExecutionConfig;
+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.GeneratorSourceReaderFactory;
+import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator;
+import org.apache.flink.api.java.ClosureCleaner;
+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.
+ *
+ * <p>The source splits the sequence into as many parallel sub-sequences as 
there are parallel
+ * source readers.
+ *
+ * <p>Users can supply a {@code GeneratorFunction} for mapping the 
(sub-)sequences of Long values
+ * into the generated events. For instance, the following code will produce 
the sequence of
+ * ["Number: 0", "Number: 2", ... , "Number: 999"] elements.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, String> generatorFunction = index -> "Number: " + 
index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunction, 1000, Types.STRING);
+ *
+ * DataStreamSource<String> stream =
+ *         env.fromSource(source,
+ *         WatermarkStrategy.noWatermarks(),
+ *         "Generator Source");
+ * }</pre>
+ *
+ * <p>The order of elements depends on the parallelism. Each sub-sequence will 
be produced in order.
+ * Consequently, if the parallelism is limited to one, this will produce one 
sequence in order from
+ * "Number: 0" to "Number: 999".
+ *
+ * <p>Note that this approach also makes it possible to produce deterministic 
watermarks at the
+ * source based on the generated events and a custom {@code WatermarkStrategy}.
+ *
+ * <p>This source has built-in support for rate limiting. The following code 
will produce an
+ * effectively unbounded (Long.MAX_VALUE from practical perspective will never 
be reached) stream of
+ * Long values at the overall source rate (across all source subtasks) of 100 
events per second.
+ *
+ * <pre>{@code
+ * GeneratorFunction<Long, Long> generatorFunction = index -> index;
+ *
+ * DataGeneratorSource<String> source =
+ *         new DataGeneratorSource<>(generatorFunctionStateless, 
Long.MAX_VALUE, 100, Types.STRING);
+ * }</pre>
+ *
+ * <p>For more sophisticates use cases, users can take full control of the 
low-level data generation
+ * details by supplying a custom {@code SourceReaderFactory}. The instantiated 
{@code SourceReader}s
+ * are expected to produce data based on processing {@code 
NumberSequenceSplit}s. A customized
+ * generator could, for instance, synchronize the data release process with 
checkpointing by making
+ * use of ({@link SourceReader#notifyCheckpointComplete(long)}). Such 
functionality could be
+ * helpful, for instance, for testing sinks that are expected to create 
specific metadata upon the
+ * arrival of a checkpoint barrier and other similar use cases.
+ *
+ * <p>This source is always bounded. For very long sequences (for example when 
the {@code count} is
+ * set to Long.MAX_VALUE), users 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 DataGeneratorSource<OUT>
+        implements Source<OUT, NumberSequenceSplit, 
Collection<NumberSequenceSplit>>,
+                ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SourceReaderFactory<OUT, NumberSequenceSplit> 
sourceReaderFactory;
+    private final TypeInformation<OUT> typeInfo;
+
+    private final NumberSequenceSource numberSource;
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            TypeInformation<OUT> typeInfo) {
+        this(generatorFunction, count, -1, typeInfo);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}.
+     *
+     * @param generatorFunction The {@code GeneratorFunction} function.
+     * @param count The number of generated data points.
+     * @param sourceRatePerSecond The overall source rate per second (across 
all source subtasks).
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            GeneratorFunction<Long, OUT> generatorFunction,
+            long count,
+            double sourceRatePerSecond,
+            TypeInformation<OUT> typeInfo) {
+        this(
+                new GeneratorSourceReaderFactory<>(generatorFunction, 
sourceRatePerSecond),
+                count,
+                typeInfo);
+        ClosureCleaner.clean(
+                generatorFunction, 
ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+    }
+
+    /**
+     * Instantiates a new {@code DataGeneratorSource}. This constructor allows 
users can take
+     * control of the low-level data generation details by supplying a custom 
{@code
+     * SourceReaderFactory}. The instantiated {@code SourceReader}s are 
expected to produce data
+     * based on processing {@code NumberSequenceSplit}s.
+     *
+     * @param sourceReaderFactory The {@link SourceReader} factory.
+     * @param count The number of generated data points.
+     * @param typeInfo The type of the produced data points.
+     */
+    public DataGeneratorSource(
+            SourceReaderFactory<OUT, NumberSequenceSplit> sourceReaderFactory,
+            long count,
+            TypeInformation<OUT> typeInfo) {
+        this.sourceReaderFactory = checkNotNull(sourceReaderFactory);
+        this.typeInfo = checkNotNull(typeInfo);
+        this.numberSource = new NumberSequenceSource(0, count - 1);
+    }
+
+    /** @return The number of records produced by this source. */
+    public long getCount() {

Review Comment:
   Why is this public?



##########
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}.
+ *
+ * <p>The {@code IteratorSourceSplit} is also responsible for taking the 
current iterator and
+ * turning it back into a split for checkpointing.
+ *
+ * @param <E> The type of events returned by the reader.
+ * @param <IterT> The type of the iterator that produces the events. This type 
exists to make the
+ *     conversion between iterator and {@code IteratorSourceSplit} type safe.
+ * @param <SplitT> 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<E>, SplitT extends 
IteratorSourceSplit<E, IterT>>
+        implements SourceReader<O, SplitT> {
+
+    /** 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<Void> 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;

Review Comment:
   I'd rather have a well-defined interface that sub-classes implement 
providing these at the right time. It would clarify what custom behavior we 
actually intend to support. The current approach of "you can change anything" 
means we can't change anything in here because something could always break.



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/GeneratingIteratorSourceReader.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.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;
+
+import java.util.Iterator;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@code SourceReader} that takes the values of an iterator, supplied via 
an {@link
+ * IteratorSourceSplit}, and applies a {@link GeneratorFunction} to them to 
perform arbitrary
+ * transformations.
+ */
+@Experimental
+public class GeneratingIteratorSourceReader<
+                E, O, IterT extends Iterator<E>, SplitT extends 
IteratorSourceSplit<E, IterT>>
+        extends IteratorSourceReaderBase<E, O, IterT, SplitT> {
+
+    private final GeneratorFunction<E, O> generatorFunction;
+
+    public GeneratingIteratorSourceReader(
+            SourceReaderContext context, GeneratorFunction<E, O> 
generatorFunction) {
+        super(context);
+        this.generatorFunction = checkNotNull(generatorFunction);
+    }
+
+    // ------------------------------------------------------------------------
+
+    @Override
+    public InputStatus pollNext(ReaderOutput<O> 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, e);
+                }
+                return InputStatus.MORE_AVAILABLE;
+            } else {
+                finishSplit();
+            }
+        }
+        return tryMoveToNextSplit();
+    }
+
+    @Override
+    public void close() throws Exception {
+        super.close();
+        generatorFunction.close();

Review Comment:
   ```suggestion
           generatorFunction.close();
           super.close();
   ```



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/GeneratorSourceReaderFactory.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.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.lib.GeneratorFunction;
+import org.apache.flink.api.connector.source.lib.NumberSequenceSource;
+import org.apache.flink.api.connector.source.lib.SourceReaderFactory;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A factory for instantiating source readers that produce elements by 
applying a user-supplied
+ * {@link GeneratorFunction}. This implementation also implicitly supports 
throttling the data rate
+ * by using a default rate limiter.
+ *
+ * @param <OUT> The type of the output elements.
+ */
+public class GeneratorSourceReaderFactory<OUT>
+        implements SourceReaderFactory<OUT, 
NumberSequenceSource.NumberSequenceSplit> {
+
+    private final GeneratorFunction<Long, OUT> generatorFunction;
+    private final double sourceRatePerSecond;
+
+    /**
+     * Instantiates a new {@code GeneratorSourceReaderFactory}.
+     *
+     * @param generatorFunction The generator function.
+     * @param sourceRatePerSecond The target source rate per second. This 
parameter specifies the
+     *     overall source rate (across all source subtasks) and does not need 
to account for the
+     *     parallelism.

Review Comment:
   should mention that rates <= 0 are disable rate-limiting.
   
   I don't quite get what `does not need to account for the parallelism` is 
meant to convey.
   
   I'd also rename it to maxSourceRatePerSecond, because technically there's no 
guarantee that all sources reach the target rate.



##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/GeneratingIteratorSourceReader.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.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;
+
+import java.util.Iterator;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@code SourceReader} that takes the values of an iterator, supplied via 
an {@link
+ * IteratorSourceSplit}, and applies a {@link GeneratorFunction} to them to 
perform arbitrary
+ * transformations.
+ */
+@Experimental
+public class GeneratingIteratorSourceReader<
+                E, O, IterT extends Iterator<E>, SplitT extends 
IteratorSourceSplit<E, IterT>>
+        extends IteratorSourceReaderBase<E, O, IterT, SplitT> {
+
+    private final GeneratorFunction<E, O> generatorFunction;
+
+    public GeneratingIteratorSourceReader(
+            SourceReaderContext context, GeneratorFunction<E, O> 
generatorFunction) {
+        super(context);
+        this.generatorFunction = checkNotNull(generatorFunction);
+    }
+
+    // ------------------------------------------------------------------------
+
+    @Override
+    public InputStatus pollNext(ReaderOutput<O> 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, e);
+                }
+                return InputStatus.MORE_AVAILABLE;
+            } else {
+                finishSplit();
+            }
+        }
+        return tryMoveToNextSplit();
+    }
+
+    @Override
+    public void close() throws Exception {
+        super.close();
+        generatorFunction.close();

Review Comment:
   it's common to use the inverse order in which they were opened, reason being 
that the function may do something (potentially via the SourceReaderContext) 
that still requires the source reader to be open.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to