dannycranmer commented on a change in pull request #18669: URL: https://github.com/apache/flink/pull/18669#discussion_r804485922
########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsStateSerializerTest.java ########## @@ -0,0 +1,76 @@ +/* + * 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.List; + +import static org.apache.flink.connector.base.sink.writer.AsyncSinkWriterTestUtils.getTestState; +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(ELEMENT_CONVERTER, this::getRequestSize); + + KinesisDataStreamsStateSerializer serializer = new KinesisDataStreamsStateSerializer(); + BufferedRequestState<PutRecordsRequestEntry> actualState = + serializer.deserialize(1, serializer.serialize(expectedState)); + assertThatBufferStatesAreEqual(actualState, expectedState); + } + + private int getRequestSize(PutRecordsRequestEntry requestEntry) { + return requestEntry.data().asByteArrayUnsafe().length; + } + + private static void assertThatBufferStatesAreEqual( + BufferedRequestState<PutRecordsRequestEntry> actual, + BufferedRequestState<PutRecordsRequestEntry> expected) { + assertEquals(actual.getStateSize(), expected.getStateSize()); + int actualLength = actual.getBufferedRequestEntries().size(); + assertEquals(actualLength, expected.getBufferedRequestEntries().size()); + List<RequestEntryWrapper<PutRecordsRequestEntry>> actualRequests = + actual.getBufferedRequestEntries(); + List<RequestEntryWrapper<PutRecordsRequestEntry>> expectedRequests = + expected.getBufferedRequestEntries(); + for (int i = 0; i < actualLength; i++) { + assertEquals( + actualRequests.get(i).getRequestEntry(), + expectedRequests.get(i).getRequestEntry()); + assertEquals(actualRequests.get(i).getSize(), expectedRequests.get(i).getSize()); + } Review comment: nit: Add new lines to make this more readable ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsStateSerializer.java ########## @@ -0,0 +1,83 @@ +/* + * 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; +import java.nio.charset.StandardCharsets; + +/** 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(StandardCharsets.UTF_8)); + } + + protected void validateExplicitHashKey(PutRecordsRequestEntry request) { + if (request.explicitHashKey() != null) { + throw new IllegalStateException( + String.format( + "KinesisDataStreamsStateSerializer is incompatible with ElementConverter." + + "Serializer version %d does not support explicit hash key.", + getVersion())); + } + } + + @Override + protected PutRecordsRequestEntry deserializeRequestFromStream( + long requestSize, DataInputStream in) throws IOException { + byte[] requestData = new byte[(int) requestSize]; + in.read(requestData); + + return PutRecordsRequestEntry.builder() + .data(SdkBytes.fromByteArray(requestData)) + .partitionKey(deserializePartitionKeyFromStream(in)) + .build(); + } + + protected String deserializePartitionKeyFromStream(DataInputStream in) throws IOException { + int partitionKeyLength = in.readInt(); + byte[] requestPartitionKeyData = new byte[(int) partitionKeyLength]; + in.read(requestPartitionKeyData); Review comment: We need to confirm that this will always read the entire length of the string. ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterStateSerializer.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.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.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 = in.readInt(); + List<RequestEntryWrapper<RequestEntryT>> serializedState = new ArrayList<>(); + for (int i = 0; i < size; i++) { + long requestSize = in.readLong(); + RequestEntryT request = deserializeRequestFromStream(requestSize, in); + serializedState.add(new RequestEntryWrapper<>(request, requestSize)); + } + + return new BufferedRequestState<>(serializedState); + } + } Review comment: nit: Add new lines to make more readable ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/BufferedRequestState.java ########## @@ -0,0 +1,84 @@ +/* + * 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.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Objects; + +/** + * Class holding internal state of buffered requests. + * + * @param <RequestEntryT> request type. + */ +@PublicEvolving Review comment: Why is this failing? ########## File path: flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTestUtils.java ########## @@ -0,0 +1,60 @@ +/* + * 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 java.io.Serializable; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** Utils class for {@link AsyncSinkWriter} related test. */ +public class AsyncSinkWriterTestUtils { + + public static <T extends Serializable> BufferedRequestState<T> getTestState( + ElementConverter<String, T> elementConverter, + Function<T, Integer> requestSizeExtractor) { + return new BufferedRequestState<>( + IntStream.range(0, 100) + .mapToObj(i -> String.format("value:%d", i)) + .map(element -> elementConverter.apply(element, null)) + .map( + request -> + new RequestEntryWrapper<>( + request, requestSizeExtractor.apply(request))) + .collect(Collectors.toList())); + } + + public static <T extends Serializable> void assertThatBufferStatesAreEqual( + BufferedRequestState<T> actual, BufferedRequestState<T> expected) { + assertEquals(actual.getStateSize(), expected.getStateSize()); + int actualLength = actual.getBufferedRequestEntries().size(); + assertEquals(actualLength, expected.getBufferedRequestEntries().size()); + List<RequestEntryWrapper<T>> actualRequests = actual.getBufferedRequestEntries(); + List<RequestEntryWrapper<T>> expectedRequests = expected.getBufferedRequestEntries(); + for (int i = 0; i < actualLength; i++) { + assertEquals( + actualRequests.get(i).getRequestEntry(), + expectedRequests.get(i).getRequestEntry()); + assertEquals(actualRequests.get(i).getSize(), expectedRequests.get(i).getSize()); + } + } Review comment: nit: add whitespace to make this more readable ########## 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); Review comment: nit: ``` this.bufferedRequestEntries = new ArrayList<>(bufferedRequestEntries); ``` ########## 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); Review comment: Please see guidance on use of java streams. Unfortunately I would consider this performance critical and therefore please remove streams: https://flink.apache.org/contributing/code-style-and-quality-java.html#java-streams ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsStateSerializer.java ########## @@ -0,0 +1,83 @@ +/* + * 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; +import java.nio.charset.StandardCharsets; + +/** 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(StandardCharsets.UTF_8)); + } + + protected void validateExplicitHashKey(PutRecordsRequestEntry request) { + if (request.explicitHashKey() != null) { + throw new IllegalStateException( + String.format( + "KinesisDataStreamsStateSerializer is incompatible with ElementConverter." + + "Serializer version %d does not support explicit hash key.", + getVersion())); + } + } + + @Override + protected PutRecordsRequestEntry deserializeRequestFromStream( + long requestSize, DataInputStream in) throws IOException { + byte[] requestData = new byte[(int) requestSize]; + in.read(requestData); + + return PutRecordsRequestEntry.builder() + .data(SdkBytes.fromByteArray(requestData)) + .partitionKey(deserializePartitionKeyFromStream(in)) + .build(); + } + + protected String deserializePartitionKeyFromStream(DataInputStream in) throws IOException { + int partitionKeyLength = in.readInt(); + byte[] requestPartitionKeyData = new byte[(int) partitionKeyLength]; + in.read(requestPartitionKeyData); Review comment: I see this is a wrapped byte array in the parent class -- 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]
