zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370102899
########## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java: ########## @@ -566,15 +567,15 @@ public void testMaxParallelismWithConnectedKeyedStream() { int maxParallelism = 42; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataStream<Integer> input1 = env.fromElements(1, 2, 3, 4).setMaxParallelism(128); - DataStream<Integer> input2 = env.fromElements(1, 2, 3, 4).setMaxParallelism(129); + DataStream<Long> input1 = env.fromSequence(1, 4).setMaxParallelism(128); + DataStream<Long> input2 = env.fromSequence(1, 4).setMaxParallelism(129); Review Comment: why are these being changed? Not necessarily in this instnace (since sequence seems more appropriate here), just curious about all the other changed streaming-java tests. ########## flink-formats/flink-parquet/src/test/resources/avro/user.avsc: ########## @@ -1,9 +0,0 @@ -{ - "namespace": "org.apache.flink.connector.datagen.source.generated", - "type": "record", - "name": "User", Review Comment: was this unused? ########## flink-architecture-tests/flink-architecture-tests-production/archunit-violations/e5126cae-f3fe-48aa-b6fb-60ae6cc3fcd5: ########## @@ -19,8 +19,8 @@ Method <org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTr Method <org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator$DeferrableCoordinator.closeAsync(long)> calls method <org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator$QuiesceableContext.quiesce()> in (RecreateOnResetOperatorCoordinator.java:361) Method <org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration.fromConfiguration(org.apache.flink.configuration.Configuration, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, java.lang.String, java.io.File)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateDefaultSlotResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, int)> in (TaskManagerConfiguration.java:244) Method <org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration.fromConfiguration(org.apache.flink.configuration.Configuration, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, java.lang.String, java.io.File)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateTotalAvailableResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec)> in (TaskManagerConfiguration.java:246) -Method <org.apache.flink.runtime.taskexecutor.TaskManagerServices.createTaskSlotTable(int, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, long, int, java.util.concurrent.Executor)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateDefaultSlotResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, int)> in (TaskManagerServices.java:433) -Method <org.apache.flink.runtime.taskexecutor.TaskManagerServices.createTaskSlotTable(int, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, long, int, java.util.concurrent.Executor)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateTotalAvailableResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec)> in (TaskManagerServices.java:431) Review Comment: These seem to be unrelated? ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java: ########## @@ -123,7 +124,8 @@ private void createChainableStream(TableTestUtil util) { } private void createNonChainableStream(TableTestUtil util) { - DataStreamSource<Integer> dataStream = util.getStreamEnv().fromElements(1, 2, 3); + DataStreamSource<Integer> dataStream = + util.getStreamEnv().fromCollection(Arrays.asList(1, 2, 3)); Review Comment: Here I wonder why we're changing things. ########## flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamExecutionEnvironmentITCase.java: ########## @@ -67,7 +79,62 @@ public void executeThrowsProgramInvocationException() throws Exception { }) .print(); - thrown.expect(ProgramInvocationException.class); - env.execute(); + assertThatThrownBy(env::execute).isInstanceOf(ProgramInvocationException.class); + } + + @Test + @SuppressWarnings("unchecked") + void testAvroSpecificRecordsInFromElements() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + User user1 = new User("Foo", 1); + User user2 = new User("Bar", 2); + User[] data = {user1, user2}; + DataStreamSource<User> stream = env.fromElements(User.class, user1, user2); + DataGeneratorSource<User> source = getSourceFromStream(stream); + FromElementsGeneratorFunction<User> generatorFunction = + (FromElementsGeneratorFunction<User>) source.getGeneratorFunction(); + + List<User> result = stream.executeAndCollect(data.length + 1); + TypeSerializer<User> serializer = generatorFunction.getSerializer(); + + assertThat(serializer).isInstanceOf(AvroSerializer.class); + assertThat(result).containsExactly(data); + } + + @Test + @SuppressWarnings("unchecked") + void testAvroGenericRecordsInFromElements() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + Schema schema = getSchemaFromResources("/avro/user.avsc"); + GenericRecord user1 = + new GenericRecordBuilder(schema).set("name", "Foo").set("age", 40).build(); + GenericRecord user2 = + new GenericRecordBuilder(schema).set("name", "Bar").set("age", 45).build(); + GenericRecord[] data = {user1, user2}; + DataStream<GenericRecord> stream = + env.fromElements(data).returns(new GenericRecordAvroTypeInfo(schema)); + DataGeneratorSource<GenericRecord> source = getSourceFromStream(stream); + FromElementsGeneratorFunction<GenericRecord> generatorFunction = + (FromElementsGeneratorFunction<GenericRecord>) source.getGeneratorFunction(); + + List<GenericRecord> result = stream.executeAndCollect(data.length + 1); + TypeSerializer<GenericRecord> serializer = generatorFunction.getSerializer(); + + assertThat(serializer).isInstanceOf(AvroSerializer.class); + assertThat(result).containsExactly(data); + } + + private Schema getSchemaFromResources(String path) throws Exception { + try (InputStream schemaStream = getClass().getResourceAsStream(path)) { + if (schemaStream == null) { + throw new IllegalStateException("Could not find " + path + " in classpath"); + } + return new Schema.Parser().parse(schemaStream); + } + } + + @SuppressWarnings("unchecked") + private static <T, S extends Source<T, ?, ?>> S getSourceFromStream(DataStream<T> stream) { + return (S) ((SourceTransformation<T, ?, ?>) stream.getTransformation()).getSource(); Review Comment: couldn't you solve this in other ways? like by having a separate variable for `env.fromElements` ########## flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e: ########## @@ -64,6 +64,19 @@ Constructor <org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher. Constructor <org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.<init>(int, org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue, org.apache.flink.connector.base.source.reader.splitreader.SplitReader, java.util.function.Consumer, java.lang.Runnable, java.util.function.Consumer, boolean)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (SplitFetcher.java:97) Constructor <org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager.<init>(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue, java.util.function.Supplier, org.apache.flink.configuration.Configuration, java.util.function.Consumer)> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (SplitFetcherManager.java:0) Constructor <org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue.<init>(int)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (FutureCompletingBlockingQueue.java:114) +Constructor <org.apache.flink.connector.datagen.functions.FromElementsGeneratorFunction.<init>(org.apache.flink.api.common.typeutils.TypeSerializer, java.lang.Iterable)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (FromElementsGeneratorFunction.java:85) Review Comment: I suppose these are fine because these classes are shipped with Flink 🤔 ########## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java: ########## @@ -1161,9 +1163,10 @@ void testYieldingOperatorChainableToTaskNotChainedToLegacySource() { */ @Test void testYieldingOperatorProperlyChainedOnLegacySources() { + // TODO: this test can be removed when the legacy SourceFunction API gets removed StreamExecutionEnvironment chainEnv = StreamExecutionEnvironment.createLocalEnvironment(1); - chainEnv.fromElements(1) + chainEnv.addSource(new LegacySource()) Review Comment: ooh so these to be changed because they require a legacy source? ########## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.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.connector.datagen.functions; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.typeutils.OutputTypeConfigurable; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Objects; + +/** + * A stream generator function that returns a sequence of elements. + * + * <p>This generator function serializes the elements using Flink's type information. That way, any + * object transport using Java serialization will not be affected by the serializability of the + * elements. + * + * <p><b>NOTE:</b> This source has a parallelism of 1. + * + * @param <OUT> The type of elements returned by this function. + */ +@Internal +public class FromElementsGeneratorFunction<OUT> + implements GeneratorFunction<Long, OUT>, OutputTypeConfigurable<OUT> { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(FromElementsGeneratorFunction.class); + + /** The (de)serializer to be used for the data elements. */ + private @Nullable TypeSerializer<OUT> serializer; + + /** The actual data elements, in serialized form. */ + private byte[] elementsSerialized; + + /** The number of elements emitted already. */ + private int numElementsEmitted; + + private final transient Iterable<OUT> elements; + private transient DataInputView input; + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, OUT... elements) + throws IOException { + this(serializer, Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, Iterable<OUT> elements) + throws IOException { + this.serializer = Preconditions.checkNotNull(serializer); + this.elements = elements; + serializeElements(); + } + + @SafeVarargs + public FromElementsGeneratorFunction(OUT... elements) { + this(Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(Iterable<OUT> elements) { + this.serializer = null; + this.elements = elements; + checkIterable(elements, Object.class); + } + + @VisibleForTesting + @Nullable + public TypeSerializer<OUT> getSerializer() { + return serializer; + } + + private void serializeElements() throws IOException { + Preconditions.checkState(serializer != null, "serializer not set"); + LOG.info("Serializing elements using " + serializer); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper wrapper = new DataOutputViewStreamWrapper(baos); + + try { + for (OUT element : elements) { + serializer.serialize(element, wrapper); + } + } catch (Exception e) { + throw new IOException("Serializing the source elements failed: " + e.getMessage(), e); + } + this.elementsSerialized = baos.toByteArray(); + } + + @Override + public void open(SourceReaderContext readerContext) throws Exception { + ByteArrayInputStream bais = new ByteArrayInputStream(elementsSerialized); + this.input = new DataInputViewStreamWrapper(bais); + } + + @Override + public OUT map(Long nextIndex) throws Exception { + // Move iterator to the required position in case of failure recovery + while (numElementsEmitted < nextIndex) { + numElementsEmitted++; + tryDeserialize(serializer, input); + } + numElementsEmitted++; + return tryDeserialize(serializer, input); + } + + private OUT tryDeserialize(TypeSerializer<OUT> serializer, DataInputView input) + throws IOException { + try { + return serializer.deserialize(input); + } catch (Exception e) { + throw new IOException( + "Failed to deserialize an element from the source. " + + "If you are using user-defined serialization (Value and Writable types), check the " + + "serialization functions.\nSerializer is " + + serializer, + e); + } + } + + @Override + public void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) { + Preconditions.checkState( + elements != null, + "The output type should've been specified before shipping the graph to the cluster"); + checkIterable(elements, outTypeInfo.getTypeClass()); + TypeSerializer<OUT> newSerializer = outTypeInfo.createSerializer(executionConfig); + if (Objects.equals(serializer, newSerializer)) { + return; + } + serializer = newSerializer; + try { + serializeElements(); + } catch (IOException ex) { + throw new UncheckedIOException(ex); + } + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Verifies that all elements in the collection are non-null, and are of the given class, or a + * subclass thereof. + * + * @param elements The collection to check. + * @param viewedAs The class to which the elements must be assignable to. + * @param <OUT> The generic type of the collection to be checked. + */ + public static <OUT> void checkCollection(Collection<OUT> elements, Class<OUT> viewedAs) { Review Comment: Actually Im wondering why we need both this and checkiterable() 🤔 ########## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.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.connector.datagen.functions; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.typeutils.OutputTypeConfigurable; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Objects; + +/** + * A stream generator function that returns a sequence of elements. + * + * <p>This generator function serializes the elements using Flink's type information. That way, any + * object transport using Java serialization will not be affected by the serializability of the + * elements. + * + * <p><b>NOTE:</b> This source has a parallelism of 1. + * + * @param <OUT> The type of elements returned by this function. + */ +@Internal +public class FromElementsGeneratorFunction<OUT> + implements GeneratorFunction<Long, OUT>, OutputTypeConfigurable<OUT> { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(FromElementsGeneratorFunction.class); + + /** The (de)serializer to be used for the data elements. */ + private @Nullable TypeSerializer<OUT> serializer; + + /** The actual data elements, in serialized form. */ + private byte[] elementsSerialized; + + /** The number of elements emitted already. */ + private int numElementsEmitted; + + private final transient Iterable<OUT> elements; + private transient DataInputView input; + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, OUT... elements) + throws IOException { + this(serializer, Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, Iterable<OUT> elements) + throws IOException { + this.serializer = Preconditions.checkNotNull(serializer); + this.elements = elements; + serializeElements(); + } + + @SafeVarargs + public FromElementsGeneratorFunction(OUT... elements) { + this(Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(Iterable<OUT> elements) { + this.serializer = null; + this.elements = elements; + checkIterable(elements, Object.class); + } + + @VisibleForTesting + @Nullable + public TypeSerializer<OUT> getSerializer() { + return serializer; + } + + private void serializeElements() throws IOException { + Preconditions.checkState(serializer != null, "serializer not set"); + LOG.info("Serializing elements using " + serializer); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper wrapper = new DataOutputViewStreamWrapper(baos); + + try { + for (OUT element : elements) { + serializer.serialize(element, wrapper); + } + } catch (Exception e) { + throw new IOException("Serializing the source elements failed: " + e.getMessage(), e); + } + this.elementsSerialized = baos.toByteArray(); + } + + @Override + public void open(SourceReaderContext readerContext) throws Exception { + ByteArrayInputStream bais = new ByteArrayInputStream(elementsSerialized); + this.input = new DataInputViewStreamWrapper(bais); + } + + @Override + public OUT map(Long nextIndex) throws Exception { + // Move iterator to the required position in case of failure recovery + while (numElementsEmitted < nextIndex) { + numElementsEmitted++; + tryDeserialize(serializer, input); + } + numElementsEmitted++; + return tryDeserialize(serializer, input); + } + + private OUT tryDeserialize(TypeSerializer<OUT> serializer, DataInputView input) + throws IOException { + try { + return serializer.deserialize(input); + } catch (Exception e) { + throw new IOException( + "Failed to deserialize an element from the source. " + + "If you are using user-defined serialization (Value and Writable types), check the " + + "serialization functions.\nSerializer is " + + serializer, + e); Review Comment: Feel like we need a better / more explicit safeguard here in case this is called more often than expected, since the count of elements is now somewhat handled by the generator source. ########## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.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.connector.datagen.functions; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.typeutils.OutputTypeConfigurable; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Objects; + +/** + * A stream generator function that returns a sequence of elements. + * + * <p>This generator function serializes the elements using Flink's type information. That way, any + * object transport using Java serialization will not be affected by the serializability of the + * elements. + * + * <p><b>NOTE:</b> This source has a parallelism of 1. Review Comment: Needs to be rephrased because it's not a source. "When this function is used the generator source must run with a parallelism of 1." Theoretically we could also fix this (in a follow-up!) but it doesn't seem worth the overhead given the number of elements. Oh I think this already works; see below comment. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OutputTypeConfigurable.java: ########## @@ -27,7 +27,10 @@ * at {@link org.apache.flink.streaming.api.graph.StreamGraph} generation. This can be useful for * cases where the output type is specified by the returns method and, thus, after the stream * operator has been created. + * + * @deprecated Use {@link org.apache.flink.api.java.typeutils.OutputTypeConfigurable} instead */ +@Deprecated Review Comment: Instead I'd suggest to move this class as-is to flink-core, without changing the package. Everyone who uses streaming-java also uses flink-core => it doesn't break compatibility, which means we don't need another ML discussions for this particular item because afaict it's out-of-scope of previous discussions. ########## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.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.connector.datagen.functions; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.typeutils.OutputTypeConfigurable; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Objects; + +/** + * A stream generator function that returns a sequence of elements. + * + * <p>This generator function serializes the elements using Flink's type information. That way, any + * object transport using Java serialization will not be affected by the serializability of the + * elements. + * + * <p><b>NOTE:</b> This source has a parallelism of 1. + * + * @param <OUT> The type of elements returned by this function. + */ +@Internal +public class FromElementsGeneratorFunction<OUT> + implements GeneratorFunction<Long, OUT>, OutputTypeConfigurable<OUT> { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(FromElementsGeneratorFunction.class); + + /** The (de)serializer to be used for the data elements. */ + private @Nullable TypeSerializer<OUT> serializer; + + /** The actual data elements, in serialized form. */ + private byte[] elementsSerialized; + + /** The number of elements emitted already. */ + private int numElementsEmitted; + + private final transient Iterable<OUT> elements; + private transient DataInputView input; + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, OUT... elements) + throws IOException { + this(serializer, Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, Iterable<OUT> elements) + throws IOException { + this.serializer = Preconditions.checkNotNull(serializer); + this.elements = elements; + serializeElements(); + } + + @SafeVarargs + public FromElementsGeneratorFunction(OUT... elements) { + this(Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(Iterable<OUT> elements) { + this.serializer = null; + this.elements = elements; + checkIterable(elements, Object.class); + } + + @VisibleForTesting + @Nullable + public TypeSerializer<OUT> getSerializer() { + return serializer; + } + + private void serializeElements() throws IOException { + Preconditions.checkState(serializer != null, "serializer not set"); + LOG.info("Serializing elements using " + serializer); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper wrapper = new DataOutputViewStreamWrapper(baos); + + try { + for (OUT element : elements) { + serializer.serialize(element, wrapper); + } + } catch (Exception e) { + throw new IOException("Serializing the source elements failed: " + e.getMessage(), e); + } + this.elementsSerialized = baos.toByteArray(); + } + + @Override + public void open(SourceReaderContext readerContext) throws Exception { + ByteArrayInputStream bais = new ByteArrayInputStream(elementsSerialized); + this.input = new DataInputViewStreamWrapper(bais); + } + + @Override + public OUT map(Long nextIndex) throws Exception { + // Move iterator to the required position in case of failure recovery Review Comment: huh. Doesn't this implicitly allow the function to run in parallel? ########## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java: ########## @@ -1188,14 +1191,14 @@ void testChainingOfOperatorsWithDifferentMaxParallelism( configuration.set( PipelineOptions.OPERATOR_CHAINING_CHAIN_OPERATORS_WITH_DIFFERENT_MAX_PARALLELISM, chainingOfOperatorsWithDifferentMaxParallelismEnabled); - configuration.set(PipelineOptions.MAX_PARALLELISM, 10); + configuration.set(PipelineOptions.MAX_PARALLELISM, 1); try (StreamExecutionEnvironment chainEnv = StreamExecutionEnvironment.createLocalEnvironment(1, configuration)) { chainEnv.fromElements(1) .map(x -> x) // should automatically break chain here .map(x -> x) - .setMaxParallelism(1) + .setMaxParallelism(10) Review Comment: Why is this being changed? ########## flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e: ########## @@ -64,6 +64,19 @@ Constructor <org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher. Constructor <org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.<init>(int, org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue, org.apache.flink.connector.base.source.reader.splitreader.SplitReader, java.util.function.Consumer, java.lang.Runnable, java.util.function.Consumer, boolean)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (SplitFetcher.java:97) Constructor <org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager.<init>(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue, java.util.function.Supplier, org.apache.flink.configuration.Configuration, java.util.function.Consumer)> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (SplitFetcherManager.java:0) Constructor <org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue.<init>(int)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (FutureCompletingBlockingQueue.java:114) +Constructor <org.apache.flink.connector.datagen.functions.FromElementsGeneratorFunction.<init>(org.apache.flink.api.common.typeutils.TypeSerializer, java.lang.Iterable)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (FromElementsGeneratorFunction.java:85) Review Comment: I suppose these are fine because these classes are shipped with Flink 🤔 ########## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.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.connector.datagen.functions; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.typeutils.OutputTypeConfigurable; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Objects; + +/** + * A stream generator function that returns a sequence of elements. + * + * <p>This generator function serializes the elements using Flink's type information. That way, any + * object transport using Java serialization will not be affected by the serializability of the + * elements. + * + * <p><b>NOTE:</b> This source has a parallelism of 1. + * + * @param <OUT> The type of elements returned by this function. + */ +@Internal +public class FromElementsGeneratorFunction<OUT> + implements GeneratorFunction<Long, OUT>, OutputTypeConfigurable<OUT> { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(FromElementsGeneratorFunction.class); + + /** The (de)serializer to be used for the data elements. */ + private @Nullable TypeSerializer<OUT> serializer; + + /** The actual data elements, in serialized form. */ + private byte[] elementsSerialized; + + /** The number of elements emitted already. */ + private int numElementsEmitted; + + private final transient Iterable<OUT> elements; + private transient DataInputView input; + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, OUT... elements) + throws IOException { + this(serializer, Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, Iterable<OUT> elements) + throws IOException { + this.serializer = Preconditions.checkNotNull(serializer); + this.elements = elements; + serializeElements(); + } + + @SafeVarargs + public FromElementsGeneratorFunction(OUT... elements) { + this(Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(Iterable<OUT> elements) { + this.serializer = null; + this.elements = elements; + checkIterable(elements, Object.class); + } + + @VisibleForTesting + @Nullable + public TypeSerializer<OUT> getSerializer() { + return serializer; + } + + private void serializeElements() throws IOException { + Preconditions.checkState(serializer != null, "serializer not set"); + LOG.info("Serializing elements using " + serializer); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper wrapper = new DataOutputViewStreamWrapper(baos); + + try { + for (OUT element : elements) { + serializer.serialize(element, wrapper); + } + } catch (Exception e) { + throw new IOException("Serializing the source elements failed: " + e.getMessage(), e); + } + this.elementsSerialized = baos.toByteArray(); + } + + @Override + public void open(SourceReaderContext readerContext) throws Exception { + ByteArrayInputStream bais = new ByteArrayInputStream(elementsSerialized); + this.input = new DataInputViewStreamWrapper(bais); + } + + @Override + public OUT map(Long nextIndex) throws Exception { + // Move iterator to the required position in case of failure recovery + while (numElementsEmitted < nextIndex) { + numElementsEmitted++; + tryDeserialize(serializer, input); + } + numElementsEmitted++; + return tryDeserialize(serializer, input); + } + + private OUT tryDeserialize(TypeSerializer<OUT> serializer, DataInputView input) + throws IOException { + try { + return serializer.deserialize(input); + } catch (Exception e) { + throw new IOException( + "Failed to deserialize an element from the source. " + + "If you are using user-defined serialization (Value and Writable types), check the " + + "serialization functions.\nSerializer is " + + serializer, + e); + } + } + + @Override + public void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) { + Preconditions.checkState( + elements != null, + "The output type should've been specified before shipping the graph to the cluster"); + checkIterable(elements, outTypeInfo.getTypeClass()); + TypeSerializer<OUT> newSerializer = outTypeInfo.createSerializer(executionConfig); + if (Objects.equals(serializer, newSerializer)) { + return; + } + serializer = newSerializer; + try { + serializeElements(); + } catch (IOException ex) { + throw new UncheckedIOException(ex); + } + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Verifies that all elements in the collection are non-null, and are of the given class, or a + * subclass thereof. + * + * @param elements The collection to check. + * @param viewedAs The class to which the elements must be assignable to. + * @param <OUT> The generic type of the collection to be checked. + */ + public static <OUT> void checkCollection(Collection<OUT> elements, Class<OUT> viewedAs) { Review Comment: Why isnt this called in the constructor and instead called from the outside before calling the constructor? ########## flink-tests/pom.xml: ########## @@ -284,6 +284,13 @@ under the License. <scope>test</scope> </dependency> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-avro</artifactId> Review Comment: This may deserve a separate commit; I cant really tell how this (and some of the flink-tests changes) related to fromElements. ########## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java: ########## @@ -566,15 +567,15 @@ public void testMaxParallelismWithConnectedKeyedStream() { int maxParallelism = 42; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataStream<Integer> input1 = env.fromElements(1, 2, 3, 4).setMaxParallelism(128); - DataStream<Integer> input2 = env.fromElements(1, 2, 3, 4).setMaxParallelism(129); + DataStream<Long> input1 = env.fromSequence(1, 4).setMaxParallelism(128); + DataStream<Long> input2 = env.fromSequence(1, 4).setMaxParallelism(129); Review Comment: Is it just for test coverage? ########## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.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.connector.datagen.functions; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.typeutils.OutputTypeConfigurable; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Objects; + +/** + * A stream generator function that returns a sequence of elements. + * + * <p>This generator function serializes the elements using Flink's type information. That way, any + * object transport using Java serialization will not be affected by the serializability of the + * elements. + * + * <p><b>NOTE:</b> This source has a parallelism of 1. + * + * @param <OUT> The type of elements returned by this function. + */ +@Internal +public class FromElementsGeneratorFunction<OUT> + implements GeneratorFunction<Long, OUT>, OutputTypeConfigurable<OUT> { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(FromElementsGeneratorFunction.class); + + /** The (de)serializer to be used for the data elements. */ + private @Nullable TypeSerializer<OUT> serializer; + + /** The actual data elements, in serialized form. */ + private byte[] elementsSerialized; + + /** The number of elements emitted already. */ + private int numElementsEmitted; + + private final transient Iterable<OUT> elements; + private transient DataInputView input; + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, OUT... elements) + throws IOException { + this(serializer, Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(TypeSerializer<OUT> serializer, Iterable<OUT> elements) + throws IOException { + this.serializer = Preconditions.checkNotNull(serializer); + this.elements = elements; + serializeElements(); + } + + @SafeVarargs + public FromElementsGeneratorFunction(OUT... elements) { + this(Arrays.asList(elements)); + } + + public FromElementsGeneratorFunction(Iterable<OUT> elements) { + this.serializer = null; + this.elements = elements; + checkIterable(elements, Object.class); + } + + @VisibleForTesting + @Nullable + public TypeSerializer<OUT> getSerializer() { + return serializer; + } + + private void serializeElements() throws IOException { + Preconditions.checkState(serializer != null, "serializer not set"); + LOG.info("Serializing elements using " + serializer); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper wrapper = new DataOutputViewStreamWrapper(baos); + + try { + for (OUT element : elements) { + serializer.serialize(element, wrapper); + } + } catch (Exception e) { + throw new IOException("Serializing the source elements failed: " + e.getMessage(), e); + } + this.elementsSerialized = baos.toByteArray(); + } + + @Override + public void open(SourceReaderContext readerContext) throws Exception { + ByteArrayInputStream bais = new ByteArrayInputStream(elementsSerialized); + this.input = new DataInputViewStreamWrapper(bais); + } + + @Override + public OUT map(Long nextIndex) throws Exception { + // Move iterator to the required position in case of failure recovery + while (numElementsEmitted < nextIndex) { + numElementsEmitted++; + tryDeserialize(serializer, input); + } + numElementsEmitted++; + return tryDeserialize(serializer, input); + } + + private OUT tryDeserialize(TypeSerializer<OUT> serializer, DataInputView input) + throws IOException { + try { + return serializer.deserialize(input); + } catch (Exception e) { + throw new IOException( + "Failed to deserialize an element from the source. " + + "If you are using user-defined serialization (Value and Writable types), check the " + + "serialization functions.\nSerializer is " + + serializer, + e); + } + } + + @Override + public void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) { + Preconditions.checkState( + elements != null, + "The output type should've been specified before shipping the graph to the cluster"); + checkIterable(elements, outTypeInfo.getTypeClass()); + TypeSerializer<OUT> newSerializer = outTypeInfo.createSerializer(executionConfig); + if (Objects.equals(serializer, newSerializer)) { + return; + } + serializer = newSerializer; + try { + serializeElements(); + } catch (IOException ex) { + throw new UncheckedIOException(ex); + } + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Verifies that all elements in the collection are non-null, and are of the given class, or a + * subclass thereof. + * + * @param elements The collection to check. + * @param viewedAs The class to which the elements must be assignable to. + * @param <OUT> The generic type of the collection to be checked. + */ + public static <OUT> void checkCollection(Collection<OUT> elements, Class<OUT> viewedAs) { Review Comment: Actually Im wondering why we need both this and checkiterable() 🤔 -- 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