Jiabao-Sun commented on code in PR #1: URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1027761305
########## flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java: ########## @@ -0,0 +1,200 @@ +/* + * 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.mongodb.sink.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.DeliveryGuarantee; + +import java.io.Serializable; +import java.util.Objects; + +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BUFFER_FLUSH_INTERVAL; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BUFFER_FLUSH_MAX_ROWS; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_MAX_RETRIES; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Configurations for MongoSink to control write operations. All the options list here could be + * configured by {@link MongoWriteOptionsBuilder}. + */ +@PublicEvolving +public final class MongoWriteOptions implements Serializable { + + private static final long serialVersionUID = 1L; + + private final int batchSize; + private final long batchIntervalMs; + private final int maxRetries; + private final long retryIntervalMs; + private final DeliveryGuarantee deliveryGuarantee; + + private MongoWriteOptions( + int batchSize, + long batchIntervalMs, + int maxRetries, + long retryIntervalMs, + DeliveryGuarantee deliveryGuarantee) { + this.batchSize = batchSize; + this.batchIntervalMs = batchIntervalMs; + this.maxRetries = maxRetries; + this.retryIntervalMs = retryIntervalMs; + this.deliveryGuarantee = deliveryGuarantee; + } + + public int getBatchSize() { + return batchSize; + } + + public long getBatchIntervalMs() { + return batchIntervalMs; + } + + public int getMaxRetries() { + return maxRetries; + } + + public long getRetryIntervalMs() { + return retryIntervalMs; + } + + public DeliveryGuarantee getDeliveryGuarantee() { + return deliveryGuarantee; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MongoWriteOptions that = (MongoWriteOptions) o; + return batchSize == that.batchSize + && batchIntervalMs == that.batchIntervalMs + && maxRetries == that.maxRetries + && retryIntervalMs == that.retryIntervalMs + && deliveryGuarantee == that.deliveryGuarantee; + } + + @Override + public int hashCode() { + return Objects.hash( + batchSize, batchIntervalMs, maxRetries, retryIntervalMs, deliveryGuarantee); + } + + public static MongoWriteOptionsBuilder builder() { + return new MongoWriteOptionsBuilder(); + } + + /** Builder for {@link MongoWriteOptions}. */ + @PublicEvolving + public static class MongoWriteOptionsBuilder { + private int batchSize = BUFFER_FLUSH_MAX_ROWS.defaultValue(); + private long batchIntervalMs = BUFFER_FLUSH_INTERVAL.defaultValue().toMillis(); + private int maxRetries = SINK_MAX_RETRIES.defaultValue(); + private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis(); + private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE; + + private MongoWriteOptionsBuilder() {} + + /** + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. The default flush size is 1000. + * + * @param batchSize the maximum number of actions to buffer per bulk request. + * @return this builder + */ + public MongoWriteOptionsBuilder setBatchSize(int batchSize) { + checkArgument( + batchSize == -1 || batchSize > 0, + "Max number of batch size must be larger than 0."); + this.batchSize = batchSize; + return this; + } + + /** + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. + * + * @param batchIntervalMs the batch flush interval, in milliseconds. + * @return this builder + */ + public MongoWriteOptionsBuilder setBatchIntervalMs(long batchIntervalMs) { + checkArgument( + batchIntervalMs == -1 || batchIntervalMs >= 0, + "The batch flush interval (in milliseconds) between each flush must be larger than " + + "or equal to 0."); Review Comment: There are currently three ways to write: 1. flush only on checkpoint 2. flush for for every single element 3. flush when time interval or batch size exceed limi | batch interval \\ batch size | -1 | 1 | > 1 | |--|--|--|--| | -1 | flush on checkpoint | flush for every single element | flush when batch size exceed limit | | 0 | flush for every single element | flush for every single element | flush for every single element | | > 0 | flush when time interval exceed limit | flush for every single element | flush when time interval or batch size exceed limit | -- 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