Re: SourceEvent Task Failure Handling

2022-03-02 Thread Qingsheng Ren
Hi Mason, 

Unfortunately from SplitEnumerator’s aspect, events will be lost if they are 
not delivered to the SourceReader. 

On OperatorCoordinator level, if an OperatorEvent is not delivered 
successfully, the coordinator will trigger a subtask failover, then 
OperatorCoordinator#subtaskReset will be invoked so that coordinator could be 
able to handle it. However, SourceCoordinator only handles splits on subtask 
reset (via addSplitsBack) so state of split assignment will be consistent, but 
other custom events will be lost :-( I think a FLIP is needed to resolve this 
because an interface change on SplitEnumerator is required to handle custom 
event loss. 

Best regards, 

Qingsheng Ren


> On Feb 26, 2022, at 4:50 AM, Mason Chen  wrote:
> 
> Hi Devs,
> 
> I am implementing a custom connector with the FLIP27 interface and I have a
> question about how source events are handled with respect to
> task/taskmanager failure. In my implementation, the enumerator sends source
> events to readers so that readers can react to split changes detected by
> the enumerator.
> 
> How are source events handled in the cases of task failure/task manager
> failure? Are they completely lost or is there an internal mechanism to
> re-apply source events upon reader/task/taskmanager restart?
> 
> I'm thinking I would need to keep track of processed source events in
> enumerator state and resend them if they weren't processed before
> reader/task/taskmanager failure. I'd like to avoid the extra work of this
> mechanism and it sounds very similar to the addSplitsBack functionality, so
> I'm looking for suggestions for what else I might be able to leverage.
> 
> Best,
> Mason



Re: Re: [VOTE] Release 1.14.4, release candidate #1

2022-03-02 Thread Yun Gao
+1 (binding)

- Checked the signatures of the release artifacts
- Checked the WordCount example run successfully with the binary distributions 
for both of scala 2.11/2.12
- Verified the source compiled to binary successfully (withouth running the 
tests)
- Reviewed the flink-web PR
- Reviewed the staged artifacts randomly for some modules.

Best,
Yun Gao


 --Original Mail --
Sender:Yun Tang 
Send Date:Wed Mar 2 17:05:05 2022
Recipients:dev 
Subject:Re: [VOTE] Release 1.14.4, release candidate #1
+1 non-binding

- Checked the signatures for pending release artifacts.
- Download the pre-built flink-dist of both scala_2.11 and scala_2.12 versions 
and run them locally with the StateMachine example.
- Reviewed the flink-web PR

Best
Yun Tang

From: Seth Wiesman 
Sent: Monday, February 28, 2022 23:02
To: dev 
Subject: Re: [VOTE] Release 1.14.4, release candidate #1

+1 non-binding

- built from source
- checked hashes and signatures
- started locally and deployed wordcount / stopped with savepoint /
restarted
- reviewed announcement post

Thanks for managing the release!

Seth

On Fri, Feb 25, 2022 at 7:30 AM Konstantin Knauf  wrote:

> Hi everyone,
>
> Please review and vote on the release candidate #1 for the version 1.14.4,
> as follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
>
>  The complete staging area is available for your review, which includes:
> * JIRA release notes [1],
> * the official Apache source release and binary convenience releases to be
> deployed to dist.apache.org [2], which are signed with the key with
> fingerprint 8C3FB007FE60 DEFA [3],
> * all artifacts to be deployed to the Maven Central Repository [4],
> * source code tag "release-1.14.4-rc1" [5],
> * website pull request listing the new release and adding announcement blog
> post [6].
>
> The vote will be open for at least 72 hours. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
>
> Thanks,
> Konstantin
>
> [1]
>
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12351231
> [2] https://dist.apache.org/repos/dist/dev/flink/flink-1.14.4-rc1/
> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> [4]
> https://repository.apache.org/content/repositories/orgapacheflink-1487/
> [5] https://github.com/apache/flink/tree/release-1.14.4-rc1
> [6] https://github.com/apache/flink-web/pull/510
>
> --
>
> Konstantin Knauf
>
> https://twitter.com/snntrable
>
> https://github.com/knaufk
>


[jira] [Created] (FLINK-26458) Rename Accumulator to MergeFunction

2022-03-02 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-26458:


 Summary: Rename Accumulator to MergeFunction
 Key: FLINK-26458
 URL: https://issues.apache.org/jira/browse/FLINK-26458
 Project: Flink
  Issue Type: Sub-task
  Components: Table Store
Reporter: Jingsong Lee
 Fix For: table-store-0.1.0


See org.apache.flink.table.store.file.mergetree.compact.Accumulator.

Actually, it is not an accumulator, but a merger. The naming of the accumulator 
is misleading.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (FLINK-26457) Introduce Join Accumulator for Wide table

2022-03-02 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-26457:


 Summary: Introduce Join Accumulator for Wide table
 Key: FLINK-26457
 URL: https://issues.apache.org/jira/browse/FLINK-26457
 Project: Flink
  Issue Type: Sub-task
  Components: Table Store
Reporter: Jingsong Lee
 Fix For: table-store-0.1.0


Consider a Join Accumulator, It will merge two records, completing the not-null 
fields.

It is very useful for wide tables, where two source tables join together to 
form a wide table.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Re: [DISCUSS] Manual savepoint triggering in flink-kubernetes-operator

2022-03-02 Thread Yang Wang
I agree that we could start with the annotation approach and collect the
feedback at the same time.

Best,
Yang

Őrhidi Mátyás  于2022年3月2日周三 20:06写道:

> Thank you for your feedback!
>
> The annotation on the
>
> @ControllerConfiguration(generationAwareEventProcessing = false)
> FlinkDeploymentController
>
> already enables the event triggering based on metadata changes. It was set
> earlier to support some failure scenarios. (It can be used for example to
> manually reenable the reconcile loop when it got stuck in an error phase)
>
> I will go ahead and propose a PR using annotations then.
>
> Cheers,
> Matyas
>
> On Wed, Mar 2, 2022 at 12:47 PM Yang Wang  wrote:
>
> > I also like the annotation approach since it is more natural.
> > But I am not sure about whether the meta data change will trigger an
> event
> > in java-operator-sdk.
> >
> >
> > Best,
> > Yang
> >
> > Gyula Fóra  于2022年3月2日周三 16:29写道:
> >
> > > Thanks Matyas,
> > >
> > > From a user perspective I think the annotation is pretty nice and user
> > > friendly so I personally prefer that approach.
> > >
> > > You said:
> > >  "It seems, the java-operator-sdk handles the changes of the .metadata
> > and
> > > .spec fields of custom resources differently."
> > >
> > > What implications does this have on the above mentioned 2 approaches?
> > Does
> > > it make one more difficult than the other?
> > >
> > > Cheers
> > > Gyula
> > >
> > >
> > >
> > > On Tue, Mar 1, 2022 at 1:52 PM Őrhidi Mátyás 
> > > wrote:
> > >
> > > > Hi All!
> > > >
> > > > I'd like to start a quick discussion about the way we allow users to
> > > > trigger savepoints manually in the operator [FLINK-26181]
> > > > . There are
> > existing
> > > > solutions already for this functionality in other operators, for
> > example:
> > > > - counter based
> > > > <
> > > >
> > >
> >
> https://github.com/spotify/flink-on-k8s-operator/blob/master/docs/savepoints_guide.md#2-taking-savepoints-by-updating-the-flinkcluster-custom-resource
> > > > >
> > > > - annotation based
> > > > <
> > > >
> > >
> >
> https://github.com/spotify/flink-on-k8s-operator/blob/master/docs/savepoints_guide.md#3-taking-savepoints-by-attaching-annotation-to-the-flinkcluster-custom-resource
> > > > >
> > > >
> > > > We could implement any of these or both or come up with our own
> > approach.
> > > > It seems, the java-operator-sdk handles the changes of the .metadata
> > and
> > > > .spec fields of custom resources differently. For further info see
> the
> > > > chapter Generation Awareness and Event Filtering in the docs
> > > > .
> > > >
> > > > Let me know what you think.
> > > >
> > > > Cheers,
> > > > Matyas
> > > >
> > >
> >
>


Re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread Shuo Cheng
+1, a lot of convenience will be brought then. 

On Wed, Mar 2, 2022 at 7:29 PM Leonard Xu  wrote:

> +1 from my side, thanks for driving this improvement.
>
> Best,
> Leonard
>
> > 2022年3月2日 下午7:13,Jiangang Liu  写道:
> >
> > +1 for the feature. Good style management can help the developer a lot.
> >
> > Marios Trivyzas  于2022年3月2日周三 18:19写道:
> >
> >> +1 from me as well, Having a unified auto-formatter for scala would be
> >> great.
> >> Currently we don't have consistency in our code base, and this makes it
> >> more difficult
> >> to read and work on the scala code.
> >>
> >> Best,
> >> Marios
> >>
> >> On Wed, Mar 2, 2022 at 11:41 AM wenlong.lwl 
> >> wrote:
> >>
> >>> +1, currently the scalastyle does not work well actually, there are a
> lot
> >>> of style differences in different files. It would be great if the code
> >> can
> >>> be auto formatted.
> >>>
> >>> Best,
> >>> Wenlong
> >>>
> >>> On Wed, 2 Mar 2022 at 16:34, Jingsong Li 
> wrote:
> >>>
>  +1.
> 
>  Thanks for driving.
> 
>  I wrote some scala code, the style of our flink's scala is messy. We
>  can do better.
> 
>  Best,
>  Jingsong
> 
>  On Wed, Mar 2, 2022 at 4:19 PM Yun Tang  wrote:
> >
> > +1
> >
> > I also noticed that the project of scalafmt [1] is much more active
> >>> than
>  scalatyle [2], which has no release in the past 4 years.
> >
> >
> > [1] https://github.com/scalameta/scalafmt/releases
> > [2] https://github.com/scalastyle/scalastyle/tags
> >
> > Best
> > Yun Tang
> >
> > 
> > From: Konstantin Knauf 
> > Sent: Wednesday, March 2, 2022 15:01
> > To: dev 
> > Subject: Re: [DISCUSS] Enable scala formatting check
> >
> > +1 I've never written any Scala in Flink, but this makes a lot of
> >> sense
>  to
> > me. Converging on a smaller set of tools and simplifying the build is
> > always a good idea and the Community already concluded before that
>  spotless
> > is generally a good approach.
> >
> > On Tue, Mar 1, 2022 at 5:52 PM Francesco Guardiani <
>  france...@ververica.com>
> > wrote:
> >
> >> Hi all,
> >>
> >> I want to propose to enable the spotless scalafmt integration and
>  remove
> >> the scalastyle plugin.
> >>
> >> From an initial analysis, scalafmt can do everything scalastyle can
>  do, and
> >> the integration with spotless looks easy to enable:
> >> https://github.com/diffplug/spotless/tree/main/plugin-maven#scala.
> >>> The
> >> scalafmt conf file gets picked up automatically from every IDE, and
> >>> it
>  can
> >> be heavily tuned.
> >>
> >> This way we can unify the formatting and integrate with our CI
> >>> without
>  any
> >> additional configurations. And we won't need scalastyle anymore, as
> >> scalafmt will take care of the checks:
> >>
> >> * mvn spotless:check will check both java and scala
> >> * mvn spotless:apply will format both java and scala
> >>
> >> WDYT?
> >>
> >> FG
> >>
> >>
> >>
> >> --
> >>
> >> Francesco Guardiani | Software Engineer
> >>
> >> france...@ververica.com
> >>
> >>
> >> 
> >>
> >> Follow us @VervericaData
> >>
> >> --
> >>
> >> Join Flink Forward  - The Apache Flink
> >> Conference
> >>
> >> Stream Processing | Event Driven | Real Time
> >>
> >> --
> >>
> >> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
> >>
> >> --
> >>
> >> Ververica GmbH
> >>
> >> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> >>
> >> Managing Directors: Karl Anton Wehner, Holger Temme, Yip Park Tung
>  Jason,
> >> Jinwei (Kevin) Zhang
> >>
> >
> >
> > --
> >
> > Konstantin Knauf
> >
> > https://twitter.com/snntrable
> >
> > https://github.com/knaufk
> 
> >>>
> >>
> >>
> >> --
> >> Marios
> >>
>
>


Re: Change of focus

2022-03-02 Thread Shuo Cheng
Thanks for your efforts, Till. Good Luck!

On Thu, Mar 3, 2022 at 12:00 AM Peter Huang 
wrote:

> Hi Till,
>
> Thank you Till for monitoring and reviewing yarn related PRs. Good luck in
> your next chapter!
>
> Best Regards
> Peter Huang
>
> On Wed, Mar 2, 2022 at 3:34 AM Yu Li  wrote:
>
> > Thanks for everything, Till! You are such a great mentor and workmate,
> and
> > we will definitely miss you...
> >
> > Wish you all the best for the new adventure, and be sure to let me know
> > when you plan to visit Beijing someday in the future!
> >
> > Best Regards,
> > Yu
> >
> >
> > On Wed, 2 Mar 2022 at 16:09, Jingsong Li  wrote:
> >
> > > Hi Till,
> > >
> > > Good luck with the next chapter, and thanks for all of your efforts.
> > >
> > > Best,
> > > Jingsong
> > >
> > > On Wed, Mar 2, 2022 at 2:57 PM Yufei Zhang  wrote:
> > > >
> > > > Hi Till,
> > > >
> > > > Thank you Till, and good luck in your next chapter :)
> > > >
> > > > Cheers,
> > > > Yufei
> > > >
> > > > On Mon, Feb 28, 2022 at 6:59 PM Till Rohrmann 
> > > wrote:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > > I wanted to let you know that I will be less active in the
> community
> > > > > because I’ve decided to start a new chapter in my life. Hence,
> please
> > > don’t
> > > > > wonder if I might no longer be very responsive on mails and JIRA
> > > issues.
> > > > >
> > > > > It is great being part of such a great community with so many
> amazing
> > > > > people. Over the past 7,5 years, I’ve learned a lot thanks to you
> and
> > > > > together we have shaped how people think about stream processing
> > > nowadays.
> > > > > This is something we can be very proud of. I am sure that the
> > community
> > > > > will continue innovating and setting the pace for what is possible
> > with
> > > > > real time processing. I wish you all godspeed!
> > > > >
> > > > > Cheers,
> > > > > Till
> > > > >
> > >
> >
>


[jira] [Created] (FLINK-26456) StreamExecutionEnvironmentTests::test_add_python_file failed with ModuleNotFoundError

2022-03-02 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-26456:


 Summary: StreamExecutionEnvironmentTests::test_add_python_file 
failed with ModuleNotFoundError
 Key: FLINK-26456
 URL: https://issues.apache.org/jira/browse/FLINK-26456
 Project: Flink
  Issue Type: Bug
  Components: API / Python
Affects Versions: 1.14.0
Reporter: Huang Xingbo



{code:java}
2022-03-02T16:33:43.6649755Z Mar 02 16:33:43 Traceback (most recent call last):
2022-03-02T16:33:43.6650839Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/.tox/py38/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 289, in _execute
2022-03-02T16:33:43.6651405Z Mar 02 16:33:43 response = task()
2022-03-02T16:33:43.6652257Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/.tox/py38/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 362, in 
2022-03-02T16:33:43.6652846Z Mar 02 16:33:43 lambda: 
self.create_worker().do_instruction(request), request)
2022-03-02T16:33:43.6653655Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/.tox/py38/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 606, in do_instruction
2022-03-02T16:33:43.6654200Z Mar 02 16:33:43 return getattr(self, 
request_type)(
2022-03-02T16:33:43.6654969Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/.tox/py38/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 644, in process_bundle
2022-03-02T16:33:43.6655549Z Mar 02 16:33:43 
bundle_processor.process_bundle(instruction_id))
2022-03-02T16:33:43.6656337Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/.tox/py38/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py",
 line 999, in process_bundle
2022-03-02T16:33:43.6656936Z Mar 02 16:33:43 
input_op_by_transform_id[element.transform_id].process_encoded(
2022-03-02T16:33:43.6657773Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/.tox/py38/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py",
 line 228, in process_encoded
2022-03-02T16:33:43.6658326Z Mar 02 16:33:43 self.output(decoded_value)
2022-03-02T16:33:43.6658829Z Mar 02 16:33:43   File 
"apache_beam/runners/worker/operations.py", line 357, in 
apache_beam.runners.worker.operations.Operation.output
2022-03-02T16:33:43.6659460Z Mar 02 16:33:43   File 
"apache_beam/runners/worker/operations.py", line 359, in 
apache_beam.runners.worker.operations.Operation.output
2022-03-02T16:33:43.6660261Z Mar 02 16:33:43   File 
"apache_beam/runners/worker/operations.py", line 221, in 
apache_beam.runners.worker.operations.SingletonConsumerSet.receive
2022-03-02T16:33:43.6661064Z Mar 02 16:33:43   File 
"apache_beam/runners/worker/operations.py", line 319, in 
apache_beam.runners.worker.operations.Operation.process
2022-03-02T16:33:43.6661902Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/pyflink/fn_execution/beam/beam_operations_slow.py", line 
132, in process
2022-03-02T16:33:43.6662440Z Mar 02 16:33:43 
self._output_processor.process_outputs(o, self.process_element(value))
2022-03-02T16:33:43.6663200Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/pyflink/fn_execution/beam/beam_operations_slow.py", line 
63, in process_outputs
2022-03-02T16:33:43.6663750Z Mar 02 16:33:43 
self._consumer.process(windowed_value.with_value(results))
2022-03-02T16:33:43.6664475Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/pyflink/fn_execution/beam/beam_operations_slow.py", line 
131, in process
2022-03-02T16:33:43.6664969Z Mar 02 16:33:43 for value in o.value:
2022-03-02T16:33:43.6665754Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/pyflink/fn_execution/datastream/operations.py", line 
179, in wrapped_func
2022-03-02T16:33:43.280Z Mar 02 16:33:43 yield from 
_emit_results(timestamp, watermark, results)
2022-03-02T16:33:43.6667010Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/pyflink/fn_execution/datastream/input_handler.py", line 
101, in _emit_results
2022-03-02T16:33:43.6667492Z Mar 02 16:33:43 for result in results:
2022-03-02T16:33:43.6668154Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/pyflink/datastream/data_stream.py", line 271, in 
process_element
2022-03-02T16:33:43.6668626Z Mar 02 16:33:43 yield self._map_func(value)
2022-03-02T16:33:43.6669335Z Mar 02 16:33:43   File 
"/__w/2/s/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py",
 line 350, in plus_two_map
2022-03-02T16:33:43.6669848Z Mar 02 16:33:43 from test_dep1 import add_two
2022-03-02T16:33:43.6670561Z Mar 02 16:33:43 ModuleNotFoundError: No module 
named 'test_dep1'
{code}
https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=32448=logs=9cada3cb-c1d3-5621-16da-0f718fb86602=c67e71ed-6451-5d26-8920-5a8cf9651901




--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (FLINK-26455) [Changelog] Materialization interleaved with task cancellation can fail the job

2022-03-02 Thread Roman Khachatryan (Jira)
Roman Khachatryan created FLINK-26455:
-

 Summary: [Changelog] Materialization interleaved with task 
cancellation can fail the job
 Key: FLINK-26455
 URL: https://issues.apache.org/jira/browse/FLINK-26455
 Project: Flink
  Issue Type: Bug
  Components: Runtime / State Backends
Affects Versions: 1.15.0
Reporter: Roman Khachatryan
 Fix For: 1.15.0






--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Re: Help with pom dependencies for Flink with Table

2022-03-02 Thread Jing Ge
Hi Adesh,

have you tried building your project with a clean classpath, i.e. mvn clean
...? Additionally, you could also try deleting org/apache/flink from your
local .m2 repository to have an even cleaner repo. It looks like a
BulkReaderFormatFactory from the current master branch [1] has been
referenced while Flink 1.14.3 [2] is used.

[1]
https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/BulkReaderFormatFactory.java
[2]
https://github.com/apache/flink/blob/release-1.14.3/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/BulkReaderFormatFactory.java


Best regards,
Jing

On Wed, Mar 2, 2022 at 6:34 PM Adesh Dsilva  wrote:

> Hi
>
> I am struggling to run my test Flink project with Table API
>
> I am trying to run a simple piece of code:
>
> final StreamExecutionEnvironment env =
> StreamExecutionEnvironment.getExecutionEnvironment();
> env.setRuntimeMode(RuntimeExecutionMode.BATCH);
>
> StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
>
> Path in = Path.fromLocalFile(new File("part-v001-o000-r-00330.avro"));
> AvroInputFormat users = new AvroInputFormat<>(in,
> BidSample.class);
> DataStream bidsDS = env.createInput(users);
>
> Table bidsTable = tableEnv.fromDataStream(bidsDS);
> bidsTable.printSchema();
>
>
> And here is my pom dependencies
>
> 
>org.apache.flink
>flink-streaming-java_2.12
>1.14.3
> 
> 
>org.apache.flink
>flink-clients_2.12
>1.14.3
> 
> 
>org.apache.flink
>flink-table-api-java-bridge_2.12
>1.14.3
> 
> 
>org.apache.flink
>flink-table-planner_2.12
>1.14.3
> 
> 
>org.apache.flink
>flink-table-common
>1.14.3
> 
> 
>org.apache.flink
>flink-table-api-java
>1.14.3
> 
>
>
>
> But I keep getting below error:
> Caused by: java.lang.ClassNotFoundException:
> org.apache.flink.connector.file.table.factories.BulkReaderFormatFactory
> at
> java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:766)
> at
> java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178)
>
>
>
> Any help to know why this is happening and fix it would be much
> appreciated.
>
> Many thanks,
> Adesh DSilva


[jira] [Created] (FLINK-26454) Improve operator logging

2022-03-02 Thread Gyula Fora (Jira)
Gyula Fora created FLINK-26454:
--

 Summary: Improve operator logging
 Key: FLINK-26454
 URL: https://issues.apache.org/jira/browse/FLINK-26454
 Project: Flink
  Issue Type: Sub-task
  Components: Kubernetes Operator
Reporter: Gyula Fora


At the moment the way information is logged throughout the operator is very 
inconsistent. Some parts log the name of the deployment, some the name + 
namespace, some neither of these.

We should try to clean this up and unify it across the operator.

I see basically 2 possible ways:
 1. Add a log formatter utility to always attach name + namespace information 
to each logged line
 2. Remove namespace + name from everywhere and extract this as part of the 
logger setting from MDC information the operator sdk already provides 
([https://javaoperatorsdk.io/docs/features)]

We should discuss this on the mailing list as part of this work



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Help with pom dependencies for Flink with Table

2022-03-02 Thread Adesh Dsilva
Hi

I am struggling to run my test Flink project with Table API

I am trying to run a simple piece of code:

final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
env.setRuntimeMode(RuntimeExecutionMode.BATCH);

StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);

Path in = Path.fromLocalFile(new File("part-v001-o000-r-00330.avro"));
AvroInputFormat users = new AvroInputFormat<>(in, BidSample.class);
DataStream bidsDS = env.createInput(users);

Table bidsTable = tableEnv.fromDataStream(bidsDS);
bidsTable.printSchema();


And here is my pom dependencies


   org.apache.flink
   flink-streaming-java_2.12
   1.14.3


   org.apache.flink
   flink-clients_2.12
   1.14.3


   org.apache.flink
   flink-table-api-java-bridge_2.12
   1.14.3


   org.apache.flink
   flink-table-planner_2.12
   1.14.3


   org.apache.flink
   flink-table-common
   1.14.3


   org.apache.flink
   flink-table-api-java
   1.14.3




But I keep getting below error:
Caused by: java.lang.ClassNotFoundException: 
org.apache.flink.connector.file.table.factories.BulkReaderFormatFactory
at 
java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:766)
at 
java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178)



Any help to know why this is happening and fix it would be much appreciated.

Many thanks,
Adesh DSilva

[jira] [Created] (FLINK-26453) execution.allow-client-job-configurations not checked for executeAsync

2022-03-02 Thread Marios Trivyzas (Jira)
Marios Trivyzas created FLINK-26453:
---

 Summary: execution.allow-client-job-configurations not checked for 
executeAsync
 Key: FLINK-26453
 URL: https://issues.apache.org/jira/browse/FLINK-26453
 Project: Flink
  Issue Type: Bug
  Components: API / DataStream
Affects Versions: 1.15.0
Reporter: Marios Trivyzas
Assignee: Fabian Paul


* *checkNotAllowedConfigurations()* should be called by  
{*}{*}{*}StreamContextEnvironment#executeAsync(){*}
 *  Description of the *DeploymentOption* should be more clear, and it's not 
only checked by application mode.
 * When using a combination of TableAPI and DataStreamApi, the check for 
overriding config options is not applied:

Modified code of StreamSQLExample to pass extra config
{noformat}
final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();

Configuration conf = new Configuration();
conf.set(
ExecutionConfigOptions.TABLE_EXEC_SINK_NOT_NULL_ENFORCER,
ExecutionConfigOptions.NotNullEnforcer.DROP);

// set up the Java Table API
final StreamTableEnvironment tableEnv =
StreamTableEnvironment.create(env, 
EnvironmentSettings.fromConfiguration(conf));

final DataStream orderA =
env.fromCollection(
Arrays.asList(
new Order(1L, "beer", 3),
new Order(1L, "diaper", 4),
new Order(3L, "rubber", 2)));

final DataStream orderB =
env.fromCollection(
Arrays.asList(
new Order(2L, "pen", 3),
new Order(2L, "rubber", 3),
new Order(4L, "beer", 1)));

// convert the first DataStream to a Table object
// it will be used "inline" and is not registered in a catalog
final Table tableA = tableEnv.fromDataStream(orderA);

// convert the second DataStream and register it as a view
// it will be accessible under a name
tableEnv.createTemporaryView("TableB", orderB);

// union the two tables
final Table result =
tableEnv.sqlQuery(
"SELECT * FROM "
+ tableA
+ " WHERE amount > 2 UNION ALL "
+ "SELECT * FROM TableB WHERE amount < 2");

// convert the Table back to an insert-only DataStream of type `Order`
tableEnv.toDataStream(result, Order.class).print();

// after the table program is converted to a DataStream program,
// we must use `env.execute()` to submit the job
env.execute();


{noformat}
ExecutionConfigOptions.TABLE_EXEC_SINK_NOT_NULL_ENFORCER is set to ERROR in 
flink-conf.yaml and yet no exception is thrown, that is because in

StreamTableEnvironmentImpl:
{noformat}
public static StreamTableEnvironment create(
StreamExecutionEnvironment executionEnvironment,
EnvironmentSettings settings,
TableConfig tableConfig) {{noformat}
we use the 
{noformat}
public static Executor lookupExecutor(
ClassLoader classLoader,
String executorIdentifier,
StreamExecutionEnvironment executionEnvironment) {{noformat}
so we don't follow the same path to call the 
StreamContextEnvironment#setAsContext

which checks for overriding options depending on the new flag.

 

 

 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Re: Change of focus

2022-03-02 Thread Peter Huang
Hi Till,

Thank you Till for monitoring and reviewing yarn related PRs. Good luck in
your next chapter!

Best Regards
Peter Huang

On Wed, Mar 2, 2022 at 3:34 AM Yu Li  wrote:

> Thanks for everything, Till! You are such a great mentor and workmate, and
> we will definitely miss you...
>
> Wish you all the best for the new adventure, and be sure to let me know
> when you plan to visit Beijing someday in the future!
>
> Best Regards,
> Yu
>
>
> On Wed, 2 Mar 2022 at 16:09, Jingsong Li  wrote:
>
> > Hi Till,
> >
> > Good luck with the next chapter, and thanks for all of your efforts.
> >
> > Best,
> > Jingsong
> >
> > On Wed, Mar 2, 2022 at 2:57 PM Yufei Zhang  wrote:
> > >
> > > Hi Till,
> > >
> > > Thank you Till, and good luck in your next chapter :)
> > >
> > > Cheers,
> > > Yufei
> > >
> > > On Mon, Feb 28, 2022 at 6:59 PM Till Rohrmann 
> > wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > I wanted to let you know that I will be less active in the community
> > > > because I’ve decided to start a new chapter in my life. Hence, please
> > don’t
> > > > wonder if I might no longer be very responsive on mails and JIRA
> > issues.
> > > >
> > > > It is great being part of such a great community with so many amazing
> > > > people. Over the past 7,5 years, I’ve learned a lot thanks to you and
> > > > together we have shaped how people think about stream processing
> > nowadays.
> > > > This is something we can be very proud of. I am sure that the
> community
> > > > will continue innovating and setting the pace for what is possible
> with
> > > > real time processing. I wish you all godspeed!
> > > >
> > > > Cheers,
> > > > Till
> > > >
> >
>


[jira] [Created] (FLINK-26452) Flink deploy on k8s when kubeconfig server is hostname not ip

2022-03-02 Thread hjw (Jira)
hjw created FLINK-26452:
---

 Summary:  Flink deploy on k8s when  kubeconfig  server is hostname 
not ip
 Key: FLINK-26452
 URL: https://issues.apache.org/jira/browse/FLINK-26452
 Project: Flink
  Issue Type: Bug
  Components: Deployment / Kubernetes
Affects Versions: 1.13.6
Reporter: hjw


~/.kube/config

apiVersion:v1
kind:config
cluster:
-name: "yf-dev-cluster1"
  cluster:
server: "https://in-acpmanager.test.*.cn/k8s/clusters/c-t5h2t;
certificate-authority-data : “……"


{code:java}
2022-03-02 18:59:30 | ^[[31mWARN ^[[0;39m | ^[[1;33mOkHttp 
https://in-acpmanager.test.*.cn/...^[[0;39m | 
^[[1;32mio.fabric8.kubernetes.client.dsl.internal.WatcherWebSocketListener^[[0;39m:76]
 Exec Failure javax.net.ssl.SSLPeerUnverifi
edException Hostname in-acpmanager.test.yfzx.cn not verified:
certificate: sha256/cw2T2s+Swhl7z+H35/3C1dTLxL26OOMO5VoEN9kAZCA=
DN: CN=in-acpmanager.test.yfzx.cn
subjectAltNames: []
io.fabric8.kubernetes.client.KubernetesClientException: Failed to start 
websocket
at 
io.fabric8.kubernetes.client.dsl.internal.WatcherWebSocketListener.onFailure(WatcherWebSocketListener.java:77)
at 
org.apache.flink.kubernetes.shaded.okhttp3.internal.ws.RealWebSocket.failWebSocket(RealWebSocket.java:570)
at 
org.apache.flink.kubernetes.shaded.okhttp3.internal.ws.RealWebSocket$1.onFailure(RealWebSocket.java:216)
at 
org.apache.flink.kubernetes.shaded.okhttp3.RealCall$AsyncCall.execute(RealCall.java:180)
at 
org.apache.flink.kubernetes.shaded.okhttp3.internal.NamedRunnable.run(NamedRunnable.java:32)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
Suppressed: java.lang.Throwable: waiting here
at 
io.fabric8.kubernetes.client.utils.Utils.waitUntilReady(Utils.java:164)
at 
io.fabric8.kubernetes.client.utils.Utils.waitUntilReadyOrFail(Utils.java:175)
at 
io.fabric8.kubernetes.client.dsl.internal.WatcherWebSocketListener.waitUntilReady(WatcherWebSocketListener.java:120)
at 
io.fabric8.kubernetes.client.dsl.internal.WatchConnectionManager.waitUntilReady(WatchConnectionManager.java:82)
at 
io.fabric8.kubernetes.client.dsl.base.BaseOperation.watch(BaseOperation.java:705)
at 
io.fabric8.kubernetes.client.dsl.base.BaseOperation.watch(BaseOperation.java:678)
at 
io.fabric8.kubernetes.client.dsl.base.BaseOperation.watch(BaseOperation.java:
{code}




--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (FLINK-26451) Test GCS FileSystem w/ RecoverableWriter manually

2022-03-02 Thread Galen Warren (Jira)
Galen Warren created FLINK-26451:


 Summary: Test GCS FileSystem w/ RecoverableWriter manually
 Key: FLINK-26451
 URL: https://issues.apache.org/jira/browse/FLINK-26451
 Project: Flink
  Issue Type: Improvement
  Components: FileSystems
Reporter: Galen Warren
 Fix For: 1.15.0


This is a manual testing issue for 
[FLINK-11838|https://github.com/apache/flink/pull/15599].

The linked issue adds support for a Flink FileSystem backed by Google Cloud 
Storage (GCS). This FileSystem implementation includes RecoverableWriter 
support, which allows it to be used with [FileSystem 
connector|https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/filesystem/]
 sinks.

Suggested things to test:
 * Validate writing checkpoints and/or savepoints to a GCS bucket via a *gs://* 
endpoint
 * Validate reading from and writing to *gs://* endpoints via FileSystem 
connector sources and sinks

Note that a Google Cloud account is required to access GCS buckets. 
Instructions for how to provide the necessary credentials are included in the 
documentation.

Documentation:
 * 
https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/filesystems/gcs/

 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (FLINK-26450) FileStateHandle.discardState does not process return value

2022-03-02 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-26450:
-

 Summary: FileStateHandle.discardState does not process return value
 Key: FLINK-26450
 URL: https://issues.apache.org/jira/browse/FLINK-26450
 Project: Flink
  Issue Type: Bug
  Components: Connectors / FileSystem, Runtime / Coordination
Affects Versions: 1.14.3, 1.13.6, 1.15.0
Reporter: Matthias Pohl


The retryable cleanup does not work properly if there's an error appearing 
during the {{FileSystem.delete}} call which is used within 
{{FileStateHandle.discardState}}. Some {{FileSystem}} implementations (e.g. S3 
presto) return {{false}} in case of an error which will be swalled in 
{{FileStateHandle.discardState}}.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (FLINK-26449) Enable tests in KafkaSourceLegacyITCase for test stability issues

2022-03-02 Thread Jing Ge (Jira)
Jing Ge created FLINK-26449:
---

 Summary: Enable tests in KafkaSourceLegacyITCase for test 
stability issues
 Key: FLINK-26449
 URL: https://issues.apache.org/jira/browse/FLINK-26449
 Project: Flink
  Issue Type: Bug
  Components: Tests
Reporter: Jing Ge


this the follow-up ticket of FLINK-26448



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (FLINK-26448) Disable tests in KafkaSourceLegacyITCase for test stability issues

2022-03-02 Thread Jing Ge (Jira)
Jing Ge created FLINK-26448:
---

 Summary: Disable tests in KafkaSourceLegacyITCase for test 
stability issues
 Key: FLINK-26448
 URL: https://issues.apache.org/jira/browse/FLINK-26448
 Project: Flink
  Issue Type: Bug
  Components: Tests
Reporter: Jing Ge


Currently, there are some issues with the Kafka test env with 1 broker.

Disable testBrokerFailure() and testMultipleTopicsWithKafkaSerializer () in 
KafkaSourceLegacyITCase. After disabling them, Test - kafka/gelly will take 29m 
42s, now is hung for 3h 52m with that tests enabled.

They will be enabled again after the kafka test env has been improved.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Re: [DISCUSS] Manual savepoint triggering in flink-kubernetes-operator

2022-03-02 Thread Őrhidi Mátyás
Thank you for your feedback!

The annotation on the

@ControllerConfiguration(generationAwareEventProcessing = false)
FlinkDeploymentController

already enables the event triggering based on metadata changes. It was set
earlier to support some failure scenarios. (It can be used for example to
manually reenable the reconcile loop when it got stuck in an error phase)

I will go ahead and propose a PR using annotations then.

Cheers,
Matyas

On Wed, Mar 2, 2022 at 12:47 PM Yang Wang  wrote:

> I also like the annotation approach since it is more natural.
> But I am not sure about whether the meta data change will trigger an event
> in java-operator-sdk.
>
>
> Best,
> Yang
>
> Gyula Fóra  于2022年3月2日周三 16:29写道:
>
> > Thanks Matyas,
> >
> > From a user perspective I think the annotation is pretty nice and user
> > friendly so I personally prefer that approach.
> >
> > You said:
> >  "It seems, the java-operator-sdk handles the changes of the .metadata
> and
> > .spec fields of custom resources differently."
> >
> > What implications does this have on the above mentioned 2 approaches?
> Does
> > it make one more difficult than the other?
> >
> > Cheers
> > Gyula
> >
> >
> >
> > On Tue, Mar 1, 2022 at 1:52 PM Őrhidi Mátyás 
> > wrote:
> >
> > > Hi All!
> > >
> > > I'd like to start a quick discussion about the way we allow users to
> > > trigger savepoints manually in the operator [FLINK-26181]
> > > . There are
> existing
> > > solutions already for this functionality in other operators, for
> example:
> > > - counter based
> > > <
> > >
> >
> https://github.com/spotify/flink-on-k8s-operator/blob/master/docs/savepoints_guide.md#2-taking-savepoints-by-updating-the-flinkcluster-custom-resource
> > > >
> > > - annotation based
> > > <
> > >
> >
> https://github.com/spotify/flink-on-k8s-operator/blob/master/docs/savepoints_guide.md#3-taking-savepoints-by-attaching-annotation-to-the-flinkcluster-custom-resource
> > > >
> > >
> > > We could implement any of these or both or come up with our own
> approach.
> > > It seems, the java-operator-sdk handles the changes of the .metadata
> and
> > > .spec fields of custom resources differently. For further info see the
> > > chapter Generation Awareness and Event Filtering in the docs
> > > .
> > >
> > > Let me know what you think.
> > >
> > > Cheers,
> > > Matyas
> > >
> >
>


Re: [DISCUSS] Manual savepoint triggering in flink-kubernetes-operator

2022-03-02 Thread Yang Wang
I also like the annotation approach since it is more natural.
But I am not sure about whether the meta data change will trigger an event
in java-operator-sdk.


Best,
Yang

Gyula Fóra  于2022年3月2日周三 16:29写道:

> Thanks Matyas,
>
> From a user perspective I think the annotation is pretty nice and user
> friendly so I personally prefer that approach.
>
> You said:
>  "It seems, the java-operator-sdk handles the changes of the .metadata and
> .spec fields of custom resources differently."
>
> What implications does this have on the above mentioned 2 approaches? Does
> it make one more difficult than the other?
>
> Cheers
> Gyula
>
>
>
> On Tue, Mar 1, 2022 at 1:52 PM Őrhidi Mátyás 
> wrote:
>
> > Hi All!
> >
> > I'd like to start a quick discussion about the way we allow users to
> > trigger savepoints manually in the operator [FLINK-26181]
> > . There are existing
> > solutions already for this functionality in other operators, for example:
> > - counter based
> > <
> >
> https://github.com/spotify/flink-on-k8s-operator/blob/master/docs/savepoints_guide.md#2-taking-savepoints-by-updating-the-flinkcluster-custom-resource
> > >
> > - annotation based
> > <
> >
> https://github.com/spotify/flink-on-k8s-operator/blob/master/docs/savepoints_guide.md#3-taking-savepoints-by-attaching-annotation-to-the-flinkcluster-custom-resource
> > >
> >
> > We could implement any of these or both or come up with our own approach.
> > It seems, the java-operator-sdk handles the changes of the .metadata and
> > .spec fields of custom resources differently. For further info see the
> > chapter Generation Awareness and Event Filtering in the docs
> > .
> >
> > Let me know what you think.
> >
> > Cheers,
> > Matyas
> >
>


[jira] [Created] (FLINK-26447) Clean up webhook jar and dependency management

2022-03-02 Thread Gyula Fora (Jira)
Gyula Fora created FLINK-26447:
--

 Summary: Clean up webhook jar and dependency management
 Key: FLINK-26447
 URL: https://issues.apache.org/jira/browse/FLINK-26447
 Project: Flink
  Issue Type: Sub-task
  Components: Kubernetes Operator
Reporter: Gyula Fora


Currently the webhook module builds it's own shaded jar which includes the 
operator shaded jar contents as well.

This is unnecessary and simply adds to the size of the image.

Operator dependencies should be in the provided scope and the operator shaded 
jar simply put on the classpath when the webhook starts.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread Leonard Xu
+1 from my side, thanks for driving this improvement.

Best,
Leonard

> 2022年3月2日 下午7:13,Jiangang Liu  写道:
> 
> +1 for the feature. Good style management can help the developer a lot.
> 
> Marios Trivyzas  于2022年3月2日周三 18:19写道:
> 
>> +1 from me as well, Having a unified auto-formatter for scala would be
>> great.
>> Currently we don't have consistency in our code base, and this makes it
>> more difficult
>> to read and work on the scala code.
>> 
>> Best,
>> Marios
>> 
>> On Wed, Mar 2, 2022 at 11:41 AM wenlong.lwl 
>> wrote:
>> 
>>> +1, currently the scalastyle does not work well actually, there are a lot
>>> of style differences in different files. It would be great if the code
>> can
>>> be auto formatted.
>>> 
>>> Best,
>>> Wenlong
>>> 
>>> On Wed, 2 Mar 2022 at 16:34, Jingsong Li  wrote:
>>> 
 +1.
 
 Thanks for driving.
 
 I wrote some scala code, the style of our flink's scala is messy. We
 can do better.
 
 Best,
 Jingsong
 
 On Wed, Mar 2, 2022 at 4:19 PM Yun Tang  wrote:
> 
> +1
> 
> I also noticed that the project of scalafmt [1] is much more active
>>> than
 scalatyle [2], which has no release in the past 4 years.
> 
> 
> [1] https://github.com/scalameta/scalafmt/releases
> [2] https://github.com/scalastyle/scalastyle/tags
> 
> Best
> Yun Tang
> 
> 
> From: Konstantin Knauf 
> Sent: Wednesday, March 2, 2022 15:01
> To: dev 
> Subject: Re: [DISCUSS] Enable scala formatting check
> 
> +1 I've never written any Scala in Flink, but this makes a lot of
>> sense
 to
> me. Converging on a smaller set of tools and simplifying the build is
> always a good idea and the Community already concluded before that
 spotless
> is generally a good approach.
> 
> On Tue, Mar 1, 2022 at 5:52 PM Francesco Guardiani <
 france...@ververica.com>
> wrote:
> 
>> Hi all,
>> 
>> I want to propose to enable the spotless scalafmt integration and
 remove
>> the scalastyle plugin.
>> 
>> From an initial analysis, scalafmt can do everything scalastyle can
 do, and
>> the integration with spotless looks easy to enable:
>> https://github.com/diffplug/spotless/tree/main/plugin-maven#scala.
>>> The
>> scalafmt conf file gets picked up automatically from every IDE, and
>>> it
 can
>> be heavily tuned.
>> 
>> This way we can unify the formatting and integrate with our CI
>>> without
 any
>> additional configurations. And we won't need scalastyle anymore, as
>> scalafmt will take care of the checks:
>> 
>> * mvn spotless:check will check both java and scala
>> * mvn spotless:apply will format both java and scala
>> 
>> WDYT?
>> 
>> FG
>> 
>> 
>> 
>> --
>> 
>> Francesco Guardiani | Software Engineer
>> 
>> france...@ververica.com
>> 
>> 
>> 
>> 
>> Follow us @VervericaData
>> 
>> --
>> 
>> Join Flink Forward  - The Apache Flink
>> Conference
>> 
>> Stream Processing | Event Driven | Real Time
>> 
>> --
>> 
>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>> 
>> --
>> 
>> Ververica GmbH
>> 
>> Registered at Amtsgericht Charlottenburg: HRB 158244 B
>> 
>> Managing Directors: Karl Anton Wehner, Holger Temme, Yip Park Tung
 Jason,
>> Jinwei (Kevin) Zhang
>> 
> 
> 
> --
> 
> Konstantin Knauf
> 
> https://twitter.com/snntrable
> 
> https://github.com/knaufk
 
>>> 
>> 
>> 
>> --
>> Marios
>> 



Re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread Jiangang Liu
+1 for the feature. Good style management can help the developer a lot.

Marios Trivyzas  于2022年3月2日周三 18:19写道:

> +1 from me as well, Having a unified auto-formatter for scala would be
> great.
> Currently we don't have consistency in our code base, and this makes it
> more difficult
> to read and work on the scala code.
>
> Best,
> Marios
>
> On Wed, Mar 2, 2022 at 11:41 AM wenlong.lwl 
> wrote:
>
> > +1, currently the scalastyle does not work well actually, there are a lot
> > of style differences in different files. It would be great if the code
> can
> > be auto formatted.
> >
> > Best,
> > Wenlong
> >
> > On Wed, 2 Mar 2022 at 16:34, Jingsong Li  wrote:
> >
> > > +1.
> > >
> > > Thanks for driving.
> > >
> > > I wrote some scala code, the style of our flink's scala is messy. We
> > > can do better.
> > >
> > > Best,
> > > Jingsong
> > >
> > > On Wed, Mar 2, 2022 at 4:19 PM Yun Tang  wrote:
> > > >
> > > > +1
> > > >
> > > > I also noticed that the project of scalafmt [1] is much more active
> > than
> > > scalatyle [2], which has no release in the past 4 years.
> > > >
> > > >
> > > > [1] https://github.com/scalameta/scalafmt/releases
> > > > [2] https://github.com/scalastyle/scalastyle/tags
> > > >
> > > > Best
> > > > Yun Tang
> > > >
> > > > 
> > > > From: Konstantin Knauf 
> > > > Sent: Wednesday, March 2, 2022 15:01
> > > > To: dev 
> > > > Subject: Re: [DISCUSS] Enable scala formatting check
> > > >
> > > > +1 I've never written any Scala in Flink, but this makes a lot of
> sense
> > > to
> > > > me. Converging on a smaller set of tools and simplifying the build is
> > > > always a good idea and the Community already concluded before that
> > > spotless
> > > > is generally a good approach.
> > > >
> > > > On Tue, Mar 1, 2022 at 5:52 PM Francesco Guardiani <
> > > france...@ververica.com>
> > > > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I want to propose to enable the spotless scalafmt integration and
> > > remove
> > > > > the scalastyle plugin.
> > > > >
> > > > > From an initial analysis, scalafmt can do everything scalastyle can
> > > do, and
> > > > > the integration with spotless looks easy to enable:
> > > > > https://github.com/diffplug/spotless/tree/main/plugin-maven#scala.
> > The
> > > > > scalafmt conf file gets picked up automatically from every IDE, and
> > it
> > > can
> > > > > be heavily tuned.
> > > > >
> > > > > This way we can unify the formatting and integrate with our CI
> > without
> > > any
> > > > > additional configurations. And we won't need scalastyle anymore, as
> > > > > scalafmt will take care of the checks:
> > > > >
> > > > > * mvn spotless:check will check both java and scala
> > > > > * mvn spotless:apply will format both java and scala
> > > > >
> > > > > WDYT?
> > > > >
> > > > > FG
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > >
> > > > > Francesco Guardiani | Software Engineer
> > > > >
> > > > > france...@ververica.com
> > > > >
> > > > >
> > > > > 
> > > > >
> > > > > Follow us @VervericaData
> > > > >
> > > > > --
> > > > >
> > > > > Join Flink Forward  - The Apache Flink
> > > > > Conference
> > > > >
> > > > > Stream Processing | Event Driven | Real Time
> > > > >
> > > > > --
> > > > >
> > > > > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
> > > > >
> > > > > --
> > > > >
> > > > > Ververica GmbH
> > > > >
> > > > > Registered at Amtsgericht Charlottenburg: HRB 158244 B
> > > > >
> > > > > Managing Directors: Karl Anton Wehner, Holger Temme, Yip Park Tung
> > > Jason,
> > > > > Jinwei (Kevin) Zhang
> > > > >
> > > >
> > > >
> > > > --
> > > >
> > > > Konstantin Knauf
> > > >
> > > > https://twitter.com/snntrable
> > > >
> > > > https://github.com/knaufk
> > >
> >
>
>
> --
> Marios
>


Re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread Ryan Skraba
+1 for me -- I've used spotless and scalafmt together in the past, and
especially appreciated how consistent it is between using on the command
line and in the IDE.

All my best, Ryan


On Wed, Mar 2, 2022 at 11:19 AM Marios Trivyzas  wrote:

> +1 from me as well, Having a unified auto-formatter for scala would be
> great.
> Currently we don't have consistency in our code base, and this makes it
> more difficult
> to read and work on the scala code.
>
> Best,
> Marios
>
> On Wed, Mar 2, 2022 at 11:41 AM wenlong.lwl 
> wrote:
>
> > +1, currently the scalastyle does not work well actually, there are a lot
> > of style differences in different files. It would be great if the code
> can
> > be auto formatted.
> >
> > Best,
> > Wenlong
> >
> > On Wed, 2 Mar 2022 at 16:34, Jingsong Li  wrote:
> >
> > > +1.
> > >
> > > Thanks for driving.
> > >
> > > I wrote some scala code, the style of our flink's scala is messy. We
> > > can do better.
> > >
> > > Best,
> > > Jingsong
> > >
> > > On Wed, Mar 2, 2022 at 4:19 PM Yun Tang  wrote:
> > > >
> > > > +1
> > > >
> > > > I also noticed that the project of scalafmt [1] is much more active
> > than
> > > scalatyle [2], which has no release in the past 4 years.
> > > >
> > > >
> > > > [1] https://github.com/scalameta/scalafmt/releases
> > > > [2] https://github.com/scalastyle/scalastyle/tags
> > > >
> > > > Best
> > > > Yun Tang
> > > >
> > > > 
> > > > From: Konstantin Knauf 
> > > > Sent: Wednesday, March 2, 2022 15:01
> > > > To: dev 
> > > > Subject: Re: [DISCUSS] Enable scala formatting check
> > > >
> > > > +1 I've never written any Scala in Flink, but this makes a lot of
> sense
> > > to
> > > > me. Converging on a smaller set of tools and simplifying the build is
> > > > always a good idea and the Community already concluded before that
> > > spotless
> > > > is generally a good approach.
> > > >
> > > > On Tue, Mar 1, 2022 at 5:52 PM Francesco Guardiani <
> > > france...@ververica.com>
> > > > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I want to propose to enable the spotless scalafmt integration and
> > > remove
> > > > > the scalastyle plugin.
> > > > >
> > > > > From an initial analysis, scalafmt can do everything scalastyle can
> > > do, and
> > > > > the integration with spotless looks easy to enable:
> > > > > https://github.com/diffplug/spotless/tree/main/plugin-maven#scala.
> > The
> > > > > scalafmt conf file gets picked up automatically from every IDE, and
> > it
> > > can
> > > > > be heavily tuned.
> > > > >
> > > > > This way we can unify the formatting and integrate with our CI
> > without
> > > any
> > > > > additional configurations. And we won't need scalastyle anymore, as
> > > > > scalafmt will take care of the checks:
> > > > >
> > > > > * mvn spotless:check will check both java and scala
> > > > > * mvn spotless:apply will format both java and scala
> > > > >
> > > > > WDYT?
> > > > >
> > > > > FG
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > >
> > > > > Francesco Guardiani | Software Engineer
> > > > >
> > > > > france...@ververica.com
> > > > >
> > > > >
> > > > > 
> > > > >
> > > > > Follow us @VervericaData
> > > > >
> > > > > --
> > > > >
> > > > > Join Flink Forward  - The Apache Flink
> > > > > Conference
> > > > >
> > > > > Stream Processing | Event Driven | Real Time
> > > > >
> > > > > --
> > > > >
> > > > > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
> > > > >
> > > > > --
> > > > >
> > > > > Ververica GmbH
> > > > >
> > > > > Registered at Amtsgericht Charlottenburg: HRB 158244 B
> > > > >
> > > > > Managing Directors: Karl Anton Wehner, Holger Temme, Yip Park Tung
> > > Jason,
> > > > > Jinwei (Kevin) Zhang
> > > > >
> > > >
> > > >
> > > > --
> > > >
> > > > Konstantin Knauf
> > > >
> > > > https://twitter.com/snntrable
> > > >
> > > > https://github.com/knaufk
> > >
> >
>
>
> --
> Marios
>


[jira] [Created] (FLINK-26446) Update Feature Radar in Apache Flink Roadmap

2022-03-02 Thread Konstantin Knauf (Jira)
Konstantin Knauf created FLINK-26446:


 Summary: Update Feature Radar in Apache Flink Roadmap 
 Key: FLINK-26446
 URL: https://issues.apache.org/jira/browse/FLINK-26446
 Project: Flink
  Issue Type: Improvement
  Components: Project Website
Reporter: Konstantin Knauf
 Fix For: 1.15.0


Deployment/Coordination:
 * Java 8 -> Deprecation
 * Add Deployment Modes
 ** Application Mode -> Stable
 ** Session Mode -> Stable
 ** Per-Job Mode -> Deprecated
 * Adaptive Scheduler -> Ready & Evolving



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread Marios Trivyzas
+1 from me as well, Having a unified auto-formatter for scala would be
great.
Currently we don't have consistency in our code base, and this makes it
more difficult
to read and work on the scala code.

Best,
Marios

On Wed, Mar 2, 2022 at 11:41 AM wenlong.lwl  wrote:

> +1, currently the scalastyle does not work well actually, there are a lot
> of style differences in different files. It would be great if the code can
> be auto formatted.
>
> Best,
> Wenlong
>
> On Wed, 2 Mar 2022 at 16:34, Jingsong Li  wrote:
>
> > +1.
> >
> > Thanks for driving.
> >
> > I wrote some scala code, the style of our flink's scala is messy. We
> > can do better.
> >
> > Best,
> > Jingsong
> >
> > On Wed, Mar 2, 2022 at 4:19 PM Yun Tang  wrote:
> > >
> > > +1
> > >
> > > I also noticed that the project of scalafmt [1] is much more active
> than
> > scalatyle [2], which has no release in the past 4 years.
> > >
> > >
> > > [1] https://github.com/scalameta/scalafmt/releases
> > > [2] https://github.com/scalastyle/scalastyle/tags
> > >
> > > Best
> > > Yun Tang
> > >
> > > 
> > > From: Konstantin Knauf 
> > > Sent: Wednesday, March 2, 2022 15:01
> > > To: dev 
> > > Subject: Re: [DISCUSS] Enable scala formatting check
> > >
> > > +1 I've never written any Scala in Flink, but this makes a lot of sense
> > to
> > > me. Converging on a smaller set of tools and simplifying the build is
> > > always a good idea and the Community already concluded before that
> > spotless
> > > is generally a good approach.
> > >
> > > On Tue, Mar 1, 2022 at 5:52 PM Francesco Guardiani <
> > france...@ververica.com>
> > > wrote:
> > >
> > > > Hi all,
> > > >
> > > > I want to propose to enable the spotless scalafmt integration and
> > remove
> > > > the scalastyle plugin.
> > > >
> > > > From an initial analysis, scalafmt can do everything scalastyle can
> > do, and
> > > > the integration with spotless looks easy to enable:
> > > > https://github.com/diffplug/spotless/tree/main/plugin-maven#scala.
> The
> > > > scalafmt conf file gets picked up automatically from every IDE, and
> it
> > can
> > > > be heavily tuned.
> > > >
> > > > This way we can unify the formatting and integrate with our CI
> without
> > any
> > > > additional configurations. And we won't need scalastyle anymore, as
> > > > scalafmt will take care of the checks:
> > > >
> > > > * mvn spotless:check will check both java and scala
> > > > * mvn spotless:apply will format both java and scala
> > > >
> > > > WDYT?
> > > >
> > > > FG
> > > >
> > > >
> > > >
> > > > --
> > > >
> > > > Francesco Guardiani | Software Engineer
> > > >
> > > > france...@ververica.com
> > > >
> > > >
> > > > 
> > > >
> > > > Follow us @VervericaData
> > > >
> > > > --
> > > >
> > > > Join Flink Forward  - The Apache Flink
> > > > Conference
> > > >
> > > > Stream Processing | Event Driven | Real Time
> > > >
> > > > --
> > > >
> > > > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
> > > >
> > > > --
> > > >
> > > > Ververica GmbH
> > > >
> > > > Registered at Amtsgericht Charlottenburg: HRB 158244 B
> > > >
> > > > Managing Directors: Karl Anton Wehner, Holger Temme, Yip Park Tung
> > Jason,
> > > > Jinwei (Kevin) Zhang
> > > >
> > >
> > >
> > > --
> > >
> > > Konstantin Knauf
> > >
> > > https://twitter.com/snntrable
> > >
> > > https://github.com/knaufk
> >
>


-- 
Marios


Re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread wenlong.lwl
+1, currently the scalastyle does not work well actually, there are a lot
of style differences in different files. It would be great if the code can
be auto formatted.

Best,
Wenlong

On Wed, 2 Mar 2022 at 16:34, Jingsong Li  wrote:

> +1.
>
> Thanks for driving.
>
> I wrote some scala code, the style of our flink's scala is messy. We
> can do better.
>
> Best,
> Jingsong
>
> On Wed, Mar 2, 2022 at 4:19 PM Yun Tang  wrote:
> >
> > +1
> >
> > I also noticed that the project of scalafmt [1] is much more active than
> scalatyle [2], which has no release in the past 4 years.
> >
> >
> > [1] https://github.com/scalameta/scalafmt/releases
> > [2] https://github.com/scalastyle/scalastyle/tags
> >
> > Best
> > Yun Tang
> >
> > 
> > From: Konstantin Knauf 
> > Sent: Wednesday, March 2, 2022 15:01
> > To: dev 
> > Subject: Re: [DISCUSS] Enable scala formatting check
> >
> > +1 I've never written any Scala in Flink, but this makes a lot of sense
> to
> > me. Converging on a smaller set of tools and simplifying the build is
> > always a good idea and the Community already concluded before that
> spotless
> > is generally a good approach.
> >
> > On Tue, Mar 1, 2022 at 5:52 PM Francesco Guardiani <
> france...@ververica.com>
> > wrote:
> >
> > > Hi all,
> > >
> > > I want to propose to enable the spotless scalafmt integration and
> remove
> > > the scalastyle plugin.
> > >
> > > From an initial analysis, scalafmt can do everything scalastyle can
> do, and
> > > the integration with spotless looks easy to enable:
> > > https://github.com/diffplug/spotless/tree/main/plugin-maven#scala. The
> > > scalafmt conf file gets picked up automatically from every IDE, and it
> can
> > > be heavily tuned.
> > >
> > > This way we can unify the formatting and integrate with our CI without
> any
> > > additional configurations. And we won't need scalastyle anymore, as
> > > scalafmt will take care of the checks:
> > >
> > > * mvn spotless:check will check both java and scala
> > > * mvn spotless:apply will format both java and scala
> > >
> > > WDYT?
> > >
> > > FG
> > >
> > >
> > >
> > > --
> > >
> > > Francesco Guardiani | Software Engineer
> > >
> > > france...@ververica.com
> > >
> > >
> > > 
> > >
> > > Follow us @VervericaData
> > >
> > > --
> > >
> > > Join Flink Forward  - The Apache Flink
> > > Conference
> > >
> > > Stream Processing | Event Driven | Real Time
> > >
> > > --
> > >
> > > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
> > >
> > > --
> > >
> > > Ververica GmbH
> > >
> > > Registered at Amtsgericht Charlottenburg: HRB 158244 B
> > >
> > > Managing Directors: Karl Anton Wehner, Holger Temme, Yip Park Tung
> Jason,
> > > Jinwei (Kevin) Zhang
> > >
> >
> >
> > --
> >
> > Konstantin Knauf
> >
> > https://twitter.com/snntrable
> >
> > https://github.com/knaufk
>


Re: [VOTE] Release 1.14.4, release candidate #1

2022-03-02 Thread Yun Tang
+1 non-binding

- Checked the signatures for pending release artifacts.
- Download the pre-built flink-dist of both scala_2.11 and scala_2.12 versions 
and run them locally with the StateMachine example.
- Reviewed the flink-web PR

Best
Yun Tang

From: Seth Wiesman 
Sent: Monday, February 28, 2022 23:02
To: dev 
Subject: Re: [VOTE] Release 1.14.4, release candidate #1

+1 non-binding

- built from source
- checked hashes and signatures
- started locally and deployed wordcount / stopped with savepoint /
restarted
- reviewed announcement post

Thanks for managing the release!

Seth

On Fri, Feb 25, 2022 at 7:30 AM Konstantin Knauf  wrote:

> Hi everyone,
>
> Please review and vote on the release candidate #1 for the version 1.14.4,
> as follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
>
>  The complete staging area is available for your review, which includes:
> * JIRA release notes [1],
> * the official Apache source release and binary convenience releases to be
> deployed to dist.apache.org [2], which are signed with the key with
> fingerprint 8C3FB007FE60 DEFA [3],
> * all artifacts to be deployed to the Maven Central Repository [4],
> * source code tag "release-1.14.4-rc1" [5],
> * website pull request listing the new release and adding announcement blog
> post [6].
>
> The vote will be open for at least 72 hours. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
>
> Thanks,
> Konstantin
>
> [1]
>
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12351231
> [2] https://dist.apache.org/repos/dist/dev/flink/flink-1.14.4-rc1/
> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> [4]
> https://repository.apache.org/content/repositories/orgapacheflink-1487/
> [5] https://github.com/apache/flink/tree/release-1.14.4-rc1
> [6] https://github.com/apache/flink-web/pull/510
>
> --
>
> Konstantin Knauf
>
> https://twitter.com/snntrable
>
> https://github.com/knaufk
>


Re: Change of focus

2022-03-02 Thread Yu Li
Thanks for everything, Till! You are such a great mentor and workmate, and
we will definitely miss you...

Wish you all the best for the new adventure, and be sure to let me know
when you plan to visit Beijing someday in the future!

Best Regards,
Yu


On Wed, 2 Mar 2022 at 16:09, Jingsong Li  wrote:

> Hi Till,
>
> Good luck with the next chapter, and thanks for all of your efforts.
>
> Best,
> Jingsong
>
> On Wed, Mar 2, 2022 at 2:57 PM Yufei Zhang  wrote:
> >
> > Hi Till,
> >
> > Thank you Till, and good luck in your next chapter :)
> >
> > Cheers,
> > Yufei
> >
> > On Mon, Feb 28, 2022 at 6:59 PM Till Rohrmann 
> wrote:
> >
> > > Hi everyone,
> > >
> > > I wanted to let you know that I will be less active in the community
> > > because I’ve decided to start a new chapter in my life. Hence, please
> don’t
> > > wonder if I might no longer be very responsive on mails and JIRA
> issues.
> > >
> > > It is great being part of such a great community with so many amazing
> > > people. Over the past 7,5 years, I’ve learned a lot thanks to you and
> > > together we have shaped how people think about stream processing
> nowadays.
> > > This is something we can be very proud of. I am sure that the community
> > > will continue innovating and setting the pace for what is possible with
> > > real time processing. I wish you all godspeed!
> > >
> > > Cheers,
> > > Till
> > >
>


re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread ??????
+1


I still remember my first pr. Lack of experience, I had to pay attention to 
Scala code format and corrected the format manually, which made me a 
littleembarrassed(though I'm a big fan of Scala). I think this proposal 
will lighten the burden of writing Scala code.


Shouwei Liu


----
??: 
   "dev"
https://github.com/diffplug/spotless/tree/main/plugin-maven#scala. The
 scalafmt conf file gets picked up automatically from every IDE, and it can
 be heavily tuned.

 This way we can unify the formatting and integrate with our CI without any
 additional configurations. And we won't need scalastyle anymore, as
 scalafmt will take care of the checks:

 * mvn spotless:check will check both java and scala
 * mvn spotless:apply will format both java and scala

 WDYT?

 FG



 --

 Francesco Guardiani | Software Engineer

 france...@ververica.com


 

Re: [DISCUSS] Manual savepoint triggering in flink-kubernetes-operator

2022-03-02 Thread Gyula Fóra
Thanks Matyas,

>From a user perspective I think the annotation is pretty nice and user
friendly so I personally prefer that approach.

You said:
 "It seems, the java-operator-sdk handles the changes of the .metadata and
.spec fields of custom resources differently."

What implications does this have on the above mentioned 2 approaches? Does
it make one more difficult than the other?

Cheers
Gyula



On Tue, Mar 1, 2022 at 1:52 PM Őrhidi Mátyás 
wrote:

> Hi All!
>
> I'd like to start a quick discussion about the way we allow users to
> trigger savepoints manually in the operator [FLINK-26181]
> . There are existing
> solutions already for this functionality in other operators, for example:
> - counter based
> <
> https://github.com/spotify/flink-on-k8s-operator/blob/master/docs/savepoints_guide.md#2-taking-savepoints-by-updating-the-flinkcluster-custom-resource
> >
> - annotation based
> <
> https://github.com/spotify/flink-on-k8s-operator/blob/master/docs/savepoints_guide.md#3-taking-savepoints-by-attaching-annotation-to-the-flinkcluster-custom-resource
> >
>
> We could implement any of these or both or come up with our own approach.
> It seems, the java-operator-sdk handles the changes of the .metadata and
> .spec fields of custom resources differently. For further info see the
> chapter Generation Awareness and Event Filtering in the docs
> .
>
> Let me know what you think.
>
> Cheers,
> Matyas
>


Re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread Jingsong Li
+1.

Thanks for driving.

I wrote some scala code, the style of our flink's scala is messy. We
can do better.

Best,
Jingsong

On Wed, Mar 2, 2022 at 4:19 PM Yun Tang  wrote:
>
> +1
>
> I also noticed that the project of scalafmt [1] is much more active than 
> scalatyle [2], which has no release in the past 4 years.
>
>
> [1] https://github.com/scalameta/scalafmt/releases
> [2] https://github.com/scalastyle/scalastyle/tags
>
> Best
> Yun Tang
>
> 
> From: Konstantin Knauf 
> Sent: Wednesday, March 2, 2022 15:01
> To: dev 
> Subject: Re: [DISCUSS] Enable scala formatting check
>
> +1 I've never written any Scala in Flink, but this makes a lot of sense to
> me. Converging on a smaller set of tools and simplifying the build is
> always a good idea and the Community already concluded before that spotless
> is generally a good approach.
>
> On Tue, Mar 1, 2022 at 5:52 PM Francesco Guardiani 
> wrote:
>
> > Hi all,
> >
> > I want to propose to enable the spotless scalafmt integration and remove
> > the scalastyle plugin.
> >
> > From an initial analysis, scalafmt can do everything scalastyle can do, and
> > the integration with spotless looks easy to enable:
> > https://github.com/diffplug/spotless/tree/main/plugin-maven#scala. The
> > scalafmt conf file gets picked up automatically from every IDE, and it can
> > be heavily tuned.
> >
> > This way we can unify the formatting and integrate with our CI without any
> > additional configurations. And we won't need scalastyle anymore, as
> > scalafmt will take care of the checks:
> >
> > * mvn spotless:check will check both java and scala
> > * mvn spotless:apply will format both java and scala
> >
> > WDYT?
> >
> > FG
> >
> >
> >
> > --
> >
> > Francesco Guardiani | Software Engineer
> >
> > france...@ververica.com
> >
> >
> > 
> >
> > Follow us @VervericaData
> >
> > --
> >
> > Join Flink Forward  - The Apache Flink
> > Conference
> >
> > Stream Processing | Event Driven | Real Time
> >
> > --
> >
> > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
> >
> > --
> >
> > Ververica GmbH
> >
> > Registered at Amtsgericht Charlottenburg: HRB 158244 B
> >
> > Managing Directors: Karl Anton Wehner, Holger Temme, Yip Park Tung Jason,
> > Jinwei (Kevin) Zhang
> >
>
>
> --
>
> Konstantin Knauf
>
> https://twitter.com/snntrable
>
> https://github.com/knaufk


Re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread Yun Tang
+1

I also noticed that the project of scalafmt [1] is much more active than 
scalatyle [2], which has no release in the past 4 years.


[1] https://github.com/scalameta/scalafmt/releases
[2] https://github.com/scalastyle/scalastyle/tags

Best
Yun Tang


From: Konstantin Knauf 
Sent: Wednesday, March 2, 2022 15:01
To: dev 
Subject: Re: [DISCUSS] Enable scala formatting check

+1 I've never written any Scala in Flink, but this makes a lot of sense to
me. Converging on a smaller set of tools and simplifying the build is
always a good idea and the Community already concluded before that spotless
is generally a good approach.

On Tue, Mar 1, 2022 at 5:52 PM Francesco Guardiani 
wrote:

> Hi all,
>
> I want to propose to enable the spotless scalafmt integration and remove
> the scalastyle plugin.
>
> From an initial analysis, scalafmt can do everything scalastyle can do, and
> the integration with spotless looks easy to enable:
> https://github.com/diffplug/spotless/tree/main/plugin-maven#scala. The
> scalafmt conf file gets picked up automatically from every IDE, and it can
> be heavily tuned.
>
> This way we can unify the formatting and integrate with our CI without any
> additional configurations. And we won't need scalastyle anymore, as
> scalafmt will take care of the checks:
>
> * mvn spotless:check will check both java and scala
> * mvn spotless:apply will format both java and scala
>
> WDYT?
>
> FG
>
>
>
> --
>
> Francesco Guardiani | Software Engineer
>
> france...@ververica.com
>
>
> 
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward  - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
>
> Ververica GmbH
>
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
>
> Managing Directors: Karl Anton Wehner, Holger Temme, Yip Park Tung Jason,
> Jinwei (Kevin) Zhang
>


--

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk


Re: Change of focus

2022-03-02 Thread Jingsong Li
Hi Till,

Good luck with the next chapter, and thanks for all of your efforts.

Best,
Jingsong

On Wed, Mar 2, 2022 at 2:57 PM Yufei Zhang  wrote:
>
> Hi Till,
>
> Thank you Till, and good luck in your next chapter :)
>
> Cheers,
> Yufei
>
> On Mon, Feb 28, 2022 at 6:59 PM Till Rohrmann  wrote:
>
> > Hi everyone,
> >
> > I wanted to let you know that I will be less active in the community
> > because I’ve decided to start a new chapter in my life. Hence, please don’t
> > wonder if I might no longer be very responsive on mails and JIRA issues.
> >
> > It is great being part of such a great community with so many amazing
> > people. Over the past 7,5 years, I’ve learned a lot thanks to you and
> > together we have shaped how people think about stream processing nowadays.
> > This is something we can be very proud of. I am sure that the community
> > will continue innovating and setting the pace for what is possible with
> > real time processing. I wish you all godspeed!
> >
> > Cheers,
> > Till
> >


Re: [DISCUSS] Enable scala formatting check

2022-03-02 Thread Konstantin Knauf
+1 I've never written any Scala in Flink, but this makes a lot of sense to
me. Converging on a smaller set of tools and simplifying the build is
always a good idea and the Community already concluded before that spotless
is generally a good approach.

On Tue, Mar 1, 2022 at 5:52 PM Francesco Guardiani 
wrote:

> Hi all,
>
> I want to propose to enable the spotless scalafmt integration and remove
> the scalastyle plugin.
>
> From an initial analysis, scalafmt can do everything scalastyle can do, and
> the integration with spotless looks easy to enable:
> https://github.com/diffplug/spotless/tree/main/plugin-maven#scala. The
> scalafmt conf file gets picked up automatically from every IDE, and it can
> be heavily tuned.
>
> This way we can unify the formatting and integrate with our CI without any
> additional configurations. And we won't need scalastyle anymore, as
> scalafmt will take care of the checks:
>
> * mvn spotless:check will check both java and scala
> * mvn spotless:apply will format both java and scala
>
> WDYT?
>
> FG
>
>
>
> --
>
> Francesco Guardiani | Software Engineer
>
> france...@ververica.com
>
>
> 
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward  - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
>
> Ververica GmbH
>
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
>
> Managing Directors: Karl Anton Wehner, Holger Temme, Yip Park Tung Jason,
> Jinwei (Kevin) Zhang
>


-- 

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk