dannycranmer commented on code in PR #20245:
URL: https://github.com/apache/flink/pull/20245#discussion_r919721608


##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java:
##########
@@ -436,9 +435,12 @@ private void completeRequest(
         ackTime = System.currentTimeMillis();
 
         inFlightRequestsCount--;
-        inFlightMessages -= batchSize;
-
-        updateInFlightMessagesLimit(failedRequestEntries.size() == 0);
+        rateLimitingStrategy.registerCompletedRequest(
+                RequestInfo.builder()

Review Comment:
   Another variation of `RequestInfo` used with varying fields set. 



##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java:
##########
@@ -344,44 +343,42 @@ public void write(InputT element, Context context) throws 
IOException, Interrupt
      * </ul>
      */
     private void nonBlockingFlush() throws InterruptedException {
-        while (!isInFlightRequestOrMessageLimitExceeded()
+        while (!rateLimitingStrategy.shouldBlock(createRequestInfo())
                 && (bufferedRequestEntries.size() >= getNextBatchSizeLimit()
-                        || bufferedRequestEntriesTotalSizeInBytes >= 
maxBatchSizeInBytes)) {
+                || bufferedRequestEntriesTotalSizeInBytes >= 
maxBatchSizeInBytes)) {
             flush();
         }
     }
 
-    /**
-     * Determines if the sink should block and complete existing in flight 
requests before it may
-     * prudently create any new ones. This is exactly determined by if the 
number of requests
-     * currently in flight exceeds the maximum supported by the sink OR if the 
number of in flight
-     * messages exceeds the maximum determined to be appropriate by the rate 
limiting strategy.
-     */
-    private boolean isInFlightRequestOrMessageLimitExceeded() {
-        return inFlightRequestsCount >= maxInFlightRequests
-                || inFlightMessages >= rateLimitingStrategy.getRateLimit();
+    private RequestInfo createRequestInfo() {
+        int batchSize = getNextBatchSize();
+        return RequestInfo.builder().setBatchSize(batchSize).build();
     }
 
     /**
      * Persists buffered RequestsEntries into the destination by invoking 
{@code
      * submitRequestEntries} with batches according to the user specified 
buffering hints.
      *
-     * <p>The method blocks if too many async requests are in flight.
+     * <p>The method checks with the {@code rateLimitingStrategy} to see if it 
should block the
+     * request.
      */
     private void flush() throws InterruptedException {
-        while (isInFlightRequestOrMessageLimitExceeded()) {
+        RequestInfo requestInfo = createRequestInfo();
+        while (rateLimitingStrategy.shouldBlock(requestInfo)) {
             mailboxExecutor.yield();
+            requestInfo = createRequestInfo();
         }
 
-        List<RequestEntryT> batch = createNextAvailableBatch();
+        List<RequestEntryT> batch = createNextAvailableBatch(requestInfo);
         int batchSize = batch.size();
-
-        if (batch.size() == 0) {
+        if (batchSize == 0) {
             return;
         }
+        requestInfo.setBatchSize(batchSize);
 
         long timestampOfRequest = System.currentTimeMillis();
-        Consumer<List<RequestEntryT>> requestResult =
+        requestInfo.setRequestStartTime(timestampOfRequest);

Review Comment:
   This is a code smell to have fields of an object `null` in some cases and 
set in others. If `createNextAvailableBatch()` only requires batch size suggest 
passing in the `int` since it is private anyway. Then try to make  
`requestInfo` immutable 



##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java:
##########
@@ -344,44 +343,42 @@ public void write(InputT element, Context context) throws 
IOException, Interrupt
      * </ul>
      */
     private void nonBlockingFlush() throws InterruptedException {
-        while (!isInFlightRequestOrMessageLimitExceeded()
+        while (!rateLimitingStrategy.shouldBlock(createRequestInfo())
                 && (bufferedRequestEntries.size() >= getNextBatchSizeLimit()
-                        || bufferedRequestEntriesTotalSizeInBytes >= 
maxBatchSizeInBytes)) {
+                || bufferedRequestEntriesTotalSizeInBytes >= 
maxBatchSizeInBytes)) {
             flush();
         }
     }
 
-    /**
-     * Determines if the sink should block and complete existing in flight 
requests before it may
-     * prudently create any new ones. This is exactly determined by if the 
number of requests
-     * currently in flight exceeds the maximum supported by the sink OR if the 
number of in flight
-     * messages exceeds the maximum determined to be appropriate by the rate 
limiting strategy.
-     */
-    private boolean isInFlightRequestOrMessageLimitExceeded() {
-        return inFlightRequestsCount >= maxInFlightRequests
-                || inFlightMessages >= rateLimitingStrategy.getRateLimit();
+    private RequestInfo createRequestInfo() {
+        int batchSize = getNextBatchSize();
+        return RequestInfo.builder().setBatchSize(batchSize).build();
     }
 
     /**
      * Persists buffered RequestsEntries into the destination by invoking 
{@code
      * submitRequestEntries} with batches according to the user specified 
buffering hints.
      *
-     * <p>The method blocks if too many async requests are in flight.
+     * <p>The method checks with the {@code rateLimitingStrategy} to see if it 
should block the
+     * request.
      */
     private void flush() throws InterruptedException {
-        while (isInFlightRequestOrMessageLimitExceeded()) {
+        RequestInfo requestInfo = createRequestInfo();
+        while (rateLimitingStrategy.shouldBlock(requestInfo)) {
             mailboxExecutor.yield();
+            requestInfo = createRequestInfo();
         }
 
-        List<RequestEntryT> batch = createNextAvailableBatch();
+        List<RequestEntryT> batch = createNextAvailableBatch(requestInfo);
         int batchSize = batch.size();
-
-        if (batch.size() == 0) {
+        if (batchSize == 0) {
             return;
         }
+        requestInfo.setBatchSize(batchSize);

Review Comment:
   Why do we need to update the `batchSize` here? Is it incase the batchSize is 
LESS than the original batch size specified by `createRequestInfo()` ?
   
   Ideally we should make `RequestInfo` immutable to follow general best 
practise. 



##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/strategy/RequestInfo.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.strategy;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** Dataclass to encapsulate information about starting / completing requests. 
*/
+@PublicEvolving
+public class RequestInfo {
+    public final int failedMessages;
+    public int batchSize;
+    public long requestStartTime;

Review Comment:
   Use private fields with getters



##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/strategy/CongestionControlRateLimitingStrategy.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.strategy;
+
+import org.apache.flink.util.Preconditions;
+
+/**
+ * A {@code RateLimitingStrategy} implementation that does the following:
+ *
+ * <ul>
+ *   <li>Scales up when any request is successful.
+ *   <li>Scales down when any message in a request is unsuccessful.
+ *   <li>Uses Additive Increase / Multiplicative Decrease (AIMD) strategy to 
scale up/down.
+ * </ul>
+ *
+ * <p>This strategy works well for throughput-limited record-based sinks (e.g. 
Kinesis, Kafka).
+ */
+public class CongestionControlRateLimitingStrategy implements 
RateLimitingStrategy {

Review Comment:
   This needs `@PublicEvolving`



##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/strategy/RequestInfo.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.strategy;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** Dataclass to encapsulate information about starting / completing requests. 
*/
+@PublicEvolving
+public class RequestInfo {
+    public final int failedMessages;
+    public int batchSize;
+    public long requestStartTime;
+
+    private RequestInfo(int failedMessages, int batchSize, long 
requestStartTime) {
+        this.failedMessages = failedMessages;
+        this.batchSize = batchSize;
+        this.requestStartTime = requestStartTime;
+    }
+
+    public void setBatchSize(int batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    public void setRequestStartTime(long requestStartTime) {
+        this.requestStartTime = requestStartTime;
+    }

Review Comment:
   as mentioned before, can we make this immutable? Seems like a smell to have 
a dataclass with a Builder and private constructor to have setters



##########
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/strategy/AIMDScalingStrategyTest.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.strategy;
+
+import org.junit.Test;

Review Comment:
   We should be using Junit 5 now, can you migrate, see this commit for 
example: 
   - 
https://github.com/apache/flink/commit/a10fd2314269e0686d9fbbd3eacb470a03432ef3



##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java:
##########
@@ -221,10 +196,12 @@ protected abstract void submitRequestEntries(
      * be the same).
      *
      * @param requestEntry the requestEntry for which we want to know the size
+     *
      * @return the size of the requestEntry, as defined previously
      */
     protected abstract long getSizeInBytes(RequestEntryT requestEntry);
 
+    @Deprecated

Review Comment:
   nit: When deprecating it is nice to add javadoc that tells the caller what 
to use in preference



##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/strategy/CongestionControlRateLimitingStrategy.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.strategy;
+
+import org.apache.flink.util.Preconditions;
+
+/**
+ * A {@code RateLimitingStrategy} implementation that does the following:
+ *
+ * <ul>
+ *   <li>Scales up when any request is successful.
+ *   <li>Scales down when any message in a request is unsuccessful.
+ *   <li>Uses Additive Increase / Multiplicative Decrease (AIMD) strategy to 
scale up/down.
+ * </ul>
+ *
+ * <p>This strategy works well for throughput-limited record-based sinks (e.g. 
Kinesis, Kafka).
+ */
+public class CongestionControlRateLimitingStrategy implements 
RateLimitingStrategy {
+
+    private final int maxInFlightRequests;
+    private final AIMDScalingStrategy aimdScalingStrategy;

Review Comment:
   Why can this not be an interface so we can pass in arbitrary scaling 
strategies? This was part of the original design, which I know was 
"simplified", but it results in a code smell IMO, We have a coupling here, are 
we sure this is the right thing to do? 



##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/strategy/AIMDScalingStrategy.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.strategy;
+
+import org.apache.flink.util.Preconditions;
+
+/**
+ * AIMDScalingStrategy scales up linearly and scales down multiplicatively. See
+ * https://en.wikipedia.org/wiki/Additive_increase/multiplicative_decrease for 
more details
+ */
+public class AIMDScalingStrategy {

Review Comment:
   This needs `@PublicEvolving`



##########
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/strategy/RequestInfo.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.strategy;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** Dataclass to encapsulate information about starting / completing requests. 
*/
+@PublicEvolving
+public class RequestInfo {
+    public final int failedMessages;
+    public int batchSize;
+    public long requestStartTime;
+
+    private RequestInfo(int failedMessages, int batchSize, long 
requestStartTime) {
+        this.failedMessages = failedMessages;
+        this.batchSize = batchSize;
+        this.requestStartTime = requestStartTime;
+    }
+
+    public void setBatchSize(int batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    public void setRequestStartTime(long requestStartTime) {
+        this.requestStartTime = requestStartTime;
+    }
+
+    @PublicEvolving
+    public static RequestInfoBuilder builder() {
+        return new RequestInfoBuilder();
+    }
+
+    /** Builder for {@link RequestInfo} dataclass. */
+    public static class RequestInfoBuilder {
+        private int failedMessages;
+        private int batchSize;
+        private long requestStartTime;
+

Review Comment:
   The fact these fields might be `0` under certain conditions which means "not 
set" reinforces the previous concerns I have mentioned. If we genuinely want to 
reuse this object and leverage setters we would need to distinguish between 
`failedMessages` as 0 or unset. Could be achieved with `-1`, but prefer refactor



-- 
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]

Reply via email to