Re: [DISCUSS] Use SET statement to set table config in Flink SQL and implement a unified SQL call method

2020-03-09 Thread Timo Walther
Hi Weike, thanks for your feedback. Your use case is definitely on our agenda. The redesign of big parts of the API is still in progress. In the mid-term, most of the SQL Client commands should be present in the SQL API as well such that platform teams can built their custom logic (like REST A

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-10 Thread Timo Walther
1] https://docs.microsoft.com/en-us/sql/t-sql/queries/hints-transact-sql-query?view=sql-server-ver15 Best, Danny Chan 在 2020年3月9日 +0800 PM5:46,Timo Walther ,写道: Hi Danny, thanks for the proposal. I agree with Jark and Jingsong. Planner hints and table hints are orthogonal topics that should be discus

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-10 Thread Timo Walther
data or a start option. So in total, FLIP-110 uses the offset more like a Hive partition prune, we can do that if we have an offset column, but most of the case we do not define that, so there is actually no conflict or overlap. Best, Danny Chan 在 2020年3月10日 +0800 PM4:28,Timo Walther ,写道: Hi D

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-11 Thread Timo Walther
f all the source connectors in the query session, e.g. "table.sources.start-offset". This is possible now because `TableSourceFactory.Context` has `getConfiguration` method to get the session configuration, and use it to create an adapted TableSource. Then we can also expose to SQL CLI via S

Re: [DISCUSS] Features of Apache Flink 1.11

2020-03-11 Thread Timo Walther
Hi Zhijiang and Piotr, from the SQL side we also plan to rework the source and sink interfaces in 1.11. The FLIP is not yet published but already reserved and requirement for FLIP-105: FLIP-95: New TableSource and TableSink interfaces Thanks for compiling the list! Regards, Timo On 11.03.

[DISCUSS] FLIP-95: New TableSource and TableSink interfaces

2020-03-16 Thread Timo Walther
Hi everyone, I'm happy to present the results of long discussions that we had internally. Jark, Dawid, Aljoscha, Kurt, Jingsong, me, and many more have contributed to this design document. We would like to propose new long-term table source and table sink interfaces: https://cwiki.apache.o

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-17 Thread Timo Walther
ose, what if we have 10 tables to join? However, what I want to propose (should be another thread) is a global configuration to reset start offsets of all the source connectors in the query session, e.g. "table.sources.start-offset". This is possible now because `TableSourceFactory.C

Re: [DISCUSS] FLIP-95: New TableSource and TableSink interfaces

2020-03-18 Thread Timo Walther
es of table modules more clear, I like it very much. Timo Walther 于2020年3月17日周二 上午1:36写道: Hi everyone, I'm happy to present the results of long discussions that we had internally. Jark, Dawid, Aljoscha, Kurt, Jingsong, me, and many more have contributed to this design document. We would like

Re: [DISCUSS] (FLINK-16648) Clean up the input parameter StreamQueryConfig of the API in StreamTableEnvironment

2020-03-18 Thread Timo Walther
Hi Jinhai, yes, we can remove this deprecated method. Actually, I started with it but I stopped because we were missing a validation feature in ConfigOptions that ensures that min and max retention time satisfies some conditions (e.g. min is always smaller than max and some minimum difference

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-18 Thread Timo Walther
o add some additional logic in each connector factories in order to merge these properties (and the logic are almost the same), what do you think about this? [1] https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html Best, Danny Chan 在 2020年3月17日 +0800 PM10:

Re: [DISCUSS] FLIP-95: New TableSource and TableSink interfaces

2020-03-23 Thread Timo Walther
. I added a new section `Factory Interfaces` to the design document. This should be helpful to understand the big picture and connecting the concepts. Please let me know what you think? Thanks, Timo On 18.03.20 13:43, Timo Walther wrote: Hi Benchao, this is a very good question. I will update

Re: [DISCUSS] FLIP-95: New TableSource and TableSink interfaces

2020-03-24 Thread Timo Walther
kowicz wrote: Hi Timo, Thank you for the proposal. I think it is an important improvement that will benefit many parts of the Table API. The proposal looks really good to me and personally I would be comfortable with voting on the current state. Best, Dawid On 23/03/2020 18:53, Timo Walther wrote:

Re: [DISCUSS] FLIP-95: New TableSource and TableSink interfaces

2020-03-24 Thread Timo Walther
ign doc. We can also consider @Experimental too. But I am +1 to @PublicEvolving, we should be confident in the current change. Best, Jingsong Lee On Tue, Mar 24, 2020 at 4:30 PM Timo Walther wrote: @Becket: We totally agree that we don't need table specific connectors during runtime

Re: [DISCUSS] FLIP-110: Support LIKE clause in CREATE TABLE

2020-03-24 Thread Timo Walther
Hi Dawid, thanks for your design document. LIKE vs. INHERITS: I would also not start creating transitive dependencies for table metadata. This is very complicated to maintain in a long-term, esp. when we ALTER or DELETE a table. Instead the new table metadata should be materialized immediate

Re: [DISCUSS] FLIP-95: New TableSource and TableSink interfaces

2020-03-25 Thread Timo Walther
On Tue, Mar 24, 2020 at 5:03 PM Jingsong Li < jingsongl...@gmail.com> < jingsongl...@gmail.com> wrote: +1. Thanks Timo for the design doc. We can also consider @Experimental too. But I am +1 to @PublicEvolving, we should be confident in the current change. Best, Jin

Re: [DISCUSS] FLIP-84 Feedback Summary

2020-03-25 Thread Timo Walther
Hi Godfrey, thanks for starting the discussion on the mailing list. And sorry again for the late reply to FLIP-84. I have updated the Google doc one more time to incorporate the offline discussions. From Dawid's and my view, it is fine to postpone the multiline support to a separate method.

Re: [DISCUSS] FLIP-84 Feedback Summary

2020-03-25 Thread Timo Walther
: Hi Godfrey, The changes sounds good to me. +1 to start another voting. A minor question: does the ResultKind contain an ERROR kind? Best, Jark On Wed, 25 Mar 2020 at 18:51, Timo Walther wrote: Hi Godfrey, thanks for starting the discussion on the mailing list. And sorry again for the

Re: [DISCUSS] FLIP-84 Feedback Summary

2020-03-26 Thread Timo Walther
c @Jark Wu I will update the agreed parts to the document first. Best, Godfrey Timo Walther 于2020年3月25日周三 下午6:51写道: Hi Godfrey, thanks for starting the discussion on the mailing list. And sorry again for the late reply to FLIP-84. I have updated the Google doc one more time to incorporate th

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-26 Thread Timo Walther
e#diff-6e569a6dd124fd2091c18e2790fb49c5 [2] https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92 Best, Danny Chan 在 2020年3月18日 +0800 PM9:10,Timo Walt

Re: [DISCUSS] FLIP-95: New TableSource and TableSink interfaces

2020-03-26 Thread Timo Walther
pure SQL perspective. And I am not even sure if this would meet all the requirements for SQL, but the benefit is that the connector developers just need to know how to write an ExpressionToParquetFilter in order to make it work for Table, without having to understand the entire SQL concept. Thanks,

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-26 Thread Timo Walther
enabled" should belong to ` *OptimizerConfigOptions*` Best, Kurt On Thu, Mar 26, 2020 at 4:40 PM Timo Walther wrote: Thanks for the update Danny. +1 for this proposal. Regards, Timo On 26.03.20 04:51, Danny Chan wrote: Thanks everyone who engaged in this discussion ~ Our goal is "

Re: [DISCUSS] FLIP-84 Feedback Summary

2020-03-26 Thread Timo Walther
ming. Best, Godfrey Timo Walther 于2020年3月26日周四 下午4:29写道: Hi Godfrey, having control over the job after submission is a requirement that was requested frequently (some examples are [1], [2]). Users would like to get insights about the running or completed job. Including the jobId, jobGraph

Re: [DISCUSS] FLIP-95: New TableSource and TableSink interfaces

2020-03-26 Thread Timo Walther
bably have an offline call to accelerate the discussion. I think it is a good idea. Can we do that? Thanks, Jiangjie (Becket) Qin On Thu, Mar 26, 2020 at 5:28 PM Timo Walther wrote: Hi Becket, Regarding "PushDown/NestedPushDown which is internal to optimizer": Those concepts cannot be

[VOTE] FLIP-95: New TableSource and TableSink interfaces

2020-03-30 Thread Timo Walther
Hi all, I would like to start the vote for FLIP-95 [1], which is discussed and reached a consensus in the discussion thread [2]. The vote will be open until April 2nd (72h), unless there is an objection or not enough votes. Thanks, Timo [1] https://cwiki.apache.org/confluence/display/FLINK/

Re: [DISCUSS] FLIP-110: Support LIKE clause in CREATE TABLE

2020-03-30 Thread Timo Walther
Hi Dawid, thanks for updating the FLIP. One minor comment from my side, should we move the LIKE clause to the very end? CREATE TABLE X () WITH () LIKE ... Otherwise, the LIKE clause looks a bit lost if there are options afterwards. Otherwise, +1 for start a vote from my side. Regards, Timo

Re: [DISCUSS] FLIP-122: New Connector Property Keys for New Factory

2020-03-30 Thread Timo Walther
are two versions in the old design. One is property version > > > "connector.property-version" which can be used for backward > > compatibility. > > > The other one is "connector.version" which defines the version of > > external &g

Re: [DISCUSS] FLIP-84 Feedback Summary

2020-03-30 Thread Timo Walther
t public interface TableResult { /** * return JobClient for DQL and DML in async mode, else return Optional.empty */ Optional getJobClient(); } what do you think? Best, Godfrey Timo Walther 于2020年3月26日周四 下午9:15写道: Hi Godfrey, executing streaming queries must be our top priority be

Re: [DISCUSS] FLIP-122: New Connector Property Keys for New Factory

2020-03-31 Thread Timo Walther
年3月31日 +0800 AM12:37,Jark Wu ,写道: Hi all, Thanks for the feedbacks. It seems that we have a conclusion to put the version into the factory identifier. I'm also fine with this. If we have this outcome, the interface of Factory#factoryVersion is not needed anymore, this can simplify the learning

Re: [VOTE] FLIP-110: Support LIKE clause in CREATE TABLE

2020-03-31 Thread Timo Walther
+1 this will reduce manual schema work a lot! Thanks, Timo On 31.03.20 09:33, Dawid Wysakowicz wrote: Hi all, I would like to start the vote for FLIP-110 [1], which is discussed and reached a consensus in the discussion thread [2]. The vote will be open until April 3rd (72h), unless there is

Re: [VOTE] FLIP-84: Improve & Refactor API of TableEnvironment & Table

2020-03-31 Thread Timo Walther
-1 The current discussion has not completed. The last comments were sent less than 24h ago. Let's wait a bit longer to collect feedback from all stakeholders. Thanks, Timo On 31.03.20 08:31, godfrey he wrote: Hi everyone, I'd like to start the vote of FLIP-84[1] again, because we have some

Re: [DISCUSS] FLIP-84 Feedback Summary

2020-03-31 Thread Timo Walther
e it will cause a lot of confusion and problems if users don't deeply keep the "sync" in mind (e.g. client hangs). Besides, the streaming mode is still the majority use cases of Flink and Flink SQL. We should put the usability at a high priority. Best, Jark On Mon, 30 Mar 2020 at 2

Re: [DISCUSS] Change default planner to blink planner in 1.11

2020-04-01 Thread Timo Walther
Hi Kurt, +1 for making the Blink planner the default in 1.11. One prerequisite that I see would be to resolve the remaining @Ignore test cases in the code base. One example that I recently spotted was here: https://github.com/apache/flink/blob/master/flink-table/flink-table-planner-blink/src

Re: [DISCUSS] FLIP-84 Feedback Summary

2020-04-01 Thread Timo Walther
from the cluster at once. I think we should also use Iterator for TableEnvironment#executeMultilineSql(String statements): Iterator. Best, Dawid On 31/03/2020 19:27, Timo Walther wrote: Hi Godfrey, Aljoscha, Dawid, Klou, and I had another discussion around FLIP-84. In particular, we discu

Re: [VOTE] FLIP-95: New TableSource and TableSink interfaces

2020-04-02 Thread Timo Walther
, Kurt On Mon, Mar 30, 2020 at 4:08 PM Benchao Li wrote: +1 (non-binding) Jark Wu 于2020年3月30日周一 下午3:57写道: +1 from my side. Thanks Timo for driving this. Best, Jark On Mon, 30 Mar 2020 at 15:36, Timo Walther wrote: Hi all, I would like to start the vote for FLIP-95 [1], which is discu

Re: [VOTE] FLIP-95: New TableSource and TableSink interfaces

2020-04-02 Thread Timo Walther
Then `TableSchema` can continue to serve as a pure table schema and it stays in a good package. Best, Jark On Thu, 2 Apr 2020 at 19:39, Timo Walther wrote: Hi Dawid, thanks for your feedback. I agree with your concerns. I also observed that TableSchema might be used at too many different place

Re: [VOTE] FLIP-122: New Connector Property Keys for New Factory

2020-04-02 Thread Timo Walther
+1 Thanks, Timo On 02.04.20 17:22, Jark Wu wrote: Hi all, I would like to start the vote for FLIP-122 [1], which is discussed and reached a consensus in the discussion thread [2]. The vote will be open for at least 72h, unless there is an objection or not enough votes. Thanks, Timo [1] http

Re: Function input type validation

2015-11-08 Thread Timo Walther
The reason for input validation is to check if the Function is fully compatible. Actually only the return types are necessary, but it prohibits stupid implementation mistakes and undesired behavior. E.g. if you implement a "class MyMapper extends MapFunctionString>{}" and use it for "env.fromEl

Re: [DISCUSSION] Type hints versus TypeInfoParser

2015-11-11 Thread Timo Walther
and type safe than the String/Parser way of giving hints, I was wondering whether we should add this and deprecate the String variant. If we do that, 1.0 is the time to do that. What do you think about this idea? @Timo Walther Since you worked a lot on types/parser/etc - what is your take on this? Greetings, Stephan

Re: Error in during TypeExtraction

2015-11-12 Thread Timo Walther
This looks like a bug. Can you open an issue for that? I will look into it later. Regards, Timo On 12.11.2015 13:16, Gyula Fóra wrote: Hey, I get a weird error when I try to execute my job on the cluster. Locally this works fine but running it from the command line fails during typeextractio

Add BigDecimal and BigInteger as types

2015-11-18 Thread Timo Walther
Hey everyone, I'm not sure if we already had a discussion about it but as we are currently adding new types like the Either type, I would like to discuss it again. I think especially for business or scientific applications it makes sense to support the BigInteger and BigDecimal types natively.

Re: [DISCUSSION] Type hints versus TypeInfoParser

2015-11-18 Thread Timo Walther
t; method is probably redundant anyways, as the hints are only necessary in generics are involved (where class does not help anyways). @Timo: What can the String do what the TypeHint cannot do? On Wed, Nov 11, 2015 at 9:06 PM, Timo Walther wrote: +1 It's still hacky but we don't h

Re: [DISCUSSION] Type hints versus TypeInfoParser

2015-11-19 Thread Timo Walther
TypeExtractor, though). What do you think? Stephan On Wed, Nov 18, 2015 at 6:03 PM, Timo Walther wrote: If the TypeExtractor is not able to handle the fields of a Pojo correctly, the String parser is quite useful to say "org.my.Pojo>". Doing this with TypeInformation classes is

Re: Add BigDecimal and BigInteger as types

2015-11-19 Thread Timo Walther
y are currently generic types and handled by Kryo, which has (AFAIK) proper serializers for them. Are there more benefits of native support (other than more compact serialization) that you are thinking of? On Wed, Nov 18, 2015 at 5:55 PM, Timo Walther wrote: Hey everyone, I'm not sure if we a

withParameters() for Streaming API

2015-11-24 Thread Timo Walther
Hi all, I want to set the Configuration of a streaming operator and access it via the open method of the RichFunction. There is no possibility to set the Configuration of the open method at the moment, right? Can I open an issue for a withParameters() equivalent for the Stremaing API? Regard

Re: withParameters() for Streaming API

2015-11-24 Thread Timo Walther
. Sax wrote: We had this discussion a while ago. If I recall correctly, "withParameters()" is not encourage to be used in DataSet either. This is the thread: https://mail-archives.apache.org/mod_mbox/flink-dev/201509.mbox/%3C55EC69CD.1070003%40apache.org%3E -Matthias On 11/24/2015 02:1

Re: The null in Flink

2015-11-25 Thread Timo Walther
Hi Chengxiang, I totally agree that the Table API should fully support NULL values. The Table API is a logical API and therefore we should be as close to ANSI SQL as possible. Rows need to be nullable in the near future. 2. i, ii, iii and iv sound reasonable. But v, vi and vii sound to much

Re: Either not NotSerializableException and InvalidTypesException

2015-11-29 Thread Timo Walther
Hi Vasia, regarding your TypeExtractor problem. The TypeExtractor works correctly. The with() function of the JoinOperator calls the wrong TypeExtractor method that does not allow missing type info. This is a bug. Can open an issue for that? Regards, Timo On 28.11.2015 20:18, Vasiliki Kalav

Re: Input type validation is killing me

2016-03-02 Thread Timo Walther
The TypeExtractor's input type validation was designed for the built-in TypeInformation classes. In your case of a new, unknown TypeInformation, the validation should simply skipped, because we can assume that you user knows what he is doing. I can open a PR for that. On 02.03.2016 11:34, Al

Re: Input type validation is killing me

2016-03-02 Thread Timo Walther
:00, Gyula Fóra wrote: Hi! Yes I think, that sounds good :) We just need to make sure that this works with things like the TupleTypeInfo which is built-on but I can still mix in new Types for the fields. Thanks, Gyula Timo Walther ezt írta (időpont: 2016. márc. 2., Sze, 14:02): The

Re: Input type validation is killing me

2016-03-02 Thread Timo Walther
Can you open an issue with an example of your custom TypeInfo? I will then open a suitable PR for it. On 02.03.2016 15:33, Gyula Fóra wrote: Would that work with generic classes? Timo Walther ezt írta (időpont: 2016. márc. 2., Sze, 15:22): After thinking about it, I think an even better

Re: Table API / SQL Queries and Code Generation

2016-04-05 Thread Timo Walther
Hi Gábor, the code generation in the Table API is in a very early stage and contains not much optimization logic so far. Currently we extend the functionality to support the most important SQL operations. It will need some time until we can further optimize the generated code (e.g. for tracki

Re: [PROPOSAL] Structure the Flink Open Source Development

2016-05-13 Thread Timo Walther
+1 for from my side too On 13.05.2016 06:13, Chiwan Park wrote: +1 for this proposal

Re: Preparing Table API & SQL for Flink 1.1.0

2016-05-20 Thread Timo Walther
l leave in one month for a two-months parental leave and I don't know how much I can contribute in that time. So if somebody would like to step up and help coordinating, please let me and the others know. Cheers, Fabian -- Freundliche Grüße / Kind Regards Timo Walther Follow me:

Re: primitiveDefaultValue in CodeGenUtils in Table API

2016-06-29 Thread Timo Walther
Hi Cody, default values are needed in cases where NULL values are not supported. This happens if the null check is disabled in TableConfig for efficiency reasons. Using 0 to DataType.MAX_VALUE for numeric types and -1 as a NULL equivalent in special cases seems more reasonable to me. Hope th

Re: Flink Table & SQL doesn't work in very simple example

2016-07-20 Thread Timo Walther
ableList (with flink shaded)? As per the error "/apache/flink/shaded/calcite/com/google/common/collect/ImmutableList;" -Original Message- From: Maximilian Michels [mailto:m...@apache.org] Sent: Wednesday, July 20, 2016 11:52 AM To: dev@flink.apache.org Cc: Timo Walther Subject: R

Re: Flink Table & SQL doesn't work in very simple example

2016-07-20 Thread Timo Walther
/common/collect/ImmutableList (with flink shaded)? As per the error "/apache/flink/shaded/calcite/com/google/common/collect/ImmutableList;" -Original Message- From: Maximilian Michels [mailto:m...@apache.org] Sent: Wednesday, July 20, 2016 11:52 AM To: dev@flink.apache.org Cc: Tim

Re: Flink Table & SQL doesn't work in very simple example

2016-07-20 Thread Timo Walther
ral, flink-tableXXX.jar already includes everyting it needs (calcite, avative, eigenbase, etc.). Hope that helps. Am 20/07/16 um 14:14 schrieb Timo Walther: You can always find the latest nightly snapshot version here: http://flink.apache.org/contribute-code.html (at the end of the page) Am 20/07/1

Re: Use case

2016-07-28 Thread Timo Walther
Why can't this be enabled? My iterative stream is also quite small: only one value. Is there a better way to keep track of a largest value in a stream? Regards, Kevin -- Freundliche Grüße / Kind Regards Timo Walther Follow me: @twalthr https://www.linkedin.com/in/twalthr

Re: <无主题>

2016-08-16 Thread Timo Walther
Hi Renkai, thanks for the interest in Flink and the Table API/SQL! I would recommend to maybe have a look into previous Pull Requests for new SQL features such as INTERSECT [1] to get an overview of layers and classes. I also recommend the design document [2] that describes the all planned fe

Re: [DISCUSS] Some thoughts about unify Stream SQL and Batch SQL grammer

2016-08-18 Thread Timo Walther
ust my thoughts :) What do you think about this ? [1] https://calcite.apache.org/docs/stream.html - Jark Wu -- Freundliche Grüße / Kind Regards Timo Walther Follow me: @twalthr https://www.linkedin.com/in/twalthr

Re: [DISCUSS] Some thoughts about unify Stream SQL and Batch SQL grammer

2016-08-23 Thread Timo Walther
bian [1] http://calcite.apache.org/docs/stream.html#tumbling-windows-improved [2] https://issues.apache.org/jira/browse/CALCITE-1345 -- Freundliche Grüße / Kind Regards Timo Walther Follow me: @twalthr https://www.linkedin.com/in/twalthr

Re: [DISCUSS] Some thoughts about unify Stream SQL and Batch SQL grammer

2016-08-29 Thread Timo Walther
e next days and ask about the status of StreamSQL. Best, Fabian [1] http://calcite.apache.org/docs/stream.html#tumbling- windows-improved [2] https://issues.apache.org/jira/browse/CALCITE-1345 -- Freundliche Grüße / Kind Regards Timo Walther Follow me: @twalthr https://www.linkedin.com/in/twalthr

Re: [DISCUSS] Some thoughts about unify Stream SQL and Batch SQL grammer

2016-08-29 Thread Timo Walther
exception. Because we register the table as a regular table not streamable. - Jark Wu 在 2016年8月29日,下午8:13,Timo Walther 写道: Hi Jark, your code looks good and it also simplifies many parts. So the STREAM keyword is not optional but invalid now, right? What happens if there is keyword in

Re: [DISCUSS] Some thoughts about unify Stream SQL and Batch SQL grammer

2016-08-30 Thread Timo Walther
think the TableSourceITCase.testCsvTableSource in batch and stream package can explain it. Am I right ? - Jark Wu 在 2016年8月29日,下午8:59,Timo Walther 写道: At first glance, I thought we are losing the possibility to distingish between choosing a batch or streaming table if a TableSource

Re: Extending FLIP template

2016-09-01 Thread Timo Walther
e FLIP can be implemented. The described subtasks will then be reflected as individual JIRA issues. What do you think? Cheers, Fabian -- Freundliche Grüße / Kind Regards Timo Walther Follow me: @twalthr https://www.linkedin.com/in/twalthr

[DISCUSS] FLIP-11: Table API Stream Aggregations

2016-09-01 Thread Timo Walther
Hi all! Fabian and I worked on a FLIP for Stream Aggregations in the Table API. You can find the FLIP-11 here: https://cwiki.apache.org/confluence/display/FLINK/FLIP-11%3A+Table+API+Stream+Aggregations Motivation for the FLIP: The Table API is a declarative API to define queries on static an

Re: [DISCUSS] FLIP-155: Introduce a few convenient operations in Table API

2021-01-07 Thread Timo Walther
pydata.org/docs/reference/api/pandas.DataFrame.fillna.html> [2] https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.dropna.html <https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.dropna.html> [3] https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.replace.html

[DISCUSS] Dealing with deprecated and legacy code in Flink

2021-01-15 Thread Timo Walther
Hi everyone, I would like to start a discussion how we treat deprecated and legacy code in Flink in the future. During the last years, our code base has grown quite a bit and a couple of interfaces and components have been reworked on the way. I'm sure each component has a few legacy parts t

Re: [Vote] FLIP-157 Migrate Flink Documentation from Jekyll to Hugo

2021-01-18 Thread Timo Walther
+1 Thanks for upgrading our docs infrastructure. Regards, Timo On 18.01.21 15:29, Seth Wiesman wrote: Hi devs, The discussion of the FLIP-157 [1] seems has reached a consensus through the mailing thread [2]. I would like to start a vote for it. The vote will be opened until 20th January (72h

Re: [DISCUSS] Correct time-related function behavior in Flink SQL

2021-01-21 Thread Timo Walther
Hi Leonard, thanks for working on this topic. I agree that time handling is not easy in Flink at the moment. We added new time data types (and some are still not supported which even further complicates things like TIME(9)). We should definitely improve this situation for users. This is a pr

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-01-21 Thread Timo Walther
Now we have 2 discussion threads on 3 mailing lists. Which one should have prioity? Should I repost my large email here again? I think it is good to inform and invite in the user mailing lists but let's keep the FLIP discussion on the dev@ ML only. Regards, Timo On 21.01.21 16:50, Leonard Xu

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-01-22 Thread Timo Walther
most all user problems,the divergence is whether we need to spend pretty energy just to get a bit more accurate semantics? I think we need a tradeoff. Best, Leonard [1] https://trino.io/docs/current/functions/datetime.html#current_timestamp < https://trino.io/docs/current/functions/datetime.html#curre

Re: [DISCUSS] Dealing with deprecated and legacy code in Flink

2021-01-26 Thread Timo Walther
raised a point sometime ago, that in the recent past, we developed a habit of marking everything as `@Experimental` or `@PublicEvolving` and leaving it as that forever. Maybe we should also include deadlines (2 releases since introduction?) for changing `@Experimental`/`@PublicEvolving` into `

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-01-28 Thread Timo Walther
. In one word, both your suggestion and my proposal can resolve almost all user problems,the divergence is whether we need to spend pretty energy just to get a bit more accurate semantics? I think we need a tradeoff. Best, Leonard [1] https://trino.io/docs/current/functions/datetime.h

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-01-28 Thread Timo Walther
and my proposal can resolve almost all user problems,the divergence is whether we need to spend pretty energy just to get a bit more accurate semantics? I think we need a tradeoff. Best, Leonard [1] https://trino.io/docs/current/functions/datetime.html#current_timestamp < https://tr

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-01-29 Thread Timo Walther
hether we need to spend pretty energy just to get a bit more accurate semantics? I think we need a tradeoff. Best, Leonard [1] https://trino.io/docs/current/functions/datetime.html#current_timestamp < https://trino.io/docs/current/functions/datetime.html#current_timestamp> [2] http

Re: [DISCUSS] FLINK-21045: Support 'load module' and 'unload module' SQL syntax

2021-02-01 Thread Timo Walther
Jark Wu and Nicholas Jiang proposed to use `CREATE/DROP MODULE` instead of `LOAD/UNLOAD MODULE` because 1) From a pure SQL user's perspective, maybe `CREATE MODULE + USE MODULE` is easier to use rather than `LOAD/UNLOAD`. 2) This will be very similar to what the catalog used now.

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-02-01 Thread Timo Walther
osal can resolve almost all user problems,the divergence is whether we need to spend pretty energy just to get a bit more accurate semantics? I think we need a tradeoff. Best, Leonard [1] https://trino.io/docs/current/functions/datetime.html#current_timestamp < https://trin

Re: [DISCUSS] FLINK-21045: Support 'load module' and 'unload module' SQL syntax

2021-02-01 Thread Timo Walther
rading the priority of the loaded module(s). 2. `LOAD/UNLOAD MODULE` v.s. `CREATE/DROP MODULE` syntax Jark Wu and Nicholas Jiang proposed to use `CREATE/DROP MODULE` instead of `LOAD/UNLOAD MODULE` because 1) From a pure SQL user's perspective, maybe `CREATE MODULE + USE MODUL

Re: [DISCUSS] FLINK-21045: Support 'load module' and 'unload module' SQL syntax

2021-02-01 Thread Timo Walther
deprecated `TableFactory` class. Regarding #1, I think the point lies in whether changing the resolution order implies an `unload` operation explicitly (i.e., users could sense it). What do others think? Best, Jane On Mon, Feb 1, 2021 at 6:41 PM Timo Walther wrote: IMHO I would rather un

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-02-01 Thread Timo Walther
esult when user use their streaming pipeline sql to run a batch pipeline(e.g backfilling), and user also can not control these function behavior. How do you think ? Thanks, Leonard 在 2021年2月1日,18:23,Timo Walther 写道: Parts of the FLIP can already be implemented without a completed votin

Re: [DISCUSS] FLINK-21045: Support 'load module' and 'unload module' SQL syntax

2021-02-01 Thread Timo Walther
Feb 1, 2021 at 10:02 PM Timo Walther wrote: +1 to Jark's proposal I like the difference between just loading and actually enabling these modules. @Rui: I would use the same behavior as catalogs here. You cannot `USE` a catalog without creating it before. Another question is whether a LOAD

Re: [DISCUSS] FLINK-21045: Support 'load module' and 'unload module' SQL syntax

2021-02-01 Thread Timo Walther
g the LOAD statement instead of CREATE, so I think it's fine that it does some implicit things. Best, Jark On Tue, 2 Feb 2021 at 00:48, Timo Walther wrote: Not the module itself but the ModuleManager should handle this case, yes. Regards, Timo On 01.02.21 17:35, Jane Chan wrote: +1 to Jark&#x

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-02-02 Thread Timo Walther
vide consistent results. Thus, I think we may need to think more from the users' perspective. Best, Jark On Mon, 1 Feb 2021 at 23:06, Timo Walther wrote: Hi Leonard, thanks for considering this issue as well. +1 for the proposed config option. Let's start a voting thread once the

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-02 Thread Timo Walther
Thanks for this great proposal Shengkai. This will give the SQL Client a very good update and make it production ready. Here is some feedback from my side: 1) SQL client specific options I don't think that `sql-client.planner` and `sql-client.execution.mode` are SQL Client specific. Similar t

Re: [DISCUSS] FLINK-21045: Support 'load module' and 'unload module' SQL syntax

2021-02-03 Thread Timo Walther
xecution for all of them or #2 enabled the rest modules and return a warning to users? My personal preference goes to #1 for simplicity. What do you think? Best, Jane On Tue, Feb 2, 2021 at 3:53 PM Timo Walther wrote: +1 @Jane Can you summarize our discussion in the JIRA issue? Thanks, T

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-03 Thread Timo Walther
se `ADD JAR` -> `CREATE JAR`, `DELETE JAR` -> `DROP JAR`, `LIST JAR` -> `SHOW JAR`. *Regarding #5*: I agree with you that we'd better keep consistent. *Regarding #6*: Yes. Most of the commands should belong to the table environment. In the Summary section, I use the tag to identify

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-05 Thread Timo Walther
about REMOVE vs DELETE though. While flink doesn't need to follow hive syntax, as far as I know, most users who are requesting these features are previously hive users. So I wonder whether we can support both LIST/SHOW JARS and REMOVE/DELETE JARS as synonyms? It's just like lots

[DISCUSS] FLIP-164: Improve Schema Handling in Catalogs

2021-02-05 Thread Timo Walther
Hi everyone, you might have seen that we discussed a better schema API in past as part of FLIP-129 and FLIP-136. We also discussed this topic during different releases: https://issues.apache.org/jira/browse/FLINK-17793 Jark and I had an offline discussion how we can finally fix this shortco

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-08 Thread Timo Walther
o Alternative 1: We consider batch/streaming mode and block for batch INSERT INTO and async for streaming INSERT INTO/STATEMENT SET. And this behavior is consistent across CLI and files. Best, Jark [1]: https://github.com/apache/flink/blob/master/flink-end-to-end-tests/flink-end-to-end-tests-comm

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-08 Thread Timo Walther
tible, and provides flexible configurable behavior 3) sync for both batch and streaming DML, and can be set to async via a configuration. ==> +0 for this, because it breaks all the compatibility, esp. our main users. Best, Jark On Mon, 8 Feb 2021 at 17:34, Timo Walther wrote: Hi Jark, Hi

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-08 Thread Timo Walther
th a Flink job detach mode. How about `table.dml-async`? Thanks, Timo On 08.02.21 15:55, Jark Wu wrote: Thanks Timo, I'm +1 for option#2 too. I think we have addressed all the concerns and can start a vote. Best, Jark On Mon, 8 Feb 2021 at 22:19, Timo Walther wrote: Hi Jark, you are

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-09 Thread Timo Walther
Hi everyone, I understand Rui's concerns. `table.dml-sync` should not apply to regular `executeSql`. Actually, this option makes only sense when executing multi statements. Once we have a `TableEnvironment.executeMultiSql()` this config could be considered. Maybe we can find a better generic

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-09 Thread Timo Walther
uteMultiSql() in the future, right? Best, Jark On Tue, 9 Feb 2021 at 16:37, Timo Walther wrote: Hi everyone, I understand Rui's concerns. `table.dml-sync` should not apply to regular `executeSql`. Actually, this option makes only sense when executing multi statements. Once we have a `TableEn

Re: [DISCUSS] FLIP-164: Improve Schema Handling in Catalogs

2021-02-09 Thread Timo Walther
look good. I agree it is an important step towards FLIP-129 and FLIP-136. Personally I feel comfortable voting on the document. Best, Dawid On 05/02/2021 16:09, Timo Walther wrote: Hi everyone, you might have seen that we discussed a better schema API in past as part of FLIP-129 and FLIP-13

Re: [DISCUSS] FLIP-164: Improve Schema Handling in Catalogs

2021-02-09 Thread Timo Walther
to me overall. I have two questions. 1. When should we use a resolved schema and when to use an unresolved one? 2. The FLIP mentions only resolved tables/views can be stored into a catalog. Does that mean the getTable method should also return a resolved object? On Tue, Feb 9, 2021 at 6:29 PM Timo Wa

Re: [DISCUSS] FLIP-164: Improve Schema Handling in Catalogs

2021-02-10 Thread Timo Walther
? If we want to introduce a new stack, it would be better to have a different name, otherwise, it's easy to use a wrong class for users. Best, Jark On Wed, 10 Feb 2021 at 09:49, Rui Li wrote: I see. Makes sense to me. Thanks Timo for the detailed explanation! On Tue, Feb 9, 2021 at 9:48 PM

Re: [DISCUSS] FLIP-164: Improve Schema Handling in Catalogs

2021-02-12 Thread Timo Walther
declaration. Regards, Timo On 10.02.21 12:12, Timo Walther wrote: Hi Jark, I don't think many users use WatermarkSpec. UniqueConstraint could cause some confusion but this mostly affects catalog or connector implementers. After deprecating the old APIs it should be obvious when an out

[VOTE] FLIP-164: Improve Schema Handling in Catalogs

2021-02-12 Thread Timo Walther
Hi everyone, I'd like to start a vote on FLIP-164 [1] which was discussed in [2]. The vote will be open for at least 72 hours. Unless there are any objections, I'll close it by February 17th, 2021 (due to weekend) if we have received sufficient votes. [1] https://cwiki.apache.org/confluence/

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-12 Thread Timo Walther
I am fine with the new option name. Best, Shengkai Timo Walther 于2021年2月9日 周二下午5:35写道: Yes, `TableEnvironment#executeMultiSql()` can be future work. @Rui, Shengkai: Are you also fine with this conclusion? Thanks, Timo On 09.02.21 10:14, Jark Wu wrote: I'm fine with `table.multi-dml-

<    1   2   3   4   5   6   7   8   9   10   >