Skip to content

Commit

Permalink
[FLINK-16200][table-planner] Refactor CustomizedConvertRule
Browse files Browse the repository at this point in the history
This closes apache#16691.
  • Loading branch information
Airblader authored and twalthr committed Aug 12, 2021
1 parent 334082d commit db2e333
Show file tree
Hide file tree
Showing 25 changed files with 1,120 additions and 436 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.table.planner.expressions.converter;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.catalog.DataTypeFactory;
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.Expression;
Expand All @@ -29,6 +30,7 @@
import java.util.Optional;

/** Rule to convert {@link CallExpression}. */
@Internal
public interface CallExpressionConvertRule {

/**
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.table.planner.expressions.converter;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.functions.FunctionDefinition;
Expand All @@ -36,6 +37,7 @@
* A {@link CallExpressionConvertRule} that performs a simple one-to-one mapping between {@link
* FunctionDefinition} and a corresponding {@link SqlOperator}.
*/
@Internal
public class DirectConvertRule implements CallExpressionConvertRule {

private static final Map<FunctionDefinition, SqlOperator> DEFINITION_OPERATOR_MAP =
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.planner.expressions.converter.converters;

import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.planner.expressions.converter.CallExpressionConvertRule;
import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable;

import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexNode;

import java.util.List;

import static org.apache.flink.table.planner.expressions.converter.ExpressionConverter.toRexNodes;

/** Conversion for {@link BuiltInFunctionDefinitions#ARRAY}. */
class ArrayConverter extends CustomizedConverter {
@Override
public RexNode convert(CallExpression call, CallExpressionConvertRule.ConvertContext context) {
List<RexNode> childrenRexNode = toRexNodes(context, call.getChildren());
RelDataType relDataType =
context.getTypeFactory()
.createFieldTypeFromLogicalType(call.getOutputDataType().getLogicalType());
return context.getRelBuilder()
.getRexBuilder()
.makeCall(
relDataType,
FlinkSqlOperatorTable.ARRAY_VALUE_CONSTRUCTOR,
childrenRexNode);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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.planner.expressions.converter.converters;

import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.ValueLiteralExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.planner.expressions.converter.CallExpressionConvertRule;

import org.apache.calcite.rex.RexNode;

import static org.apache.flink.table.planner.expressions.converter.ExpressionConverter.extractValue;

/** Conversion for {@link BuiltInFunctionDefinitions#AS}. */
class AsConverter extends CustomizedConverter {
@Override
public RexNode convert(CallExpression call, CallExpressionConvertRule.ConvertContext context) {
checkArgumentNumber(call, 2);
String name =
extractValue((ValueLiteralExpression) call.getChildren().get(1), String.class);
RexNode child = context.toRexNode(call.getChildren().get(0));
return context.getRelBuilder().alias(child, name);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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.planner.expressions.converter.converters;

import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.planner.expressions.converter.CallExpressionConvertRule;
import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable;

import org.apache.calcite.rex.RexNode;

import java.util.List;

import static org.apache.flink.table.planner.expressions.converter.ExpressionConverter.toRexNodes;

/** Conversion for {@link BuiltInFunctionDefinitions#BETWEEN}. */
class BetweenConverter extends CustomizedConverter {
@Override
public RexNode convert(CallExpression call, CallExpressionConvertRule.ConvertContext context) {
checkArgumentNumber(call, 3);
List<RexNode> childrenRexNode = toRexNodes(context, call.getChildren());
RexNode expr = childrenRexNode.get(0);
RexNode lowerBound = childrenRexNode.get(1);
RexNode upperBound = childrenRexNode.get(2);
return context.getRelBuilder()
.and(
context.getRelBuilder()
.call(
FlinkSqlOperatorTable.GREATER_THAN_OR_EQUAL,
expr,
lowerBound),
context.getRelBuilder()
.call(FlinkSqlOperatorTable.LESS_THAN_OR_EQUAL, expr, upperBound));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* 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.planner.expressions.converter.converters;

import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.TypeLiteralExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.planner.expressions.converter.CallExpressionConvertRule;

import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexNode;

/** Conversion for {@link BuiltInFunctionDefinitions#CAST}. */
class CastConverter extends CustomizedConverter {
@Override
public RexNode convert(CallExpression call, CallExpressionConvertRule.ConvertContext context) {
checkArgumentNumber(call, 2);

final RexNode child = context.toRexNode(call.getChildren().get(0));
final TypeLiteralExpression targetType = (TypeLiteralExpression) call.getChildren().get(1);
final RelDataType targetRelDataType =
context.getTypeFactory()
.createFieldTypeFromLogicalType(
targetType.getOutputDataType().getLogicalType());

return context.getRelBuilder().getRexBuilder().makeAbstractCast(targetRelDataType, child);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.planner.expressions.converter.converters;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.planner.expressions.converter.CallExpressionConvertRule;
import org.apache.flink.table.planner.expressions.converter.CustomizedConvertRule;

import org.apache.calcite.rex.RexNode;

/** Customized converter used by {@link CustomizedConvertRule}. */
@Internal
public abstract class CustomizedConverter {

public abstract RexNode convert(
CallExpression call, CallExpressionConvertRule.ConvertContext context);

// ---------------------------------------------------------------------------------------------

protected static void checkArgumentNumber(CallExpression call, int... validArgumentCounts) {
boolean hasValidArgumentCount = false;
for (int argumentCount : validArgumentCounts) {
if (call.getChildren().size() == argumentCount) {
hasValidArgumentCount = true;
break;
}
}

checkArgument(call, hasValidArgumentCount);
}

protected static void checkArgument(CallExpression call, boolean check) {
if (!check) {
throw new TableException("Invalid arguments for call: " + call);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.planner.expressions.converter.converters;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.planner.expressions.converter.CustomizedConvertRule;
import org.apache.flink.table.planner.functions.InternalFunctionDefinitions;

import java.util.HashMap;
import java.util.Map;
import java.util.Optional;

/** Registry of customized converters used by {@link CustomizedConvertRule}. */
@Internal
public class CustomizedConverters {
private static final Map<FunctionDefinition, CustomizedConverter> CONVERTERS = new HashMap<>();

static {
CONVERTERS.put(BuiltInFunctionDefinitions.CAST, new CastConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.REINTERPRET_CAST, new ReinterpretCastConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.IN, new InConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.GET, new GetConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.TRIM, new TrimConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.AS, new AsConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.BETWEEN, new BetweenConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.NOT_BETWEEN, new NotBetweenConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.REPLACE, new ReplaceConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.PLUS, new PlusConverter());
CONVERTERS.put(
BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS, new TemporalOverlapsConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.TIMESTAMP_DIFF, new TimestampDiffConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.ARRAY, new ArrayConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.MAP, new MapConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.ROW, new RowConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.ORDER_ASC, new OrderAscConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.SQRT, new SqrtConverter());
CONVERTERS.put(BuiltInFunctionDefinitions.JSON_EXISTS, new JsonExistsConverter());
CONVERTERS.put(InternalFunctionDefinitions.THROW_EXCEPTION, new ThrowExceptionConverter());
}

public Optional<CustomizedConverter> getConverter(FunctionDefinition functionDefinition) {
return Optional.ofNullable(CONVERTERS.get(functionDefinition));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.planner.expressions.converter.converters;

import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.ExpressionUtils;
import org.apache.flink.table.expressions.ValueLiteralExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.planner.expressions.converter.CallExpressionConvertRule;

import org.apache.calcite.rex.RexNode;

import java.util.Optional;

import static org.apache.flink.table.planner.expressions.converter.ExpressionConverter.extractValue;

/** Conversion for {@link BuiltInFunctionDefinitions#GET}. */
class GetConverter extends CustomizedConverter {
@Override
public RexNode convert(CallExpression call, CallExpressionConvertRule.ConvertContext context) {
checkArgumentNumber(call, 2);
RexNode child = context.toRexNode(call.getChildren().get(0));
ValueLiteralExpression keyLiteral = (ValueLiteralExpression) call.getChildren().get(1);
Optional<Integer> indexOptional =
ExpressionUtils.extractValue(keyLiteral, String.class)
.map(child.getType().getFieldNames()::indexOf);
int index = indexOptional.orElseGet(() -> extractValue(keyLiteral, Integer.class));
return context.getRelBuilder().getRexBuilder().makeFieldAccess(child, index);
}
}
Loading

0 comments on commit db2e333

Please sign in to comment.