maedhroz commented on a change in pull request #1045:
URL: https://github.com/apache/cassandra/pull/1045#discussion_r687273415



##########
File path: 
src/java/org/apache/cassandra/utils/concurrent/NonBlockingRateLimiter.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cassandra.utils.concurrent;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Ticker;
+
+import javax.annotation.concurrent.ThreadSafe;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A rate limiter implementation that allows callers to reserve permits that 
may only be available 
+ * in the future, delegating to them decisions about how to schedule/delay 
work and whether or not
+ * to block execution to do so.
+ */
+@SuppressWarnings("UnstableApiUsage")
+@ThreadSafe
+public class NonBlockingRateLimiter
+{
+    private static final long DEFAULT_BURST_NANOS = 
TimeUnit.SECONDS.toNanos(1);
+    
+    /** a starting time for elapsed time calculations */
+    private final long startedNanos;
+    
+    /** 
+     * nanoseconds of "burst time"
+     * 
+     * ex. If this is configured at the default of 1 second, the limiter will 
allow a
+     *     number of permits equal to the configured number of permits/second 
to be issued 
+     *     in a "burst" prior to the limiter reaching a steady state. This 
allows the limiter 
+     *     to adhere more closely to the configured rate, as it can bring 
forward short
+     *     periods of inactivity if the rate of requests oscillates around the 
configured limit.
+     */
+    private final long burstNanos;
+
+    /** nanoseconds from start time corresponding to the next available permit 
*/
+    private final AtomicLong nextAvailable = new AtomicLong();
+    
+    private volatile Ticker ticker;
+    
+    private volatile int permitsPerSecond;
+    
+    /** time in nanoseconds between permits on the timeline */
+    private volatile long intervalNanos;
+
+    public NonBlockingRateLimiter(int permitsPerSecond)
+    {
+        this(permitsPerSecond, DEFAULT_BURST_NANOS, Ticker.systemTicker());
+    }
+
+    @VisibleForTesting
+    public NonBlockingRateLimiter(int permitsPerSecond, long burstNanos, 
Ticker ticker)
+    {
+        this.startedNanos = ticker.read();
+        this.burstNanos = burstNanos;
+        setRate(permitsPerSecond, ticker);
+    }
+
+    public void setRate(int permitsPerSecond)
+    {
+        setRate(permitsPerSecond, Ticker.systemTicker());
+    }
+
+    @VisibleForTesting
+    public synchronized void setRate(int permitsPerSecond, Ticker ticker)
+    {
+        Preconditions.checkArgument(permitsPerSecond > 0, "permits/second must 
be positive");
+
+        this.ticker = ticker;
+        this.permitsPerSecond = permitsPerSecond;
+        intervalNanos = Math.max(TimeUnit.SECONDS.toNanos(1) / 
permitsPerSecond, 1);
+        nextAvailable.set(nanosElapsed());
+    }
+
+    /**
+     * @return the number of available permits per second
+     */
+    public int getRate()
+    {
+        return permitsPerSecond;
+    }
+
+    /**
+     * Reserves a single permit slot on the timeline which may not yet be 
available.
+     *
+     * @return microseconds until the reserved permit will be available (or 
zero if it already is)
+     */
+    public long reserveAndGetWaitMicros()
+    {
+        long nowNanos = nanosElapsed();
+
+        for (;;)
+        {
+            long prev = nextAvailable.get();
+            long interval = this.intervalNanos;
+
+            // Push the first available permit slot up to the burst window if 
necessary.
+            long firstAvailable = Math.max(prev, nowNanos - burstNanos);
+
+            // Advance the configured interval starting from the bounded 
previous permit slot.
+            if (nextAvailable.compareAndSet(prev, firstAvailable + interval))
+                // If the time now is before the first available slot, return 
the delay.  
+                return TimeUnit.NANOSECONDS.toMicros(Math.max(0,  
firstAvailable - nowNanos));
+        }
+    }
+
+    /**
+     * Reserves a single permit slot on the timeline, but only if one is 
available.
+     *
+     * @return true if a permit is available, false if one is not
+     */
+    public boolean tryReserve()

Review comment:
       I settled on `reserveAndGetDelay()` with a configurable delay unit. I 
can only go so terse ;)




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



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to