zentol commented on code in PR #20757:
URL: https://github.com/apache/flink/pull/20757#discussion_r965797922


##########
flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/RateLimitedSourceReader.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.api.connector.source.lib.util;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.ReaderOutput;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.flink.core.io.InputStatus;
+
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Wraps the actual {@link SourceReader} and rate limits its data emission. */
+@Experimental
+public class RateLimitedSourceReader<E, SplitT extends SourceSplit>
+        implements SourceReader<E, SplitT> {
+
+    private final SourceReader<E, SplitT> sourceReader;
+    private final RateLimiter rateLimiter;
+
+    /**
+     * Instantiates a new rate-limited source reader.
+     *
+     * @param sourceReader The actual source reader.
+     * @param rateLimiter The rate limiter.
+     */
+    public RateLimitedSourceReader(SourceReader<E, SplitT> sourceReader, 
RateLimiter rateLimiter) {
+        checkNotNull(sourceReader);
+        checkNotNull(rateLimiter);
+        this.sourceReader = sourceReader;
+        this.rateLimiter = rateLimiter;
+    }
+
+    // ------------------------------------------------------------------------
+
+    @Override
+    public void start() {
+        sourceReader.start();
+    }
+
+    @Override
+    public InputStatus pollNext(ReaderOutput<E> output) throws Exception {
+        rateLimiter.acquire();

Review Comment:
   > Can't we use tryAcquire() for such implementation directly?
   
   No, that would imply hot-looping if there if we have currently hit the rate 
limit.
   
   > The call to sourceReader.pollNext(output) in the sketch is unrestricted
   
   That's not quite true. The contract of the `SourceReader`, as I understand 
it, is that if you return `NOTHING_AVAILABLE` then `isAvailable()` is called, 
and `pollNext()` is only called again once the returned future is complete.
   By never returning `MORE_AVAILABLE` `pollNext()` should never be called 
multiple times in a row.
   
   > The underlying sourceReader is going to keep internally calling 
output.collect(iterator.next()) without any limitations
   
   This is also an issue in the current PR. SourceReaders are also encouraged, 
per the javadocs, to not emit multiple records within a single `pollNext()` 
call.
   
   >  Do I get it wrong?
   
   Yes, I think so. There's no busy waiting because we either a) wait for the 
future to complete or b) poll values.



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