Hi Dian,
I proposed a solution that should be backwards compatible and solves our
Maven dependency problems in the corresponding issue.
I'm happy about feedback.
Regards,
Timo
Am 11.12.18 um 11:23 schrieb fudian.fd:
Hi Timo,
Thanks a lot for your reply. I think the cause to this problem is that
TableEnvironment.getTableEnvironment() returns the actual TableEnvironment
implementations instead of an interface or an abstract base class. Even the
porting of FLINK-11067 is done, I'm afraid that the problem may still exist.
For example, for batch TableEnvironment, both java.BatchTableEnvironment and
api.BatchTableEnvironment may be prompted for import. Could you share more
information about what you want to do with the 7 TableEnvironments in
FLINK-11067? Especially api.BatchTableEnvironment, api.StreamTableEnvironment
and TableEnvironment.
Thanks,
Dian
在 2018年12月11日,下午3:41,jincheng sun <sunjincheng...@gmail.com> 写道:
Hi Xuefu,
Thanks for your feedback, and mention the compatibility issues.
You are right the change will result version incompatibility. And we my
plan it's will be released in the version of 1.8.x.
To be frank, we have considered the compatibility approach, which is to
retain the current TableEnvironment, and then create a new one, such as
"GeneralTableEnvironment" for unified abstraction, and then Deprecated the
TableEnvironment. But we feel that the code is not clean enough, and the
long-term goal is that we need to make StreamTableEnvironment and
BatchTableEnvironment transparent to the user, so I tend to release this
change in 1.8.x, keeping the status quo in 1.7.x. What do you think? Any
feedback is welcome!
Thanks,
Jincheng
Zhang, Xuefu <xuef...@alibaba-inc.com> 于2018年12月11日周二 下午1:13写道:
Hi Jincheng,
Thanks for bringing this up. It seems making good sense to me. However,
one concern I have is about backward compatibility. Could you clarify
whether existing user program will break with the proposed changes?
The answer to the question would largely determine when this can be
introduced.
Thanks,
Xuefu
------------------------------------------------------------------
Sender:jincheng sun <sunjincheng...@gmail.com>
Sent at:2018 Dec 10 (Mon) 18:14
Recipient:dev <dev@flink.apache.org>
Subject:[DISCUSS] Enhance convenience of TableEnvironment in TableAPI/SQL
Hi All,
According to the feedback from users, the design of TableEnvironment is
very inconvenient for users, and often mistakenly imported by IDE,
especially for Java users, such as:
ExecutionEnvironment env = ...BatchTableEnvironment tEnv =
TableEnvironment.getTableEnvironment(env);
The user does not know which BatchTableEnvironment should be imported,
because there are three implementations of BatchTableEnvironment, shown as
below:
1. org.apache.flink.table.api.BatchTableEnvironment 2.
org.apache.flink.table.api.java.BatchTableEnvironment 3.
org.apache.flink.table.api.scala.BatchTableEnvironment
[image.png]
This brings unnecessary inconveniences to the flink user. To solve this
problem, Wei Zhong, Hequn Cheng, Dian Fu, Shaoxuan Wang and myself
discussed offline a bit and propose to change the inheritance diagram of
TableEnvironment is shown as follows:
1. AbstractTaleEnvironment - rename current TableEnvironment to
AbstractTableEnvironment, The functionality implemented by Abstract
TableEnvironment is stream and batch shared.2. TableEnvironment - Create a
new TableEnvironment(abstract), and defined all methods in
(java/scala)StreamTableEnvironment and (java/scala)BatchTableEnvironment.
In the implementation of BatchTableEnviroment and StreamTableEnviroment,
the unsupported operations will be reported as an error.
[image.png]
Then the usage as follows:
ExecutionEnvironment env = …TableEnvironment tEnv =
TableEnvironment.getTableEnvironment(env)
For detailed proposals please refer to the Google doc:
https://docs.google.com/document/d/1t-AUGuaChADddyJi6e0WLsTDEnf9ZkupvvBiQ4yTTEI/edit?usp=sharing
Any mail feedback and Google doc comment are welcome.
Thanks,
Jincheng