Skip to content

Commit

Permalink
[FLINK-6334] [table] Refactor Table API TableFunction join methods.
Browse files Browse the repository at this point in the history
This closes apache#3791.
  • Loading branch information
Xpray authored and fhueske committed May 3, 2017
1 parent 8ed85fe commit c969237
Show file tree
Hide file tree
Showing 10 changed files with 404 additions and 192 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ abstract class TableEnvironment(val config: TableConfig) {
private val rootSchema: SchemaPlus = internalSchema.plus()

// Table API/SQL function catalog
private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns
private[flink] val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns

// the configuration to create a Calcite planner
private lazy val frameworkConfig: FrameworkConfig = Frameworks
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.api.scala

import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.table.api.Table
import org.apache.flink.table.expressions._
import org.apache.flink.table.functions.TableFunction
import org.apache.flink.table.plan.logical.LogicalTableFunctionCall

/**
* Holds methods to convert a [[TableFunction]] call in the Scala Table API into a [[Table]].
*
* @param tf The TableFunction to convert.
*/
class TableFunctionConversions[T](tf: TableFunction[T]) {

/**
* Creates a [[Table]] from a [[TableFunction]] in Scala Table API.
*
* @param args The arguments of the table function call.
* @return A [[Table]] with which represents the [[LogicalTableFunctionCall]].
*/
final def apply(args: Expression*)(implicit typeInfo: TypeInformation[T]): Table = {

val resultType = if (tf.getResultType == null) typeInfo else tf.getResultType

new Table(
tableEnv = null, // Table environment will be set later.
LogicalTableFunctionCall(
tf.getClass.getCanonicalName,
tf,
args.toList,
resultType,
Array.empty,
child = null // Child will be set later.
)
)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import org.apache.flink.api.scala.DataSet
import org.apache.flink.streaming.api.scala.DataStream
import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnv}
import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnv}
import org.apache.flink.table.functions.TableFunction

import _root_.scala.language.implicitConversions

Expand Down Expand Up @@ -89,4 +90,7 @@ package object scala extends ImplicitExpressionConversions {
tableEnv.toDataStream[Row](table)
}

implicit def tableFunctionCall2Table[T](tf: TableFunction[T]): TableFunctionConversions[T] = {
new TableFunctionConversions[T](tf)
}
}
Loading

0 comments on commit c969237

Please sign in to comment.