Skip to content

Commit

Permalink
[hotfix][table-common] Move function definitions to functions package
Browse files Browse the repository at this point in the history
  • Loading branch information
twalthr committed Jun 17, 2019
1 parent a55227b commit 290e218
Show file tree
Hide file tree
Showing 51 changed files with 139 additions and 125 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,10 @@
package org.apache.flink.table.api;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.expressions.BuiltInFunctionDefinitions;
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ExpressionParser;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;

import java.util.Collections;
import java.util.List;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,14 @@

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.table.expressions.AggregateFunctionDefinition;
import org.apache.flink.table.expressions.BuiltInFunctionDefinitions;
import org.apache.flink.table.expressions.FunctionDefinition;
import org.apache.flink.table.expressions.ScalarFunctionDefinition;
import org.apache.flink.table.expressions.TableFunctionDefinition;
import org.apache.flink.table.expressions.catalog.FunctionDefinitionCatalog;
import org.apache.flink.table.functions.AggregateFunctionDefinition;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.functions.ScalarFunction;
import org.apache.flink.table.functions.ScalarFunctionDefinition;
import org.apache.flink.table.functions.TableFunction;
import org.apache.flink.table.functions.TableFunctionDefinition;
import org.apache.flink.table.functions.UserDefinedAggregateFunction;
import org.apache.flink.table.functions.UserFunctionsTypeHelper;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.Table;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.types.DataType;

import java.util.Arrays;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.table.expressions;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.util.Preconditions;

import java.util.ArrayList;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.expressions.catalog.FunctionDefinitionCatalog;
import org.apache.flink.table.functions.FunctionDefinition;

import java.util.List;
import java.util.stream.Collectors;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
package org.apache.flink.table.expressions.catalog;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.expressions.FunctionDefinition;
import org.apache.flink.table.functions.FunctionDefinition;

import java.util.Optional;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,11 @@
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.FieldReferenceExpression;
import org.apache.flink.table.expressions.FunctionDefinition;
import org.apache.flink.table.expressions.LocalReferenceExpression;
import org.apache.flink.table.expressions.LookupCallExpression;
import org.apache.flink.table.expressions.TableReferenceExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.functions.FunctionDefinition;

import java.util.LinkedHashMap;
import java.util.List;
Expand All @@ -37,11 +38,11 @@
import static org.apache.flink.table.expressions.ApiExpressionUtils.call;
import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.AS;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.WINDOW_PROPERTIES;
import static org.apache.flink.table.expressions.ExpressionUtils.extractValue;
import static org.apache.flink.table.expressions.ExpressionUtils.isFunctionOfType;
import static org.apache.flink.table.expressions.FunctionDefinition.Type.AGGREGATE_FUNCTION;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AS;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.WINDOW_PROPERTIES;
import static org.apache.flink.table.functions.FunctionDefinition.Type.AGGREGATE_FUNCTION;

/**
* Utility methods for transforming {@link Expression} to use them in {@link QueryOperation}s.
Expand Down Expand Up @@ -110,7 +111,7 @@ public static CategorizedExpressions extractAggregationsAndProperties(List<Expre
* <li>{@link FieldReferenceExpression}</li>
* <li>{@link TableReferenceExpression}</li>
* <li>{@link LocalReferenceExpression}</li>
* <li>{@link org.apache.flink.table.expressions.BuiltInFunctionDefinitions#AS}</li>
* <li>{@link BuiltInFunctionDefinitions#AS}</li>
* </ul>
*
* @param expressions list of expressions to extract names from
Expand All @@ -126,7 +127,7 @@ public static List<Optional<String>> extractNames(List<Expression> expressions)
* <li>{@link FieldReferenceExpression}</li>
* <li>{@link TableReferenceExpression}</li>
* <li>{@link LocalReferenceExpression}</li>
* <li>{@link org.apache.flink.table.expressions.BuiltInFunctionDefinitions#AS}</li>
* <li>{@link BuiltInFunctionDefinitions#AS}</li>
* </ul>
*
* @param expression expression to extract name from
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,11 +29,11 @@
import org.apache.flink.table.api.Types;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.expressions.ApiExpressionDefaultVisitor;
import org.apache.flink.table.expressions.BuiltInFunctionDefinitions;
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ExpressionUtils;
import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.types.AtomicDataType;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,9 @@

import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.expressions.BuiltInFunctionDefinitions;
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.FieldReferenceExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;

import org.junit.Test;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.table.expressions;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.util.Preconditions;

import java.util.ArrayList;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.table.expressions;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.functions.FunctionDefinition;

import java.util.Optional;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,13 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions;
package org.apache.flink.table.functions;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.table.functions.UserDefinedAggregateFunction;
import org.apache.flink.util.Preconditions;

import static org.apache.flink.table.expressions.FunctionDefinition.Type.AGGREGATE_FUNCTION;
import static org.apache.flink.table.functions.FunctionDefinition.Type.AGGREGATE_FUNCTION;

/**
* The function definition of an user-defined aggregate function.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions;
package org.apache.flink.table.functions;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.api.TableException;
Expand All @@ -29,9 +29,9 @@
import java.util.List;
import java.util.Set;

import static org.apache.flink.table.expressions.FunctionDefinition.Type.AGGREGATE_FUNCTION;
import static org.apache.flink.table.expressions.FunctionDefinition.Type.OTHER_FUNCTION;
import static org.apache.flink.table.expressions.FunctionDefinition.Type.SCALAR_FUNCTION;
import static org.apache.flink.table.functions.FunctionDefinition.Type.AGGREGATE_FUNCTION;
import static org.apache.flink.table.functions.FunctionDefinition.Type.OTHER_FUNCTION;
import static org.apache.flink.table.functions.FunctionDefinition.Type.SCALAR_FUNCTION;

/**
* Dictionary of function definitions for all built-in functions.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions;
package org.apache.flink.table.functions;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.util.Preconditions;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,12 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions;
package org.apache.flink.table.functions;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.functions.ScalarFunction;
import org.apache.flink.util.Preconditions;

import static org.apache.flink.table.expressions.FunctionDefinition.Type.SCALAR_FUNCTION;
import static org.apache.flink.table.functions.FunctionDefinition.Type.SCALAR_FUNCTION;

/**
* The function definition of an user-defined scalar function.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,13 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions;
package org.apache.flink.table.functions;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.table.functions.TableFunction;
import org.apache.flink.util.Preconditions;

import static org.apache.flink.table.expressions.FunctionDefinition.Type.TABLE_FUNCTION;
import static org.apache.flink.table.functions.FunctionDefinition.Type.TABLE_FUNCTION;

/**
* The function definition of an user-defined table function.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.functions.ScalarFunction;
import org.apache.flink.table.functions.ScalarFunctionDefinition;

import org.junit.Rule;
import org.junit.Test;
Expand All @@ -32,8 +33,8 @@

import static java.util.Arrays.asList;
import static java.util.Collections.singletonList;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.AND;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.EQUALS;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AND;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.EQUALS;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,29 +18,30 @@

package org.apache.flink.table.expressions;

import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.types.DataType;

import java.util.Arrays;
import java.util.Collections;
import java.util.List;

import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.AND;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.CAST;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.CONCAT;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.DIVIDE;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.EQUALS;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.GREATER_THAN;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.IF;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.IS_NULL;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.LESS_THAN;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.MINUS;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.MOD;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.NOT;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.OR;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.PLUS;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.REINTERPRET_CAST;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.TIMES;
import static org.apache.flink.table.expressions.InternalFunctionDefinitions.THROW_EXCEPTION;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AND;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.CAST;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.CONCAT;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.DIVIDE;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.EQUALS;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.GREATER_THAN;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.IF;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.IS_NULL;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.LESS_THAN;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.MINUS;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.MOD;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.NOT;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.OR;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.PLUS;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.REINTERPRET_CAST;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.TIMES;
import static org.apache.flink.table.functions.InternalFunctionDefinitions.THROW_EXCEPTION;

/**
* Builder for {@link Expression}s.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@
import org.apache.flink.table.calcite.RexAggLocalVariable;
import org.apache.flink.table.calcite.RexDistinctKeyVariable;
import org.apache.flink.table.dataformat.Decimal;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.functions.InternalFunctionDefinitions;
import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable;
import org.apache.flink.table.types.logical.BigIntType;
import org.apache.flink.table.types.logical.DateType;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,9 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions;
package org.apache.flink.table.functions;

import static org.apache.flink.table.expressions.FunctionDefinition.Type.SCALAR_FUNCTION;
import static org.apache.flink.table.functions.FunctionDefinition.Type.SCALAR_FUNCTION;

/**
* Dictionary of function definitions for all internal used functions.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,27 +18,27 @@

package org.apache.flink.table.sources

import java.sql.Timestamp

import com.google.common.collect.ImmutableList
import org.apache.calcite.plan.RelOptCluster
import org.apache.calcite.rel.RelNode
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.logical.LogicalValues
import org.apache.calcite.rex.{RexLiteral, RexNode}
import org.apache.calcite.tools.RelBuilder
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.common.typeutils.CompositeType
import org.apache.flink.table.api.{DataTypes, ValidationException}
import org.apache.flink.table.calcite.FlinkTypeFactory
import org.apache.flink.table.expressions.{BuiltInFunctionDefinitions, CallExpression, PlannerResolvedFieldReference, ResolvedFieldReference, RexNodeConverter, TypeLiteralExpression}
import org.apache.flink.table.expressions._
import org.apache.flink.table.functions.BuiltInFunctionDefinitions
import org.apache.flink.table.types.LogicalTypeDataTypeConverter
import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
import org.apache.flink.table.types.logical.{LogicalType, TimestampKind, TimestampType, TinyIntType}
import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo
import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo

import com.google.common.collect.ImmutableList
import org.apache.calcite.plan.RelOptCluster
import org.apache.calcite.rel.RelNode
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.logical.LogicalValues
import org.apache.calcite.rex.{RexLiteral, RexNode}
import org.apache.calcite.tools.RelBuilder

import java.sql.Timestamp

import scala.collection.JavaConversions._

/** Util class for [[TableSource]]. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.table.api.{Types, ValidationException}
import org.apache.flink.table.descriptors.Rowtime
import org.apache.flink.table.expressions._
import org.apache.flink.table.functions.BuiltInFunctionDefinitions
import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType

import java.util
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,17 +18,15 @@

package org.apache.flink.table.validate

import java.util

import org.apache.calcite.sql._
import org.apache.flink.table.calcite.FlinkTypeFactory
import org.apache.flink.table.expressions._
import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{createAggregateSqlFunction, createScalarSqlFunction, createTableSqlFunction}
import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction}
import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction, _}
import org.apache.flink.table.types.DataType
import org.apache.flink.table.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo

import org.apache.calcite.sql._

import java.util

import _root_.scala.collection.JavaConversions._

/**
Expand Down
Loading

0 comments on commit 290e218

Please sign in to comment.