This is an automated email from the ASF dual-hosted git repository.

mhubail pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 59f0dad28f [ASTERIXDB-3530][OTH] Add throttle count to cloud rate 
limiter
59f0dad28f is described below

commit 59f0dad28fef36bd33f40a747d1876e479f97f4b
Author: Murtadha Hubail <[email protected]>
AuthorDate: Tue Nov 19 21:13:50 2024 +0300

    [ASTERIXDB-3530][OTH] Add throttle count to cloud rate limiter
    
    - user model changes: no
    - storage format changes: no
    - interface changes: yes
    
    Details:
    
    - Refactor cloud request rate limiter classes.
    - Add read/write throttle count to cloud rate limiter.
    
    Ex-ref MB-62928
    
    Change-Id: I174f90bfe5fd3a6b183dd9cf74d2a2ed89dae930
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/19094
    Integration-Tests: Jenkins <[email protected]>
    Tested-by: Jenkins <[email protected]>
    Reviewed-by: Murtadha Hubail <[email protected]>
    Reviewed-by: Hussain Towaileb <[email protected]>
---
 ...r.java => AbstractCloudRequestRateLimiter.java} | 27 +++++++++++-----
 .../cloud/clients/aws/s3/S3RequestRateLimiter.java | 37 +++-------------------
 .../azure/blobstorage/AzureRequestRateLimiter.java | 36 +++------------------
 .../clients/google/gcs/GCSRequestRateLimiter.java  | 36 +++------------------
 .../profiler/CountRequestProfilerLimiter.java      | 10 ++++++
 .../clients/profiler/IRequestProfilerLimiter.java  |  3 ++
 .../profiler/NoOpRequestProfilerLimiter.java       | 10 ++++++
 .../profiler/RequestLimiterNoOpProfiler.java       | 10 ++++++
 .../clients/profiler/limiter/IRateLimiter.java     |  7 ++++
 .../profiler/limiter/IRequestRateLimiter.java      | 14 ++++++++
 .../clients/profiler/limiter/NoOpRateLimiter.java  |  5 +++
 .../profiler/limiter/NoOpRequestLimiter.java       | 10 ++++++
 .../profiler/limiter/TokenBasedRateLimiter.java    |  7 ++++
 13 files changed, 107 insertions(+), 105 deletions(-)

diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java
similarity index 68%
copy from 
asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java
copy to 
asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java
index 6a76952c94..b1c1c94100 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java
@@ -16,21 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.cloud.clients.azure.blobstorage;
+package org.apache.asterix.cloud.clients;
 
 import org.apache.asterix.cloud.clients.profiler.limiter.IRateLimiter;
 import org.apache.asterix.cloud.clients.profiler.limiter.IRequestRateLimiter;
 import org.apache.asterix.cloud.clients.profiler.limiter.NoOpRateLimiter;
 import org.apache.asterix.cloud.clients.profiler.limiter.TokenBasedRateLimiter;
 
-public final class AzureRequestRateLimiter implements IRequestRateLimiter {
-    private final IRateLimiter writeLimiter;
-    private final IRateLimiter readLimiter;
+public abstract class AbstractCloudRequestRateLimiter implements 
IRequestRateLimiter {
 
-    public AzureRequestRateLimiter(AzBlobStorageClientConfig config) {
-        long tokenAcquireTimeout = config.getTokenAcquireTimeout();
-        this.writeLimiter = 
createLimiter(config.getWriteMaxRequestsPerSeconds(), tokenAcquireTimeout);
-        this.readLimiter = 
createLimiter(config.getReadMaxRequestsPerSeconds(), tokenAcquireTimeout);
+    protected final IRateLimiter writeLimiter;
+    protected final IRateLimiter readLimiter;
+
+    public AbstractCloudRequestRateLimiter(int writeMaxRequestsPerSeconds, int 
readMaxRequestsPerSeconds,
+            long tokenAcquireTimeout) {
+        this.writeLimiter = createLimiter(writeMaxRequestsPerSeconds, 
tokenAcquireTimeout);
+        this.readLimiter = createLimiter(readMaxRequestsPerSeconds, 
tokenAcquireTimeout);
     }
 
     @Override
@@ -48,6 +49,16 @@ public final class AzureRequestRateLimiter implements 
IRequestRateLimiter {
         readLimiter.acquire();
     }
 
+    @Override
+    public long getReadThrottleCount() {
+        return readLimiter.getThrottleCount();
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return writeLimiter.getThrottleCount();
+    }
+
     private static IRateLimiter createLimiter(int maxRequestsPerSecond, long 
tokeAcquireTimeout) {
         if (maxRequestsPerSecond > 0) {
             return new TokenBasedRateLimiter(maxRequestsPerSecond, 
tokeAcquireTimeout);
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/aws/s3/S3RequestRateLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/aws/s3/S3RequestRateLimiter.java
index 37387a664a..c2d730be9e 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/aws/s3/S3RequestRateLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/aws/s3/S3RequestRateLimiter.java
@@ -18,41 +18,12 @@
  */
 package org.apache.asterix.cloud.clients.aws.s3;
 
-import org.apache.asterix.cloud.clients.profiler.limiter.IRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.IRequestRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.NoOpRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.TokenBasedRateLimiter;
+import org.apache.asterix.cloud.clients.AbstractCloudRequestRateLimiter;
 
-public final class S3RequestRateLimiter implements IRequestRateLimiter {
-    private final IRateLimiter writeLimiter;
-    private final IRateLimiter readLimiter;
+public final class S3RequestRateLimiter extends 
AbstractCloudRequestRateLimiter {
 
     public S3RequestRateLimiter(S3ClientConfig config) {
-        long tokenAcquireTimeout = config.getTokenAcquireTimeout();
-        this.writeLimiter = 
createLimiter(config.getWriteMaxRequestsPerSeconds(), tokenAcquireTimeout);
-        this.readLimiter = 
createLimiter(config.getReadMaxRequestsPerSeconds(), tokenAcquireTimeout);
-    }
-
-    @Override
-    public void writeRequest() {
-        writeLimiter.acquire();
-    }
-
-    @Override
-    public void readRequest() {
-        readLimiter.acquire();
-    }
-
-    @Override
-    public void listRequest() {
-        // List requests in S3 are considered as PUT
-        writeLimiter.acquire();
-    }
-
-    private static IRateLimiter createLimiter(int maxRequestsPerSecond, long 
tokeAcquireTimeout) {
-        if (maxRequestsPerSecond > 0) {
-            return new TokenBasedRateLimiter(maxRequestsPerSecond, 
tokeAcquireTimeout);
-        }
-        return NoOpRateLimiter.INSTANCE;
+        super(config.getWriteMaxRequestsPerSeconds(), 
config.getReadMaxRequestsPerSeconds(),
+                config.getTokenAcquireTimeout());
     }
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java
index 6a76952c94..c97f3e6056 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java
@@ -18,40 +18,12 @@
  */
 package org.apache.asterix.cloud.clients.azure.blobstorage;
 
-import org.apache.asterix.cloud.clients.profiler.limiter.IRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.IRequestRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.NoOpRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.TokenBasedRateLimiter;
+import org.apache.asterix.cloud.clients.AbstractCloudRequestRateLimiter;
 
-public final class AzureRequestRateLimiter implements IRequestRateLimiter {
-    private final IRateLimiter writeLimiter;
-    private final IRateLimiter readLimiter;
+public final class AzureRequestRateLimiter extends 
AbstractCloudRequestRateLimiter {
 
     public AzureRequestRateLimiter(AzBlobStorageClientConfig config) {
-        long tokenAcquireTimeout = config.getTokenAcquireTimeout();
-        this.writeLimiter = 
createLimiter(config.getWriteMaxRequestsPerSeconds(), tokenAcquireTimeout);
-        this.readLimiter = 
createLimiter(config.getReadMaxRequestsPerSeconds(), tokenAcquireTimeout);
-    }
-
-    @Override
-    public void writeRequest() {
-        writeLimiter.acquire();
-    }
-
-    @Override
-    public void readRequest() {
-        readLimiter.acquire();
-    }
-
-    @Override
-    public void listRequest() {
-        readLimiter.acquire();
-    }
-
-    private static IRateLimiter createLimiter(int maxRequestsPerSecond, long 
tokeAcquireTimeout) {
-        if (maxRequestsPerSecond > 0) {
-            return new TokenBasedRateLimiter(maxRequestsPerSecond, 
tokeAcquireTimeout);
-        }
-        return NoOpRateLimiter.INSTANCE;
+        super(config.getWriteMaxRequestsPerSeconds(), 
config.getReadMaxRequestsPerSeconds(),
+                config.getTokenAcquireTimeout());
     }
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSRequestRateLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSRequestRateLimiter.java
index 71f6b8ccd0..a68c3f9417 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSRequestRateLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSRequestRateLimiter.java
@@ -18,40 +18,12 @@
  */
 package org.apache.asterix.cloud.clients.google.gcs;
 
-import org.apache.asterix.cloud.clients.profiler.limiter.IRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.IRequestRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.NoOpRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.TokenBasedRateLimiter;
+import org.apache.asterix.cloud.clients.AbstractCloudRequestRateLimiter;
 
-public class GCSRequestRateLimiter implements IRequestRateLimiter {
-    private final IRateLimiter writeLimiter;
-    private final IRateLimiter readLimiter;
+public class GCSRequestRateLimiter extends AbstractCloudRequestRateLimiter {
 
     public GCSRequestRateLimiter(GCSClientConfig config) {
-        long tokenAcquireTimeout = config.getTokenAcquireTimeout();
-        this.writeLimiter = 
createLimiter(config.getWriteMaxRequestsPerSeconds(), tokenAcquireTimeout);
-        this.readLimiter = 
createLimiter(config.getReadMaxRequestsPerSeconds(), tokenAcquireTimeout);
-    }
-
-    @Override
-    public void writeRequest() {
-        writeLimiter.acquire();
-    }
-
-    @Override
-    public void readRequest() {
-        readLimiter.acquire();
-    }
-
-    @Override
-    public void listRequest() {
-        readLimiter.acquire();
-    }
-
-    private static IRateLimiter createLimiter(int maxRequestsPerSecond, long 
tokeAcquireTimeout) {
-        if (maxRequestsPerSecond > 0) {
-            return new TokenBasedRateLimiter(maxRequestsPerSecond, 
tokeAcquireTimeout);
-        }
-        return NoOpRateLimiter.INSTANCE;
+        super(config.getWriteMaxRequestsPerSeconds(), 
config.getReadMaxRequestsPerSeconds(),
+                config.getTokenAcquireTimeout());
     }
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/CountRequestProfilerLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/CountRequestProfilerLimiter.java
index 16ffb7a18e..95383f6dc8 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/CountRequestProfilerLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/CountRequestProfilerLimiter.java
@@ -140,6 +140,16 @@ public class CountRequestProfilerLimiter implements 
IRequestProfilerLimiter {
         return multipartDownloadCounter.get();
     }
 
+    @Override
+    public long getReadThrottleCount() {
+        return limiter.getReadThrottleCount();
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return limiter.getWriteThrottleCount();
+    }
+
     private void log() {
         if (LOGGER.isEnabled(LOG_LEVEL)) {
             long currentTime = System.nanoTime();
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/IRequestProfilerLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/IRequestProfilerLimiter.java
index b86cd485d2..fc0cbef6f1 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/IRequestProfilerLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/IRequestProfilerLimiter.java
@@ -47,4 +47,7 @@ public interface IRequestProfilerLimiter {
 
     long objectMultipartDownloadCount();
 
+    long getReadThrottleCount();
+
+    long getWriteThrottleCount();
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/NoOpRequestProfilerLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/NoOpRequestProfilerLimiter.java
index ab658f50ac..832457b277 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/NoOpRequestProfilerLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/NoOpRequestProfilerLimiter.java
@@ -93,4 +93,14 @@ public class NoOpRequestProfilerLimiter implements 
IRequestProfilerLimiter {
     public long objectMultipartDownloadCount() {
         return 0;
     }
+
+    @Override
+    public long getReadThrottleCount() {
+        return 0;
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return 0;
+    }
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/RequestLimiterNoOpProfiler.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/RequestLimiterNoOpProfiler.java
index cce2f8e0f7..9d5118aca0 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/RequestLimiterNoOpProfiler.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/RequestLimiterNoOpProfiler.java
@@ -96,4 +96,14 @@ public final class RequestLimiterNoOpProfiler implements 
IRequestProfilerLimiter
     public long objectMultipartDownloadCount() {
         return 0;
     }
+
+    @Override
+    public long getReadThrottleCount() {
+        return limiter.getReadThrottleCount();
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return limiter.getWriteThrottleCount();
+    }
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRateLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRateLimiter.java
index 741735bb21..883990d22a 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRateLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRateLimiter.java
@@ -26,4 +26,11 @@ public interface IRateLimiter {
      * Acquire permit or wait if rate limit exceeded
      */
     void acquire();
+
+    /**
+     * Get the number of throttled requests
+     *
+     * @return the number of throttled requests
+     */
+    long getThrottleCount();
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRequestRateLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRequestRateLimiter.java
index 98b2eab558..bc5bdbd3fb 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRequestRateLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRequestRateLimiter.java
@@ -37,4 +37,18 @@ public interface IRequestRateLimiter {
      * Perform a list request
      */
     void listRequest();
+
+    /**
+     * Get the number of throttled read requests
+     *
+     * @return the number of throttled read requests
+     */
+    long getReadThrottleCount();
+
+    /**
+     * Get the number of throttled write requests
+     *
+     * @return the number of throttled write requests
+     */
+    long getWriteThrottleCount();
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRateLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRateLimiter.java
index 4dd8c88997..356538e97d 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRateLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRateLimiter.java
@@ -28,4 +28,9 @@ public class NoOpRateLimiter implements IRateLimiter {
     public void acquire() {
         // NoOp
     }
+
+    @Override
+    public long getThrottleCount() {
+        return 0;
+    }
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRequestLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRequestLimiter.java
index ea89a2e68d..182d9255c7 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRequestLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRequestLimiter.java
@@ -38,4 +38,14 @@ public final class NoOpRequestLimiter implements 
IRequestRateLimiter {
     public void listRequest() {
         // NoOp
     }
+
+    @Override
+    public long getReadThrottleCount() {
+        return 0;
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return 0;
+    }
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/TokenBasedRateLimiter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/TokenBasedRateLimiter.java
index a0273fb523..564e1c2e00 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/TokenBasedRateLimiter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/TokenBasedRateLimiter.java
@@ -28,6 +28,7 @@ import org.apache.logging.log4j.Logger;
 public final class TokenBasedRateLimiter implements IRateLimiter {
     private static final Logger LOGGER = LogManager.getLogger();
     private static final long SECOND_NANO = TimeUnit.SECONDS.toNanos(1);
+    private final AtomicLong throttleCount = new AtomicLong();
     private final long acquireTimeoutNano;
     private final int maxTokensPerSecond;
     private final Semaphore semaphore;
@@ -54,6 +55,7 @@ public final class TokenBasedRateLimiter implements 
IRateLimiter {
                 if (semaphore.tryAcquire(acquireTimeoutNano, 
TimeUnit.NANOSECONDS)) {
                     return;
                 }
+                throttleCount.incrementAndGet();
             } catch (InterruptedException e) {
                 Thread.currentThread().interrupt();
                 LOGGER.debug("Interrupted while waiting for acquiring a 
request token", e);
@@ -62,6 +64,11 @@ public final class TokenBasedRateLimiter implements 
IRateLimiter {
         }
     }
 
+    @Override
+    public long getThrottleCount() {
+        return throttleCount.get();
+    }
+
     private void refillTokens() {
         long refillTime = lastRefillTime.get();
         long now = System.nanoTime();

Reply via email to