Hi guys,

So far, there is a PR[1] that implements the proposal in this thread.

I look forward to your reviews or start a vote if required.

Best,
tison.

[1] https://github.com/apache/flink/pull/10526


tison <wander4...@gmail.com> 于2020年3月6日周五 下午4:11写道:

> Hi,
>
> Thanks for your rapid replies!
>
> Correctly when the dependencies to flink-client & flink-runtime broken
> we can make flink-streaming-java scala-free. And even part of the goal,
> i.e., that to flink-client reversed we can directly compile streaming
> pipeline in flink-client instead of relying on reflection.
>
> For the impact to user, I agree with Stephan that we should spell it out
> prominently in release note.
>
> For exiting build setting when user bump their flink version and see
> ClassNotFoundException he need to add a dependency to
> flink-client explicitly. Since we effectively remove the dependency
> from flink-streaming-java to flink-client it will only happen when user
> directly depends on flink-client. So far, it is not a common case. And
> since
> it is not Flink who uses flink-client and cause CNFE, it might be hard
> we inject error message because user code directly cause CNFE.
>
> If we agree on the motivation, we can move forward to the corresponding
> JIRAs[1][2].
>
> Best,
> tison.
>
> [1] https://jira.apache.org/jira/browse/FLINK-15090
> [2] https://jira.apache.org/jira/browse/FLINK-16427
>
>
>
> Hequn Cheng <he...@apache.org> 于2020年3月6日周五 下午3:06写道:
>
>> Hi,
>>
>> +1 to make flink-streaming-java an API only module and solve it sooner
>> rather than later.
>> It would be more clear to only expose an SDK module for writing jobs.
>>
>> Another benefit I can see is: the flink-streaming-java would be scala-free
>> if we reverse the dependencies and this would be really nice for the Java
>> API module.
>>
>> As for the issue of dependencies setup of users, I agree with Stephan that
>> it's ok to do so
>> if we add corresponding document and runtime error messages about the
>> changes.
>>
>> Best,
>> Hequn
>>
>>
>> On Fri, Mar 6, 2020 at 3:03 AM Kostas Kloudas <kklou...@gmail.com> wrote:
>>
>> > Big +1 also from my side.
>> >
>> > This will eliminate some work-arounds used so far to bypass the module
>> > structure (like code using reflection to extract a JobGraph from a
>> > Pipeline).
>> >
>> > I agree with Stephan that with proper documentation, release notes and
>> > tooling update, it will hopefully not be a big hassle for users to
>> > migrate.
>> > Also I think it should be done as early in the release as possible, so
>> > that we can give it enough exposure and testing. In the past, such
>> > deep changes late in the release have led to longer release-testing
>> > periods and, eventually, longer release cycles.
>> >
>> > Cheers,
>> > Kostas
>> >
>> > On Thu, Mar 5, 2020 at 3:35 PM Stephan Ewen <se...@apache.org> wrote:
>> > >
>> > > +1 to this fix, in general.
>> > >
>> > > If the main issue is that users have to now add "flink-clients"
>> > explicitly,
>> > > then I think this is okay, if we spell it out prominently in the
>> release
>> > > notes, and make sure quickstarts / etc are updated, and have a good
>> error
>> > > message when client/runtime classes are not found.
>> > >
>> > > On Thu, Mar 5, 2020 at 2:56 PM Aljoscha Krettek <aljos...@apache.org>
>> > wrote:
>> > >
>> > > > Hi,
>> > > >
>> > > > thanks for starting the discussion, Tison!
>> > > >
>> > > > I'd like to fix this dependency mess rather sooner than later, but
>> we
>> > do
>> > > > have to consider the fact that we are breaking the dependency setup
>> of
>> > > > users. If they they only had a dependency on flink-streaming-java
>> > before
>> > > > but used classes from flink-clients they would have to explicitly
>> add
>> > > > this dependency now.
>> > > >
>> > > > Let's see what others think.
>> > > >
>> > > > Best,
>> > > > Aljoscha
>> > > >
>> > > > On 05.03.20 02:53, tison wrote:
>> > > > > Hi devs,
>> > > > >
>> > > > > Here is a proposal to reverse the dependency from
>> > flink-streaming-java to
>> > > > > flink-client, for a proper
>> > > > > module dependency graph. Since it changes current structure, it
>> > should be
>> > > > > discussed publicly.
>> > > > >
>> > > > > The original idea comes from that flink-streaming-java acts as an
>> API
>> > > > only
>> > > > > module just as what
>> > > > > we do in its batch companion flink-java. If a Flink user want to
>> > write a
>> > > > > minimum DataStream
>> > > > > program, the only dependency should be flink-streaming java.
>> > > > >
>> > > > > However, currently as it is implemented, flink-client and even
>> > > > > flink-runtime are transitively polluted
>> > > > > in when user depends on flink-streaming-java. These dependencies
>> > polluted
>> > > > > in as
>> > > > >
>> > > > > flink-client:
>> > > > >    - previously, ClusterClient, which is removed by FLIP-73
>> Executors
>> > > > >    - accidentally, ProgramInvocationException, we just throw in
>> > place as
>> > > > it
>> > > > > is accessible.
>> > > > >    - transitively, flink-optimizer, for one utility.
>> > > > >    - transitively, flink-java, for several utilities.
>> > > > > flink-runtime:
>> > > > >    - mainly for JobGraph generating.
>> > > > >
>> > > > > With a previous discussion with @Aljoscha Krettek <
>> > aljos...@apache.org>
>> > > > our
>> > > > > goal is briefly making flink-streaming-java
>> > > > > an API only module. As a first step we can break the dependency
>> from
>> > > > > flink-streaming-java to
>> > > > > flink-client[1][2].
>> > > > >
>> > > > > With this first step, continuously we factor out common utilities
>> in
>> > > > > flink-java to
>> > > > > flink-core and eventually eliminate dependencies from streaming to
>> > batch;
>> > > > > while
>> > > > > orthogonally, we factor out job compilation logic into
>> > > > > flink-streaming-compiler module and
>> > > > > break the dependency to flink-runtime. The final dependency graph
>> > will
>> > > > be:
>> > > > >
>> > > > >
>> > > > > flink-client -> flink-streaming-compiler -> flink-runtime
>> > > > >                                                           \->
>> > > > > flink-streaming-java
>> > > > >
>> > > > > Looking forward to your feedback. Basically whether or not it is
>> in a
>> > > > right
>> > > > > direction, and if so,
>> > > > > how the community integrates this proposal.
>> > > > >
>> > > > > Best,
>> > > > > tison.
>> > > > >
>> > > > > [1] https://issues.apache.org/jira/browse/FLINK-15090
>> > > > > [2] https://issues.apache.org/jira/browse/FLINK-16427
>> > > > >
>> > > >
>> >
>>
>

Reply via email to