AHeise commented on a change in pull request #16966: URL: https://github.com/apache/flink/pull/16966#discussion_r698066088
########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/CompoundReducingUpsertWriterStateSerializer.java ########## @@ -0,0 +1,123 @@ +/* + * 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.streaming.connectors.kafka.table; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.table.data.RowData; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class CompoundReducingUpsertWriterStateSerializer<WriterState> + implements SimpleVersionedSerializer<CompoundReducingUpsertWriterState<WriterState>> { + + private final TypeSerializer<RowData> rowDataTypeSerializer; + @Nullable private final SimpleVersionedSerializer<WriterState> wrappedStateSerializer; + + CompoundReducingUpsertWriterStateSerializer( + TypeSerializer<RowData> rowDataTypeSerializer, Review comment: Does that need to be versioned? @twalthr is there something like `SimpleVersionedSerializer` or do we simply stick to: "we don't support version upgrades"? ########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/CompoundReducingUpsertWriterState.java ########## @@ -0,0 +1,68 @@ +/* + * 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.streaming.connectors.kafka.table; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.data.RowData; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class CompoundReducingUpsertWriterState<WriterState> { Review comment: I'd drop the compound here. Imho many writers will have compounds states and I wouldn't want to make it the default. ########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java ########## @@ -73,8 +88,17 @@ } @Override - public Optional<SimpleVersionedSerializer<WriterState>> getWriterStateSerializer() { - return wrappedSink.getWriterStateSerializer(); + public Optional<SimpleVersionedSerializer<CompoundReducingUpsertWriterState<WriterState>>> + getWriterStateSerializer() { + final Optional<SimpleVersionedSerializer<WriterState>> wrappedStateSerializer = + wrappedSink.getWriterStateSerializer(); + if (wrappedStateSerializer.isPresent()) { + return Optional.of( + new CompoundReducingUpsertWriterStateSerializer<WriterState>( + rowDataTypeSerializer, wrappedStateSerializer.get())); + } + return Optional.of( + new CompoundReducingUpsertWriterStateSerializer<>(rowDataTypeSerializer)); Review comment: ```suggestion return Optional.of( new CompoundReducingUpsertWriterStateSerializer<>(rowDataTypeSerializer, wrappedSink.getWriterStateSerializer().orElse(null))); ``` ########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/CompoundReducingUpsertWriterStateSerializer.java ########## @@ -0,0 +1,123 @@ +/* + * 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.streaming.connectors.kafka.table; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.table.data.RowData; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class CompoundReducingUpsertWriterStateSerializer<WriterState> + implements SimpleVersionedSerializer<CompoundReducingUpsertWriterState<WriterState>> { + + private final TypeSerializer<RowData> rowDataTypeSerializer; + @Nullable private final SimpleVersionedSerializer<WriterState> wrappedStateSerializer; + + CompoundReducingUpsertWriterStateSerializer( + TypeSerializer<RowData> rowDataTypeSerializer, + @Nullable SimpleVersionedSerializer<WriterState> wrappedStateSerializer) { + this.wrappedStateSerializer = wrappedStateSerializer; + this.rowDataTypeSerializer = checkNotNull(rowDataTypeSerializer); + } + + CompoundReducingUpsertWriterStateSerializer(TypeSerializer<RowData> rowDataTypeSerializer) { + this(rowDataTypeSerializer, null); + } + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(CompoundReducingUpsertWriterState<WriterState> state) + throws IOException { + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputViewStreamWrapper out = new DataOutputViewStreamWrapper(baos)) { + final List<WriterState> wrappedStates = state.getWrappedStates(); + if (wrappedStateSerializer != null) { + out.writeInt(wrappedStateSerializer.getVersion()); + out.writeInt(wrappedStates.size()); + for (final WriterState wrappedState : wrappedStates) { + final byte[] serializedWrappedState = + wrappedStateSerializer.serialize(wrappedState); + out.writeInt(serializedWrappedState.length); + out.write(wrappedStateSerializer.serialize(wrappedState)); + } + } + + final Map<RowData, Tuple2<RowData, Long>> reduceBuffer = state.getReduceBuffer(); + out.writeInt(reduceBuffer.size()); + for (final Map.Entry<RowData, Tuple2<RowData, Long>> entry : reduceBuffer.entrySet()) { + rowDataTypeSerializer.serialize(entry.getKey(), out); + rowDataTypeSerializer.serialize(entry.getValue().f0, out); + out.writeLong(entry.getValue().f1); + } + return baos.toByteArray(); + } + } + + @Override + public CompoundReducingUpsertWriterState<WriterState> deserialize( + int version, byte[] serialized) throws IOException { + try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + final DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(bais); ) { + List<WriterState> wrappedStates = null; + if (wrappedStateSerializer != null) { + wrappedStates = new ArrayList<>(); + final int wrappedStateSerializerVersion = in.readInt(); + final int wrappedStatesSize = in.readInt(); + for (int ignored = 0; ignored < wrappedStatesSize; ignored++) { + final int wrappedStateSize = in.readInt(); + final ByteBuffer buffer = ByteBuffer.allocate(wrappedStateSize); + for (int i = 0; i < wrappedStateSize; i++) { + buffer.put(in.readByte()); + } + wrappedStates.add( + wrappedStateSerializer.deserialize( + wrappedStateSerializerVersion, buffer.array())); Review comment: ```suggestion byte[] data = new byte[wrappedStateSize]; in.readFully(data); wrappedStates.add( wrappedStateSerializer.deserialize(wrappedStateSerializerVersion, data)); ``` ########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/CompoundReducingUpsertWriterStateSerializer.java ########## @@ -0,0 +1,123 @@ +/* + * 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.streaming.connectors.kafka.table; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.table.data.RowData; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class CompoundReducingUpsertWriterStateSerializer<WriterState> + implements SimpleVersionedSerializer<CompoundReducingUpsertWriterState<WriterState>> { + + private final TypeSerializer<RowData> rowDataTypeSerializer; + @Nullable private final SimpleVersionedSerializer<WriterState> wrappedStateSerializer; + + CompoundReducingUpsertWriterStateSerializer( + TypeSerializer<RowData> rowDataTypeSerializer, + @Nullable SimpleVersionedSerializer<WriterState> wrappedStateSerializer) { + this.wrappedStateSerializer = wrappedStateSerializer; + this.rowDataTypeSerializer = checkNotNull(rowDataTypeSerializer); + } + + CompoundReducingUpsertWriterStateSerializer(TypeSerializer<RowData> rowDataTypeSerializer) { + this(rowDataTypeSerializer, null); + } + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(CompoundReducingUpsertWriterState<WriterState> state) + throws IOException { + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputViewStreamWrapper out = new DataOutputViewStreamWrapper(baos)) { + final List<WriterState> wrappedStates = state.getWrappedStates(); + if (wrappedStateSerializer != null) { + out.writeInt(wrappedStateSerializer.getVersion()); + out.writeInt(wrappedStates.size()); + for (final WriterState wrappedState : wrappedStates) { + final byte[] serializedWrappedState = + wrappedStateSerializer.serialize(wrappedState); + out.writeInt(serializedWrappedState.length); + out.write(wrappedStateSerializer.serialize(wrappedState)); + } Review comment: Extract to `SimpleVersionedSerialization`? ########## File path: flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/CompoundReducingUpsertWriterStateSerializerTest.java ########## @@ -0,0 +1,86 @@ +/* + * 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.streaming.connectors.kafka.table; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap; + +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +import static org.apache.flink.table.runtime.util.StreamRecordUtils.binaryrow; +import static org.junit.Assert.assertEquals; + +/** Tests for {@link CompoundReducingUpsertWriterStateSerializer}. */ +public class CompoundReducingUpsertWriterStateSerializerTest { + + private static final CompoundReducingUpsertWriterStateSerializer<Integer> COMPOUND_SERIALIZER = + new CompoundReducingUpsertWriterStateSerializer<>( + new RowDataSerializer( + DataTypes.STRING().getLogicalType(), DataTypes.INT().getLogicalType()), + new WrappedSerializer()); + private static final int VERSION = 1; + + @Test + public void testSerDe() throws IOException { + final CompoundReducingUpsertWriterState<Integer> state = + new CompoundReducingUpsertWriterState<>( + ImmutableMap.of( + binaryrow("row1", 1), Tuple2.of(binaryrow("row1", 2), 42L), + binaryrow("row2", 2), Tuple2.of(binaryrow("row2", 3), 43L)), + ImmutableList.of(5, 4)); + final CompoundReducingUpsertWriterState<Integer> actual = + COMPOUND_SERIALIZER.deserialize(VERSION, COMPOUND_SERIALIZER.serialize(state)); + assertEquals(state, actual); + } + Review comment: Add a test without serializer. -- 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