Conversation
…vice), add cli option for using proxy
mrocklin
left a comment
There was a problem hiding this comment.
but I left some code in because I was somewhat undecided on where proxying should occur. At first, I built a new proxying service but soon realized that the full service was unnecessary and could just run as another route in on the scheduler's dashboard. If reviewers are ok with running the proxying as another route I can remove all of distributed/proxy/core.py and the Proxy object in proxy_html.py.
+1 on running this as an additional route inside of our existing Tornado HTTP server
|
Also thanks for doing this! I'm excited about getting access to these dashboards. I also think that once we have them more publicly available we'll probably start investing more time in adding to them. |
|
cc @jcrist who might find this interesting as well |
| <td> {{ len(ws.has_what) }} </td> | ||
| {% if 'bokeh' in ws.services %} | ||
| <td> <a href="http://{{ ws.host }}:{{ ws.services['bokeh'] }}">bokeh</a> </td> | ||
| <td> <a href="http://{{ ws.host }}:{{ scheduler.services['bokeh'].port }}/proxy/{{ ws.services['bokeh'] }}/{{ ws.host }}/status">bokeh</a> </td> |
There was a problem hiding this comment.
@mrocklin This is the new url: /proxy/{port}/{host}/{proxied_path}.
There was a problem hiding this comment.
Oh wow, how did you manage that?
There was a problem hiding this comment.
@mrocklin I realized that I could manually overwrite the request uri (something I always had trouble with using ngxin) and a bit of regex in the handler made things smooth for having nicer URLs
Ah. Great!
There was a problem hiding this comment.
This is cool!
Is ws.host coming from a trusted source? If it is user controlled, it could be used maliciously to bypass CORS / Origin checking.
There was a problem hiding this comment.
This is the address that a Dask worker advertises as its contact address. A user that has access to launch a Dask worker and connect it to this scheduler can manipulate this value, yes.
However, such a user already has the ability to connec to the scheduler and run arbitrary Python code there, or on any other worker-connected machine, so I suspect that at this point we're ok with this.
There was a problem hiding this comment.
With the introduction of scheduler here we need to include that all uses of this template also include the scheduler= keyword,
class Worker(RequestHandler):
def get(self, worker):
worker = escape.url_unescape(worker)
with log_errors():
self.render(
"worker.html",
title="Worker: " + worker,
Worker=worker,
+ scheduler=self.server,
**toolz.merge(self.server.__dict__, ns, self.extra)
)There was a problem hiding this comment.
This is something that should have shown up on tests ideally. If you felt the desire, it might be nice to add this to some test that checks the info pages (assuming that something like this already exists)
There was a problem hiding this comment.
In my last commit, I fixed the worker and task handlers. The test were more or less already written in test_scheduler_bokeh_html.py . The tests previously passed because the workers did not setup bokeh dashboards. Now they do and tests pass again with the suggested addition above
|
@mrocklin I realized that I could manually overwrite the request uri (something I always had trouble with using ngxin) and a bit of regex in the handler made things smooth for having nicer URLs |
| assert response_proxy.code == 200 | ||
| assert b"Crossfilter" in response_proxy.body | ||
| assert response_direct.code == 200 | ||
| assert b"Crossfilter" in response_direct.body |
|
Thanks for the review @mrocklin I think I've address all of your comments. Another thought occurring to me as I've just added For context, notebook is ~7MBs |
Then maybe we should make this optional. Presumably we would try-except around the internal import of our own module when adding the route. |
Or maybe our proxy handler sends back a "Oops, couldn't proxy, please |
… handler when jupyter-server-proxy is not installed
|
ProxyHandler also inherits from IPythonHandler, which probably has implications around authentication - IIRC, we currently require / depend on notebook authentication to allow access. |
|
@yuvipanda that's a good point! I think, though could be wrong, that as long as dask internals don't implement a LoginHandler/self.login_handler we should be ok. Fortunately for us the, the |
… to include bokeh for workers to properly test routes
We would have needed the scheduler's bokeh address instead, but relative is probably safer regardless? It's also the convention used within this template, so lets stick to that for conformity and future reviewers..
|
@quasiben is it ok to rename proxy_html.py to proxy.py ? |
|
👍 |
| <td> {{ len(ws.has_what) }} </td> | ||
| {% if 'bokeh' in ws.services %} | ||
| <td> <a href="http://{{ ws.host }}:{{ scheduler.services['bokeh'].port }}/proxy/{{ ws.services['bokeh'] }}/{{ ws.host }}/status">bokeh</a> </td> | ||
| <td> <a href="../../proxy/{{ ws.services['bokeh'] }}/{{ ws.host }}/status">bokeh</a> </td> |
|
OK, done. I plan to merge this after tests run. I'll rename bokeh to dashboard shortly after |
|
thanks for the edits to |
|
My pleasure. Dashboard work is particularly satisfying. |
|
This is in. Thanks @quasiben ! |
|
<3 great to see jupyter-server-proxy get used this way! \o/ |
PR resolves #2160
This PR is largely done but I left some code in because I was somewhat undecided on where proxying should occur. At first, I built a new proxying service but soon realized that the full service was unnecessary and could just run as another route in on the scheduler's dashboard. If reviewers are ok with running the proxying as another route I can remove all of distributed/proxy/core.py and the
Proxyobject in proxy_html.py.How it Works:
@yuvipanda if you have time, this would work would greatly benefit from a review from you