+1 for creating a channel.
Kostas
On Wed, Sep 11, 2019 at 10:57 AM Zili Chen wrote:
>
> Hi Aljoscha,
>
> I'm OK to use the ASF slack.
>
> Best,
> tison.
>
>
> Jeff Zhang 于2019年9月11日周三 下午4:48写道:
>>
>> +1 for using slack for instant communication
>>
>> Aljoscha Krettek 于2019年9月11日周三 下午4:44写道:
Hi Aljoscha,
I'm OK to use the ASF slack.
Best,
tison.
Jeff Zhang 于2019年9月11日周三 下午4:48写道:
> +1 for using slack for instant communication
>
> Aljoscha Krettek 于2019年9月11日周三 下午4:44写道:
>
>> Hi,
>>
>> We could try and use the ASF slack for this purpose, that would probably
>> be easiest. See
+1 for using slack for instant communication
Aljoscha Krettek 于2019年9月11日周三 下午4:44写道:
> Hi,
>
> We could try and use the ASF slack for this purpose, that would probably
> be easiest. See https://s.apache.org/slack-invite. We could create a
> dedicated channel for our work and would still use
Hi,
We could try and use the ASF slack for this purpose, that would probably be
easiest. See https://s.apache.org/slack-invite. We could create a dedicated
channel for our work and would still use the open ASF infrastructure and people
can have a look if they are interested because discussion
Here is the issue, and I will keep on updating it as I find more issues.
https://issues.apache.org/jira/browse/FLINK-13954
This will also cover the refactoring of the Executors that we discussed
in this thread, without any additional functionality (such as the job client).
Kostas
On Wed, Sep
Great idea Tison!
I will create the umbrella issue and post it here so that we are all
on the same page!
Cheers,
Kostas
On Wed, Sep 4, 2019 at 11:36 AM Zili Chen wrote:
>
> Hi Kostas & Aljoscha,
>
> I notice that there is a JIRA(FLINK-13946) which could be included
> in this refactor thread.
Hi Kostas & Aljoscha,
I notice that there is a JIRA(FLINK-13946) which could be included
in this refactor thread. Since we agree on most of directions in
big picture, is it reasonable that we create an umbrella issue for
refactor client APIs and also linked subtasks? It would be a better
way that
Great Kostas! Looking forward to your POC!
Best,
tison.
Jeff Zhang 于2019年8月30日周五 下午11:07写道:
> Awesome, @Kostas Looking forward your POC.
>
> Kostas Kloudas 于2019年8月30日周五 下午8:33写道:
>
> > Hi all,
> >
> > I am just writing here to let you know that I am working on a POC that
> > tries to
Awesome, @Kostas Looking forward your POC.
Kostas Kloudas 于2019年8月30日周五 下午8:33写道:
> Hi all,
>
> I am just writing here to let you know that I am working on a POC that
> tries to refactor the current state of job submission in Flink.
> I want to stress out that it introduces NO CHANGES to the
Hi all,
I am just writing here to let you know that I am working on a POC that
tries to refactor the current state of job submission in Flink.
I want to stress out that it introduces NO CHANGES to the current
behaviour of Flink. It just re-arranges things and introduces the
notion of an Executor,
Hi Zili,
It make sense to me that a dedicated cluster is started for a per-job
cluster and will not accept more jobs.
Just have a question about the command line.
Currently we could use the following commands to start different clusters.
*per-job cluster*
./bin/flink run -d -p 5 -ynm
Hi Till,
Thanks for your update. Nice to hear :-)
Best,
tison.
Till Rohrmann 于2019年8月23日周五 下午10:39写道:
> Hi Tison,
>
> just a quick comment concerning the class loading issues when using the per
> job mode. The community wants to change it so that the
> StandaloneJobClusterEntryPoint actually
Hi Tison,
just a quick comment concerning the class loading issues when using the per
job mode. The community wants to change it so that the
StandaloneJobClusterEntryPoint actually uses the user code class loader
with child first class loading [1]. Hence, I hope that this problem will be
resolved
Hi all,
On the topic of web submission, I agree with Till that it only seems
to complicate things.
It is bad for security, job isolation (anybody can submit/cancel jobs), and its
implementation complicates some parts of the code. So, if it were to
redesign the
WebUI, maybe this part could be left
Hi Yang,
It would be helpful if you check Stephan's last comment,
which states that isolation is important.
For per-job mode, we run a dedicated cluster(maybe it
should have been a couple of JM and TMs during FLIP-6
design) for a specific job. Thus the process is prevented
from other jobs.
In
>From the user's perspective, it is really confused about the scope of
per-job cluster.
If it means a flink cluster with single job, so that we could get better
isolation.
Now it does not matter how we deploy the cluster, directly deploy(mode1)
or start a flink cluster and then submit job
Thanks for the clarification.
The idea JobDeployer ever came into my mind when I was muddled with
how to execute per-job mode and session mode with the same user code
and framework codepath.
With the concept JobDeployer we back to the statement that environment
knows every configs of cluster
Till has made some good comments here.
Two things to add:
- The job mode is very nice in the way that it runs the client inside the
cluster (in the same image/process that is the JM) and thus unifies both
applications and what the Spark world calls the "driver mode".
- Another thing I would
I would not be in favour of getting rid of the per-job mode since it
simplifies the process of running Flink jobs considerably. Moreover, it is
not only well suited for container deployments but also for deployments
where you want to guarantee job isolation. For example, a user could use
the
In my opinion the client should not use any environment to get the Job
graph because the jar should reside ONLY on the cluster (and not in the
client classpath otherwise there are always inconsistencies between client
and Flink Job manager's classpath).
In the YARN, Mesos and Kubernetes scenarios
I would like to involve Till & Stephan here to clarify some concept of
per-job mode.
The term per-job is one of modes a cluster could run on. It is mainly aimed
at spawn
a dedicated cluster for a specific job while the job could be packaged with
Flink
itself and thus the cluster initialized with
Hi Aljoscha,
Thanks for your reply and participance. The Google Doc you linked to
requires
permission and I think you could use a share link instead.
I agree with that we almost reach a consensus that JobClient is necessary to
interacte with a running Job.
Let me check your open questions one
Hi,
I read both Jeffs initial design document and the newer document by Tison. I
also finally found the time to collect our thoughts on the issue, I had quite
some discussions with Kostas and this is the result: [1].
I think overall we agree that this part of the code is in dire need of some
Thanks tison for the effort. I left a few comments.
Zili Chen 于2019年7月31日周三 下午8:24写道:
> Hi Flavio,
>
> Thanks for your reply.
>
> Either current impl and in the design, ClusterClient
> never takes responsibility for generating JobGraph.
> (what you see in current codebase is several class
Hi Flavio,
Thanks for your reply.
Either current impl and in the design, ClusterClient
never takes responsibility for generating JobGraph.
(what you see in current codebase is several class methods)
Instead, user describes his program in the main method
with ExecutionEnvironment apis and calls
Just one note on my side: it is not clear to me whether the client needs to
be able to generate a job graph or not.
In my opinion, the job jar must resides only on the server/jobManager side
and the client requires a way to get the job graph.
If you really want to access to the job graph, I'd add
Hi all,
Here is a document[1] on client api enhancement from our perspective.
We have investigated current implementations. And we propose
1. Unify the implementation of cluster deployment and job submission in
Flink.
2. Provide programmatic interfaces to allow flexible job and cluster
Thanks Stephan, I will follow up this issue in next few weeks, and will
refine the design doc. We could discuss more details after 1.9 release.
Stephan Ewen 于2019年7月24日周三 上午12:58写道:
> Hi all!
>
> This thread has stalled for a bit, which I assume ist mostly due to the
> Flink 1.9 feature freeze
Hi Stephan,
Thanks for waking up this thread.
Jeff and I had a discussion yesterday sharing our respective
observations and ideas of client api enhancements. We are
glad to make some progress in Flink 1.10.
It's really nice to hear that you're gonna participate this
thread soon. In ML threads
Hi all!
This thread has stalled for a bit, which I assume ist mostly due to the
Flink 1.9 feature freeze and release testing effort.
I personally still recognize this issue as one important to be solved. I'd
be happy to help resume this discussion soon (after the 1.9 release) and
see if we can
That's exactly what I suggested a long time ago: the Flink REST client
should not require any Flink dependency, only http library to call the REST
services to submit and monitor a job.
What I suggested also in [1] was to have a way to automatically suggest the
user (via a UI) the available main
Hi, Tison,
Thanks for your comments. Overall I agree with you that it is difficult for
down stream project to integrate with flink and we need to refactor the
current flink client api.
And I agree that CliFrontend should only parsing command line arguments and
then pass them to
Hi all,
After a closer look on our client apis, I can see there are two major
issues to consistency and integration, namely different deployment of
job cluster which couples job graph creation and cluster deployment,
and submission via CliFrontend confusing control flow of job graph
compilation
Hi Jeff,
Thanks for raising this thread and the design document!
As @Thomas Weise mentioned above, extending config to flink
requires far more effort than it should be. Another example
is we achieve detach mode by introduce another execution
environment which also hijack #execute method.
I
Hi Jeff and Flavio,
Thanks Jeff a lot for proposing the design document.
We are also working on refactoring ClusterClient to allow flexible and
efficient job management in our real-time platform.
We would like to draft a document to share our ideas with you.
I think it's a good idea to have
Is there any possibility to have something like Apache Livy [1] also for
Flink in the future?
[1] https://livy.apache.org/
On Tue, Jun 11, 2019 at 5:23 PM Jeff Zhang wrote:
> >>> Any API we expose should not have dependencies on the runtime
> (flink-runtime) package or other implementation
>>> Any API we expose should not have dependencies on the runtime
(flink-runtime) package or other implementation details. To me, this means
that the current ClusterClient cannot be exposed to users because it uses
quite some classes from the optimiser and runtime packages.
We should change
Some points to consider:
* Any API we expose should not have dependencies on the runtime (flink-runtime)
package or other implementation details. To me, this means that the current
ClusterClient cannot be exposed to users because it uses quite some classes
from the optimiser and runtime
Sorry folks, the design doc is late as you expected. Here's the design doc
I drafted, welcome any comments and feedback.
https://docs.google.com/document/d/1VavBrYn8vJeZs-Mhu5VzKO6xrWCF40aY0nlQ_UVVTRg/edit?usp=sharing
Stephan Ewen 于2019年2月14日周四 下午8:43写道:
> Nice that this discussion is
Nice that this discussion is happening.
In the FLIP, we could also revisit the entire role of the environments
again.
Initially, the idea was:
- the environments take care of the specific setup for standalone (no
setup needed), yarn, mesos, etc.
- the session ones have control over the
Hi folks,
Sorry for late response, It seems we reach consensus on this, I will create
FLIP for this with more detailed design
Thomas Weise 于2018年12月21日周五 上午11:43写道:
> Great to see this discussion seeded! The problems you face with the
> Zeppelin integration are also affecting other downstream
Great to see this discussion seeded! The problems you face with the
Zeppelin integration are also affecting other downstream projects, like
Beam.
We just enabled the savepoint restore option in RemoteStreamEnvironment [1]
and that was more difficult than it should be. The main issue is that
>>> I'm not so sure whether the user should be able to define where the job
runs (in your example Yarn). This is actually independent of the job
development and is something which is decided at deployment time.
User don't need to specify execution mode programmatically. They can also
pass the
You are probably right that we have code duplication when it comes to the
creation of the ClusterClient. This should be reduced in the future.
I'm not so sure whether the user should be able to define where the job
runs (in your example Yarn). This is actually independent of the job
development
Hi Till,
Thanks for the feedback. You are right that I expect better programmatic
job submission/control api which could be used by downstream project. And
it would benefit for the flink ecosystem. When I look at the code of flink
scala-shell and sql-client (I believe they are not the core of
Hi Jeff,
what you are proposing is to provide the user with better programmatic job
control. There was actually an effort to achieve this but it has never been
completed [1]. However, there are some improvement in the code base now.
Look for example at the NewClusterClient interface which offers
Hi Folks,
I am trying to integrate flink into apache zeppelin which is an interactive
notebook. And I hit several issues that is caused by flink client api. So
I'd like to proposal the following changes for flink client api.
1. Support nonblocking execution. Currently,
47 matches
Mail list logo