-
Notifications
You must be signed in to change notification settings - Fork 4.2k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[BEAM-12272] Python - Backport Firestore connector's ramp-up throttli…
…ng to Datastore connector (#14723) * Initial Python implementation of ramp-up throttler. * Finish implementation, add tests. * Add f-string. * Fix Beam user agent, bump Datastore version * Fix formatting * Add throttling counter, update var name in comment. * Add missing license * Fix timestamp test on Windows. * Remove custom to_runner_api_parameter
- Loading branch information
Showing
8 changed files
with
218 additions
and
18 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
95 changes: 95 additions & 0 deletions
95
sdks/python/apache_beam/io/gcp/datastore/v1new/rampup_throttling_fn.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,95 @@ | ||
# | ||
# 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. | ||
# | ||
|
||
import datetime | ||
import logging | ||
import time | ||
from typing import TypeVar | ||
|
||
from apache_beam import typehints | ||
from apache_beam.io.gcp.datastore.v1new import util | ||
from apache_beam.metrics.metric import Metrics | ||
from apache_beam.transforms import DoFn | ||
from apache_beam.utils.retry import FuzzedExponentialIntervals | ||
|
||
T = TypeVar('T') | ||
|
||
_LOG = logging.getLogger(__name__) | ||
|
||
|
||
@typehints.with_input_types(T) | ||
@typehints.with_output_types(T) | ||
class RampupThrottlingFn(DoFn): | ||
"""A ``DoFn`` that throttles ramp-up following an exponential function. | ||
An implementation of a client-side throttler that enforces a gradual ramp-up, | ||
broadly in line with Datastore best practices. See also | ||
https://cloud.google.com/datastore/docs/best-practices#ramping_up_traffic. | ||
""" | ||
|
||
_BASE_BUDGET = 500 | ||
_RAMP_UP_INTERVAL = datetime.timedelta(minutes=5) | ||
|
||
def __init__(self, num_workers, *unused_args, **unused_kwargs): | ||
"""Initializes a ramp-up throttler transform. | ||
Args: | ||
num_workers: A hint for the expected number of workers, used to derive | ||
the local rate limit. | ||
""" | ||
super(RampupThrottlingFn, self).__init__(*unused_args, **unused_kwargs) | ||
self._num_workers = num_workers | ||
self._successful_ops = util.MovingSum(window_ms=1000, bucket_ms=1000) | ||
self._first_instant = datetime.datetime.now() | ||
self._throttled_secs = Metrics.counter( | ||
RampupThrottlingFn, "cumulativeThrottlingSeconds") | ||
|
||
def _calc_max_ops_budget( | ||
self, | ||
first_instant: datetime.datetime, | ||
current_instant: datetime.datetime): | ||
"""Function that returns per-second budget according to best practices. | ||
The exact function is `500 / num_workers * 1.5^max(0, (x-5)/5)`, where x is | ||
the number of minutes since start time. | ||
""" | ||
timedelta_since_first = current_instant - first_instant | ||
growth = max( | ||
0.0, (timedelta_since_first - self._RAMP_UP_INTERVAL) / | ||
self._RAMP_UP_INTERVAL) | ||
max_ops_budget = int(self._BASE_BUDGET / self._num_workers * (1.5**growth)) | ||
return max(1, max_ops_budget) | ||
|
||
def process(self, element, **kwargs): | ||
backoff = iter( | ||
FuzzedExponentialIntervals(initial_delay_secs=1, num_retries=10000)) | ||
|
||
while True: | ||
instant = datetime.datetime.now() | ||
max_ops_budget = self._calc_max_ops_budget(self._first_instant, instant) | ||
current_op_count = self._successful_ops.sum(instant.timestamp() * 1000) | ||
available_ops = max_ops_budget - current_op_count | ||
|
||
if available_ops > 0: | ||
self._successful_ops.add(instant.timestamp() * 1000, 1) | ||
yield element | ||
break | ||
else: | ||
backoff_ms = next(backoff) | ||
_LOG.info('Delaying by %sms to conform to gradual ramp-up.', backoff_ms) | ||
time.sleep(backoff_ms) | ||
self._throttled_secs.inc(backoff_ms) |
62 changes: 62 additions & 0 deletions
62
sdks/python/apache_beam/io/gcp/datastore/v1new/rampup_throttling_fn_test.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,62 @@ | ||
# | ||
# 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. | ||
# | ||
|
||
import datetime | ||
import unittest | ||
|
||
from mock import patch | ||
|
||
from apache_beam.io.gcp.datastore.v1new.rampup_throttling_fn import RampupThrottlingFn | ||
|
||
DATE_ZERO = datetime.datetime( | ||
year=1970, month=1, day=1, tzinfo=datetime.timezone.utc) | ||
|
||
|
||
class _RampupDelayException(Exception): | ||
pass | ||
|
||
|
||
class RampupThrottlerTransformTest(unittest.TestCase): | ||
@patch('datetime.datetime') | ||
@patch('time.sleep') | ||
def test_rampup_throttling(self, mock_sleep, mock_datetime): | ||
mock_datetime.now.return_value = DATE_ZERO | ||
throttling_fn = RampupThrottlingFn(num_workers=1) | ||
rampup_schedule = [ | ||
(DATE_ZERO + datetime.timedelta(seconds=0), 500), | ||
(DATE_ZERO + datetime.timedelta(milliseconds=1), 0), | ||
(DATE_ZERO + datetime.timedelta(seconds=1), 500), | ||
(DATE_ZERO + datetime.timedelta(seconds=1, milliseconds=1), 0), | ||
(DATE_ZERO + datetime.timedelta(minutes=5), 500), | ||
(DATE_ZERO + datetime.timedelta(minutes=10), 750), | ||
(DATE_ZERO + datetime.timedelta(minutes=15), 1125), | ||
(DATE_ZERO + datetime.timedelta(minutes=30), 3796), | ||
(DATE_ZERO + datetime.timedelta(minutes=60), 43248), | ||
] | ||
|
||
mock_sleep.side_effect = _RampupDelayException() | ||
for date, expected_budget in rampup_schedule: | ||
mock_datetime.now.return_value = date | ||
for _ in range(expected_budget): | ||
next(throttling_fn.process(None)) | ||
# Delay after budget is exhausted | ||
with self.assertRaises(_RampupDelayException): | ||
next(throttling_fn.process(None)) | ||
|
||
|
||
if __name__ == '__main__': | ||
unittest.main() |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters