Hi Thomas,

IIUC this "launcher" should run on client endpoint instead
of dispatcher endpoint. "jar run" will extract the job graph
and submit it to the dispatcher, which has mismatched
semantic from your willing.

Could you run it with CliFrontend? Or propose that "jar run"
supports running directly the main method instead of extraction?

Best,
tison.


Thomas Weise <t...@apache.org> 于2019年7月26日周五 下午11:38写道:

> Hi Till,
>
> Thanks for taking a look!
>
> The Beam job server does not currently have the ability to just output the
> job graph (and related artifacts) that could then be used with the
> JobSubmitHandler. It is itself using StreamExecutionEnvironment, which in
> turn will lead to a REST API submission.
>
> Here I'm looking at what happens before the Beam job server gets involved:
> the interaction of the k8s operator with the Flink deployment. The jar run
> endpoint (ignoring the current handler implementation) is generic and
> pretty much exactly matches what we would need for a uniform entry point.
> It's just that in the Beam case the jar file would itself be a "launcher"
> that doesn't provide the job graph itself, but the dependencies and
> mechanism to invoke the actual client.
>
> I could accomplish what I'm looking for by creating a separate REST
> endpoint that looks almost the same. But I would prefer to reuse the Flink
> REST API interaction that is already implemented for the Flink Java jobs to
> reduce the complexity of the deployment.
>
> Thomas
>
>
>
>
> On Fri, Jul 26, 2019 at 2:29 AM Till Rohrmann <trohrm...@apache.org>
> wrote:
>
> > Hi Thomas,
> >
> > quick question: Why do you wanna use the JarRunHandler? If another
> process
> > is building the JobGraph, then one could use the JobSubmitHandler which
> > expects a JobGraph and then starts executing it.
> >
> > Cheers,
> > Till
> >
> > On Thu, Jul 25, 2019 at 7:45 PM Thomas Weise <t...@apache.org> wrote:
> >
> > > Hi,
> > >
> > > While considering different options to launch Beam jobs through the
> Flink
> > > REST API, I noticed that the implementation of JarRunHandler places
> > quite a
> > > few restrictions on how the entry point shall construct a Flink job, by
> > > extracting and manipulating the job graph.
> > >
> > > That's normally not a problem for Flink Java programs, but in the
> > scenario
> > > I'm looking at, the job graph would be constructed by a different
> process
> > > and isn't available to the REST handler. Instead, I would like to be
> able
> > > to just respond with the job ID of the already launched job.
> > >
> > > For context, please see:
> > >
> > >
> > >
> >
> https://docs.google.com/document/d/1z3LNrRtr8kkiFHonZ5JJM_L4NWNBBNcqRc_yAf6G0VI/edit#heading=h.fh2f571kms4d
> > >
> > > The current JarRunHandler code is here:
> > >
> > >
> > >
> >
> https://github.com/apache/flink/blob/f3c5dd960ff81a022ece2391ed3aee86080a352a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java#L82
> > >
> > > It would be nice if there was an option to delegate the responsibility
> > for
> > > job submission to the user code / entry point. That would be useful for
> > > Beam and other frameworks built on top of Flink that dynamically
> create a
> > > job graph from a different representation.
> > >
> > > Possible ways to get there:
> > >
> > > * an interface that the main class can be implement end when present,
> the
> > > jar run handler calls instead of main.
> > >
> > > * an annotated method
> > >
> > > Either way query parameters like savepoint path and parallelism would
> be
> > > forwarded to the user code and the result would be the ID of the
> launched
> > > job.
> > >
> > > Thougths?
> > >
> > > Thanks,
> > > Thomas
> > >
> >
>

Reply via email to