CrynetLogistics commented on a change in pull request #17345: URL: https://github.com/apache/flink/pull/17345#discussion_r756760460
########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsSinkBuilder.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.connector.kinesis.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.sink.AsyncSinkBaseBuilder; + +import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry; + +import java.util.Properties; + +/** + * Builder to construct {@link KinesisDataStreamsSink}. + * + * <p>The following example shows the minimum setup to create a {@link KinesisDataStreamsSink} that + * writes String values to a Kinesis Data Streams stream named your_stream_here. + * + * <pre>{@code + * ElementConverter<String, PutRecordsRequestEntry> elementConverter = + * KinesisDataStreamsSinkElementConverter.<String>builder() + * .setSerializationSchema(new SimpleStringSchema()) + * .setPartitionKeyGenerator(element -> String.valueOf(element.hashCode())) + * .build(); + * + * KinesisDataStreamsSink<String> kdsSink = + * KinesisDataStreamsSink.<String>builder() + * .setElementConverter(elementConverter) + * .setStreamName("your_stream_name") + * .build(); + * }</pre> + * + * <p>If the following parameters are not set in this builder, the following defaults will be used: + * + * <ul> + * <li>{@code maxBatchSize} will be 200 + * <li>{@code maxInFlightRequests} will be 16 + * <li>{@code maxBufferedRequests} will be 10000 + * <li>{@code flushOnBufferSizeInBytes} will be 64MB i.e. {@code 64 * 1024 * 1024} + * <li>{@code maxTimeInBufferMS} will be 5000ms + * <li>{@code failOnError} will be false + * </ul> + * + * @param <InputT> type of elements that should be persisted in the destination + */ +@PublicEvolving +public class KinesisDataStreamsSinkBuilder<InputT> + extends AsyncSinkBaseBuilder< + InputT, PutRecordsRequestEntry, KinesisDataStreamsSinkBuilder<InputT>> { + + private static final int DEFAULT_MAX_BATCH_SIZE = 200; + private static final int DEFAULT_MAX_IN_FLIGHT_REQUESTS = 16; + private static final int DEFAULT_MAX_BUFFERED_REQUESTS = 10000; + private static final long DEFAULT_FLUSH_ON_BUFFER_SIZE_IN_B = 64 * 1024 * 1024; + private static final long DEFAULT_MAX_TIME_IN_BUFFER_MS = 5000; + private static final boolean DEFAULT_FAIL_ON_ERROR = false; Review comment: It has been decided that we will: 1. introduce parameters `maxBatchBytes` and `maxRecordSize` in the base sink, 2. modify the base sink behaviour to `(batchSize < maxBatchSize) AND (batchBytes < maxBatchBytes)` 3. alert immediately if the sink receives a record of size greater than `maxRecordSize` in the base sink. -- 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]
