Hi Timo, Thanks for your summary of design in FLINK-11067's discuss! This proposal has two core objectives which I mentioned:
1. Must solve user import problems; 2. Unify interface definitions of TableEnvironment for stream and batch. I think FLINK-11067 can fit the #1. and we need create a new JIRA. for #2. i.e. User only take one TableEnvironment, e.g.: ExecutionEnvironment env = … TableEnvironment tEnv = TableEnvironment.getTableEnvironment(env) Of course, stream and batch unification will undermine existing compatibility, but I still think this is an effort we must make. At the SQL/TableAPI level, users don't need to know the existence of `BatchTableEnvironment` and `StreamTableEnvironment`. What do you think? Thanks, Jincheng jincheng sun <sunjincheng...@gmail.com> 于2018年12月12日周三 下午3:44写道: > HI Timo, > > Thanks for your feedback! And I'm glad to hear that you are already > thinking about import issues! > > 1. I commented on the solution you mentioned in FLINK-11067. I have the > same questions with Dian Fu, about the design of compatibility in the > google doc, I look forward to your reply. > > 2. About unified stream batch interface definition > >> However, I don't like the design of putting all methods of Batch and >> Stream environments into the base class and throw exceptions if not >> supported by base classes. This sounds not like a nice object oriented >> design and confuses users. > > > At present, we have unified the stream and batch interface definitions on > the Table, such as the `orderBy` operator. Although it only supports time > order on the stream, we still have the interface definition unified, check > it at runtime, if you want `orderBy` string on the stream, will throw an > exception. So we should unify the interface definition of > TableEnvironment in some way. When the stream and batch execution modes are > unified and Stream/BatchSource/sink are unified , a job can be run in > mix(Stream/Batch) mode. By then, a table can be either a toDataSet or a > ToDataStream. > > 3. About Scala dependencies > IMO. It is not expected to solve the Scala dependency problem in this > proposal(keep status quo). The Scala dependency problem is the goal of > FLIP-28. > This proposal has two core objectives: > 1) Must solve user import problems; > 2) Do our best to unify interface definitions of TableEnvironment for > stream and batch. > > So, I think we can solve the user problem and unify the interface between > Stream and Batch firstly. Regarding the separation of Scala and Java, I > agree that when we do FLIP-28, we can have a Java abstraction and a Scala > abstraction in `flink-table-api.java` and `flink-table-api.scala` > respectively which we mentioned in Q/A session of google doc. > > Best, > Jincheng > > > Timo Walther <twal...@apache.org> 于2018年12月11日周二 下午3:13写道: > >> Hi Jincheng, >> >> thanks for the proposal. I totally agree with the problem of having 3 >> StreamTableEnvironments and 3 BatchTableEnvironments. We also identified >> this problem when doing Flink trainings and introductions to the Table & >> SQL API. >> >> Actually, @Dawid and I were already discussing to remove this >> shortcoming while working on FLINK-11067 [1]. The porting allows to fix >> the class hierarchy because some visibilities of members change as well >> from Scala to Java. This would not break backwards compatibility as the >> base classes should not be used by users anyway. >> >> However, I don't like the design of putting all methods of Batch and >> Stream environments into the base class and throw exceptions if not >> supported by base classes. This sounds not like a nice object oriented >> design and confuses users. >> >> I added some comments to the document. I think we can improve the >> current situation without breaking backwards compatibility. Methods that >> interact with Scala and Java API such as toDataSet/toDataStream should >> not be moved to an abstract class as they would otherwise pull in Scala >> dependencies transitively or do not incoperate with the type extraction >> logic of the target API. >> >> Regards, >> Timo >> >> [1] https://issues.apache.org/jira/browse/FLINK-11067 >> >> >> Am 11.12.18 um 06:12 schrieb Zhang, Xuefu: >> > 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 >> > >> >>