-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-3418] Send worker_id in all grpc channels to runner harness #4587
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,59 @@ | ||
| # | ||
| # 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. | ||
| # | ||
| """Client Interceptor to inject worker_id""" | ||
| from __future__ import absolute_import | ||
| from __future__ import division | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we need
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No, we don't need these imports. Should I remove them?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's remove them, if they are not needed now. I do not see print() or / being used here.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sure! |
||
| from __future__ import print_function | ||
|
|
||
| import collections | ||
| import os | ||
| import uuid | ||
|
|
||
| import grpc | ||
|
|
||
|
|
||
| class _ClientCallDetails( | ||
| collections.namedtuple('_ClientCallDetails', | ||
| ('method', 'timeout', 'metadata', 'credentials')), | ||
| grpc.ClientCallDetails): | ||
| pass | ||
|
|
||
|
|
||
| class WorkerIdInterceptor(grpc.StreamStreamClientInterceptor): | ||
|
|
||
| # TODO: (BEAM-3904) Removed defaulting to UUID when worker_id is not present | ||
| # and throw exception in worker_id_interceptor.py after we have rolled out | ||
| # the corresponding container changes. | ||
| # Unique worker Id for this worker. | ||
| _worker_id = os.environ['WORKER_ID'] if os.environ.has_key( | ||
| 'WORKER_ID') else str(uuid.uuid4()) | ||
|
|
||
| def __init__(self): | ||
| pass | ||
|
|
||
| def intercept_stream_stream(self, continuation, client_call_details, | ||
| request_iterator): | ||
| metadata = [] | ||
| if client_call_details.metadata is not None: | ||
| metadata = list(client_call_details.metadata) | ||
| if 'worker_id' in metadata: | ||
| raise RuntimeError('Header metadata alreay have worker_id.') | ||
| metadata.append(('worker_id', self._worker_id)) | ||
| new_client_details = _ClientCallDetails( | ||
| client_call_details.method, client_call_details.timeout, metadata, | ||
| client_call_details.credentials) | ||
| return continuation(new_client_details, request_iterator) | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,75 @@ | ||
| # | ||
| # 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. | ||
| # | ||
| """Test for WorkerIdInterceptor""" | ||
| from __future__ import absolute_import | ||
|
|
||
| import collections | ||
| import logging | ||
| import unittest | ||
|
|
||
| import grpc | ||
|
|
||
| from apache_beam.runners.worker.worker_id_interceptor import WorkerIdInterceptor | ||
|
|
||
|
|
||
| class _ClientCallDetails( | ||
| collections.namedtuple('_ClientCallDetails', | ||
| ('method', 'timeout', 'metadata', 'credentials')), | ||
| grpc.ClientCallDetails): | ||
| pass | ||
|
|
||
|
|
||
| class WorkerIdInterceptorTest(unittest.TestCase): | ||
|
|
||
| def test_worker_id_insertion(self): | ||
| worker_id_key = 'worker_id' | ||
| headers_holder = {} | ||
|
|
||
| def continuation(client_details, request_iterator): | ||
| headers_holder.update({ | ||
| worker_id_key: dict(client_details.metadata).get(worker_id_key) | ||
| }) | ||
|
|
||
| WorkerIdInterceptor._worker_id = 'my_worker_id' | ||
|
|
||
| WorkerIdInterceptor().intercept_stream_stream(continuation, | ||
| _ClientCallDetails( | ||
| None, None, None, None), | ||
| []) | ||
| self.assertEqual(headers_holder[worker_id_key], 'my_worker_id', | ||
| 'worker_id_key not set') | ||
|
|
||
| def test_failure_when_worker_id_exists(self): | ||
| worker_id_key = 'worker_id' | ||
| headers_holder = {} | ||
|
|
||
| def continuation(client_details, request_iterator): | ||
| headers_holder.update({ | ||
| worker_id_key: dict(client_details.metadata).get(worker_id_key) | ||
| }) | ||
|
|
||
| WorkerIdInterceptor._worker_id = 'my_worker_id' | ||
|
|
||
| with self.assertRaises(RuntimeError): | ||
| WorkerIdInterceptor().intercept_stream_stream( | ||
| continuation, _ClientCallDetails(None, None, {'worker_id': '1'}, | ||
| None), []) | ||
|
|
||
|
|
||
| if __name__ == '__main__': | ||
| logging.getLogger().setLevel(logging.INFO) | ||
| unittest.main() |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -99,7 +99,7 @@ def get_version(): | |
| 'avro>=1.8.1,<2.0.0', | ||
| 'crcmod>=1.7,<2.0', | ||
| 'dill==0.2.6', | ||
| 'grpcio>=1.0,<2', | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What is the reason for this change? AFAIK, some runner (Dataflow) requires grpcio 1.3? Also, it is became more restrictive for users each time we reduce the list of allowed versions for a dependency.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. grpcio 1.8 is the first python version which allows passing/receiving client headers
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We use header to send worker_id in all channels and GRPC headers are only supported after 1.8.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sounds good. Just try running dataflow runner test before finishing this PR.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sure, Tried running apache_beam.runners.dataflow.dataflow_runner_test.DataflowRunnerTest
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I meant running a job on dataflow using: https://github.com/apache/beam/blob/master/.test-infra/jenkins/job_beam_PostCommit_Python_Verify.groovy Commenting "Run Python PostCommit" on the PR should trigger one. |
||
| 'grpcio>=1.8,<2', | ||
| 'hdfs>=2.1.0,<3.0.0', | ||
| 'httplib2>=0.8,<0.10', | ||
| 'mock>=1.0.1,<3.0.0', | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not Simplifying to keep readability.