Skip to content
Merged
Show file tree
Hide file tree
Changes from 17 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
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
#
# 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 with Vertex AI.
"""

import argparse
import logging

import apache_beam as beam
from apache_beam.io.components.rate_limiter import EnvoyRateLimiter
from apache_beam.ml.inference.base import RunInference
from apache_beam.ml.inference.vertex_ai_inference import VertexAIModelHandlerJSON
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.options.pipeline_options import SetupOptions


def run(argv=None):
parser = argparse.ArgumentParser()
parser.add_argument(
'--project',
dest='project',
help='The Google Cloud project ID for Vertex AI.')
parser.add_argument(
'--location',
dest='location',
help='The Google Cloud location (e.g. us-central1) for Vertex AI.')
parser.add_argument(
'--endpoint_id',
dest='endpoint_id',
help='The ID of the Vertex AI endpoint.')
parser.add_argument(
'--rls_address',
dest='rls_address',
help='The address of the Envoy Rate Limit Service (e.g. localhost:8081).')

known_args, pipeline_args = parser.parse_known_args(argv)
pipeline_options = PipelineOptions(pipeline_args)
pipeline_options.view_as(SetupOptions).save_main_session = True

# Initialize the EnvoyRateLimiter
rate_limiter = EnvoyRateLimiter(
service_address=known_args.rls_address,
domain="mongo_cps",
descriptors=[{
"database": "users"
}],
namespace='example_pipeline')

# Initialize the VertexAIModelHandler with the rate limiter
model_handler = VertexAIModelHandlerJSON(
endpoint_id=known_args.endpoint_id,
project=known_args.project,
location=known_args.location,
rate_limiter=rate_limiter)

# Input features for the model
features = [[1.0, 2.0, 3.0], [4.0, 5.0, 6.0], [7.0, 8.0, 9.0],
[10.0, 11.0, 12.0], [13.0, 14.0, 15.0]]

with beam.Pipeline(options=pipeline_options) as p:
_ = (
p
| 'CreateInputs' >> beam.Create(features)
| 'RunInference' >> RunInference(model_handler)
| 'PrintPredictions' >> beam.Map(logging.info))


if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
run()
15 changes: 14 additions & 1 deletion sdks/python/apache_beam/io/components/rate_limiter.py
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ def __init__(
timeout: float = 5.0,
block_until_allowed: bool = True,
retries: int = 3,
namespace: str = ""):
namespace: str = ''):
"""
Args:
service_address: Address of the Envoy RLS (e.g., 'localhost:8081').
Expand Down Expand Up @@ -224,3 +224,16 @@ def throttle(self, hits_added: int = 1) -> bool:
response.overall_code)
break
return throttled

def __getstate__(self):
state = self.__dict__.copy()
if '_lock' in state:
del state['_lock']
if '_stub' in state:
del state['_stub']
return state

def __setstate__(self, state):
self.__dict__.update(state)
self._lock = threading.Lock()
self._stub = None
21 changes: 20 additions & 1 deletion sdks/python/apache_beam/ml/inference/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@

import apache_beam as beam
from apache_beam.io.components.adaptive_throttler import ReactiveThrottler
from apache_beam.io.components.rate_limiter import RateLimiter
from apache_beam.utils import multi_process_shared
from apache_beam.utils import retry
from apache_beam.utils import shared
Expand Down Expand Up @@ -349,7 +350,8 @@ def __init__(
*,
window_ms: int = 1 * _MILLISECOND_TO_SECOND,
bucket_ms: int = 1 * _MILLISECOND_TO_SECOND,
overload_ratio: float = 2):
overload_ratio: float = 2,
rate_limiter: Optional[RateLimiter] = None):
"""Initializes a ReactiveThrottler class for enabling
client-side throttling for remote calls to an inference service. Also wraps
provided calls to the service with retry logic.
Expand All @@ -372,6 +374,7 @@ def __init__(
overload_ratio: the target ratio between requests sent and successful
requests. This is "K" in the formula in
https://landing.google.com/sre/book/chapters/handling-overload.html.
rate_limiter: A RateLimiter object for setting a global rate limit.
"""
# Configure ReactiveThrottler for client-side throttling behavior.
self.throttler = ReactiveThrottler(
Expand All @@ -383,6 +386,9 @@ def __init__(
self.logger = logging.getLogger(namespace)
self.num_retries = num_retries
self.retry_filter = retry_filter
self._rate_limiter = rate_limiter
self._shared_rate_limiter = None
self._shared_handle = shared.Shared()

def __init_subclass__(cls):
if cls.load_model is not RemoteModelHandler.load_model:
Expand Down Expand Up @@ -431,6 +437,19 @@ def run_inference(
Returns:
An Iterable of Predictions.
"""
if self._rate_limiter:
if self._shared_rate_limiter is None:

def init_limiter():
return self._rate_limiter

self._shared_rate_limiter = self._shared_handle.acquire(init_limiter)

if not self._shared_rate_limiter.throttle(hits_added=len(batch)):
raise RuntimeError(
"Rate Limit Exceeded, "
"Could not process this batch.")

self.throttler.throttle()

try:
Expand Down
64 changes: 64 additions & 0 deletions sdks/python/apache_beam/ml/inference/base_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -2071,6 +2071,70 @@ def run_inference(self,
responses.append(model.predict(example))
return responses

def test_run_inference_with_rate_limiter(self):
class FakeRateLimiter(base.RateLimiter):
def __init__(self):
super().__init__(namespace='test_namespace')

def throttle(self, hits_added=1):
self.requests_counter.inc()
return True
Comment on lines +2079 to +2081
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is going back to the base implementation of the rate limiter, but throttle() returning True logically makes me think that the request should be throttled, not that it's approved. See the adaptive throttler definition of throttle_request() (

)

Copy link
Contributor Author

@tarun-google tarun-google Jan 7, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Context is different in AdaptiveThrottler and RateLimiter. In AdaptiveThrottler throttling is always possible throttle_request is giving caller info to either apply throttle delay or allow. In RateLimiter throttling might not be always possible, and if possible, delay is already applied by RateLimiter. So true indicates that it is throttled and false indicates its not.

If the naming of throttle() is confusing for people coming from AdaptiveThrottler context we can change it to allow/acquire()

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sorry, I'm still not following this logic based based on the description in the RateLimiter class:

@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

"Check if request should be throttled" as a docstring, but then responding with a True if the request is allowed feels backwards. throttle() returning False sounds like the request isn't being throttled or limited in any way, when that isn't the case. That's where the disconnect is here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for pointing to the source of confusion. I have updated the doc string to set the expectations of the functions.


limiter = FakeRateLimiter()

with TestPipeline() as pipeline:
examples = [1, 5]

class ConcreteRemoteModelHandler(base.RemoteModelHandler):
def create_client(self):
return FakeModel()

def request(self, batch, model, inference_args=None):
return [model.predict(example) for example in batch]

model_handler = ConcreteRemoteModelHandler(
rate_limiter=limiter, namespace='test_namespace')

pcoll = pipeline | 'start' >> beam.Create(examples)
actual = pcoll | base.RunInference(model_handler)

expected = [2, 6]
assert_that(actual, equal_to(expected))

result = pipeline.run()
result.wait_until_finish()

metrics_filter = MetricsFilter().with_name(
'RatelimitRequestsTotal').with_namespace('test_namespace')
metrics = result.metrics().query(metrics_filter)
self.assertGreaterEqual(metrics['counters'][0].committed, 0)

def test_run_inference_with_rate_limiter_exceeded(self):
class FakeRateLimiter(base.RateLimiter):
def __init__(self):
super().__init__(namespace='test_namespace')

def throttle(self, hits_added=1):
return False

with self.assertRaises(RuntimeError):
with TestPipeline() as pipeline:
examples = [1]

class ConcreteRemoteModelHandler(base.RemoteModelHandler):
def create_client(self):
return FakeModel()

def request(self, batch, model, inference_args=None):
return [model.predict(example) for example in batch]

model_handler = ConcreteRemoteModelHandler(
rate_limiter=FakeRateLimiter(),
namespace='test_namespace',
num_retries=0)
pcoll = pipeline | 'start' >> beam.Create(examples)
_ = pcoll | base.RunInference(model_handler)


if __name__ == '__main__':
unittest.main()
Loading