Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
91 changes: 91 additions & 0 deletions sdks/python/apache_beam/examples/rate_limiter_simple.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
#
# 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.
#

"""A simple example demonstrating usage of the EnvoyRateLimiter in a Beam pipeline."""

import argparse
import logging
import time

import apache_beam as beam
from apache_beam.utils import shared
from apache_beam.io.components.rate_limiter import EnvoyRateLimiter
from apache_beam.options.pipeline_options import PipelineOptions


class SampleApiDoFn(beam.DoFn):
"""A DoFn that simulates calling an external API with rate limiting."""
def __init__(self, rls_address, domain, descriptors):
self.rls_address = rls_address
self.domain = domain
self.descriptors = descriptors
self._shared = shared.Shared()
self.rate_limiter = None

def setup(self):
# Initialize the rate limiter in setup()
# We use shared.Shared() to ensure only one RateLimiter instance is created
# per worker and shared across threads.
def init_limiter():
logging.info(f"Connecting to Envoy RLS at {self.rls_address}")
return EnvoyRateLimiter(
service_address=self.rls_address,
domain=self.domain,
descriptors=self.descriptors,
namespace='example_pipeline')

self.rate_limiter = self._shared.acquire(init_limiter)

def process(self, element):
self.rate_limiter.throttle()

# Process the element mock API call
logging.info(f"Processing element: {element}")
time.sleep(0.1)
yield element


def parse_known_args(argv):
"""Parses args for the workflow."""
parser = argparse.ArgumentParser()
parser.add_argument(
'--rls_address',
default='localhost:8081',
help='Address of the Envoy Rate Limit Service')
return parser.parse_known_args(argv)


def run(argv=None):
known_args, pipeline_args = parse_known_args(argv)
pipeline_options = PipelineOptions(pipeline_args)

with beam.Pipeline(options=pipeline_options) as p:
(
p
| 'Create' >> beam.Create(range(100))
| 'RateLimit' >> beam.ParDo(
SampleApiDoFn(
rls_address=known_args.rls_address,
domain="mongo_cps",
descriptors=[{
"database": "users"
}])))


if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
run()
206 changes: 206 additions & 0 deletions sdks/python/apache_beam/io/components/rate_limiter.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,206 @@
#
# 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.
#

"""
Rate Limiter classes for controlling access to external resources.
"""

import abc
import logging
import random
import threading
import time
from typing import Dict
from typing import List

import grpc
from envoy.extensions.common.ratelimit.v3 import ratelimit_pb2
from envoy.service.ratelimit.v3 import rls_pb2
from envoy.service.ratelimit.v3 import rls_pb2_grpc

from apache_beam.io.components import adaptive_throttler
from apache_beam.metrics import Metrics

_LOGGER = logging.getLogger(__name__)

_MAX_CONNECTION_RETRIES = 5
_RETRY_DELAY_SECONDS = 10


class RateLimiter(abc.ABC):
"""Abstract base class for RateLimiters."""
def __init__(self, namespace: str = ""):
# Metrics collected from the RateLimiter
# Metric updates are thread safe
self.throttling_signaler = adaptive_throttler.ThrottlingSignaler(
namespace=namespace)
self.requests_counter = Metrics.counter(
namespace, 'envoyRatelimitRequestsTotal')
self.requests_allowed = Metrics.counter(
namespace, 'envoyRatelimitRequestsAllowed')
self.requests_throttled = Metrics.counter(
namespace, 'envoyRatelimitRequestsThrottled')
self.rpc_errors = Metrics.counter(namespace, 'envoyRatelimitRpcErrors')
self.rpc_retries = Metrics.counter(namespace, 'envoyRatelimitRpcRetries')
self.rpc_latency = Metrics.distribution(
namespace, 'envoyRatelimitRpcLatencyMs')

@abc.abstractmethod
def throttle(self, **kwargs) -> bool:
"""Check if request should be throttled.

Args:
**kwargs: Keyword arguments specific to the RateLimiter implementation.

Returns:
bool: True if the request is allowed, False if retries exceeded.

Raises:
Exception: If an underlying infrastructure error occurs (e.g. RPC failure).
"""
pass


class EnvoyRateLimiter(RateLimiter):
"""
Rate limiter implementation that uses an external Envoy Rate Limit Service.
"""
def __init__(
self,
service_address: str,
domain: str,
descriptors: List[Dict[str, str]],
timeout: float = 5.0,
block_until_allowed: bool = True,
retries: int = 3,
namespace: str = ""):
"""
Args:
service_address: Address of the Envoy RLS (e.g., 'localhost:8081').
domain: The rate limit domain.
descriptors: List of descriptors (key-value pairs).
retries: Number of retries to attempt if rate limited, respected only if
block_until_allowed is False.
timeout: gRPC timeout in seconds.
block_until_allowed: If enabled blocks until RateLimiter gets
the token.
namespace: the namespace to use for logging and signaling
throttling is occurring.
"""
super().__init__(namespace=namespace)

self.service_address = service_address
self.domain = domain
self.descriptors = descriptors
self.retries = retries
self.timeout = timeout
self.block_until_allowed = block_until_allowed
self._stub = None
self._lock = threading.Lock()

def init_connection(self):
if self._stub is None:
# Acquire lock to safegaurd againest multiple DoFn threads sharing the same
# RateLimiter instance, which is the case when using Shared().
with self._lock:
if self._stub is None:
channel = grpc.insecure_channel(self.service_address)
self._stub = rls_pb2_grpc.RateLimitServiceStub(channel)

def throttle(self, hits_added: int = 1) -> bool:
"""Calls the Envoy RLS to check for rate limits.

Args:
hits_added: Number of hits to add to the rate limit.

Returns:
bool: True if the request is allowed, False if retries exceeded.
"""
self.init_connection()

# execute thread-safe gRPC call
# Convert descriptors to proto format
proto_descriptors = []
for d in self.descriptors:
entries = []
for k, v in d.items():
entries.append(ratelimit_pb2.RateLimitDescriptor.Entry(key=k, value=v))
proto_descriptors.append(ratelimit_pb2.RateLimitDescriptor(entries=entries))

request = rls_pb2.RateLimitRequest(
domain=self.domain,
descriptors=proto_descriptors,
hits_addend=hits_added)

self.requests_counter.inc()
attempt = 0
throttled = False
while True:
if not self.block_until_allowed and attempt > self.retries:
break

# Connection retry loop
for conn_attempt in range(_MAX_CONNECTION_RETRIES):
try:
start_time = time.time()
response = self._stub.ShouldRateLimit(request, timeout=self.timeout)
self.rpc_latency.update(int((time.time() - start_time) * 1000))
break
except grpc.RpcError as e:
if conn_attempt == _MAX_CONNECTION_RETRIES - 1:
_LOGGER.error("[EnvoyRateLimiter] Connection Failed: %s", e)
self.rpc_errors.inc()
raise e
self.rpc_retries.inc()
_LOGGER.warning(
"[EnvoyRateLimiter] Connection Failed, retrying: %s", e)
time.sleep(_RETRY_DELAY_SECONDS)

if response.overall_code == rls_pb2.RateLimitResponse.OK:
self.requests_allowed.inc()
throttled = True
break
elif response.overall_code == rls_pb2.RateLimitResponse.OVER_LIMIT:
self.requests_throttled.inc()
# Ratelimit exceeded, sleep for duration until reset and retry
# multiple rules can be set in the RLS config, so we need to find the max duration
sleep_s = 0.0
if response.statuses:
for status in response.statuses:
if status.code == rls_pb2.RateLimitResponse.OVER_LIMIT:
dur = status.duration_until_reset
# duration_until_reset is google.protobuf.Duration
val = dur.seconds + dur.nanos / 1e9
if val > sleep_s:
sleep_s = val

# Add 1% additive jitter to prevent thundering herd
# This adds jitter in the order of ms
jitter = random.uniform(0, 0.01 * sleep_s)
sleep_s += jitter

_LOGGER.warning("[EnvoyRateLimiter] Throttled for %s seconds", sleep_s)
# signal throttled time to backend
self.throttling_signaler.signal_throttled(int(sleep_s))
time.sleep(sleep_s)
attempt += 1
else:
_LOGGER.error(
"[EnvoyRateLimiter] Unknown code from RLS: %s",
response.overall_code)
break
return throttled
Loading
Loading