[
https://issues.apache.org/jira/browse/BEAM-8372?focusedWorklogId=334545&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-334545
]
ASF GitHub Bot logged work on BEAM-8372:
----------------------------------------
Author: ASF GitHub Bot
Created on: 26/Oct/19 16:09
Start Date: 26/Oct/19 16:09
Worklog Time Spent: 10m
Work Description: robertwb commented on issue #9844: [BEAM-8372] Support
both flink_master and flink_master_url parameter
URL: https://github.com/apache/beam/pull/9844#issuecomment-546616164
If `flink_master` is the option, and has been for a long time, then we
should just use that.
As for adapting #9775, REST does require specification of a protocol.
Currently if we just pass `--runner=FlinkRunner --flink_master=localhost:8081`
we get
```
File
"/Users/robertwb/Work/beam/incubator-beam/sdks/python/apache_beam/runners/portability/portable_runner.py",
line 315, in run_pipeline
timeout=portable_options.job_server_timeout)
File
"/Users/robertwb/Work/beam/incubator-beam/sdks/python/apache_beam/runners/portability/abstract_job_service.py",
line 53, in Prepare
request.pipeline_options)
File
"/Users/robertwb/Work/beam/incubator-beam/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py",
line 77, in create_beam_job
self.executable_jar(),
File
"/Users/robertwb/Work/beam/incubator-beam/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py",
line 66, in executable_jar
'runners:flink:%s:job-server:shadowJar' % self.flink_version()))
File
"/Users/robertwb/Work/beam/incubator-beam/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py",
line 70, in flink_version
'%s/v1/config' % self._master_url).json()['flink-version']
File
"/Users/robertwb/Work/beam/venv3/lib/python3.6/site-packages/requests-2.22.0-py3.6.egg/requests/api.py",
line 75, in get
return request('get', url, params=params, **kwargs)
File
"/Users/robertwb/Work/beam/venv3/lib/python3.6/site-packages/requests-2.22.0-py3.6.egg/requests/api.py",
line 60, in request
return session.request(method=method, url=url, **kwargs)
File
"/Users/robertwb/Work/beam/venv3/lib/python3.6/site-packages/requests-2.22.0-py3.6.egg/requests/sessions.py",
line 533, in request
resp = self.send(prep, **send_kwargs)
File
"/Users/robertwb/Work/beam/venv3/lib/python3.6/site-packages/requests-2.22.0-py3.6.egg/requests/sessions.py",
line 640, in send
adapter = self.get_adapter(url=request.url)
File
"/Users/robertwb/Work/beam/venv3/lib/python3.6/site-packages/requests-2.22.0-py3.6.egg/requests/sessions.py",
line 731, in get_adapter
raise InvalidSchema("No connection adapters were found for '%s'" % url)
requests.exceptions.InvalidSchema: No connection adapters were found for
'localhost:8081/v1/config'
```
because we're using requests to attempt to communicate with the flink
master. (Note that in this case there is no `FlinkJobServerDriver`, etc.) We
could automatically append `http` or `https` if we could figure out which one,
but it seems strange design to have to read a config file when the address is
passed explicitly.
Taking a step back, here's what I think the ideal experience should be. If
one passes just `--runner=FlinkRunner` then it should start up a local runner
and run against that, automatically enabling `LOOPBACK` mode if one hasn't been
explicitly chosen for ease of use. If one passes `--runner=FlinkRunner
--flink_master=address` it would submit the job to the cluster, without having
to even download jars and invoke java. It's unclear how `[auto]` would get this
behavior (or indeed what the behavior of `[auto]` even is).
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 334545)
Time Spent: 7h 50m (was: 7h 40m)
> Allow submission of Flink UberJar directly to flink cluster.
> ------------------------------------------------------------
>
> Key: BEAM-8372
> URL: https://issues.apache.org/jira/browse/BEAM-8372
> Project: Beam
> Issue Type: New Feature
> Components: sdk-py-core
> Reporter: Robert Bradshaw
> Assignee: Robert Bradshaw
> Priority: Major
> Time Spent: 7h 50m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)