>From Murtadha Hubail <[email protected]>: Murtadha Hubail has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/19094 )
Change subject: [ASTERIXDB-3530][OTH] Add throttle count to cloud rate limiter ...................................................................... [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 --- M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/aws/s3/S3RequestRateLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/NoOpRequestProfilerLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSRequestRateLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRequestRateLimiter.java A asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRateLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/CountRequestProfilerLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/RequestLimiterNoOpProfiler.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/IRequestProfilerLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/TokenBasedRateLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRequestLimiter.java M asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRateLimiter.java 13 files changed, 176 insertions(+), 97 deletions(-) git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/94/19094/1 diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java new file mode 100644 index 0000000..b1c1c94 --- /dev/null +++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java @@ -0,0 +1,68 @@ +/* + * 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.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 abstract class AbstractCloudRequestRateLimiter implements IRequestRateLimiter { + + 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 + public void writeRequest() { + writeLimiter.acquire(); + } + + @Override + public void readRequest() { + readLimiter.acquire(); + } + + @Override + public void listRequest() { + 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); + } + return NoOpRateLimiter.INSTANCE; + } +} 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 37387a6..c2d730b 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 6a76952..c97f3e6 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 71f6b8c..a68c3f9 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 16ffb7a..95383f6 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 @@ 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 b86cd48..fc0cbef 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 @@ 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 ab658f5..832457b 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 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 cce2f8e..9d5118a 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 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 741735b..883990d 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 @@ * 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 98b2eab..bc5bdbd 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 @@ * 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 4dd8c88..356538e 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 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 ea89a2e..182d925 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 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 a0273fb..564e1c2 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 @@ 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 @@ 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 @@ } } + @Override + public long getThrottleCount() { + return throttleCount.get(); + } + private void refillTokens() { long refillTime = lastRefillTime.get(); long now = System.nanoTime(); -- To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/19094 To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings Gerrit-Project: asterixdb Gerrit-Branch: master Gerrit-Change-Id: I174f90bfe5fd3a6b183dd9cf74d2a2ed89dae930 Gerrit-Change-Number: 19094 Gerrit-PatchSet: 1 Gerrit-Owner: Murtadha Hubail <[email protected]> Gerrit-MessageType: newchange
