[jira] [Comment Edited] (FLINK-11067) Port TableEnvironments to Java

2018-12-14 Thread Timo Walther (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-11067?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16721330#comment-16721330
 ] 

Timo Walther edited comment on FLINK-11067 at 12/14/18 12:07 PM:
-

Sorry, [~dian.fu] and [~sunjincheng121] I forgot that {{BatchTableEnvironment}} 
and {{StreamTableEnvironment}} are also in different packages. Yes, I think 
[~dawidwys] suggestion is a better solution with a common base class in 
{{flink-table-api-base}}. In summary:

{code}
// in flink-tabe-api-base
interface BaseTableEnvironment {

// methods independent of batch/streaming or Java/Scala

// contains main logic for within the table ecosystem
// (reading from table sources and writing to table sinks)
}

// in flink-table-api-java package
interface TableEnvironment extends BaseTableEnvironment{

// only methods for creating a table environment in Java!

static BatchTableEnvironment getTableEnvironment(ExecutionEnvironment 
env);

static StreamTableEnvironment 
getTableEnvironment(StreamExecutionEnvironment env);

// and methods specific for Java i.e. UDF registration
}
interface BatchTableEnvironment extends TableEnvironment {
// methods specific for batch and Java (toDataSet, fromDataSet)
}
interface StreamTableEnvironment extends TableEnvironment {
// methods specific for streaming and Java (toRetractStream, ...)
}

// in flink-table-api-scala package
interface TableEnvironment extends BaseTableEnvironment {

// only methods for creating a table environment in Scala!

static BatchTableEnvironment getTableEnvironment(ExecutionEnvironment 
env);

static StreamTableEnvironment 
getTableEnvironment(StreamExecutionEnvironment env);

// and methods specific for Scala i.e. UDF registration
}
interface BatchTableEnvironment extends TableEnvironment {
// methods specific for batch and Scala (toDataSet, fromDataSet)
}
interface StreamTableEnvironment extends TableEnvironment {
// methods specific for streaming and Scala (toRetractStream, ...)
}
{code}

What do you think?


was (Author: twalthr):
Sorry, [~dian.fu] and [~sunjincheng121] I forgot that {{BatchTableEnvironment}} 
and {{StreamTableEnvironment}} are also in different packages. Yes, I think 
[~dawidwys] suggestion is a better solution with a common base class in 
{{flink-table-api-base}}. In summary:

// in flink-tabe-api-base
interface BaseTableEnvironment {

// methods independent of batch/streaming or Java/Scala

// contains main logic for within the table ecosystem
// (reading from table sources and writing to table sinks)
}

// in flink-table-api-java package
interface TableEnvironment extends BaseTableEnvironment{

// only methods for creating a table environment in Java!

static BatchTableEnvironment getTableEnvironment(ExecutionEnvironment 
env);

static StreamTableEnvironment 
getTableEnvironment(StreamExecutionEnvironment env);

// and methods specific for Java i.e. UDF registration
}
interface BatchTableEnvironment extends TableEnvironment {
// methods specific for batch and Java (toDataSet, fromDataSet)
}
interface StreamTableEnvironment extends TableEnvironment {
// methods specific for streaming and Java (toRetractStream, ...)
}

// in flink-table-api-scala package
interface TableEnvironment extends BaseTableEnvironment {

// only methods for creating a table environment in Scala!

static BatchTableEnvironment getTableEnvironment(ExecutionEnvironment 
env);

static StreamTableEnvironment 
getTableEnvironment(StreamExecutionEnvironment env);

// and methods specific for Scala i.e. UDF registration
}
interface BatchTableEnvironment extends TableEnvironment {
// methods specific for batch and Scala (toDataSet, fromDataSet)
}
interface StreamTableEnvironment extends TableEnvironment {
// methods specific for streaming and Scala (toRetractStream, ...)
}

What do you think?

> Port TableEnvironments to Java
> --
>
> Key: FLINK-11067
> URL: https://issues.apache.org/jira/browse/FLINK-11067
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API  SQL
>Reporter: Timo Walther
>Assignee: Dawid Wysakowicz
>Priority: Major
>
> This task includes porting {{TableEnvironment}}, {{StreamTableEnvironment}}, 
> {{BatchTableEnvironment}} to Java. API-breaking changes need to be avoided 
> and discussed. Some refactoring and clean up might be necessary.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Comment Edited] (FLINK-11067) Port TableEnvironments to Java

2018-12-12 Thread Dawid Wysakowicz (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-11067?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16719091#comment-16719091
 ] 

Dawid Wysakowicz edited comment on FLINK-11067 at 12/12/18 3:29 PM:


I think as user should only import either {{flink-tabe-api-java}} or 
{{flink-tabe-api-scala}} never both of them we can have:

{code}

// in flink-table-api-java module
package org.apache.flink.table.api;

public final class TableEnvironment {

   static org.apache.flink.table.api.java.BatchTableEnvironment 
getTableEnvironment(BatchEnvironment);

   static org.apache.flink.table.api.java.StreamTableEnvironment 
getTableEnvironment(StreamEnvironment);
   
   private TableEnvironment() {
   }
}

// in flink-table-api-scala module
package org.apache.flink.table.api;

object TableEnvironment {
   def getTableEnvironment(BatchEnvironment): 
org.apache.flink.table.api.scala.BatchTableEnvironment = ...

   def getTableEnvironment(StreamEnvironment): 
org.apache.flink.table.api.scala.StreamTableEnvironment = 
}

{code}

The rest would remain the same as in [~twalthr] suggestion. I think that would 
retain backwards compatibility.


was (Author: dawidwys):
I think as user should only import either {{flink-tabe-api-java}} or 
{{flink-tabe-api-scala}} never both of them. I think we can have:

{code}

// in flink-table-api-java module
package org.apache.flink.table.api;

public final class TableEnvironment {

   static org.apache.flink.table.api.java.BatchTableEnvironment 
getTableEnvironment(BatchEnvironment);

   static org.apache.flink.table.api.java.StreamTableEnvironment 
getTableEnvironment(StreamEnvironment);
   
   private TableEnvironment() {
   }
}

// in flink-table-api-scala module
package org.apache.flink.table.api;

object TableEnvironment {
   def getTableEnvironment(BatchEnvironment): 
org.apache.flink.table.api.scala.BatchTableEnvironment = ...

   def getTableEnvironment(StreamEnvironment): 
org.apache.flink.table.api.scala.StreamTableEnvironment = 
}

{code}

The rest would remain the same as in [~twalthr] suggestion. I think that would 
retain backwards compatibility.

> Port TableEnvironments to Java
> --
>
> Key: FLINK-11067
> URL: https://issues.apache.org/jira/browse/FLINK-11067
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API  SQL
>Reporter: Timo Walther
>Assignee: Dawid Wysakowicz
>Priority: Major
>
> This task includes porting {{TableEnvironment}}, {{StreamTableEnvironment}}, 
> {{BatchTableEnvironment}} to Java. API-breaking changes need to be avoided 
> and discussed. Some refactoring and clean up might be necessary.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)