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


##########
flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.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.streaming.api.operators.OutputTypeConfigurable;
+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.EOFException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.NoSuchElementException;
+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.
+ *
+ * @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 transient Iterable<OUT> elements;
+    private transient DataInputView input;
+
+    @SafeVarargs
+    public FromElementsGeneratorFunction(TypeInformation<OUT> typeInfo, OUT... 
elements) {
+        this(typeInfo, new ExecutionConfig(), Arrays.asList(elements));
+    }
+
+    public FromElementsGeneratorFunction(
+            TypeInformation<OUT> typeInfo, ExecutionConfig config, 
Iterable<OUT> elements) {
+        // must not have null elements and mixed elements
+        checkIterable(elements, typeInfo.getTypeClass());
+        this.serializer = typeInfo.createSerializer(config);

Review Comment:
   hmm, wondering if we should fully rely on `setOutputType`, because it has 
the added benefit of ensuring that we see the final ExecutionConfig. 🤔 



##########
flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.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.streaming.api.operators.OutputTypeConfigurable;
+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.EOFException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.NoSuchElementException;
+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.
+ *
+ * @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 transient Iterable<OUT> elements;
+    private transient DataInputView input;
+
+    @SafeVarargs
+    public FromElementsGeneratorFunction(TypeInformation<OUT> typeInfo, OUT... 
elements) {
+        this(typeInfo, new ExecutionConfig(), Arrays.asList(elements));
+    }
+
+    public FromElementsGeneratorFunction(
+            TypeInformation<OUT> typeInfo, ExecutionConfig config, 
Iterable<OUT> elements) {
+        // must not have null elements and mixed elements
+        checkIterable(elements, typeInfo.getTypeClass());
+        this.serializer = typeInfo.createSerializer(config);

Review Comment:
   hmm, wondering if we should fully rely on `setOutputType`, because it has 
the added benefit of ensuring that we see the final ExecutionConfig. 🤔 



-- 
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