Skip to content

Commit

Permalink
[FLINK-25920] Refactor & Revise SinkWriterOperatorTestBase
Browse files Browse the repository at this point in the history
The stateful SinkWriterOperatorTestBase test cases used EOI to manipulate the state which was never clean. In particular, it also stored the input elements in state until EOI arrived and emitted them all at once. For state restoration tests, we emitted records after EOI arrived.

This commit changed the writer state completely to just capture the record count, which is much more realistic than storing actual payload. The tests now directly assert on the state instead of output.

This commit also introduces an adaptor for serializing basic types in the writer state and replaces the hard-to-maintain SinkAndSuppliers with an InspectableSink in the sink writer tests that require an abstraction on top of the different Sink flavors.

(cherry picked from commit 4217408)
  • Loading branch information
AHeise committed Nov 14, 2024
1 parent f3a33a5 commit 4934ef8
Show file tree
Hide file tree
Showing 18 changed files with 403 additions and 475 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.core.io;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputDeserializer;
import org.apache.flink.core.memory.DataOutputSerializer;

import java.io.IOException;
import java.io.Serializable;

/**
* Adapter for {@link TypeSerializer} to {@link SimpleVersionedSerializer}. The implementation is
* naive and should only be used for non-critical paths and tests.
*/
@Internal
public class SimpleVersionedSerializerAdapter<T>
implements SimpleVersionedSerializer<T>, Serializable {
private final TypeSerializer<T> serializer;

public SimpleVersionedSerializerAdapter(TypeSerializer<T> serializer) {
this.serializer = serializer;
}

public int getVersion() {
return serializer.snapshotConfiguration().getCurrentVersion();
}

public byte[] serialize(T value) throws IOException {
DataOutputSerializer dataOutputSerializer = new DataOutputSerializer(10);
serializer.serialize(value, dataOutputSerializer);
return dataOutputSerializer.getCopyOfBuffer();
}

public T deserialize(int version, byte[] serialized) throws IOException {
DataInputDeserializer dataInputDeserializer = new DataInputDeserializer(serialized);
T value = serializer.deserialize(dataInputDeserializer);
dataInputDeserializer.releaseArrays();
return value;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -122,13 +122,7 @@ public void generateWriterCommitterGlobalCommitterTopology() {
public void generateWriterGlobalCommitterTopology() {
final StreamGraph streamGraph =
buildGraph(
TestSink.newBuilder()
.setCommittableSerializer(
TestSink.StringCommittableSerializer.INSTANCE)
.setGlobalCommittableSerializer(
TestSink.StringCommittableSerializer.INSTANCE)
.setDefaultGlobalCommitter()
.build(),
TestSink.newBuilder().setDefaultGlobalCommitter().build(),
runtimeExecutionMode);

final StreamNode sourceNode = findNodeName(streamGraph, node -> node.contains("Source"));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.api.connector.sink2.Committer;
import org.apache.flink.api.connector.sink2.CommitterInitContext;
import org.apache.flink.api.connector.sink2.CommittingSinkWriter;
Expand All @@ -54,6 +55,7 @@
import org.apache.flink.configuration.PipelineOptions;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.core.io.SimpleVersionedSerializer;
import org.apache.flink.core.io.SimpleVersionedSerializerAdapter;
import org.apache.flink.core.memory.ManagedMemoryUseCase;
import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
Expand Down Expand Up @@ -115,7 +117,6 @@
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
import org.apache.flink.streaming.api.transformations.StreamExchangeMode;
import org.apache.flink.streaming.runtime.operators.sink.TestSinkV2;
import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
Expand Down Expand Up @@ -2659,7 +2660,7 @@ public DataStream<CommittableMessage<Long>> addPreCommitTopology(

@Override
public SimpleVersionedSerializer<String> getWriteResultSerializer() {
return new TestSinkV2.StringSerializer();
return new SimpleVersionedSerializerAdapter<>(StringSerializer.INSTANCE);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ static List<byte[]> toBytes(Collection<String> elements) {
static byte[] toBytes(String obj) {
try {
return SimpleVersionedSerialization.writeVersionAndSerialize(
TestSinkV2.StringSerializer.INSTANCE, obj);
TestSinkV2.COMMITTABLE_SERIALIZER, obj);
} catch (IOException e) {
throw new IllegalStateException(e);
}
Expand Down Expand Up @@ -83,7 +83,7 @@ static String fromRecord(StreamRecord<byte[]> obj) {
static String fromBytes(byte[] obj) {
try {
return SimpleVersionedSerialization.readVersionAndDeSerialize(
TestSinkV2.StringSerializer.INSTANCE, obj);
TestSinkV2.COMMITTABLE_SERIALIZER, obj);
} catch (IOException e) {
throw new IllegalStateException(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@ SinkAndCounters sinkWithPostCommit() {
(TwoPhaseCommittingSink<?, String>)
TestSinkV2.newBuilder()
.setCommitter(committer)
.setCommittableSerializer(TestSinkV2.StringSerializer.INSTANCE)
.setWithPostCommitTopology(true)
.build(),
() -> committer.successfulCommits);
Expand All @@ -47,7 +46,6 @@ SinkAndCounters sinkWithPostCommitWithRetry() {
(TwoPhaseCommittingSink<?, String>)
TestSinkV2.newBuilder()
.setCommitter(new TestSinkV2.RetryOnceCommitter())
.setCommittableSerializer(TestSinkV2.StringSerializer.INSTANCE)
.setWithPostCommitTopology(true)
.build(),
() -> 0);
Expand All @@ -60,7 +58,6 @@ SinkAndCounters sinkWithoutPostCommit() {
(TwoPhaseCommittingSink<?, String>)
TestSinkV2.newBuilder()
.setCommitter(committer)
.setCommittableSerializer(TestSinkV2.StringSerializer.INSTANCE)
.setWithPostCommitTopology(false)
.build(),
() -> committer.successfulCommits);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ SinkAndCounters sinkWithPostCommit() {
(SupportsCommitter<String>)
TestSinkV2.newBuilder()
.setCommitter(committer)
.setCommittableSerializer(TestSinkV2.StringSerializer.INSTANCE)
.setWithPostCommitTopology(true)
.build(),
() -> committer.successfulCommits);
Expand All @@ -42,7 +41,6 @@ SinkAndCounters sinkWithPostCommitWithRetry() {
(SupportsCommitter<String>)
TestSinkV2.newBuilder()
.setCommitter(new TestSinkV2.RetryOnceCommitter())
.setCommittableSerializer(TestSinkV2.StringSerializer.INSTANCE)
.setWithPostCommitTopology(true)
.build(),
() -> 0);
Expand All @@ -52,12 +50,11 @@ SinkAndCounters sinkWithPostCommitWithRetry() {
SinkAndCounters sinkWithoutPostCommit() {
ForwardingCommitter committer = new ForwardingCommitter();
return new SinkAndCounters(
(SupportsCommitter<String>)
TestSinkV2.newBuilder()
.setCommitter(committer)
.setCommittableSerializer(TestSinkV2.StringSerializer.INSTANCE)
.setWithPostCommitTopology(false)
.build(),
TestSinkV2.newBuilder()
.setCommitter(committer)
.setWithPostCommitTopology(false)
.build()
.asSupportsCommitter(),
() -> committer.successfulCommits);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,84 +18,63 @@

package org.apache.flink.streaming.runtime.operators.sink;

import org.apache.flink.api.common.eventtime.Watermark;
import org.apache.flink.api.common.operators.ProcessingTimeService;
import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.streaming.runtime.operators.sink.deprecated.TestSinkV2;

import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableList;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;

/**
* Should be removed along with {@link org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink}.
*/
@Deprecated
class SinkV2SinkWriterOperatorDeprecatedTest extends SinkWriterOperatorTestBase {

@Override
SinkAndSuppliers sinkWithoutCommitter() {
InspectableSink sinkWithoutCommitter() {
TestSinkV2.DefaultSinkWriter<Integer> sinkWriter = new TestSinkV2.DefaultSinkWriter<>();
return new SinkAndSuppliers(
TestSinkV2.<Integer>newBuilder().setWriter(sinkWriter).build(),
() -> sinkWriter.elements,
() -> sinkWriter.watermarks,
() -> -1,
TestSinkV2.StringSerializer::new);
return new InspectableSink(TestSinkV2.<Integer>newBuilder().setWriter(sinkWriter).build());
}

@Override
SinkAndSuppliers sinkWithCommitter() {
InspectableSink sinkWithCommitter() {
TestSinkV2.DefaultSinkWriter<Integer> sinkWriter =
new TestSinkV2.DefaultCommittingSinkWriter<>();
return new SinkAndSuppliers(
return new InspectableSink(
TestSinkV2.<Integer>newBuilder()
.setWriter(sinkWriter)
.setDefaultCommitter()
.build(),
() -> sinkWriter.elements,
() -> sinkWriter.watermarks,
() -> -1,
TestSinkV2.StringSerializer::new);
.setWriter(sinkWriter)
.build());
}

@Override
SinkAndSuppliers sinkWithTimeBasedWriter() {
InspectableSink sinkWithTimeBasedWriter() {
TestSinkV2.DefaultSinkWriter<Integer> sinkWriter = new TimeBasedBufferingSinkWriter();
return new SinkAndSuppliers(
return new InspectableSink(
TestSinkV2.<Integer>newBuilder()
.setWriter(sinkWriter)
.setDefaultCommitter()
.build(),
() -> sinkWriter.elements,
() -> sinkWriter.watermarks,
() -> -1,
TestSinkV2.StringSerializer::new);
.build());
}

@Override
SinkAndSuppliers sinkWithSnapshottingWriter(boolean withState, String stateName) {
SnapshottingBufferingSinkWriter sinkWriter = new SnapshottingBufferingSinkWriter();
InspectableSink sinkWithState(boolean withState, String stateName) {
TestSinkV2.DefaultSinkWriter<Integer> sinkWriter =
new TestSinkV2.DefaultStatefulSinkWriter<>();
TestSinkV2.Builder<Integer> builder =
TestSinkV2.newBuilder()
.setWriter(sinkWriter)
TestSinkV2.<Integer>newBuilder()
.setDefaultCommitter()
.setWithPostCommitTopology(true);
.setWithPostCommitTopology(true)
.setWriter(sinkWriter);
if (withState) {
builder.setWriterState(true);
}
if (stateName != null) {
builder.setCompatibleStateNames(stateName);
}
return new SinkAndSuppliers(
builder.build(),
() -> sinkWriter.elements,
() -> sinkWriter.watermarks,
() -> sinkWriter.lastCheckpointId,
() -> new TestSinkV2.StringSerializer());
return new InspectableSink(builder.build());
}

private static class TimeBasedBufferingSinkWriter
Expand Down Expand Up @@ -125,31 +104,30 @@ public void init(Sink.InitContext context) {
}
}

private static class SnapshottingBufferingSinkWriter
extends TestSinkV2.DefaultStatefulSinkWriter {
public static final int NOT_SNAPSHOTTED = -1;
long lastCheckpointId = NOT_SNAPSHOTTED;
boolean endOfInput = false;
static class InspectableSink extends AbstractInspectableSink<TestSinkV2<Integer>> {
InspectableSink(TestSinkV2<Integer> sink) {
super(sink);
}

@Override
public long getLastCheckpointId() {
return getSink().getWriter().lastCheckpointId;
}

@Override
public void flush(boolean endOfInput) throws IOException, InterruptedException {
this.endOfInput = endOfInput;
public List<String> getRecordsOfCurrentCheckpoint() {
return getSink().getWriter().elements;
}

@Override
public List<String> snapshotState(long checkpointId) throws IOException {
lastCheckpointId = checkpointId;
return super.snapshotState(checkpointId);
public List<Watermark> getWatermarks() {
return getSink().getWriter().watermarks;
}

@Override
public Collection<String> prepareCommit() {
if (!endOfInput) {
return ImmutableList.of();
}
List<String> result = elements;
elements = new ArrayList<>();
return result;
public int getRecordCountFromState() {
return ((TestSinkV2.DefaultStatefulSinkWriter<?>) getSink().getWriter())
.getRecordCount();
}
}
}
Loading

0 comments on commit 4934ef8

Please sign in to comment.