dannycranmer commented on a change in pull request #18669: URL: https://github.com/apache/flink/pull/18669#discussion_r803555992
########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterStateSerializer.java ########## @@ -0,0 +1,118 @@ +/* + * 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.base.sink.writer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Serializer class for {@link AsyncSinkWriter} state. + * + * @param <RequestEntryT> Writer Request Entry type + */ +@Internal +public abstract class AsyncSinkWriterStateSerializer<RequestEntryT extends Serializable> + implements SimpleVersionedSerializer<BufferedRequestState<RequestEntryT>> { + private static final long DATA_IDENTIFIER = -1; + + /** + * Serializes state in form of + * [DATA_IDENTIFIER,NUM_OF_ELEMENTS,SIZE1,REQUEST1,SIZE2,REQUEST2....]. + */ + @Override + public byte[] serialize(BufferedRequestState<RequestEntryT> obj) throws IOException { + Collection<RequestEntryWrapper<RequestEntryT>> bufferState = + obj.getBufferedRequestEntries(); + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream out = new DataOutputStream(baos)) { + out.writeLong(DATA_IDENTIFIER); + out.writeInt(bufferState.size()); + for (RequestEntryWrapper<RequestEntryT> wrapper : bufferState) { + out.writeLong(wrapper.getSize()); + serializeRequestToStream(wrapper.getRequestEntry(), out); + } + return baos.toByteArray(); + } + } + + @Override + public BufferedRequestState<RequestEntryT> deserialize(int version, byte[] serialized) + throws IOException { + try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + final DataInputStream in = new DataInputStream(bais)) { + validateIdentifier(in); + int size = readInt(in); + List<RequestEntryWrapper<RequestEntryT>> serializedState = new ArrayList<>(); + for (int i = 0; i < size; i++) { + long requestSize = readLong(in); + RequestEntryT request = deserializeRequestFromStream(requestSize, in); + serializedState.add(new RequestEntryWrapper<>(request, requestSize)); + } + + return new BufferedRequestState<>(serializedState); + } + } + + protected abstract void serializeRequestToStream(RequestEntryT request, DataOutputStream out) + throws IOException; + + protected abstract RequestEntryT deserializeRequestFromStream( + long requestSize, DataInputStream in) throws IOException; + + private void validateIdentifier(DataInputStream in) throws IOException { + if (readLong(in) != DATA_IDENTIFIER) { + throw new IllegalStateException("Corrupted data to deserialize"); + } + } + + protected Long readLong(DataInputStream in) throws IOException { + try { + return in.readLong(); + } catch (IndexOutOfBoundsException | EOFException e) { + throw new IllegalStateException("Incomplete serialized state"); Review comment: If we are going to rethrow we should wrap `e`. However, I would be inclined to just remove these methods `readLong`/`readInt` directly and let the exceptions bubble up. This is similar to other implementations. Generally speaking it is bad practise to catch `RuntimeException` ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsStateSerializer.java ########## @@ -0,0 +1,77 @@ +/* + * 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.kinesis.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.sink.writer.AsyncSinkWriterStateSerializer; + +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** Kinesis Streams implementation {@link AsyncSinkWriterStateSerializer}. */ +@Internal +public class KinesisDataStreamsStateSerializer + extends AsyncSinkWriterStateSerializer<PutRecordsRequestEntry> { + @Override + protected void serializeRequestToStream(PutRecordsRequestEntry request, DataOutputStream out) + throws IOException { + out.write(request.data().asByteArrayUnsafe()); + serializePartitionKeyToStream(request.partitionKey(), out); + validateExplicitHashKey(request); + } + + protected void serializePartitionKeyToStream(String partitionKey, DataOutputStream out) + throws IOException { + out.writeInt(partitionKey.length()); + out.write(partitionKey.getBytes()); + } + + protected void validateExplicitHashKey(PutRecordsRequestEntry request) { + if (request.explicitHashKey() != null) { + throw new IllegalStateException( + "Request contains field not included in serialization."); + } + } + + @Override + protected PutRecordsRequestEntry deserializeRequestFromStream( + long requestSize, DataInputStream in) throws IOException { + byte[] requestData = readBytes(in, (int) requestSize); + + return PutRecordsRequestEntry.builder() + .data(SdkBytes.fromByteArray(requestData)) + .partitionKey(deserializePartitionKeyToStream(in)) + .build(); + } + + protected String deserializePartitionKeyToStream(DataInputStream in) throws IOException { Review comment: > deserializePartitionKeyToStream `deserializePartitionKeyFromStream` ? ########## File path: flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterStateSerializerTest.java ########## @@ -0,0 +1,92 @@ +/* + * 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.base.sink.writer; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; + +/** Test class for {@link AsyncSinkWriterStateSerializer}. */ +public class AsyncSinkWriterStateSerializerTest { + + @Test + public void testSerializeAndDeSerialize() throws IOException { + AsyncSinkWriterStateSerializerImpl stateSerializer = + new AsyncSinkWriterStateSerializerImpl(); + List<String> testData = getTestData(100); + BufferedRequestState<String> state = wrapStringRequests(testData); + Collection<String> deserializedState = + stripStringEntryWrappers( + stateSerializer.deserialize(0, stateSerializer.serialize(state))); + + Assertions.assertThat(deserializedState).isEqualTo(testData); + } + + private List<String> getTestData(int size) { + List<String> testData = new ArrayList<>(); + for (int i = 1; i <= size; i++) { + testData.add(String.format("value:%d", i)); + } + return testData; + } Review comment: This is duplicated in a few places. Can we move to a test util? Also can simplify with java streams as pointed out in a nit comment above ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java ########## @@ -406,11 +426,28 @@ private void addEntryToBuffer(RequestEntryT entry, boolean insertAtHead) { * a failure/restart of the application. */ @Override - public List<Collection<RequestEntryT>> snapshotState() { - return Arrays.asList( - bufferedRequestEntries.stream() - .map(RequestEntryWrapper::getRequestEntry) - .collect(Collectors.toList())); + public List<BufferedRequestState<RequestEntryT>> snapshotState() { + return Collections.singletonList(new BufferedRequestState<>((bufferedRequestEntries))); + } + + protected void initialize(List<BufferedRequestState<RequestEntryT>> states) { + if (states.isEmpty()) { + return; + } + + BufferedRequestState<RequestEntryT> state = states.get(0); Review comment: Can we validate that there are no more than 1? Do we already do this somewhere? ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/BufferedRequestState.java ########## @@ -0,0 +1,73 @@ +/* + * 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.base.sink.writer; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Deque; +import java.util.List; + +/** + * Class holding state of {@link AsyncSinkWriter} needed at taking a snapshot. The state captures + * the {@code bufferedRequestEntries} buffer for the writer at snapshot to resume the requests. This + * guarantees at least once semantic in sending requests where restoring from a snapshot where + * buffered requests were flushed to the sink will cause duplicate requests. + * + * @param <RequestEntryT> request type. + */ +@PublicEvolving +public class BufferedRequestState<RequestEntryT extends Serializable> implements Serializable { + private final List<RequestEntryWrapper<RequestEntryT>> bufferedRequestEntries; + private final long stateSize; + + public BufferedRequestState(Deque<RequestEntryWrapper<RequestEntryT>> bufferedRequestEntries) { + this.bufferedRequestEntries = new ArrayList<>(); + this.bufferedRequestEntries.addAll(bufferedRequestEntries); + this.stateSize = + bufferedRequestEntries.stream() + .map(RequestEntryWrapper::getSize) + .reduce(Long::sum) + .orElse(0L); + } + + public BufferedRequestState(List<RequestEntryWrapper<RequestEntryT>> bufferedRequestEntries) { + this.bufferedRequestEntries = new ArrayList<>(); + this.bufferedRequestEntries.addAll(bufferedRequestEntries); + this.stateSize = + bufferedRequestEntries.stream() + .map(RequestEntryWrapper::getSize) + .reduce(Long::sum) + .orElse(0L); + } + + public List<RequestEntryWrapper<RequestEntryT>> getBufferedRequestEntries() { + return bufferedRequestEntries; + } + + public Long getStateSize() { Review comment: This should be `long` not `Long` ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsStateSerializerTest.java ########## @@ -0,0 +1,93 @@ +/* + * 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.kinesis.sink; + +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.connector.base.sink.writer.BufferedRequestState; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.connector.base.sink.writer.RequestEntryWrapper; + +import org.junit.Test; +import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** Test class for {@link KinesisDataStreamsStateSerializer}. */ +public class KinesisDataStreamsStateSerializerTest { + + private static final ElementConverter<String, PutRecordsRequestEntry> ELEMENT_CONVERTER = + KinesisDataStreamsSinkElementConverter.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) + .setPartitionKeyGenerator(element -> String.valueOf(element.hashCode())) + .build(); + + @Test + public void testSerializeAndDeserialize() throws IOException { + BufferedRequestState<PutRecordsRequestEntry> expectedState = getTestState(); + + KinesisDataStreamsStateSerializer serializer = new KinesisDataStreamsStateSerializer(); + BufferedRequestState<PutRecordsRequestEntry> actualState = + serializer.deserialize(1, serializer.serialize(expectedState)); + assertThatBufferStatesAreEqual(actualState, expectedState); + } + + private BufferedRequestState<PutRecordsRequestEntry> getTestState() { + List<String> testData = getTestData(100); Review comment: nit: ``` return new BufferedRequestState<PutRecordsRequestEntry>(IntStream.range(0, 100) .mapToObj(i -> String.format("value:%d", i)) .map(element -> ELEMENT_CONVERTER.apply(element, null)) .map(request -> new RequestEntryWrapper<>(request, getRequestSize(request))) .collect(Collectors.toList())); ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
