Skip to content

Commit

Permalink
[FLINK-13028][table] Refactor expression package structure
Browse files Browse the repository at this point in the history
  • Loading branch information
twalthr committed Jul 2, 2019
1 parent 8c1968a commit 14fe641
Show file tree
Hide file tree
Showing 72 changed files with 163 additions and 113 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import java.util.List;
import java.util.Optional;

import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedCall;
import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;

/**
* Partially defined over window with (optional) partitioning and order.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@
import org.apache.flink.table.descriptors.StreamTableDescriptor;
import org.apache.flink.table.descriptors.TableDescriptor;
import org.apache.flink.table.expressions.TableReferenceExpression;
import org.apache.flink.table.expressions.lookups.TableReferenceLookup;
import org.apache.flink.table.expressions.resolver.lookups.TableReferenceLookup;
import org.apache.flink.table.functions.ScalarFunction;
import org.apache.flink.table.operations.CatalogQueryOperation;
import org.apache.flink.table.operations.CatalogSinkModifyOperation;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
import org.apache.flink.table.catalog.FunctionLookup;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ExpressionParser;
import org.apache.flink.table.expressions.LookupCallResolver;
import org.apache.flink.table.expressions.resolver.LookupCallResolver;
import org.apache.flink.table.functions.TemporalTableFunction;
import org.apache.flink.table.functions.TemporalTableFunctionImpl;
import org.apache.flink.table.operations.JoinQueryOperation.JoinType;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,17 +16,19 @@
* limitations under the License.
*/

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

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ExpressionParser;

import java.lang.reflect.Constructor;
import java.util.List;

/**
* Parser for expressions inside a String. This parses exactly the same expressions that
* would be accepted by the Scala Expression DSL.
* Temporary utility for parsing expressions inside a String. This parses exactly the same expressions
* that would be accepted by the Scala Expression DSL.
*
* <p>{@link PlannerExpressionParser} is used by {@link ExpressionParser} to parse expressions.
*/
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.delegation.PlannerExpressionParser;

import java.util.List;

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

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

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.GroupWindow;
import org.apache.flink.table.api.OverWindow;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.catalog.FunctionLookup;
import org.apache.flink.table.expressions.lookups.FieldReferenceLookup;
import org.apache.flink.table.expressions.lookups.TableReferenceLookup;
import org.apache.flink.table.expressions.rules.ResolverRule;
import org.apache.flink.table.expressions.rules.ResolverRules;
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.LocalReferenceExpression;
import org.apache.flink.table.expressions.ResolvedExpression;
import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
import org.apache.flink.table.expressions.ValueLiteralExpression;
import org.apache.flink.table.expressions.resolver.lookups.FieldReferenceLookup;
import org.apache.flink.table.expressions.resolver.lookups.TableReferenceLookup;
import org.apache.flink.table.expressions.resolver.rules.ResolverRule;
import org.apache.flink.table.expressions.resolver.rules.ResolverRules;
import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
import org.apache.flink.table.functions.BuiltInFunctionDefinition;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.operations.QueryOperation;
Expand All @@ -42,8 +49,8 @@
import java.util.function.Function;
import java.util.stream.Collectors;

import static org.apache.flink.table.expressions.ApiExpressionUtils.typeLiteral;
import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral;
import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.typeLiteral;
import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;

/**
* Tries to resolve all unresolved expressions such as {@link UnresolvedReferenceExpression}
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.expressions.resolver;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.expressions.Expression;

import javax.annotation.Nullable;

Expand All @@ -28,7 +28,6 @@
/**
* Local over window created during expression resolution.
*/
@Internal
public final class LocalOverWindow {

private Expression alias;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,15 @@
* limitations under the License.
*/

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

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.catalog.FunctionLookup;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.LookupCallExpression;
import org.apache.flink.table.expressions.UnresolvedCallExpression;
import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;

import java.util.List;
import java.util.stream.Collectors;
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.lookups;
package org.apache.flink.table.expressions.resolver.lookups;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.TableSchema;
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.lookups;
package org.apache.flink.table.expressions.resolver.lookups;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.expressions.TableReferenceExpression;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,16 +16,16 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.expressions.ApiExpressionDefaultVisitor;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ExpressionUtils;
import org.apache.flink.table.expressions.UnresolvedCallExpression;
import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
import org.apache.flink.table.expressions.ValueLiteralExpression;
import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.util.Preconditions;

Expand All @@ -35,8 +35,8 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;

import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedCall;
import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;
import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AS;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.RANGE_TO;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.WITHOUT_COLUMNS;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,11 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions.rules;
package org.apache.flink.table.expressions.resolver.rules;

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

import java.util.List;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,15 +16,15 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.expressions.ApiExpressionDefaultVisitor;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.LocalOverWindow;
import org.apache.flink.table.expressions.UnresolvedCallExpression;
import org.apache.flink.table.expressions.ValueLiteralExpression;
import org.apache.flink.table.expressions.resolver.LocalOverWindow;
import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.types.logical.LogicalType;
Expand All @@ -34,7 +34,7 @@
import java.util.stream.Collectors;

import static java.util.Arrays.asList;
import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedCall;
import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;
import static org.apache.flink.table.types.logical.LogicalTypeRoot.BIGINT;
import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTERVAL_DAY_TIME;
import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot;
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.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.catalog.FunctionLookup;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,26 +16,27 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.FieldReferenceExpression;
import org.apache.flink.table.expressions.LocalReferenceExpression;
import org.apache.flink.table.expressions.UnresolvedCallExpression;
import org.apache.flink.table.expressions.UnresolvedReferenceExpression;

import java.util.List;
import java.util.stream.Collectors;

import static java.lang.String.format;
import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedCall;
import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;

/**
* Resolves {@link UnresolvedReferenceExpression} to either
* {@link org.apache.flink.table.expressions.FieldReferenceExpression},
* {@link org.apache.flink.table.expressions.TableReferenceExpression}, or
* {@link org.apache.flink.table.expressions.LocalReferenceExpression} in this order.
* {@link LocalReferenceExpression} in this order.
*/
@Internal
final class ReferenceResolverRule implements ResolverRule {
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.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.TableException;
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.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.typeinfo.TypeInformation;
Expand All @@ -32,7 +32,7 @@
import java.util.stream.IntStream;

import static java.util.Collections.singletonList;
import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral;
import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo;
import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType;

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

package org.apache.flink.table.expressions.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.catalog.FunctionLookup;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ExpressionResolver;
import org.apache.flink.table.expressions.LocalOverWindow;
import org.apache.flink.table.expressions.LocalReferenceExpression;
import org.apache.flink.table.expressions.lookups.FieldReferenceLookup;
import org.apache.flink.table.expressions.lookups.TableReferenceLookup;
import org.apache.flink.table.expressions.resolver.ExpressionResolver;
import org.apache.flink.table.expressions.resolver.LocalOverWindow;
import org.apache.flink.table.expressions.resolver.lookups.FieldReferenceLookup;
import org.apache.flink.table.expressions.resolver.lookups.TableReferenceLookup;
import org.apache.flink.table.functions.FunctionDefinition;

import java.util.List;
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.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,11 @@
* limitations under the License.
*/

package org.apache.flink.table.expressions.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.expressions.ApiExpressionDefaultVisitor;
import org.apache.flink.table.expressions.rules.ResolverRule.ResolutionContext;
import org.apache.flink.table.expressions.resolver.rules.ResolverRule.ResolutionContext;
import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;

/**
* Utility class for {@link ResolverRule} specific visitor that unifies access to
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.rules;
package org.apache.flink.table.expressions.resolver.rules;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.expressions.Expression;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,20 @@
* limitations under the License.
*/

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

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.expressions.ApiExpressionVisitor;
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.LocalReferenceExpression;
import org.apache.flink.table.expressions.LookupCallExpression;
import org.apache.flink.table.expressions.TableReferenceExpression;
import org.apache.flink.table.expressions.TypeLiteralExpression;
import org.apache.flink.table.expressions.UnresolvedCallExpression;
import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
import org.apache.flink.table.expressions.ValueLiteralExpression;

/**
* A utility {@link ApiExpressionVisitor} that calls {@link #defaultMethod(Expression)} by default,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,21 @@
* limitations under the License.
*/

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

import org.apache.flink.annotation.Internal;
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.expressions.CallExpression;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ExpressionUtils;
import org.apache.flink.table.expressions.LookupCallExpression;
import org.apache.flink.table.expressions.TableReferenceExpression;
import org.apache.flink.table.expressions.TypeLiteralExpression;
import org.apache.flink.table.expressions.UnresolvedCallExpression;
import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
import org.apache.flink.table.expressions.ValueLiteralExpression;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.functions.FunctionKind;
import org.apache.flink.table.types.DataType;
Expand Down
Loading

0 comments on commit 14fe641

Please sign in to comment.