Re: [DISCUSS] Deprecate SourceFunction APIs

2023-07-04 Thread Flavio Pompermaier
Hi all, I've tried to migrate my very simple Elasticsearch SourceFunction (that use scroll API and produce batch of documents) to new Source API, but I gave up because it's too complicated. It should much simpler to migrate that function to a bounded or unbounded source. Before removing

Re: Change of focus

2022-02-28 Thread Flavio Pompermaier
Good luck for your new adventure Till! On Mon, Feb 28, 2022 at 12:00 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

Re: [DISCUSS] Update Policy for old releases

2021-11-16 Thread Flavio Pompermaier
Hi to all, I'd like to point out that also official downstream-projects as CDC connector's documentation should be updated to reflect compatibility with new Flink releases[1]. The aforementioned link for example doesn't say nothing about which version is compatible with Flink 1.14 [1]

Re: Is development in FlinkML still active?

2021-01-07 Thread Flavio Pompermaier
Or also https://github.com/alibaba/Alink, I don't know if the 2 are related somehow.. On Thu, Jan 7, 2021 at 9:55 AM Flavio Pompermaier wrote: > What about Flink-AI [1]? Would you suggest its adoption Till? > > [1] https://github.com/alibaba/flink-ai-extended > > On Thu, Jan 7,

Re: Is development in FlinkML still active?

2021-01-07 Thread Flavio Pompermaier
What about Flink-AI [1]? Would you suggest its adoption Till? [1] https://github.com/alibaba/flink-ai-extended On Thu, Jan 7, 2021 at 9:38 AM Till Rohrmann wrote: > HI Badrul, > > FlinkML is unfortunately no longer under active development. However, there > is some new effort to add a machine

Re: [ANNOUNCE] New formatting rules are now in effect

2020-12-29 Thread Flavio Pompermaier
Thanks Aljoscha and Chesnay for this small but important improvement! In the new year writing new Flink features will be funnier than ever ;) On Tue, Dec 29, 2020 at 9:58 AM Till Rohrmann wrote: > Thanks a lot for this effort Aljoscha and Chesnay! Finally we have a common > code style :-) > >

Re: [DISCUSS] Programmatically submit Flink job jar to session cluster

2020-12-10 Thread Flavio Pompermaier
To me creating the PackagedProgram on the client side is very bad, at least for 2 things: 1. You must ensure to have almost the same classpath of the Flink cluster otherwise you can face problems in deserializing the submitted job graph (for example jackson automatically tries to create modules

[jira] [Created] (FLINK-20004) UpperLimitExceptionParameter description is misleading

2020-11-05 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-20004: -- Summary: UpperLimitExceptionParameter description is misleading Key: FLINK-20004 URL: https://issues.apache.org/jira/browse/FLINK-20004 Project: Flink

[jira] [Created] (FLINK-19969) CliFrontendParser does not provide any help for run-application

2020-11-04 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-19969: -- Summary: CliFrontendParser does not provide any help for run-application Key: FLINK-19969 URL: https://issues.apache.org/jira/browse/FLINK-19969 Project

Re: [DISCUSS] Enforce common opinionated coding style using Spotless

2020-10-06 Thread Flavio Pompermaier
Hi Aljoscha, I think that having the style check directly in the IDE is a very good feature so +1 on my side as a contributor (I also asked once on the mailing list if there was already something like that)..I never used Spotless so I can't say if it easy to integrate with the IDE but the nice

Re: [DISCUSS] FLIP-146: Improve new TableSource and TableSink interfaces

2020-09-24 Thread Flavio Pompermaier
Hi Kurt, in the past we had a very interesting use case in this regard: our customer (oracle) db was quite big and running too many queries in parallel was too heavy and it was causing the queries to fail. So we had to limit the source parallelism to 2 threads. After the fetching of the data the

Re: [DISCUSS] FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API)

2020-07-30 Thread Flavio Pompermaier
ing > `runCustomOperation()`. Out of curiosity, what are you using that for? > > We have definitely thought about the first two points you mentioned, > though. Especially processing-time will make it tricky to define unified > execution semantics. > > Best, > Aljoscha > > On 30.07.2

Re: [DISCUSS] FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API)

2020-07-30 Thread Flavio Pompermaier
hat it works > well on bounded input. > > On Thu, Jul 30, 2020 at 8:49 AM Flavio Pompermaier > wrote: > >> Just to contribute to the discussion, when we tried to do the migration we >> faced some problems that could make migration quite difficult. >> 1

Re: [DISCUSS] FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API)

2020-07-30 Thread Flavio Pompermaier
Just to contribute to the discussion, when we tried to do the migration we faced some problems that could make migration quite difficult. 1 - It's difficult to test because of https://issues.apache.org/jira/browse/FLINK-18647 2 - missing mapPartition 3 - missing DataSet

Re: PackagedProgram and ProgramDescription

2020-07-15 Thread Flavio Pompermaier
ct at > submission time and put them into the parameters. > My guess would be that such approach can work sufficiently for all > dataset/datastream/table API users. > > On 15/07/2020 14:05, Flavio Pompermaier wrote: > > Ok, it's not a problem for me if the community is not interested in &g

[jira] [Created] (FLINK-18608) CustomizedConvertRule#convertCast drops nullability

2020-07-15 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-18608: -- Summary: CustomizedConvertRule#convertCast drops nullability Key: FLINK-18608 URL: https://issues.apache.org/jira/browse/FLINK-18608 Project: Flink

Re: PackagedProgram and ProgramDescription

2020-07-15 Thread Flavio Pompermaier
orward. > > Best, > Aljoscha > > On Mon, Mar 30, 2020, at 12:45, Flavio Pompermaier wrote: > > I would personally like to see a way of describing a Flink job/pipeline > > (including its parameters and types) in order to enable better UIs, then > > the important thing i

Re: [DISCUSS] Semantics of our JIRA fields

2020-05-23 Thread Flavio Pompermaier
In my experience it's quite complicated for a normal reporter to be able to fill all the fields correctly (especially for new users). Usually you just wanto to report a problem, remember to add a new feature or improve code/documentation but you can't really give a priority, assign the correct

[jira] [Created] (FLINK-17622) Remove useless switch for decimal in PostresCatalog

2020-05-11 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17622: -- Summary: Remove useless switch for decimal in PostresCatalog Key: FLINK-17622 URL: https://issues.apache.org/jira/browse/FLINK-17622 Project: Flink

[jira] [Created] (FLINK-17509) Support OracleDialect

2020-05-04 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17509: -- Summary: Support OracleDialect Key: FLINK-17509 URL: https://issues.apache.org/jira/browse/FLINK-17509 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-17508) Develop OracleCatalog

2020-05-04 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17508: -- Summary: Develop OracleCatalog Key: FLINK-17508 URL: https://issues.apache.org/jira/browse/FLINK-17508 Project: Flink Issue Type: Sub-task

Re: [DISCUSS]Refactor flink-jdbc connector structure

2020-04-30 Thread Flavio Pompermaier
Very big +1 from me Best, Flavio On Thu, Apr 30, 2020 at 4:47 PM David Anderson wrote: > I'm very happy to see the jdbc connector being normalized in this way. +1 > from me. > > David > > On Thu, Apr 30, 2020 at 2:14 PM Timo Walther wrote: > > > Hi Leonard, > > > > this sounds like a nice

Re: [DISCUSS] Hierarchies in ConfigOption

2020-04-29 Thread Flavio Pompermaier
Personally I don't have any preference here. Compliance wih standard YAML parser is probably more important On Wed, Apr 29, 2020 at 5:10 PM Jark Wu wrote: > From a user's perspective, I prefer the shorter one "format=json", because > it's more concise and straightforward. The "kind" is

Re: Integration of DataSketches into Flink

2020-04-27 Thread Flavio Pompermaier
If this can encourage Lee I'm one of the Flink users that already use datasketches and I found it an amazing library. When I was trying it out (lat year) I tried to stimulate some discussion[1] but at that time it was probably too early.. I really hope that now things are mature for both

Re: [DISCUSS] Should max/min be part of the hierarchy of config option?

2020-04-27 Thread Flavio Pompermaier
+1 for Chesnay approach On Mon, Apr 27, 2020 at 2:31 PM Chesnay Schepler wrote: > +1 for xyz.[min|max]; imo it becomes obvious if think of it like a yaml > file: > > xyz: > min: > max: > > opposed to > > min-xyz: > max-xyz: > > IIRC this would also be more in-line with the

[jira] [Created] (FLINK-17366) Implement listViews

2020-04-24 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17366: -- Summary: Implement listViews Key: FLINK-17366 URL: https://issues.apache.org/jira/browse/FLINK-17366 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-17361) Support creating of a JDBC table using a custom query

2020-04-23 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17361: -- Summary: Support creating of a JDBC table using a custom query Key: FLINK-17361 URL: https://issues.apache.org/jira/browse/FLINK-17361 Project: Flink

[jira] [Created] (FLINK-17360) Support custom partitioners in JDBCReadOptions

2020-04-23 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17360: -- Summary: Support custom partitioners in JDBCReadOptions Key: FLINK-17360 URL: https://issues.apache.org/jira/browse/FLINK-17360 Project: Flink

[jira] [Created] (FLINK-17358) JDBCTableSource support FiltertableTableSource

2020-04-23 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17358: -- Summary: JDBCTableSource support FiltertableTableSource Key: FLINK-17358 URL: https://issues.apache.org/jira/browse/FLINK-17358 Project: Flink

[jira] [Created] (FLINK-17357) add "DROP catalog" DDL to blink planner

2020-04-23 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17357: -- Summary: add "DROP catalog" DDL to blink planner Key: FLINK-17357 URL: https://issues.apache.org/jira/browse/FLINK-17357 Project: Flink

[jira] [Created] (FLINK-17356) Properly set constraints (PK and UNIQUE)

2020-04-23 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17356: -- Summary: Properly set constraints (PK and UNIQUE) Key: FLINK-17356 URL: https://issues.apache.org/jira/browse/FLINK-17356 Project: Flink Issue

[jira] [Created] (FLINK-17284) Support serial field type in PostgresCatalog

2020-04-20 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-17284: -- Summary: Support serial field type in PostgresCatalog Key: FLINK-17284 URL: https://issues.apache.org/jira/browse/FLINK-17284 Project: Flink

PackagedProgram and ProgramDescription

2020-03-18 Thread Flavio Pompermaier
Hi all, what do you think if we exploit this job-submission sprint to address also the problem discussed in https://issues.apache.org/jira/browse/FLINK-10862? Best, Flavio

FLIP-117: HBase catalog

2020-03-13 Thread Flavio Pompermaier
Hello everybody, I started a new FLIP to discuss about an HBaseCatalog implementation[1] after the opening of the relative issue by Bowen [2]. I drafted a very simple version of the FLIP just to discuss about the critical points (in red) in order to decide how to proceed. Best, Flavio [1]

Re: [VOTE] [FLIP-85] Flink Application Mode

2020-03-12 Thread Flavio Pompermaier
+1 (non-binding). There's also a related issue that I opened a long time ago https://issues.apache.org/jira/browse/FLINK-10879 that could be closed once implemented this FLIP (or closed immediately and referenced as duplicated by the new JIRA ticket that would be created On Thu, Mar 12, 2020 at

Re: [DISCUSS] Remove Eclipse-specific plugin configurations

2020-03-03 Thread Flavio Pompermaier
Yes, in my experience.. I always asked myself if I was the only one using Eclipse.. :D On Tue, Mar 3, 2020 at 2:33 PM Chesnay Schepler wrote: > To clarify, the whole lifecycle-mapping business is both unnecessary and > actively harmful? > > On 03/03/2020 14:18, Flavio Pompe

Re: [DISCUSS] Remove Eclipse-specific plugin configurations

2020-03-03 Thread Flavio Pompermaier
etypes)? > > On Fri, Feb 28, 2020 at 4:10 PM Chesnay Schepler > wrote: > >> What do you have to change it to? >> >> What happens if you just remove it completely? >> >> On 28/02/2020 16:08, Flavio Pompermaier wrote: >> > I use Eclipse but the stuf

Re: Flink dev blog

2020-03-03 Thread Flavio Pompermaier
Big +1 from my side. I'd be very interested in what Jeff proposed, in particular everything related to client part (job submission, workflow management, callbacks on submission/success/failure, etc). Something I can't find anywhere is also how to query Flink states..would it be possible to have

Re: [DISCUSS] Remove Eclipse-specific plugin configurations

2020-02-28 Thread Flavio Pompermaier
I use Eclipse but the stuff added in the pom.xml to improve the out-of-the-box experience is pretty useless, I always have to change it On Fri, Feb 28, 2020 at 4:01 PM Chesnay Schepler wrote: > Hello, > > in various maven pom.xml we have some plugin definitions exclusively to > increase support

Re: [DISCUSS] Drop connectors for Elasticsearch 2.x and 5.x

2020-02-10 Thread Flavio Pompermaier
+1 for dropping all Elasticsearch connectors < 6.x On Mon, Feb 10, 2020 at 2:45 PM Dawid Wysakowicz wrote: > Hi all, > > As described in this https://issues.apache.org/jira/browse/FLINK-11720 > ticket our elasticsearch 5.x connector does not work out of the box on > some systems and requires a

Re: [Discussion] Job generation / submission hooks & Atlas integration

2020-02-05 Thread Flavio Pompermaier
Hi Gyula, thanks for taking care of integrating Flink with Atlas (and Egeria initiative in the end) that is IMHO the most important part of all the Hadoop ecosystem and that, unfortunately, was quite overlooked. I can confirm that the integration with Atlas/Egeria is absolutely of big interest.

Re: [DISCUSS] FLIP-92: JDBC catalog and Postgres catalog

2020-01-21 Thread Flavio Pompermaier
Hi all, I'm happy to see a lot of interest in easing the integration with JDBC data sources. Maybe this could be a rare situation (not in my experience however..) but what if I have to connect to the same type of source (e.g. Mysql) with 2 incompatible version...? How can I load the 2 (or more)

Re: [jira] [Created] (FLINK-15644) Add support for SQL query validation

2020-01-19 Thread Flavio Pompermaier
Ok thanks for the pointer, I wasn't awareof that! Il Dom 19 Gen 2020, 03:00 godfrey he ha scritto: > hi Flavio, TableEnvironment.getCompletionHints maybe already meet the > requirement. > > Flavio Pompermaier 于2020年1月18日周六 下午3:39写道: > > > Why not adding also a s

Re: [jira] [Created] (FLINK-15644) Add support for SQL query validation

2020-01-17 Thread Flavio Pompermaier
Why not adding also a suggest() method (also unimplemented initially) that would return the list of suitable completions/tokens on the current query? How complex eould it be to implement it in you opinion? Il Ven 17 Gen 2020, 18:32 Fabian Hueske (Jira) ha scritto: > Fabian Hueske created

Re: [DISCUSS] Remove old WebUI

2019-11-22 Thread Flavio Pompermaier
+1 to drop the old UI On Thu, Nov 21, 2019 at 1:05 PM Chesnay Schepler wrote: > Hello everyone, > > Flink 1.9 shipped with a new UI, with the old one being kept around as a > backup in case something wasn't working as expected. > > Currently there are no issues indicating any significant

Re: [DISCUSS] Semantic and implementation of per-job mode

2019-10-31 Thread Flavio Pompermaier
Hi all, we're using a lot the multiple jobs in one program and this is why: when you fetch data from a huge number of sources and, for each source, you do some transformation and then you want to write into a single directory the union of all outputs (this assumes you're doing batch). When the

Re: [DISCUSS] Stateful Functions - in which form to contribute? (same or different repository)

2019-10-15 Thread Flavio Pompermaier
Definitely on the same page..+1 to keep it in a separate repo (at least until the cose becomes "stable" and widely adopted from the community) Il Mar 15 Ott 2019, 23:17 Stephan Ewen ha scritto: > Hi Flink folks! > > After the positive reaction to the contribution proposal for Stateful >

Re: [PROPOSAL] Contribute Stateful Functions to Apache Flink

2019-10-14 Thread Flavio Pompermaier
I'm obviously pro about promoting the usage of this amazing library but, maybe, in this early stage I'd try to keep it as a separate project. However, this really depends about how frequently the code is goin to change..the Flink main repo is becoming more and more complex to handle due to the

Re: [DISCUSS] FLIP-74: Flink JobClient API

2019-09-27 Thread Flavio Pompermaier
Hi all, just a remark about the Flink REST APIs (and its client as well): almost all the times we need a way to dynamically know which jobs are contained in a jar file, and this could be exposed by the REST endpoint under /jars/:jarid/entry-points (a simple way to implement this would be to check

[jira] [Created] (FLINK-14209) Add jackson-dataformat-xml to flink-shaded

2019-09-25 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-14209: -- Summary: Add jackson-dataformat-xml to flink-shaded Key: FLINK-14209 URL: https://issues.apache.org/jira/browse/FLINK-14209 Project: Flink Issue

Re: [DISCUSS] Flink client api enhancement for downstream project

2019-08-20 Thread Flavio Pompermaier
google.com/document/d/1E-8UjOLz4QPUTxetGWbU23OlsIH9VIdodpTsxwoQTs0/edit#heading=h.na7k0ad88tix > >> < > >> > https://docs.google.com/document/d/1E-8UjOLz4QPUTxetGWbU23OlsIH9VIdodpTsxwoQTs0/edit#heading=h.na7k0ad88tix > >> > > >> > >> > On 31. Jul 2019,

Re: [DISCUSS] Flink client api enhancement for downstream project

2019-07-31 Thread Flavio Pompermaier
reeze 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 do some step towards this in Flin

Re: [DISCUSS] Drop stale class Program

2019-07-19 Thread Flavio Pompermaier
l 19, 2019 at 11:14 AM Biao Liu wrote: > To Flavio, good point for the integration suggestion. > > I think it should be considered in the "Flink client api enhancement" > discussion. But the outdated API should be deprecated somehow. > > Flavio Pompermaier 于2019年7月

Re: [DISCUSS] Drop stale class Program

2019-07-19 Thread Flavio Pompermaier
In my experience a basic "official" (but optional) program description would be very useful indeed (in order to ease the integration with other frameworks). Of course it should be extended and integrated with the REST services and the Web UI (when defined) in order to be useful.. It ease to show

Re: [DISCUSS] Flink client api enhancement for downstream project

2019-06-24 Thread Flavio Pompermaier
; > >> I'd like to write down a few of notes on configs/args pass and respect, > >> as well as decoupling job compilation and submission. Share on this > >> thread later. > >> > >> Best, > >> tison. > >> > >> > >> SHI Xiaoga

Re: [DISCUSS] Flink client api enhancement for downstream project

2019-06-17 Thread Flavio Pompermaier
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

Re: [Discuss] Add JobListener (hook) in flink job lifecycle

2019-05-09 Thread Flavio Pompermaier
Hi everybody, any news on this? For us would be VERY helpful to have such a feature because we need to execute a call to a REST service once a job ends. Right now we do this after the env.execute() but this works only if the job is submitted via the CLI client, the REST client doesn't execute

Re: [DISCUSS] FLIP-39: Flink ML pipeline and ML libs

2019-05-02 Thread Flavio Pompermaier
Hi to all, I have read many discussion about Flink ML and none of them take into account the ongoing efforts carried out of by the Streamline H2020 project [1] on this topic. Have you tried to ping them? I think that both projects could benefits from a joined effort on this side.. [1]

Re: [Discuss] Add support for Apache Arrow

2019-04-11 Thread Flavio Pompermaier
Very BIG +1 for adoption of Apache Arrow. This would simplify a lot the integration with other tools On Thu, Apr 11, 2019 at 2:21 PM Run wrote: > Hi guys, > > > Apache Arrow provides a cross-language, standardized, columnar, memory > format for data. > So it is highly desirable to import Arrow

Re: Row.setField returning row itself

2019-03-22 Thread Flavio Pompermaier
be convenient but please find a better > > example; yours can be solved easily using "Row.of(value)". > > > > On 22/03/2019 12:26, Flavio Pompermaier wrote: > >> Hi all, > >> many times I had the feeling that allowing Row.setField() to return the

Row.setField returning row itself

2019-03-22 Thread Flavio Pompermaier
Hi all, many times I had the feeling that allowing Row.setField() to return the modified object instead of void would really make the (Java) code cleaner in a very unobtrusive way. For example, I could write something like: DataSet columnData = input.map(value -> new Row(1).setField(0, value))

[jira] [Created] (FLINK-11852) Improve Processing function example

2019-03-07 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-11852: -- Summary: Improve Processing function example Key: FLINK-11852 URL: https://issues.apache.org/jira/browse/FLINK-11852 Project: Flink Issue Type

Re: Some dev questions

2019-03-01 Thread Flavio Pompermaier
You're right Chesnay, I did a git fetch and I didn't remember that it doesn't update the tag list.. After a git fetch --tags --all I was able to find the tags. It would be nice to add this info into the "build from source" documentation [1]: the page states "This page covers how to build Flink

Re: Dataset rowCount accumulator

2019-02-04 Thread Flavio Pompermaier
ccumulator directly in the input formats? On Mon, Feb 4, 2019 at 10:18 AM Flavio Pompermaier wrote: > Hi to all, > we often need to track the number of rows of a dataset. > In order to burden on the job complexitye we use accumulators to track > this information. > The problem is th

Dataset rowCount accumulator

2019-02-04 Thread Flavio Pompermaier
Hi to all, we often need to track the number of rows of a dataset. In order to burden on the job complexitye we use accumulators to track this information. The problem is that we have to extends all InputFormats that we use in order to properly handle such row-count accumulator...my question is:

Re: [DISCUSS] Support Interactive Programming in Flink Table API

2018-11-26 Thread Flavio Pompermaier
What about to add also Apache Plasma + Arrow as an alternative to Apache Ignite? [1] https://arrow.apache.org/blog/2017/08/08/plasma-in-memory-object-store/ On Mon, Nov 26, 2018 at 11:56 AM Fabian Hueske wrote: > Hi, > > Thanks for the proposal! > > To summarize, you propose a new method

Re: JIRA notifications

2018-11-21 Thread Flavio Pompermaier
+1 On Wed, Nov 21, 2018 at 12:05 PM Saar Bar wrote: >  agree > > Sent from my iPhone > > > On 21 Nov 2018, at 13:03, Maximilian Michels wrote: > > > > Hi! > > > > Do you think it would make sense to send JIRA notifications to a > separate mailing list? Some people just want to casually follow

[jira] [Created] (FLINK-10947) Document handling of null keys in the data types documentation

2018-11-20 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-10947: -- Summary: Document handling of null keys in the data types documentation Key: FLINK-10947 URL: https://issues.apache.org/jira/browse/FLINK-10947 Project

Re: REST job submission

2018-11-14 Thread Flavio Pompermaier
ch please open a JIRA. > > On 12.11.2018 12:50, Flavio Pompermaier wrote: > > Hi to all, > > in our ETL we need to call an external (REST) service once a job ends: we > > extract informations about accumulators and we update the job status. > > However this is only possi

[jira] [Created] (FLINK-10879) Align Flink clients on env.execute()

2018-11-14 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-10879: -- Summary: Align Flink clients on env.execute() Key: FLINK-10879 URL: https://issues.apache.org/jira/browse/FLINK-10879 Project: Flink Issue Type

[jira] [Created] (FLINK-10864) Support multiple Main classes per jar

2018-11-13 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-10864: -- Summary: Support multiple Main classes per jar Key: FLINK-10864 URL: https://issues.apache.org/jira/browse/FLINK-10864 Project: Flink Issue Type

[jira] [Created] (FLINK-10862) REST API does not show program descriptions of "simple" ProgramDescription

2018-11-13 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-10862: -- Summary: REST API does not show program descriptions of "simple" ProgramDescription Key: FLINK-10862 URL: https://issues.apache.org/jira/browse/F

REST job submission

2018-11-12 Thread Flavio Pompermaier
Hi to all, in our ETL we need to call an external (REST) service once a job ends: we extract informations about accumulators and we update the job status. However this is only possible if using the CLI client: if we call the job via the REST API o Web UI (that is very useful to decouple our UI

[jira] [Created] (FLINK-10795) STDDEV_POP error

2018-11-05 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-10795: -- Summary: STDDEV_POP error Key: FLINK-10795 URL: https://issues.apache.org/jira/browse/FLINK-10795 Project: Flink Issue Type: Improvement

Re: [DISCUSS] Change underlying Frontend Architecture for Flink Web Dashboard

2018-10-31 Thread Flavio Pompermaier
I think that it is important to have a nice "official" (or at least free) Flink UI, we use it to see the detail of the jobs. It's very useful for people starting working with Flink and also for those that does not have the resources to write a custom UI. How are you going to monitor the status of

Re: Snapshots and RC-candidate maven artifacts

2018-05-14 Thread Flavio Pompermaier
orked off > > the release-1.5 branch and updated the version on master to 1.6-SNAPSHOT. > > > > Best, Fabian > > > > 2018-05-14 15:51 GMT+02:00 Flavio Pompermaier <pomperma...@okkam.it>: > > > > > Hi to all. > > > we were trying to

Snapshots and RC-candidate maven artifacts

2018-05-14 Thread Flavio Pompermaier
Hi to all. we were trying to run a 1.5 Flink job and we set the version to 1.5-SNAPSHOT. Unfortunately the 1.5-SNAPSHOT version uploaded on the apache snapshot repo is very old (february 2018). Shouldn't be this version be updated as well? Best, Flavio

Re: Elasticsearch Sink

2018-05-12 Thread Flavio Pompermaier
+1. Torally agree On Sat, 12 May 2018, 18:14 Christophe Jolif, wrote: > Hi all, > > There is quite some time Flink Elasticsearch sink is broken for > Elastisearch 5.x (nearly a year): > > https://issues.apache.org/jira/browse/FLINK-7386 > > And there is no support for

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

2018-04-03 Thread Flavio Pompermaier
skipTests`? > > Cheers, > Till > > On Tue, Apr 3, 2018 at 10:34 AM, Flavio Pompermaier <pomperma...@okkam.it> > wrote: > > > Just one note. > > There are 2 suggested ways of compiling Flink: > > > >1. From github Readme: mvn clean package -DskipTests ( &

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

2018-04-03 Thread Flavio Pompermaier
; > > https://repository.apache.org/content/repositories/orgapacheflink-1154/ > > > > archetype > > > https://repository.apache.org/content/repositories/orgapacheflink-1154/ > > > > > > > > And reference that in you maven commands via --settings > path/to/settings.xml. This is useful for creating a quickstart based on the > staged release and for building against the staged jars. > -- Flavio Pompermaier Development Department OKKAM S.r.l. Tel. +(39) 0461 041809

Re: [jira] [Created] (FLINK-8356) JDBCAppendTableSink does not work for Hbase Phoenix Driver

2018-01-03 Thread Flavio Pompermaier
I had a similar problem with batch API...the problem is that you have to enable autocommit in the connection URL. Thr jdbc connector should better handle this specific case as well (IMHO). See https://issues.apache.org/jira/browse/FLINK-7605 On 3 Jan 2018 22:25, "Paul Wu (JIRA)"

[jira] [Created] (FLINK-7930) Support periodic jobs with state that gets restored and persisted in a savepoint

2017-10-26 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-7930: - Summary: Support periodic jobs with state that gets restored and persisted in a savepoint Key: FLINK-7930 URL: https://issues.apache.org/jira/browse/FLINK-7930

[jira] [Created] (FLINK-7845) Netty Exception when submitting batch job repeatedly

2017-10-16 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-7845: - Summary: Netty Exception when submitting batch job repeatedly Key: FLINK-7845 URL: https://issues.apache.org/jira/browse/FLINK-7845 Project: Flink

[jira] [Created] (FLINK-7605) JDBCOutputFormat autoCommit

2017-09-08 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-7605: - Summary: JDBCOutputFormat autoCommit Key: FLINK-7605 URL: https://issues.apache.org/jira/browse/FLINK-7605 Project: Flink Issue Type: Improvement

Re: [POLL] Who still uses Java 7 with Flink ?

2017-06-08 Thread Flavio Pompermaier
We're not using Java 7 anymore from last year already. So +1 to upgrade to java 8 from Okkam On 8 Jun 2017 5:39 pm, "Robert Metzger" wrote: > Hi all, > > as promised in March, I want to revive this discussion! > > Our users are begging for Scala 2.12 support [1], migration

Operator name

2017-05-12 Thread Flavio Pompermaier
ction") Wouldn't be more convinient to allow an operator to override the default name (at least for rich ones) by adding to *RichFunction* something like - public abstract String getOperatorName() What do you think? Best, Flavio -- Flavio Pompermaier Development Department OKKAM S.r.l.

Re: [DISCUSS] Feature Freeze

2017-04-28 Thread Flavio Pompermaier
has been merged into master so it will be > release in Flink 1.3. > > > On 28. Apr 2017, at 13:50, Flavio Pompermaier <pomperma...@okkam.it> > wrote: > > > > Sorry, you're right Aljosha..the issue number is correct, the link is > > wrong! The correct one is htt

Re: [DISCUSS] Feature Freeze

2017-04-28 Thread Flavio Pompermaier
ue number 6389, > if I’m not mistaken. The latest as I’m writing this is 6410. > > > On 28. Apr 2017, at 10:00, Flavio Pompermaier <pomperma...@okkam.it> > wrote: > > > > If it's not a problem it will be great for us to include also FLINK-6398 > > <https:

Re: [DISCUSS] Feature Freeze

2017-04-28 Thread Flavio Pompermaier
If it's not a problem it will be great for us to include also FLINK-6398 if it's not a big deal Best, Flavio On Fri, Apr 28, 2017 at 3:32 AM, Zhuoluo Yang wrote: > Hi Devs, > > Thanks for the release plan. > >

[jira] [Created] (FLINK-6271) NumericBetweenParametersProvider NullPinter

2017-04-06 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-6271: - Summary: NumericBetweenParametersProvider NullPinter Key: FLINK-6271 URL: https://issues.apache.org/jira/browse/FLINK-6271 Project: Flink Issue

Re: [VOTE] Release Apache Flink 1.2.1 (RC1)

2017-04-04 Thread Flavio Pompermaier
gt;>>>>> >>>>>>>>>>>>>>>> release would introduce a new bug. >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> On Tue, Mar 28, 2017 at 11:59 PM, Haohui Mai < >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>> ricet...@gmail.com> >>>>>>>>>>> >>>>>>>>>>>> wrote: >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> -1 (non-binding) >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> We recently found out that all jobs submitted via UI will >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> have a >>>>>>>>>>> >>>>>>>>>>>> parallelism of 1, potentially due to FLINK-5808. >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> Filed FLINK-6209 to track it. >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> ~Haohui >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> On Mon, Mar 27, 2017 at 2:59 AM Chesnay Schepler < >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> ches...@apache.org> >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> wrote: >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> If possible I would like to include FLINK-6183 & >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> FLINK-6184 >>> >>>> as >>>>>>>>>>> >>>>>>>>>>>> well. >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> They fix 2 metric-related issues that could arise when a >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> Task is >>>>>>>>>>> >>>>>>>>>>>> cancelled very early. (like, right away) >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> FLINK-6183 fixes a memory leak where the TaskMetricGroup >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> was >>>>>>> >>>>>>>> never closed >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> FLINK-6184 fixes a NullPointerExceptions in the buffer >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> metrics >>>>>>>>>>> >>>>>>>>>>>> PR here: https://github.com/apache/flink/pull/3611 >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> On 26.03.2017 12:35, Aljoscha Krettek wrote: >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> I opened a PR for FLINK-6188: >>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> https://github.com/apache/ >>> >>>> flink/pull/3616 >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> <https://github.com/apache/flink/pull/3616> >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> This improves the previously very sparse test coverage >>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> for >>> >>>> timestamp/watermark assigners and fixes the bug. >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> On 25 Mar 2017, at 10:22, Ufuk Celebi <u...@apache.org> >>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> wrote: >>>>>>>>>>> >>>>>>>>>>>> I agree with Aljoscha. >>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> -1 because of FLINK-6188 >>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 25, 2017 at 9:38 AM, Aljoscha Krettek < >>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> aljos...@apache.org> >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> wrote: >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> I filed this issue, which was observed by a user: >>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-6188 >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> I think that’s blocking for 1.2.1. >>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> On 24 Mar 2017, at 18:57, Ufuk Celebi < >>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> u...@apache.org> >>> >>>> wrote: >>>>>>>>>>>>> >>>>>>>>>>>>>> RC1 doesn't contain Stefan's backport for the >>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> Asynchronous >>>>>>>>>>> >>>>>>>>>>>> snapshots >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> for heap-based keyed state that has been merged. >>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> Should >>> >>>> we >>>>>>>>>>> >>>>>>>>>>>> create >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> RC2 >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> with that fix since the voting period only starts on >>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> Monday? >>>>>>>>>>> >>>>>>>>>>>> I think >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> it would only mean rerunning the scripts on your >>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> side, >>> >>>> right? >>>>>>>>>>>>> >>>>>>>>>>>>>> – Ufuk >>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> On Fri, Mar 24, 2017 at 3:05 PM, Robert Metzger < >>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> rmetz...@apache.org> >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> wrote: >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> Dear Flink community, >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>> Please vote on releasing the following candidate as >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> Apache >>>>>>>>>>> >>>>>>>>>>>> Flink >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> version 1.2 >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> .1. >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>> The commit to be voted on: >>>>>>>>>>>>>>>>>>>>>>>>> *732e55bd* (* >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> http://git-wip-us.apache.org/r >>>>>>>>>>>>>>>>>>>> epos/asf/flink/commit/ >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> 732e55bd >>>>>>>>>>> >>>>>>>>>>>> <http://git-wip-us.apache.org/ >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> repos/asf/flink/commit/732e55b >>>>>>>>>>>>> >>>>>>>>>>>>>> d>*) >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> Branch: >>>>>>>>>>>>>>>>>>>>>>>>> release-1.2.1-rc1 >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>> The release artifacts to be voted on can be found >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> at: >>> >>>> *http://people.apache.org/~ >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> rmetzger/flink-1.2.1-rc1/ >>> >>>> <http://people.apache.org/~ >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> rmetzger/flink-1.2.1-rc1/ >>> >>>> * >>>>>>>> >>>>>>>>> The release artifacts are signed with the key with >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> fingerprint >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> D9839159: >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> http://www.apache.org/dist/flink/KEYS >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>> The staging repository for this release can be >>>>>>>>>>>>>>>>>>>>>>>>> found >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> at: >>>>>>>>>>> >>>>>>>>>>>> https://repository.apache.org/ >>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> content/repositories/orgapache >>>>>>>>>>> >>>>>>>>>>>> flink-1116 >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> -- >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> -- >>>>>>>>>>>>> >>>>>>>>>>>>>> - >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>> The vote ends on Wednesday, March 29, 2017, 3pm >>>>>>>>>>>>>>>>>>>>>>>>> CET. >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>> [ ] +1 Release this package as Apache Flink 1.2.1 >>>>>>>>>>>>>>>>>>>>>>>>> [ ] -1 Do not release this package, because ... >>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>> >>>>>>>>> >>>>> >>> > -- Flavio Pompermaier Development Department OKKAM S.r.l. Tel. +(39) 0461 1823908

Re: [DISCUSS] Release Flink 1.1.5 / Flink 1.2.1

2017-03-17 Thread Flavio Pompermaier
I propose to fix https://issues.apache.org/jira/browse/FLINK-6103 before issue a release On Fri, Mar 17, 2017 at 8:12 AM, Ufuk Celebi wrote: > Cool! Thanks for taking care of this Gordon :-) > > On Fri, Mar 17, 2017 at 7:13 AM, Tzu-Li (Gordon) Tai > wrote:

[jira] [Created] (FLINK-6103) LocalFileSystem rename() uses File.renameTo()

2017-03-17 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-6103: - Summary: LocalFileSystem rename() uses File.renameTo() Key: FLINK-6103 URL: https://issues.apache.org/jira/browse/FLINK-6103 Project: Flink Issue

Re: Flink CSV parsing

2017-03-10 Thread Flavio Pompermaier
tation but which is standard compliant. IMO that would be a very useful contribution. Best, Fabian [1] https://issues.apache.org/jira/browse/FLINK-6016 2017-03-10 11:28 GMT+01:00 Flavio Pompermaier <pomperma...@okkam.it>: > Hi to all, > I want to discuss with the dev group something

Flink CSV parsing

2017-03-10 Thread Flavio Pompermaier
Hi to all, I want to discuss with the dev group something about CSV parsing. Since I started using Flink with CSVs I always faced some little problem here and there and the new tickets about the CSV parsing seems to confirm that this part is still problematic. In my production jobs I gave up using

Re: ElasticsearchSink Exception

2017-02-25 Thread Flavio Pompermaier
va_2.10', > version: '1.2.0' > compile group: 'org.apache.flink', name: 'flink-java', version: '1.2.0' > compile group: 'org.apache.flink', name: > 'flink-connector-kafka-0.10_2.10', version: '1.2.0' > compile group: 'org.apache.flink', name: 'flink-clients_2.10', version: > '

Re: ElasticsearchSink Exception

2017-02-25 Thread Flavio Pompermaier
Are you sure that in elasticsearch.yml you've enabled ES to listen to the http port 9300? On 25 Feb 2017 08:58, "Govindarajan Srinivasaraghavan" < govindragh...@gmail.com> wrote: Hi All, I'm getting the below exception when I start my flink job. I have verified the elastic search host and it

[jira] [Created] (FLINK-5907) RowCsvInputFormat bug on parsing tsv

2017-02-24 Thread Flavio Pompermaier (JIRA)
Flavio Pompermaier created FLINK-5907: - Summary: RowCsvInputFormat bug on parsing tsv Key: FLINK-5907 URL: https://issues.apache.org/jira/browse/FLINK-5907 Project: Flink Issue Type: Bug

Re: Redis sink in Flink 2.0

2017-02-24 Thread Flavio Pompermaier
Hi Till, is there any guide about how to use bahir connectors with Flink? Is there any pro/cons against their use? ANy compatibility matrix somewhere? Best, Flavio On Fri, Feb 24, 2017 at 10:25 AM, Till Rohrmann wrote: > Hi Imalds, > > Flink's redis connector has been

Re: [Dev] Flink 'InputFormat' Interface execution related problem

2017-01-24 Thread Flavio Pompermaier
If your column on which you want to perform the split is numeric you can use the NumericBetweenParametersProvider interface that automatically computes the splits for you. This is an example of its usage (at windows of 1000 items at a time) taken from the test class *JDBCInputFormatTest*: final

  1   2   >