Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-21386][datastream] Postpone FromElementsFunction serialization to respect later type customization #15507

Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -1096,12 +1096,7 @@ public <OUT> DataStreamSource<OUT> fromCollection(
// must not have null elements and mixed elements
FromElementsFunction.checkCollection(data, typeInfo.getTypeClass());

SourceFunction<OUT> function;
try {
function = new FromElementsFunction<>(typeInfo.createSerializer(getConfig()), data);
} catch (IOException e) {
throw new RuntimeException(e.getMessage(), e);
}
SourceFunction<OUT> function = new FromElementsFunction<>(data);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we add a single positive test for the returns method on fromElements?

Copy link
Member Author

@kezhuw kezhuw Apr 7, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add StreamExecutionEnvironmentTest.testFromElementsPostConstructionType.

return addSource(function, "Collection Source", typeInfo, Boundedness.BOUNDED)
.setParallelism(1);
}
Expand Down
Expand Up @@ -18,8 +18,11 @@
package org.apache.flink.streaming.api.functions.source;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.base.IntSerializer;
import org.apache.flink.core.memory.DataInputView;
Expand All @@ -28,37 +31,44 @@
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
import org.apache.flink.util.IterableUtils;
import org.apache.flink.util.Preconditions;

import javax.annotation.Nullable;

import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Objects;

/**
* A stream source function that returns a sequence of elements.
*
* <p>Upon construction, this source 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>This source 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 <T> The type of elements returned by this function.
*/
@PublicEvolving
public class FromElementsFunction<T> implements SourceFunction<T>, CheckpointedFunction {
public class FromElementsFunction<T>
implements SourceFunction<T>, CheckpointedFunction, OutputTypeConfigurable<T> {

private static final long serialVersionUID = 1L;

/** The (de)serializer to be used for the data elements. */
private final TypeSerializer<T> serializer;
private @Nullable TypeSerializer<T> serializer;

/** The actual data elements, in serialized form. */
private final byte[] elementsSerialized;
private byte[] elementsSerialized;

/** The number of serialized elements. */
private final int numElements;
Expand All @@ -72,30 +82,81 @@ public class FromElementsFunction<T> implements SourceFunction<T>, CheckpointedF
/** Flag to make the source cancelable. */
private volatile boolean isRunning = true;

private final transient Iterable<T> elements;

private transient ListState<Integer> checkpointedState;

@SafeVarargs
public FromElementsFunction(TypeSerializer<T> serializer, T... elements) throws IOException {
this(serializer, Arrays.asList(elements));
}

public FromElementsFunction(TypeSerializer<T> serializer, Iterable<T> elements)
throws IOException {
this.serializer = Preconditions.checkNotNull(serializer);
this.elements = elements;
this.numElements =
elements instanceof Collection
? ((Collection<T>) elements).size()
: (int) IterableUtils.toStream(elements).count();
serializeElements();
}

@SafeVarargs
public FromElementsFunction(T... elements) {
this(Arrays.asList(elements));
}

public FromElementsFunction(Iterable<T> elements) {
this.serializer = null;
this.elements = elements;
this.numElements =
elements instanceof Collection
? ((Collection<T>) elements).size()
: (int) IterableUtils.toStream(elements).count();
checkIterable(elements, Object.class);
}

@VisibleForTesting
@Nullable
public TypeSerializer<T> getSerializer() {
return serializer;
}

private void serializeElements() throws IOException {
Preconditions.checkState(serializer != null, "serializer not set");
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputViewStreamWrapper wrapper = new DataOutputViewStreamWrapper(baos);

int count = 0;
try {
for (T element : elements) {
serializer.serialize(element, wrapper);
count++;
}
} catch (Exception e) {
throw new IOException("Serializing the source elements failed: " + e.getMessage(), e);
}

this.serializer = serializer;
this.elementsSerialized = baos.toByteArray();
this.numElements = count;
}

/**
* Set element type and re-serialize element if required. Should only be called before
* serialization/deserialization of this function.
*/
@Override
public void setOutputType(TypeInformation<T> 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<T> newSerializer = outTypeInfo.createSerializer(executionConfig);
if (Objects.equals(serializer, newSerializer)) {
return;
}
serializer = newSerializer;
try {
serializeElements();
} catch (IOException ex) {
throw new UncheckedIOException(ex);
}
}

@Override
Expand Down Expand Up @@ -127,6 +188,7 @@ public void initializeState(FunctionInitializationContext context) throws Except

@Override
public void run(SourceContext<T> ctx) throws Exception {
Preconditions.checkState(serializer != null, "serializer not configured");
ByteArrayInputStream bais = new ByteArrayInputStream(elementsSerialized);
final DataInputView input = new DataInputViewStreamWrapper(bais);

Expand Down Expand Up @@ -222,6 +284,10 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception {
* @param <OUT> The generic type of the collection to be checked.
*/
public static <OUT> void checkCollection(Collection<OUT> elements, Class<OUT> viewedAs) {
checkIterable(elements, viewedAs);
}

private static <OUT> void checkIterable(Iterable<OUT> elements, Class<?> viewedAs) {
for (OUT elem : elements) {
if (elem == null) {
throw new IllegalArgumentException("The collection contains a null element");
Expand Down
Expand Up @@ -49,6 +49,7 @@
import java.util.NoSuchElementException;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;

Expand All @@ -67,6 +68,35 @@ public void fromElementsWithBaseTypeTest2() {
env.fromElements(SubClass.class, new SubClass(1, "Java"), new ParentClass(1, "hello"));
}

@Test
public void testFromElementsDeducedType() {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStreamSource<String> source = env.fromElements("a", "b");

FromElementsFunction<String> elementsFunction =
(FromElementsFunction<String>) getFunctionFromDataSource(source);
assertEquals(
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(env.getConfig()),
elementsFunction.getSerializer());
}

@Test
public void testFromElementsPostConstructionType() {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStreamSource<String> source = env.fromElements("a", "b");
TypeInformation<String> customType = new GenericTypeInfo<>(String.class);

source.returns(customType);

FromElementsFunction<String> elementsFunction =
(FromElementsFunction<String>) getFunctionFromDataSource(source);
assertNotEquals(
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(env.getConfig()),
elementsFunction.getSerializer());
assertEquals(
customType.createSerializer(env.getConfig()), elementsFunction.getSerializer());
}

@Test
@SuppressWarnings("unchecked")
public void testFromCollectionParallelism() {
Expand Down