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 <twal...@apache.org> wrote: > Hi Jark, > > you are right. Nesting STATEMENT SET and ASYNC might be too verbose. > > So let's stick to the config option approach. > > However, I strongly believe that we should not use the batch/streaming > mode for deriving semantics. This discussion is similar to time function > discussion. We should not derive sync/async submission behavior from a > flag that should only influence runtime operators and the incremental > computation. Statements for bounded streams should have the same > semantics in batch mode. > > I think your proposed option 2) is a good tradeoff. For the following > reasons: > > pros: > - by default, batch and streaming behave exactly the same > - SQL Client CLI behavior does not change compared to 1.12 and remains > async for batch and streaming > - consistent with the async Table API behavior > > con: > - batch files are not 100% SQL compliant by default > > The last item might not be an issue since we can expect that users have > long-running jobs and prefer async execution in most cases. > > Regards, > Timo > > > On 08.02.21 14:15, Jark Wu wrote: > > Hi Timo, > > > > Actually, I'm not in favor of explicit syntax `BEGIN ASYNC;... END;`. > > Because it makes submitting streaming jobs very verbose, every INSERT > INTO > > and STATEMENT SET must be wrapped in the ASYNC clause which is > > not user-friendly and not backward-compatible. > > > > I agree we will have unified behavior but this is at the cost of hurting > > our main users. > > I'm worried that end users can't understand the technical decision, and > > they would > > feel streaming is harder to use. > > > > If we want to have an unified behavior, and let users decide what's the > > desirable behavior, I prefer to have a config option. A Flink cluster can > > be set to async, then > > users don't need to wrap every DML in an ASYNC clause. This is the least > > intrusive > > way to the users. > > > > > > Personally, I'm fine with following options in priority: > > > > 1) sync for batch DML and async for streaming DML > > ==> only breaks batch behavior, but makes both happy > > > > 2) async for both batch and streaming DML, and can be set to sync via a > > configuration. > > ==> compatible, 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 <twal...@apache.org> wrote: > > > >> Hi Jark, Hi Rui, > >> > >> 1) How should we execute statements in CLI and in file? Should there be > >> a difference? > >> So it seems we have consensus here with unified bahavior. Even though > >> this means we are breaking existing batch INSERT INTOs that were > >> asynchronous before. > >> > >> 2) Should we have different behavior for batch and streaming? > >> I think also batch users prefer async behavior because usually even > >> those pipelines take some time to execute. But we need should stick to > >> standard SQL blocking semantics. > >> > >> What are your opinions on making async explicit in SQL via `BEGIN ASYNC; > >> ... END;`? This would allow us to really have unified semantics because > >> batch and streaming would behave the same? > >> > >> Regards, > >> Timo > >> > >> > >> On 07.02.21 04:46, Rui Li wrote: > >>> Hi Timo, > >>> > >>> I agree with Jark that we should provide consistent experience > regarding > >>> SQL CLI and files. Some systems even allow users to execute SQL files > in > >>> the CLI, e.g. the "SOURCE" command in MySQL. If we want to support that > >> in > >>> the future, it's a little tricky to decide whether that should be > treated > >>> as CLI or file. > >>> > >>> I actually prefer a config option and let users decide what's the > >>> desirable behavior. But if we have agreed not to use options, I'm also > >> fine > >>> with Alternative #1. > >>> > >>> On Sun, Feb 7, 2021 at 11:01 AM Jark Wu <imj...@gmail.com> wrote: > >>> > >>>> Hi Timo, > >>>> > >>>> 1) How should we execute statements in CLI and in file? Should there > be > >> a > >>>> difference? > >>>> I do think we should unify the behavior of CLI and SQL files. SQL > files > >> can > >>>> be thought of as a shortcut of > >>>> "start CLI" => "copy content of SQL files" => "past content in CLI". > >>>> Actually, we already did this in kafka_e2e.sql [1]. > >>>> I think it's hard for users to understand why SQL files behave > >> differently > >>>> from CLI, all the other systems don't have such a difference. > >>>> > >>>> If we distinguish SQL files and CLI, should there be a difference in > >> JDBC > >>>> driver and UI platform? > >>>> Personally, they all should have consistent behavior. > >>>> > >>>> 2) Should we have different behavior for batch and streaming? > >>>> I think we all agree streaming users prefer async execution, otherwise > >> it's > >>>> weird and difficult to use if the > >>>> submit script or CLI never exists. On the other hand, batch SQL users > >> are > >>>> used to SQL statements being > >>>> executed blockly. > >>>> > >>>> Either unified async execution or unified sync execution, will hurt > one > >>>> side of the streaming > >>>> batch users. In order to make both sides happy, I think we can have > >>>> different behavior for batch and streaming. > >>>> There are many essential differences between batch and stream > systems, I > >>>> think it's normal to have some > >>>> different behaviors, and the behavior doesn't break the unified batch > >>>> stream semantics. > >>>> > >>>> > >>>> Thus, I'm +1 to 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-common-kafka/src/test/resources/kafka_e2e.sql > >>>> > >>>> On Fri, 5 Feb 2021 at 21:49, Timo Walther <twal...@apache.org> wrote: > >>>> > >>>>> Hi Jark, > >>>>> > >>>>> thanks for the summary. I hope we can also find a good long-term > >>>>> solution on the async/sync execution behavior topic. > >>>>> > >>>>> It should be discussed in a bigger round because it is (similar to > the > >>>>> time function discussion) related to batch-streaming unification > where > >>>>> we should stick to the SQL standard to some degree but also need to > >> come > >>>>> up with good streaming semantics. > >>>>> > >>>>> Let me summarize the problem again to hear opinions: > >>>>> > >>>>> - Batch SQL users are used to execute SQL files sequentially (from > top > >>>>> to bottom). > >>>>> - Batch SQL users are used to SQL statements being executed blocking. > >>>>> One after the other. Esp. when moving around data with INSERT INTO. > >>>>> - Streaming users prefer async execution because unbounded stream are > >>>>> more frequent than bounded streams. > >>>>> - We decided to make Flink Table API is async because in a > programming > >>>>> language it is easy to call `.await()` on the result to make it > >> blocking. > >>>>> - INSERT INTO statements in the current SQL Client implementation are > >>>>> always submitted asynchrounous. > >>>>> - Other client's such as Ververica platform allow only one INSERT > INTO > >>>>> or a STATEMENT SET at the end of a file that will run > asynchrounously. > >>>>> > >>>>> Questions: > >>>>> > >>>>> - How should we execute statements in CLI and in file? Should there > be > >> a > >>>>> difference? > >>>>> - Should we have different behavior for batch and streaming? > >>>>> - Shall we solve parts with a config option or is it better to make > it > >>>>> explicit in the SQL job definition because it influences the > semantics > >>>>> of multiple INSERT INTOs? > >>>>> > >>>>> Let me summarize my opinion at the moment: > >>>>> > >>>>> - SQL files should always be executed blocking by default. Because > they > >>>>> could potentially contain a long list of INSERT INTO statements. This > >>>>> would be SQL standard compliant. > >>>>> - If we allow async execution, we should make this explicit in the > SQL > >>>>> file via `BEGIN ASYNC; ... END;`. > >>>>> - In the CLI, we always execute async to maintain the old behavior. > We > >>>>> can also assume that people are only using the CLI to fire statements > >>>>> and close the CLI afterwards. > >>>>> > >>>>> Alternative 1: > >>>>> - We consider batch/streaming mode and block for batch INSERT INTO > and > >>>>> async for streaming INSERT INTO/STATEMENT SET > >>>>> > >>>>> What do others think? > >>>>> > >>>>> Regards, > >>>>> Timo > >>>>> > >>>>> > >>>>> > >>>>> > >>>>> On 05.02.21 04:03, Jark Wu wrote: > >>>>>> Hi all, > >>>>>> > >>>>>> After an offline discussion with Timo and Kurt, we have reached some > >>>>>> consensus. > >>>>>> Please correct me if I am wrong or missed anything. > >>>>>> > >>>>>> 1) We will introduce "table.planner" and "table.execution-mode" > >> instead > >>>>> of > >>>>>> "sql-client" prefix, > >>>>>> and add `TableEnvironment.create(Configuration)` interface. These 2 > >>>>> options > >>>>>> can only be used > >>>>>> for tableEnv initialization. If used after initialization, Flink > >> should > >>>>>> throw an exception. We may can > >>>>>> support dynamic switch the planner in the future. > >>>>>> > >>>>>> 2) We will have only one parser, > >>>>>> i.e. org.apache.flink.table.delegation.Parser. It accepts a string > >>>>>> statement, and returns a list of Operation. It will first use regex > to > >>>>>> match some special statement, > >>>>>> e.g. SET, ADD JAR, others will be delegated to the underlying > >> Calcite > >>>>>> parser. The Parser can > >>>>>> have different implementations, e.g. HiveParser. > >>>>>> > >>>>>> 3) We only support ADD JAR, REMOVE JAR, SHOW JAR for Flink dialect. > >> But > >>>>> we > >>>>>> can allow > >>>>>> DELETE JAR, LIST JAR in Hive dialect through HiveParser. > >>>>>> > >>>>>> 4) We don't have a conclusion for async/sync execution behavior yet. > >>>>>> > >>>>>> Best, > >>>>>> Jark > >>>>>> > >>>>>> > >>>>>> > >>>>>> On Thu, 4 Feb 2021 at 17:50, Jark Wu <imj...@gmail.com> wrote: > >>>>>> > >>>>>>> Hi Ingo, > >>>>>>> > >>>>>>> Since we have supported the WITH syntax and SET command since v1.9 > >>>>> [1][2], > >>>>>>> and > >>>>>>> we have never received such complaints, I think it's fine for such > >>>>>>> differences. > >>>>>>> > >>>>>>> Besides, the TBLPROPERTIES clause of CREATE TABLE in Hive also > >>>> requires > >>>>>>> string literal keys[3], > >>>>>>> and the SET <key>=<value> doesn't allow quoted keys [4]. > >>>>>>> > >>>>>>> Best, > >>>>>>> Jark > >>>>>>> > >>>>>>> [1]: > >>>>>>> > >>>>> > >>>> > >> > https://ci.apache.org/projects/flink/flink-docs-release-1.9/dev/table/connect.html > >>>>>>> [2]: > >>>>>>> > >>>>> > >>>> > >> > https://ci.apache.org/projects/flink/flink-docs-release-1.9/dev/table/sqlClient.html#running-sql-queries > >>>>>>> [3]: > >>>>> https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL > >>>>>>> [4]: > >>>>> https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli > >>>>>>> (search "set mapred.reduce.tasks=32") > >>>>>>> > >>>>>>> On Thu, 4 Feb 2021 at 17:09, Ingo Bürk <i...@ververica.com> wrote: > >>>>>>> > >>>>>>>> Hi, > >>>>>>>> > >>>>>>>> regarding the (un-)quoted question, compatibility is of course an > >>>>>>>> important > >>>>>>>> argument, but in terms of consistency I'd find it a bit surprising > >>>> that > >>>>>>>> WITH handles it differently than SET, and I wonder if that could > >>>> cause > >>>>>>>> friction for developers when writing their SQL. > >>>>>>>> > >>>>>>>> > >>>>>>>> Regards > >>>>>>>> Ingo > >>>>>>>> > >>>>>>>> On Thu, Feb 4, 2021 at 9:38 AM Jark Wu <imj...@gmail.com> wrote: > >>>>>>>> > >>>>>>>>> Hi all, > >>>>>>>>> > >>>>>>>>> Regarding "One Parser", I think it's not possible for now because > >>>>>>>> Calcite > >>>>>>>>> parser can't parse > >>>>>>>>> special characters (e.g. "-") unless quoting them as string > >>>> literals. > >>>>>>>>> That's why the WITH option > >>>>>>>>> key are string literals not identifiers. > >>>>>>>>> > >>>>>>>>> SET table.exec.mini-batch.enabled = true and ADD JAR > >>>>>>>>> /local/my-home/test.jar > >>>>>>>>> have the same > >>>>>>>>> problems. That's why we propose two parser, one splits lines into > >>>>>>>> multiple > >>>>>>>>> statements and match special > >>>>>>>>> command through regex which is light-weight, and delegate other > >>>>>>>> statements > >>>>>>>>> to the other parser which is Calcite parser. > >>>>>>>>> > >>>>>>>>> Note: we should stick on the unquoted SET > >>>>> table.exec.mini-batch.enabled > >>>>>>>> = > >>>>>>>>> true syntax, > >>>>>>>>> both for backward-compatibility and easy-to-use, and all the > other > >>>>>>>> systems > >>>>>>>>> don't have quotes on the key. > >>>>>>>>> > >>>>>>>>> > >>>>>>>>> Regarding "table.planner" vs "sql-client.planner", > >>>>>>>>> if we want to use "table.planner", I think we should explain > >> clearly > >>>>>>>> what's > >>>>>>>>> the scope it can be used in documentation. > >>>>>>>>> Otherwise, there will be users complaining why the planner > doesn't > >>>>>>>> change > >>>>>>>>> when setting the configuration on TableEnv. > >>>>>>>>> Would be better throwing an exception to indicate users it's now > >>>>>>>> allowed to > >>>>>>>>> change planner after TableEnv is initialized. > >>>>>>>>> However, it seems not easy to implement. > >>>>>>>>> > >>>>>>>>> Best, > >>>>>>>>> Jark > >>>>>>>>> > >>>>>>>>> On Thu, 4 Feb 2021 at 15:49, godfrey he <godfre...@gmail.com> > >>>> wrote: > >>>>>>>>> > >>>>>>>>>> Hi everyone, > >>>>>>>>>> > >>>>>>>>>> Regarding "table.planner" and "table.execution-mode" > >>>>>>>>>> If we define that those two options are just used to initialize > >> the > >>>>>>>>>> TableEnvironment, +1 for introducing table options instead of > >>>>>>>> sql-client > >>>>>>>>>> options. > >>>>>>>>>> > >>>>>>>>>> Regarding "the sql client, we will maintain two parsers", I want > >> to > >>>>>>>> give > >>>>>>>>>> more inputs: > >>>>>>>>>> We want to introduce sql-gateway into the Flink project (see > >>>> FLIP-24 > >>>>> & > >>>>>>>>>> FLIP-91 for more info [1] [2]). In the "gateway" mode, the CLI > >>>> client > >>>>>>>> and > >>>>>>>>>> the gateway service will communicate through Rest API. The " ADD > >>>> JAR > >>>>>>>>>> /local/path/jar " will be executed in the CLI client machine. So > >>>> when > >>>>>>>> we > >>>>>>>>>> submit a sql file which contains multiple statements, the CLI > >>>> client > >>>>>>>>> needs > >>>>>>>>>> to pick out the "ADD JAR" line, and also statements need to be > >>>>>>>> submitted > >>>>>>>>> or > >>>>>>>>>> executed one by one to make sure the result is correct. The sql > >>>> file > >>>>>>>> may > >>>>>>>>> be > >>>>>>>>>> look like: > >>>>>>>>>> > >>>>>>>>>> SET xxx=yyy; > >>>>>>>>>> create table my_table ...; > >>>>>>>>>> create table my_sink ...; > >>>>>>>>>> ADD JAR /local/path/jar1; > >>>>>>>>>> create function my_udf as com....MyUdf; > >>>>>>>>>> insert into my_sink select ..., my_udf(xx) from ...; > >>>>>>>>>> REMOVE JAR /local/path/jar1; > >>>>>>>>>> drop function my_udf; > >>>>>>>>>> ADD JAR /local/path/jar2; > >>>>>>>>>> create function my_udf as com....MyUdf2; > >>>>>>>>>> insert into my_sink select ..., my_udf(xx) from ...; > >>>>>>>>>> > >>>>>>>>>> The lines need to be splitted into multiple statements first in > >> the > >>>>>>>> CLI > >>>>>>>>>> client, there are two approaches: > >>>>>>>>>> 1. The CLI client depends on the sql-parser: the sql-parser > splits > >>>>> the > >>>>>>>>>> lines and tells which lines are "ADD JAR". > >>>>>>>>>> pro: there is only one parser > >>>>>>>>>> cons: It's a little heavy that the CLI client depends on the > >>>>>>>> sql-parser, > >>>>>>>>>> because the CLI client is just a simple tool which receives the > >>>> user > >>>>>>>>>> commands and displays the result. The non "ADD JAR" command will > >> be > >>>>>>>>> parsed > >>>>>>>>>> twice. > >>>>>>>>>> > >>>>>>>>>> 2. The CLI client splits the lines into multiple statements and > >>>> finds > >>>>>>>> the > >>>>>>>>>> ADD JAR command through regex matching. > >>>>>>>>>> pro: The CLI client is very light-weight. > >>>>>>>>>> cons: there are two parsers. > >>>>>>>>>> > >>>>>>>>>> (personally, I prefer the second option) > >>>>>>>>>> > >>>>>>>>>> Regarding "SHOW or LIST JARS", I think we can support them both. > >>>>>>>>>> For default dialect, we support SHOW JARS, but if we switch to > >> hive > >>>>>>>>>> dialect, LIST JARS is also supported. > >>>>>>>>>> > >>>>>>>>>> > >>>>>>>>>> [1] > >>>>>>>>> > >>>>> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-24+-+SQL+Client > >>>>>>>>>> [2] > >>>>>>>>>> > >>>>>>>>>> > >>>>>>>>> > >>>>>>>> > >>>>> > >>>> > >> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-91%3A+Support+SQL+Client+Gateway > >>>>>>>>>> > >>>>>>>>>> Best, > >>>>>>>>>> Godfrey > >>>>>>>>>> > >>>>>>>>>> Rui Li <lirui.fu...@gmail.com> 于2021年2月4日周四 上午10:40写道: > >>>>>>>>>> > >>>>>>>>>>> Hi guys, > >>>>>>>>>>> > >>>>>>>>>>> Regarding #3 and #4, I agree SHOW JARS is more consistent with > >>>> other > >>>>>>>>>>> commands than LIST JARS. I don't have a strong opinion 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 of systems accept both EXIT > and > >>>>>>>> QUIT > >>>>>>>>> as > >>>>>>>>>>> the command to terminate the program. So if that's not hard to > >>>>>>>> achieve, > >>>>>>>>>> and > >>>>>>>>>>> will make users happier, I don't see a reason why we must > choose > >>>> one > >>>>>>>>> over > >>>>>>>>>>> the other. > >>>>>>>>>>> > >>>>>>>>>>> On Wed, Feb 3, 2021 at 10:33 PM Timo Walther < > twal...@apache.org > >>> > >>>>>>>>> wrote: > >>>>>>>>>>> > >>>>>>>>>>>> Hi everyone, > >>>>>>>>>>>> > >>>>>>>>>>>> some feedback regarding the open questions. Maybe we can > discuss > >>>>>>>> the > >>>>>>>>>>>> `TableEnvironment.executeMultiSql` story offline to determine > >> how > >>>>>>>> we > >>>>>>>>>>>> proceed with this in the near future. > >>>>>>>>>>>> > >>>>>>>>>>>> 1) "whether the table environment has the ability to update > >>>>>>>> itself" > >>>>>>>>>>>> > >>>>>>>>>>>> Maybe there was some misunderstanding. I don't think that we > >>>>>>>> should > >>>>>>>>>>>> support > >>>>>>>> `tEnv.getConfig.getConfiguration.setString("table.planner", > >>>>>>>>>>>> "old")`. Instead I'm proposing to support > >>>>>>>>>>>> `TableEnvironment.create(Configuration)` where planner and > >>>>>>>> execution > >>>>>>>>>>>> mode are read immediately and a subsequent changes to these > >>>>>>>> options > >>>>>>>>>> will > >>>>>>>>>>>> have no effect. We are doing it similar in `new > >>>>>>>>>>>> StreamExecutionEnvironment(Configuration)`. These two > >>>>>>>> ConfigOption's > >>>>>>>>>>>> must not be SQL Client specific but can be part of the core > >> table > >>>>>>>>> code > >>>>>>>>>>>> base. Many users would like to get a 100% preconfigured > >>>>>>>> environment > >>>>>>>>>> from > >>>>>>>>>>>> just Configuration. And this is not possible right now. We can > >>>>>>>> solve > >>>>>>>>>>>> both use cases in one change. > >>>>>>>>>>>> > >>>>>>>>>>>> 2) "the sql client, we will maintain two parsers" > >>>>>>>>>>>> > >>>>>>>>>>>> I remember we had some discussion about this and decided that > we > >>>>>>>>> would > >>>>>>>>>>>> like to maintain only one parser. In the end it is "One Flink > >>>> SQL" > >>>>>>>>>> where > >>>>>>>>>>>> commands influence each other also with respect to keywords. > It > >>>>>>>>> should > >>>>>>>>>>>> be fine to include the SQL Client commands in the Flink > parser. > >>>> Of > >>>>>>>>>>>> cource the table environment would not be able to handle the > >>>>>>>>>> `Operation` > >>>>>>>>>>>> instance that would be the result but we can introduce hooks > to > >>>>>>>>> handle > >>>>>>>>>>>> those `Operation`s. Or we introduce parser extensions. > >>>>>>>>>>>> > >>>>>>>>>>>> Can we skip `table.job.async` in the first version? We should > >>>>>>>> further > >>>>>>>>>>>> discuss whether we introduce a special SQL clause for wrapping > >>>>>>>> async > >>>>>>>>>>>> behavior or if we use a config option? Esp. for streaming > >> queries > >>>>>>>> we > >>>>>>>>>>>> need to be careful and should force users to either "one > INSERT > >>>>>>>> INTO" > >>>>>>>>>> or > >>>>>>>>>>>> "one STATEMENT SET". > >>>>>>>>>>>> > >>>>>>>>>>>> 3) 4) "HIVE also uses these commands" > >>>>>>>>>>>> > >>>>>>>>>>>> In general, Hive is not a good reference. Aligning the > commands > >>>>>>>> more > >>>>>>>>>>>> with the remaining commands should be our goal. We just had a > >>>>>>>> MODULE > >>>>>>>>>>>> discussion where we selected SHOW instead of LIST. But it is > >> true > >>>>>>>>> that > >>>>>>>>>>>> JARs are not part of the catalog which is why I would not use > >>>>>>>>>>>> CREATE/DROP. ADD/REMOVE are commonly siblings in the English > >>>>>>>>> language. > >>>>>>>>>>>> Take a look at the Java collection API as another example. > >>>>>>>>>>>> > >>>>>>>>>>>> 6) "Most of the commands should belong to the table > environment" > >>>>>>>>>>>> > >>>>>>>>>>>> Thanks for updating the FLIP this makes things easier to > >>>>>>>> understand. > >>>>>>>>> It > >>>>>>>>>>>> is good to see that most commends will be available in > >>>>>>>>>> TableEnvironment. > >>>>>>>>>>>> However, I would also support SET and RESET for consistency. > >>>>>>>> Again, > >>>>>>>>>> from > >>>>>>>>>>>> an architectural point of view, if we would allow some kind of > >>>>>>>>>>>> `Operation` hook in table environment, we could check for SQL > >>>>>>>> Client > >>>>>>>>>>>> specific options and forward to regular > >>>>>>>>> `TableConfig.getConfiguration` > >>>>>>>>>>>> otherwise. What do you think? > >>>>>>>>>>>> > >>>>>>>>>>>> Regards, > >>>>>>>>>>>> Timo > >>>>>>>>>>>> > >>>>>>>>>>>> > >>>>>>>>>>>> On 03.02.21 08:58, Jark Wu wrote: > >>>>>>>>>>>>> Hi Timo, > >>>>>>>>>>>>> > >>>>>>>>>>>>> I will respond some of the questions: > >>>>>>>>>>>>> > >>>>>>>>>>>>> 1) SQL client specific options > >>>>>>>>>>>>> > >>>>>>>>>>>>> Whether it starts with "table" or "sql-client" depends on > where > >>>>>>>> the > >>>>>>>>>>>>> configuration takes effect. > >>>>>>>>>>>>> If it is a table configuration, we should make clear what's > the > >>>>>>>>>>> behavior > >>>>>>>>>>>>> when users change > >>>>>>>>>>>>> the configuration in the lifecycle of TableEnvironment. > >>>>>>>>>>>>> > >>>>>>>>>>>>> I agree with Shengkai `sql-client.planner` and > >>>>>>>>>>>> `sql-client.execution.mode` > >>>>>>>>>>>>> are something special > >>>>>>>>>>>>> that can't be changed after TableEnvironment has been > >>>>>>>> initialized. > >>>>>>>>>> You > >>>>>>>>>>>> can > >>>>>>>>>>>>> see > >>>>>>>>>>>>> `StreamExecutionEnvironment` provides `configure()` method > to > >>>>>>>>>> override > >>>>>>>>>>>>> configuration after > >>>>>>>>>>>>> StreamExecutionEnvironment has been initialized. > >>>>>>>>>>>>> > >>>>>>>>>>>>> Therefore, I think it would be better to still use > >>>>>>>>>>> `sql-client.planner` > >>>>>>>>>>>>> and `sql-client.execution.mode`. > >>>>>>>>>>>>> > >>>>>>>>>>>>> 2) Execution file > >>>>>>>>>>>>> > >>>>>>>>>>>>> >From my point of view, there is a big difference between > >>>>>>>>>>>>> `sql-client.job.detach` and > >>>>>>>>>>>>> `TableEnvironment.executeMultiSql()` that > >>>>>>>> `sql-client.job.detach` > >>>>>>>>>> will > >>>>>>>>>>>>> affect every single DML statement > >>>>>>>>>>>>> in the terminal, not only the statements in SQL files. I > think > >>>>>>>> the > >>>>>>>>>>> single > >>>>>>>>>>>>> DML statement in the interactive > >>>>>>>>>>>>> terminal is something like tEnv#executeSql() instead of > >>>>>>>>>>>>> tEnv#executeMultiSql. > >>>>>>>>>>>>> So I don't like the "multi" and "sql" keyword in > >>>>>>>>>>> `table.multi-sql-async`. > >>>>>>>>>>>>> I just find that runtime provides a configuration called > >>>>>>>>>>>>> "execution.attached" [1] which is false by default > >>>>>>>>>>>>> which specifies if the pipeline is submitted in attached or > >>>>>>>>> detached > >>>>>>>>>>>> mode. > >>>>>>>>>>>>> It provides exactly the same > >>>>>>>>>>>>> functionality of `sql-client.job.detach`. What do you think > >>>>>>>> about > >>>>>>>>>> using > >>>>>>>>>>>>> this option? > >>>>>>>>>>>>> > >>>>>>>>>>>>> If we also want to support this config in TableEnvironment, I > >>>>>>>> think > >>>>>>>>>> it > >>>>>>>>>>>>> should also affect the DML execution > >>>>>>>>>>>>> of `tEnv#executeSql()`, not only DMLs in > >>>>>>>>> `tEnv#executeMultiSql()`. > >>>>>>>>>>>>> Therefore, the behavior may look like this: > >>>>>>>>>>>>> > >>>>>>>>>>>>> val tableResult = tEnv.executeSql("INSERT INTO ...") ==> > async > >>>>>>>> by > >>>>>>>>>>>> default > >>>>>>>>>>>>> tableResult.await() ==> manually block until finish > >>>>>>>>>>>>> > >>>>>>>> > tEnv.getConfig().getConfiguration().setString("execution.attached", > >>>>>>>>>>>> "true") > >>>>>>>>>>>>> val tableResult2 = tEnv.executeSql("INSERT INTO ...") ==> > >> sync, > >>>>>>>>>> don't > >>>>>>>>>>>> need > >>>>>>>>>>>>> to wait on the TableResult > >>>>>>>>>>>>> tEnv.executeMultiSql( > >>>>>>>>>>>>> """ > >>>>>>>>>>>>> CREATE TABLE .... ==> always sync > >>>>>>>>>>>>> INSERT INTO ... => sync, because we set configuration above > >>>>>>>>>>>>> SET execution.attached = false; > >>>>>>>>>>>>> INSERT INTO ... => async > >>>>>>>>>>>>> """) > >>>>>>>>>>>>> > >>>>>>>>>>>>> On the other hand, I think `sql-client.job.detach` > >>>>>>>>>>>>> and `TableEnvironment.executeMultiSql()` should be two > separate > >>>>>>>>>> topics, > >>>>>>>>>>>>> as Shengkai mentioned above, SQL CLI only depends on > >>>>>>>>>>>>> `TableEnvironment#executeSql()` to support multi-line > >>>>>>>> statements. > >>>>>>>>>>>>> I'm fine with making `executeMultiSql()` clear but don't want > >>>>>>>> it to > >>>>>>>>>>> block > >>>>>>>>>>>>> this FLIP, maybe we can discuss this in another thread. > >>>>>>>>>>>>> > >>>>>>>>>>>>> > >>>>>>>>>>>>> Best, > >>>>>>>>>>>>> Jark > >>>>>>>>>>>>> > >>>>>>>>>>>>> [1]: > >>>>>>>>>>>>> > >>>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>> > >>>>>>>>> > >>>>>>>> > >>>>> > >>>> > >> > https://ci.apache.org/projects/flink/flink-docs-master/deployment/config.html#execution-attached > >>>>>>>>>>>>> > >>>>>>>>>>>>> On Wed, 3 Feb 2021 at 15:33, Shengkai Fang < > fskm...@gmail.com> > >>>>>>>>>> wrote: > >>>>>>>>>>>>> > >>>>>>>>>>>>>> Hi, Timo. > >>>>>>>>>>>>>> Thanks for your detailed feedback. I have some thoughts > about > >>>>>>>> your > >>>>>>>>>>>>>> feedback. > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> *Regarding #1*: I think the main problem is whether the > table > >>>>>>>>>>>> environment > >>>>>>>>>>>>>> has the ability to update itself. Let's take a simple > program > >>>>>>>> as > >>>>>>>>> an > >>>>>>>>>>>>>> example. > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> ``` > >>>>>>>>>>>>>> TableEnvironment tEnv = TableEnvironment.create(...); > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> tEnv.getConfig.getConfiguration.setString("table.planner", > >>>>>>>> "old"); > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> tEnv.executeSql("..."); > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> ``` > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> If we regard this option as a table option, users don't have > >> to > >>>>>>>>>> create > >>>>>>>>>>>>>> another table environment manually. In that case, tEnv needs > >> to > >>>>>>>>>> check > >>>>>>>>>>>>>> whether the current mode and planner are the same as before > >>>>>>>> when > >>>>>>>>>>>> executeSql > >>>>>>>>>>>>>> or explainSql. I don't think it's easy work for the table > >>>>>>>>>> environment, > >>>>>>>>>>>>>> especially if users have a StreamExecutionEnvironment but > set > >>>>>>>> old > >>>>>>>>>>>> planner > >>>>>>>>>>>>>> and batch mode. But when we make this option as a sql client > >>>>>>>>> option, > >>>>>>>>>>>> users > >>>>>>>>>>>>>> only use the SET command to change the setting. We can > rebuild > >>>>>>>> a > >>>>>>>>> new > >>>>>>>>>>>> table > >>>>>>>>>>>>>> environment when set successes. > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> *Regarding #2*: I think we need to discuss the > implementation > >>>>>>>>> before > >>>>>>>>>>>>>> continuing this topic. In the sql client, we will maintain > two > >>>>>>>>>>> parsers. > >>>>>>>>>>>> The > >>>>>>>>>>>>>> first parser(client parser) will only match the sql client > >>>>>>>>> commands. > >>>>>>>>>>> If > >>>>>>>>>>>> the > >>>>>>>>>>>>>> client parser can't parse the statement, we will leverage > the > >>>>>>>>> power > >>>>>>>>>> of > >>>>>>>>>>>> the > >>>>>>>>>>>>>> table environment to execute. According to our blueprint, > >>>>>>>>>>>>>> TableEnvironment#executeSql is enough for the sql client. > >>>>>>>>> Therefore, > >>>>>>>>>>>>>> TableEnvironment#executeMultiSql is out-of-scope for this > >> FLIP. > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> But if we need to introduce the > >>>>>>>> `TableEnvironment.executeMultiSql` > >>>>>>>>>> in > >>>>>>>>>>>> the > >>>>>>>>>>>>>> future, I think it's OK to use the option > >>>>>>>> `table.multi-sql-async` > >>>>>>>>>>> rather > >>>>>>>>>>>>>> than option `sql-client.job.detach`. But we think the name > is > >>>>>>>> not > >>>>>>>>>>>> suitable > >>>>>>>>>>>>>> because the name is confusing for others. When setting the > >>>>>>>> option > >>>>>>>>>>>> false, we > >>>>>>>>>>>>>> just mean it will block the execution of the INSERT INTO > >>>>>>>>> statement, > >>>>>>>>>>> not > >>>>>>>>>>>> DDL > >>>>>>>>>>>>>> or others(other sql statements are always executed > >>>>>>>> synchronously). > >>>>>>>>>> So > >>>>>>>>>>>> how > >>>>>>>>>>>>>> about `table.job.async`? It only works for the sql-client > and > >>>>>>>> the > >>>>>>>>>>>>>> executeMultiSql. If we set this value false, the table > >>>>>>>> environment > >>>>>>>>>>> will > >>>>>>>>>>>>>> return the result until the job finishes. > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> *Regarding #3, #4*: I still think we should use DELETE JAR > and > >>>>>>>>> LIST > >>>>>>>>>>> JAR > >>>>>>>>>>>>>> because HIVE also uses these commands to add the jar into > the > >>>>>>>>>>> classpath > >>>>>>>>>>>> or > >>>>>>>>>>>>>> delete the jar. If we use such commands, it can reduce our > >>>>>>>> work > >>>>>>>>> for > >>>>>>>>>>>> hive > >>>>>>>>>>>>>> compatibility. > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> For SHOW JAR, I think the main concern is the jars are not > >>>>>>>>>> maintained > >>>>>>>>>>> by > >>>>>>>>>>>>>> the Catalog. If we really needs to keep consistent with SQL > >>>>>>>>> grammar, > >>>>>>>>>>>> maybe > >>>>>>>>>>>>>> we should use > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> `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 <NOTE> tag to > >>>>>>>>>> identify > >>>>>>>>>>>> which > >>>>>>>>>>>>>> commands should belong to the sql client and which commands > >>>>>>>> should > >>>>>>>>>>>> belong > >>>>>>>>>>>>>> to the table environment. I also add a new section about > >>>>>>>>>>> implementation > >>>>>>>>>>>>>> details in the FLIP. > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> Best, > >>>>>>>>>>>>>> Shengkai > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> Timo Walther <twal...@apache.org> 于2021年2月2日周二 下午6:43写道: > >>>>>>>>>>>>>> > >>>>>>>>>>>>>>> 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 to > >>>>>>>> `StreamExecutionEnvironment` > >>>>>>>>>> and > >>>>>>>>>>>>>>> `ExecutionConfig#configure` that have been added recently, > we > >>>>>>>>>> should > >>>>>>>>>>>>>>> offer a possibility for TableEnvironment. How about we > offer > >>>>>>>>>>>>>>> `TableEnvironment.create(ReadableConfig)` and add a > >>>>>>>>> `table.planner` > >>>>>>>>>>> and > >>>>>>>>>>>>>>> `table.execution-mode` to > >>>>>>>>>>>>>>> `org.apache.flink.table.api.config.TableConfigOptions`? > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> 2) Execution file > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> Did you have a look at the Appendix of FLIP-84 [1] > including > >>>>>>>> the > >>>>>>>>>>>> mailing > >>>>>>>>>>>>>>> list thread at that time? Could you further elaborate how > the > >>>>>>>>>>>>>>> multi-statement execution should work for a unified > >>>>>>>>> batch/streaming > >>>>>>>>>>>>>>> story? According to our past discussions, each line in an > >>>>>>>>> execution > >>>>>>>>>>>> file > >>>>>>>>>>>>>>> should be executed blocking which means a streaming query > >>>>>>>> needs a > >>>>>>>>>>>>>>> statement set to execute multiple INSERT INTO statement, > >>>>>>>> correct? > >>>>>>>>>> We > >>>>>>>>>>>>>>> should also offer this functionality in > >>>>>>>>>>>>>>> `TableEnvironment.executeMultiSql()`. Whether > >>>>>>>>>> `sql-client.job.detach` > >>>>>>>>>>>> is > >>>>>>>>>>>>>>> SQL Client specific needs to be determined, it could also > be > >> a > >>>>>>>>>>> general > >>>>>>>>>>>>>>> `table.multi-sql-async` option? > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> 3) DELETE JAR > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> Shouldn't the opposite of "ADD" be "REMOVE"? "DELETE" > sounds > >>>>>>>> like > >>>>>>>>>> one > >>>>>>>>>>>> is > >>>>>>>>>>>>>>> actively deleting the JAR in the corresponding path. > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> 4) LIST JAR > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> This should be `SHOW JARS` according to other SQL commands > >>>>>>>> such > >>>>>>>>> as > >>>>>>>>>>>> `SHOW > >>>>>>>>>>>>>>> CATALOGS`, `SHOW TABLES`, etc. [2]. > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> 5) EXPLAIN [ExplainDetail[, ExplainDetail]*] > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> We should keep the details in sync with > >>>>>>>>>>>>>>> `org.apache.flink.table.api.ExplainDetail` and avoid > >> confusion > >>>>>>>>>> about > >>>>>>>>>>>>>>> differently named ExplainDetails. I would vote for > >>>>>>>>> `ESTIMATED_COST` > >>>>>>>>>>>>>>> instead of `COST`. I'm sure the original author had a > reason > >>>>>>>> why > >>>>>>>>> to > >>>>>>>>>>>> call > >>>>>>>>>>>>>>> it that way. > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> 6) Implementation details > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> It would be nice to understand how we plan to implement the > >>>>>>>> given > >>>>>>>>>>>>>>> features. Most of the commands and config options should go > >>>>>>>> into > >>>>>>>>>>>>>>> TableEnvironment and SqlParser directly, correct? This way > >>>>>>>> users > >>>>>>>>>>> have a > >>>>>>>>>>>>>>> unified way of using Flink SQL. TableEnvironment would > >>>>>>>> provide a > >>>>>>>>>>>> similar > >>>>>>>>>>>>>>> user experience in notebooks or interactive programs than > the > >>>>>>>> SQL > >>>>>>>>>>>> Client. > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> [1] > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>> > >>>>>>>>> > >>>>>>>> > >>>>> > >>>> > >> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=134745878 > >>>>>>>>>>>>>>> [2] > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>> > >>>>>>>>> > >>>>>>>> > >>>>> > >>>> > >> > https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/show.html > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> Regards, > >>>>>>>>>>>>>>> Timo > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> On 02.02.21 10:13, Shengkai Fang wrote: > >>>>>>>>>>>>>>>> Sorry for the typo. I mean `RESET` is much better rather > >> than > >>>>>>>>>>> `UNSET`. > >>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>> Shengkai Fang <fskm...@gmail.com> 于2021年2月2日周二 下午4:44写道: > >>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>> Hi, Jingsong. > >>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>> Thanks for your reply. I think `UNSET` is much better. > >>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>> 1. We don't need to introduce another command `UNSET`. > >>>>>>>> `RESET` > >>>>>>>>> is > >>>>>>>>>>>>>>>>> supported in the current sql client now. Our proposal > just > >>>>>>>>>> extends > >>>>>>>>>>>> its > >>>>>>>>>>>>>>>>> grammar and allow users to reset the specified keys. > >>>>>>>>>>>>>>>>> 2. Hive beeline also uses `RESET` to set the key to the > >>>>>>>> default > >>>>>>>>>>>>>>> value[1]. > >>>>>>>>>>>>>>>>> I think it is more friendly for batch users. > >>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>> Best, > >>>>>>>>>>>>>>>>> Shengkai > >>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>> [1] > >>>>>>>>>>>>>>> > >>>>>>>>>> > >>>> https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients > >>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>> Jingsong Li <jingsongl...@gmail.com> 于2021年2月2日周二 > >> 下午1:56写道: > >>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>> Thanks for the proposal, yes, sql-client is too > outdated. > >>>>>>>> +1 > >>>>>>>>> for > >>>>>>>>>>>>>>>>>> improving it. > >>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>> About "SET" and "RESET", Why not be "SET" and "UNSET"? > >>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>> Best, > >>>>>>>>>>>>>>>>>> Jingsong > >>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>> On Mon, Feb 1, 2021 at 2:46 PM Rui Li < > >>>>>>>> lirui.fu...@gmail.com> > >>>>>>>>>>>> wrote: > >>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> Thanks Shengkai for the update! The proposed changes > look > >>>>>>>>> good > >>>>>>>>>> to > >>>>>>>>>>>>>> me. > >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> On Fri, Jan 29, 2021 at 8:26 PM Shengkai Fang < > >>>>>>>>>> fskm...@gmail.com > >>>>>>>>>>>> > >>>>>>>>>>>>>>> wrote: > >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>> Hi, Rui. > >>>>>>>>>>>>>>>>>>>> You are right. I have already modified the FLIP. > >>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>> The main changes: > >>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>> # -f parameter has no restriction about the statement > >>>>>>>> type. > >>>>>>>>>>>>>>>>>>>> Sometimes, users use the pipe to redirect the result > of > >>>>>>>>>> queries > >>>>>>>>>>> to > >>>>>>>>>>>>>>>>>>> debug > >>>>>>>>>>>>>>>>>>>> when submitting job by -f parameter. It's much > >> convenient > >>>>>>>>>>>> comparing > >>>>>>>>>>>>>>> to > >>>>>>>>>>>>>>>>>>>> writing INSERT INTO statements. > >>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>> # Add a new sql client option `sql-client.job.detach` > . > >>>>>>>>>>>>>>>>>>>> Users prefer to execute jobs one by one in the batch > >>>>>>>> mode. > >>>>>>>>>> Users > >>>>>>>>>>>>>> can > >>>>>>>>>>>>>>>>>>> set > >>>>>>>>>>>>>>>>>>>> this option false and the client will process the next > >>>>>>>> job > >>>>>>>>>> until > >>>>>>>>>>>>>> the > >>>>>>>>>>>>>>>>>>>> current job finishes. The default value of this option > >> is > >>>>>>>>>> false, > >>>>>>>>>>>>>>> which > >>>>>>>>>>>>>>>>>>>> means the client will execute the next job when the > >>>>>>>> current > >>>>>>>>>> job > >>>>>>>>>>> is > >>>>>>>>>>>>>>>>>>>> submitted. > >>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>> Best, > >>>>>>>>>>>>>>>>>>>> Shengkai > >>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>> Rui Li <lirui.fu...@gmail.com> 于2021年1月29日周五 > 下午4:52写道: > >>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>> Hi Shengkai, > >>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>> Regarding #2, maybe the -f options in flink and hive > >>>>>>>> have > >>>>>>>>>>>>>> different > >>>>>>>>>>>>>>>>>>>>> implications, and we should clarify the behavior. For > >>>>>>>>>> example, > >>>>>>>>>>> if > >>>>>>>>>>>>>>> the > >>>>>>>>>>>>>>>>>>>>> client just submits the job and exits, what happens > if > >>>>>>>> the > >>>>>>>>>> file > >>>>>>>>>>>>>>>>>>> contains > >>>>>>>>>>>>>>>>>>>>> two INSERT statements? I don't think we should treat > >>>>>>>> them > >>>>>>>>> as > >>>>>>>>>> a > >>>>>>>>>>>>>>>>>>> statement > >>>>>>>>>>>>>>>>>>>>> set, because users should explicitly write BEGIN > >>>>>>>> STATEMENT > >>>>>>>>>> SET > >>>>>>>>>>> in > >>>>>>>>>>>>>>> that > >>>>>>>>>>>>>>>>>>>>> case. And the client shouldn't asynchronously submit > >> the > >>>>>>>>> two > >>>>>>>>>>>> jobs, > >>>>>>>>>>>>>>>>>>> because > >>>>>>>>>>>>>>>>>>>>> the 2nd may depend on the 1st, right? > >>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 29, 2021 at 4:30 PM Shengkai Fang < > >>>>>>>>>>> fskm...@gmail.com > >>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> wrote: > >>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>> Hi Rui, > >>>>>>>>>>>>>>>>>>>>>> Thanks for your feedback. I agree with your > >>>>>>>> suggestions. > >>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>> For the suggestion 1: Yes. we are plan to strengthen > >>>>>>>> the > >>>>>>>>> set > >>>>>>>>>>>>>>>>>>> command. In > >>>>>>>>>>>>>>>>>>>>>> the implementation, it will just put the key-value > >> into > >>>>>>>>> the > >>>>>>>>>>>>>>>>>>>>>> `Configuration`, which will be used to generate the > >>>>>>>> table > >>>>>>>>>>>> config. > >>>>>>>>>>>>>>> If > >>>>>>>>>>>>>>>>>>> hive > >>>>>>>>>>>>>>>>>>>>>> supports to read the setting from the table config, > >>>>>>>> users > >>>>>>>>>> are > >>>>>>>>>>>>>> able > >>>>>>>>>>>>>>>>>>> to set > >>>>>>>>>>>>>>>>>>>>>> the hive-related settings. > >>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>> For the suggestion 2: The -f parameter will submit > the > >>>>>>>> job > >>>>>>>>>> and > >>>>>>>>>>>>>>> exit. > >>>>>>>>>>>>>>>>>>> If > >>>>>>>>>>>>>>>>>>>>>> the queries never end, users have to cancel the job > by > >>>>>>>>>>>>>> themselves, > >>>>>>>>>>>>>>>>>>> which is > >>>>>>>>>>>>>>>>>>>>>> not reliable(people may forget their jobs). In most > >>>>>>>> case, > >>>>>>>>>>>> queries > >>>>>>>>>>>>>>>>>>> are used > >>>>>>>>>>>>>>>>>>>>>> to analyze the data. Users should use queries in the > >>>>>>>>>>> interactive > >>>>>>>>>>>>>>>>>>> mode. > >>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>> Best, > >>>>>>>>>>>>>>>>>>>>>> Shengkai > >>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>> Rui Li <lirui.fu...@gmail.com> 于2021年1月29日周五 > >> 下午3:18写道: > >>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>> Thanks Shengkai for bringing up this discussion. I > >>>>>>>> think > >>>>>>>>> it > >>>>>>>>>>>>>>> covers a > >>>>>>>>>>>>>>>>>>>>>>> lot of useful features which will dramatically > >> improve > >>>>>>>>> the > >>>>>>>>>>>>>>>>>>> usability of our > >>>>>>>>>>>>>>>>>>>>>>> SQL Client. I have two questions regarding the > FLIP. > >>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>> 1. Do you think we can let users set arbitrary > >>>>>>>>>> configurations > >>>>>>>>>>>>>> via > >>>>>>>>>>>>>>>>>>> the > >>>>>>>>>>>>>>>>>>>>>>> SET command? A connector may have its own > >>>>>>>> configurations > >>>>>>>>>> and > >>>>>>>>>>> we > >>>>>>>>>>>>>>>>>>> don't have > >>>>>>>>>>>>>>>>>>>>>>> a way to dynamically change such configurations in > >> SQL > >>>>>>>>>>> Client. > >>>>>>>>>>>>>> For > >>>>>>>>>>>>>>>>>>> example, > >>>>>>>>>>>>>>>>>>>>>>> users may want to be able to change hive conf when > >>>>>>>> using > >>>>>>>>>> hive > >>>>>>>>>>>>>>>>>>> connector [1]. > >>>>>>>>>>>>>>>>>>>>>>> 2. Any reason why we have to forbid queries in SQL > >>>>>>>> files > >>>>>>>>>>>>>> specified > >>>>>>>>>>>>>>>>>>> with > >>>>>>>>>>>>>>>>>>>>>>> the -f option? Hive supports a similar -f option > but > >>>>>>>>> allows > >>>>>>>>>>>>>>> queries > >>>>>>>>>>>>>>>>>>> in the > >>>>>>>>>>>>>>>>>>>>>>> file. And a common use case is to run some query > and > >>>>>>>>>> redirect > >>>>>>>>>>>>>> the > >>>>>>>>>>>>>>>>>>> results > >>>>>>>>>>>>>>>>>>>>>>> to a file. So I think maybe flink users would like > to > >>>>>>>> do > >>>>>>>>>> the > >>>>>>>>>>>>>> same, > >>>>>>>>>>>>>>>>>>>>>>> especially in batch scenarios. > >>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>> [1] > >> https://issues.apache.org/jira/browse/FLINK-20590 > >>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>> On Fri, Jan 29, 2021 at 10:46 AM Sebastian Liu < > >>>>>>>>>>>>>>>>>>> liuyang0...@gmail.com> > >>>>>>>>>>>>>>>>>>>>>>> wrote: > >>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> Hi Shengkai, > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> Glad to see this improvement. And I have some > >>>>>>>> additional > >>>>>>>>>>>>>>>>>>> suggestions: > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> #1. Unify the TableEnvironment in ExecutionContext > >> to > >>>>>>>>>>>>>>>>>>>>>>>> StreamTableEnvironment for both streaming and > batch > >>>>>>>> sql. > >>>>>>>>>>>>>>>>>>>>>>>> #2. Improve the way of results retrieval: sql > client > >>>>>>>>>> collect > >>>>>>>>>>>>>> the > >>>>>>>>>>>>>>>>>>>>>>>> results > >>>>>>>>>>>>>>>>>>>>>>>> locally all at once using accumulators at present, > >>>>>>>>>>>>>>>>>>>>>>>> which may have memory issues in JM or > >> Local > >>>>>>>> for > >>>>>>>>>> the > >>>>>>>>>>>> big > >>>>>>>>>>>>>>> query > >>>>>>>>>>>>>>>>>>>>>>>> result. > >>>>>>>>>>>>>>>>>>>>>>>> Accumulator is only suitable for testing purpose. > >>>>>>>>>>>>>>>>>>>>>>>> We may change to use SelectTableSink, > >> which > >>>>>>>> is > >>>>>>>>>> based > >>>>>>>>>>>>>>>>>>>>>>>> on CollectSinkOperatorCoordinator. > >>>>>>>>>>>>>>>>>>>>>>>> #3. Do we need to consider Flink SQL gateway which > >>>>>>>> is in > >>>>>>>>>>>>>> FLIP-91. > >>>>>>>>>>>>>>>>>>> Seems > >>>>>>>>>>>>>>>>>>>>>>>> that this FLIP has not moved forward for a long > >> time. > >>>>>>>>>>>>>>>>>>>>>>>> Provide a long running service out of > the > >>>>>>>> box to > >>>>>>>>>>>>>>> facilitate > >>>>>>>>>>>>>>>>>>> the > >>>>>>>>>>>>>>>>>>>>>>>> sql > >>>>>>>>>>>>>>>>>>>>>>>> submission is necessary. > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> What do you think of these? > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> [1] > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>> > >>>>>>>>> > >>>>>>>> > >>>>> > >>>> > >> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-91%3A+Support+SQL+Client+Gateway > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> Shengkai Fang <fskm...@gmail.com> 于2021年1月28日周四 > >>>>>>>>> 下午8:54写道: > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> Hi devs, > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> Jark and I want to start a discussion about > >>>>>>>>> FLIP-163:SQL > >>>>>>>>>>>>>> Client > >>>>>>>>>>>>>>>>>>>>>>>>> Improvements. > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> Many users have complained about the problems of > >> the > >>>>>>>>> sql > >>>>>>>>>>>>>> client. > >>>>>>>>>>>>>>>>>>> For > >>>>>>>>>>>>>>>>>>>>>>>>> example, users can not register the table > proposed > >>>>>>>> by > >>>>>>>>>>>> FLIP-95. > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> The main changes in this FLIP: > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> - use -i parameter to specify the sql file to > >>>>>>>>> initialize > >>>>>>>>>>> the > >>>>>>>>>>>>>>>>>>> table > >>>>>>>>>>>>>>>>>>>>>>>>> environment and deprecated YAML file; > >>>>>>>>>>>>>>>>>>>>>>>>> - add -f to submit sql file and deprecated '-u' > >>>>>>>>>> parameter; > >>>>>>>>>>>>>>>>>>>>>>>>> - add more interactive commands, e.g ADD JAR; > >>>>>>>>>>>>>>>>>>>>>>>>> - support statement set syntax; > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> For more detailed changes, please refer to > >>>>>>>> FLIP-163[1]. > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> Look forward to your feedback. > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> Best, > >>>>>>>>>>>>>>>>>>>>>>>>> Shengkai > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> [1] > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>> > >>>>>>>>> > >>>>>>>> > >>>>> > >>>> > >> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-163%3A+SQL+Client+Improvements > >>>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> -- > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> *With kind regards > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>> ------------------------------------------------------------ > >>>>>>>>>>>>>>>>>>>>>>>> Sebastian Liu 刘洋 > >>>>>>>>>>>>>>>>>>>>>>>> Institute of Computing Technology, Chinese Academy > >> of > >>>>>>>>>>> Science > >>>>>>>>>>>>>>>>>>>>>>>> Mobile\WeChat: +86—15201613655 > >>>>>>>>>>>>>>>>>>>>>>>> E-mail: liuyang0...@gmail.com < > >> liuyang0...@gmail.com > >>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>>> QQ: 3239559* > >>>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>>>>>> -- > >>>>>>>>>>>>>>>>>>>>>>> Best regards! > >>>>>>>>>>>>>>>>>>>>>>> Rui Li > >>>>>>>>>