in flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java [119:339]
private StandardConvertletTable() {
super();
// Register aliases (operators which have a different name but
// identical behavior to other operators).
addAlias(SqlLibraryOperators.LENGTH, SqlStdOperatorTable.CHAR_LENGTH);
addAlias(SqlStdOperatorTable.CHARACTER_LENGTH, SqlStdOperatorTable.CHAR_LENGTH);
addAlias(SqlStdOperatorTable.IS_UNKNOWN, SqlStdOperatorTable.IS_NULL);
addAlias(SqlStdOperatorTable.IS_NOT_UNKNOWN, SqlStdOperatorTable.IS_NOT_NULL);
addAlias(SqlLibraryOperators.NULL_SAFE_EQUAL, SqlStdOperatorTable.IS_NOT_DISTINCT_FROM);
addAlias(SqlStdOperatorTable.PERCENT_REMAINDER, SqlStdOperatorTable.MOD);
addAlias(SqlLibraryOperators.IFNULL, SqlLibraryOperators.NVL);
// Register convertlets for specific objects.
registerOp(SqlStdOperatorTable.CAST, this::convertCast);
registerOp(SqlLibraryOperators.INFIX_CAST, this::convertCast);
registerOp(
SqlStdOperatorTable.IS_DISTINCT_FROM,
(cx, call) -> convertIsDistinctFrom(cx, call, false));
registerOp(
SqlStdOperatorTable.IS_NOT_DISTINCT_FROM,
(cx, call) -> convertIsDistinctFrom(cx, call, true));
registerOp(SqlStdOperatorTable.PLUS, this::convertPlus);
registerOp(
SqlStdOperatorTable.MINUS,
(cx, call) -> {
final RexCall e = (RexCall) StandardConvertletTable.this.convertCall(cx, call);
switch (e.getOperands().get(0).getType().getSqlTypeName()) {
case DATE:
case TIME:
case TIMESTAMP:
return convertDatetimeMinus(cx, SqlStdOperatorTable.MINUS_DATE, call);
default:
return e;
}
});
// DATE(string) is equivalent to CAST(string AS DATE),
// but other DATE variants are treated as regular functions.
registerOp(
SqlLibraryOperators.DATE,
(cx, call) -> {
final RexCall e = (RexCall) StandardConvertletTable.this.convertCall(cx, call);
if (e.getOperands().size() == 1
&& SqlTypeUtil.isString(e.getOperands().get(0).getType())) {
return cx.getRexBuilder().makeCast(e.type, e.getOperands().get(0));
}
return e;
});
registerOp(SqlLibraryOperators.LTRIM, new TrimConvertlet(SqlTrimFunction.Flag.LEADING));
registerOp(SqlLibraryOperators.RTRIM, new TrimConvertlet(SqlTrimFunction.Flag.TRAILING));
registerOp(SqlLibraryOperators.GREATEST, new GreatestConvertlet());
registerOp(SqlLibraryOperators.LEAST, new GreatestConvertlet());
registerOp(
SqlLibraryOperators.SUBSTR_BIG_QUERY, new SubstrConvertlet(SqlLibrary.BIG_QUERY));
registerOp(SqlLibraryOperators.SUBSTR_MYSQL, new SubstrConvertlet(SqlLibrary.MYSQL));
registerOp(SqlLibraryOperators.SUBSTR_ORACLE, new SubstrConvertlet(SqlLibrary.ORACLE));
registerOp(
SqlLibraryOperators.SUBSTR_POSTGRESQL, new SubstrConvertlet(SqlLibrary.POSTGRESQL));
registerOp(SqlLibraryOperators.DATE_ADD, new TimestampAddConvertlet());
registerOp(SqlLibraryOperators.DATE_DIFF, new TimestampDiffConvertlet());
registerOp(SqlLibraryOperators.DATE_SUB, new TimestampSubConvertlet());
registerOp(SqlLibraryOperators.DATETIME_ADD, new TimestampAddConvertlet());
registerOp(SqlLibraryOperators.DATETIME_DIFF, new TimestampDiffConvertlet());
registerOp(SqlLibraryOperators.DATETIME_SUB, new TimestampSubConvertlet());
registerOp(SqlLibraryOperators.TIME_ADD, new TimestampAddConvertlet());
registerOp(SqlLibraryOperators.TIME_DIFF, new TimestampDiffConvertlet());
registerOp(SqlLibraryOperators.TIME_SUB, new TimestampSubConvertlet());
registerOp(SqlLibraryOperators.TIMESTAMP_ADD2, new TimestampAddConvertlet());
registerOp(SqlLibraryOperators.TIMESTAMP_DIFF3, new TimestampDiffConvertlet());
registerOp(SqlLibraryOperators.TIMESTAMP_SUB, new TimestampSubConvertlet());
registerOp(SqlLibraryOperators.NVL, StandardConvertletTable::convertNvl);
registerOp(SqlLibraryOperators.DECODE, StandardConvertletTable::convertDecode);
registerOp(SqlLibraryOperators.IF, StandardConvertletTable::convertIf);
// Expand "x NOT LIKE y" into "NOT (x LIKE y)"
registerOp(
SqlStdOperatorTable.NOT_LIKE,
(cx, call) ->
cx.convertExpression(
SqlStdOperatorTable.NOT.createCall(
SqlParserPos.ZERO,
SqlStdOperatorTable.LIKE.createCall(
SqlParserPos.ZERO, call.getOperandList()))));
// Expand "x NOT ILIKE y" into "NOT (x ILIKE y)"
registerOp(
SqlLibraryOperators.NOT_ILIKE,
(cx, call) ->
cx.convertExpression(
SqlStdOperatorTable.NOT.createCall(
SqlParserPos.ZERO,
SqlLibraryOperators.ILIKE.createCall(
SqlParserPos.ZERO, call.getOperandList()))));
// Expand "x NOT RLIKE y" into "NOT (x RLIKE y)"
registerOp(
SqlLibraryOperators.NOT_RLIKE,
(cx, call) ->
cx.convertExpression(
SqlStdOperatorTable.NOT.createCall(
SqlParserPos.ZERO,
SqlLibraryOperators.RLIKE.createCall(
SqlParserPos.ZERO, call.getOperandList()))));
// Expand "x NOT SIMILAR y" into "NOT (x SIMILAR y)"
registerOp(
SqlStdOperatorTable.NOT_SIMILAR_TO,
(cx, call) ->
cx.convertExpression(
SqlStdOperatorTable.NOT.createCall(
SqlParserPos.ZERO,
SqlStdOperatorTable.SIMILAR_TO.createCall(
SqlParserPos.ZERO, call.getOperandList()))));
// Unary "+" has no effect, so expand "+ x" into "x".
registerOp(
SqlStdOperatorTable.UNARY_PLUS,
(cx, call) -> cx.convertExpression(call.operand(0)));
// "DOT"
registerOp(
SqlStdOperatorTable.DOT,
(cx, call) ->
cx.getRexBuilder()
.makeFieldAccess(
cx.convertExpression(call.operand(0)),
call.operand(1).toString(),
false));
// "ITEM"
registerOp(SqlStdOperatorTable.ITEM, this::convertItem);
// "AS" has no effect, so expand "x AS id" into "x".
registerOp(SqlStdOperatorTable.AS, (cx, call) -> cx.convertExpression(call.operand(0)));
// "SQRT(x)" is equivalent to "POWER(x, .5)"
registerOp(
SqlStdOperatorTable.SQRT,
(cx, call) ->
cx.convertExpression(
SqlStdOperatorTable.POWER.createCall(
SqlParserPos.ZERO,
call.operand(0),
SqlLiteral.createExactNumeric("0.5", SqlParserPos.ZERO))));
// REVIEW jvs 24-Apr-2006: This only seems to be working from within a
// windowed agg. I have added an optimizer rule
// org.apache.calcite.rel.rules.AggregateReduceFunctionsRule which handles
// other cases post-translation. The reason I did that was to defer the
// implementation decision; e.g. we may want to push it down to a foreign
// server directly rather than decomposed; decomposition is easier than
// recognition.
// Convert "avg(<expr>)" to "cast(sum(<expr>) / count(<expr>) as
// <type>)". We don't need to handle the empty set specially, because
// the SUM is already supposed to come out as NULL in cases where the
// COUNT is zero, so the null check should take place first and prevent
// division by zero. We need the cast because SUM and COUNT may use
// different types, say BIGINT.
//
// Similarly STDDEV_POP and STDDEV_SAMP, VAR_POP and VAR_SAMP.
registerOp(SqlStdOperatorTable.AVG, new AvgVarianceConvertlet(SqlKind.AVG));
registerOp(SqlStdOperatorTable.STDDEV_POP, new AvgVarianceConvertlet(SqlKind.STDDEV_POP));
registerOp(SqlStdOperatorTable.STDDEV_SAMP, new AvgVarianceConvertlet(SqlKind.STDDEV_SAMP));
registerOp(SqlStdOperatorTable.STDDEV, new AvgVarianceConvertlet(SqlKind.STDDEV_SAMP));
registerOp(SqlStdOperatorTable.VAR_POP, new AvgVarianceConvertlet(SqlKind.VAR_POP));
registerOp(SqlStdOperatorTable.VAR_SAMP, new AvgVarianceConvertlet(SqlKind.VAR_SAMP));
registerOp(SqlStdOperatorTable.VARIANCE, new AvgVarianceConvertlet(SqlKind.VAR_SAMP));
registerOp(SqlStdOperatorTable.COVAR_POP, new RegrCovarianceConvertlet(SqlKind.COVAR_POP));
registerOp(
SqlStdOperatorTable.COVAR_SAMP, new RegrCovarianceConvertlet(SqlKind.COVAR_SAMP));
registerOp(SqlStdOperatorTable.REGR_SXX, new RegrCovarianceConvertlet(SqlKind.REGR_SXX));
registerOp(SqlStdOperatorTable.REGR_SYY, new RegrCovarianceConvertlet(SqlKind.REGR_SYY));
final SqlRexConvertlet floorCeilConvertlet = new FloorCeilConvertlet();
registerOp(SqlStdOperatorTable.FLOOR, floorCeilConvertlet);
registerOp(SqlStdOperatorTable.CEIL, floorCeilConvertlet);
registerOp(SqlStdOperatorTable.TIMESTAMP_ADD, new TimestampAddConvertlet());
registerOp(SqlStdOperatorTable.TIMESTAMP_DIFF, new TimestampDiffConvertlet());
registerOp(SqlStdOperatorTable.INTERVAL, StandardConvertletTable::convertInterval);
// Convert "element(<expr>)" to "$element_slice(<expr>)", if the
// expression is a multiset of scalars.
if (false) {
registerOp(
SqlStdOperatorTable.ELEMENT,
(cx, call) -> {
assert call.operandCount() == 1;
final SqlNode operand = call.operand(0);
final RelDataType type = cx.getValidator().getValidatedNodeType(operand);
if (!getComponentTypeOrThrow(type).isStruct()) {
return cx.convertExpression(
SqlStdOperatorTable.ELEMENT_SLICE.createCall(
SqlParserPos.ZERO, operand));
}
// fallback on default behavior
return StandardConvertletTable.this.convertCall(cx, call);
});
}
// Convert "$element_slice(<expr>)" to "element(<expr>).field#0"
if (false) {
registerOp(
SqlStdOperatorTable.ELEMENT_SLICE,
(cx, call) -> {
assert call.operandCount() == 1;
final SqlNode operand = call.operand(0);
final RexNode expr =
cx.convertExpression(
SqlStdOperatorTable.ELEMENT.createCall(
SqlParserPos.ZERO, operand));
return cx.getRexBuilder().makeFieldAccess(expr, 0);
});
}
}