[
https://issues.apache.org/jira/browse/FLINK-11067?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16717291#comment-16717291
]
Timo Walther commented on FLINK-11067:
--------------------------------------
This would one solution of getting a proper package structure while remaining
backwards compatible:
{code}
// common interface of environments that can be implemented by Scala and Java
environments
StreamEnvironment
BatchEnvironment
// in flink-tabe-api-base
interface TableEnvironment {
// methods independent of batch/streaming or Java/Scala
static BatchTableEnvironment getTableEnvironment(BatchEnvironment env);
static StreamTableEnvironment getTableEnvironment(StreamEnvironment
env);
// implement empty interfaces for StreamTableEnvironment and
BatchTableEnvironment
// in this module, they are overriden by concrete classes and are not
part of the jar
}
// in flink-table-api-java package
interface JavaTableEnvironment extends TableEnvironment {
// methods specific for Java i.e. UDF registration
}
interface BatchTableEnvironment extends JavaTableEnvironment {
// methods specific for batch and Java (toDataSet, fromDataSet)
}
interface StreamTableEnvironment extends JavaTableEnvironment {
// methods specific for streaming and Java (toRetractStream, ...)
}
// in flink-table-api-scala package
interface ScalaTableEnvironment extends TableEnvironment {
// methods specific for Scala i.e. UDF registration
}
interface BatchTableEnvironment extends ScalaTableEnvironment {
// methods specific for batch and Scala (toDataSet, fromDataSet)
}
interface StreamTableEnvironment extends ScalaTableEnvironment {
// methods specific for streaming and Scala (toRetractStream, ...)
}
{code}
The goal would be to have interfaces in the front end while having a
implementation class stack in the background. The method
{{TableEnvironment.getTableEnvironment()}} is the biggest blocker. However, we
can enable it by compiling it against facade classes that are replaced by
individual language-specific classes.
> 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)