[feature](Nereids)add type coercion rule for nereids (#11802)

- add an interface ExpectsInputTypes to Expression
- add an interface ImplicitCastInputTypes to Expression
- add a Expression rewrite rule for type coercion
- add a Check Analysis Rule to check whether Plan is Semantically correct

if Expression implements ImplicitCastInputTypes, type coercion rule will automatic rewrite its children that casting it to the most suitable type.
If Expression implements ExpectsInputTypes, Check Analysis will check its children's type whether accepted by expects input types.
This commit is contained in:
morrySnow
2022-08-22 23:06:02 +08:00
committed by GitHub
parent b55195bd80
commit caec862d91
125 changed files with 4197 additions and 588 deletions

View File

@ -19,6 +19,7 @@ package org.apache.doris.nereids.analyzer;
import org.apache.doris.nereids.CascadesContext;
import org.apache.doris.nereids.jobs.batch.AnalyzeRulesJob;
import org.apache.doris.nereids.jobs.batch.CheckAnalysisJob;
import org.apache.doris.nereids.jobs.batch.FinalizeAnalyzeJob;
import org.apache.doris.nereids.rules.analysis.Scope;
@ -45,6 +46,8 @@ public class NereidsAnalyzer {
public void analyze() {
new AnalyzeRulesJob(cascadesContext, outerScope).execute();
new FinalizeAnalyzeJob(cascadesContext).execute();
// check whether analyze result is meaningful
new CheckAnalysisJob(cascadesContext).execute();
}
public CascadesContext getCascadesContext() {

View File

@ -20,7 +20,7 @@ package org.apache.doris.nereids.analyzer;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
@ -53,7 +53,7 @@ public class UnboundAlias extends NamedExpression implements UnaryExpression, Un
}
@Override
public Expression withChildren(List<Expression> children) {
public UnboundAlias withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 1);
return new UnboundAlias(children.get(0));
}

View File

@ -79,7 +79,7 @@ public class UnboundFunction extends Expression implements Unbound {
}
@Override
public Expression withChildren(List<Expression> children) {
public UnboundFunction withChildren(List<Expression> children) {
return new UnboundFunction(name, isDistinct, isStar, children);
}

View File

@ -18,8 +18,8 @@
package org.apache.doris.nereids.analyzer;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.LeafExpression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.shape.LeafExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.util.Utils;

View File

@ -20,39 +20,27 @@ package org.apache.doris.nereids.glue.translator;
import org.apache.doris.analysis.ArithmeticExpr;
import org.apache.doris.analysis.BinaryPredicate;
import org.apache.doris.analysis.BinaryPredicate.Operator;
import org.apache.doris.analysis.BoolLiteral;
import org.apache.doris.analysis.CaseExpr;
import org.apache.doris.analysis.CaseWhenClause;
import org.apache.doris.analysis.CastExpr;
import org.apache.doris.analysis.CompoundPredicate;
import org.apache.doris.analysis.Expr;
import org.apache.doris.analysis.FloatLiteral;
import org.apache.doris.analysis.FunctionCallExpr;
import org.apache.doris.analysis.FunctionParams;
import org.apache.doris.analysis.IntLiteral;
import org.apache.doris.analysis.LikePredicate;
import org.apache.doris.analysis.NullLiteral;
import org.apache.doris.analysis.StringLiteral;
import org.apache.doris.analysis.TimestampArithmeticExpr;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.And;
import org.apache.doris.nereids.trees.expressions.Arithmetic;
import org.apache.doris.nereids.trees.expressions.Between;
import org.apache.doris.nereids.trees.expressions.BigIntLiteral;
import org.apache.doris.nereids.trees.expressions.BooleanLiteral;
import org.apache.doris.nereids.trees.expressions.BinaryArithmetic;
import org.apache.doris.nereids.trees.expressions.CaseWhen;
import org.apache.doris.nereids.trees.expressions.Cast;
import org.apache.doris.nereids.trees.expressions.DateLiteral;
import org.apache.doris.nereids.trees.expressions.DateTimeLiteral;
import org.apache.doris.nereids.trees.expressions.DoubleLiteral;
import org.apache.doris.nereids.trees.expressions.EqualTo;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.GreaterThan;
import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
import org.apache.doris.nereids.trees.expressions.InPredicate;
import org.apache.doris.nereids.trees.expressions.IntegerLiteral;
import org.apache.doris.nereids.trees.expressions.LessThan;
import org.apache.doris.nereids.trees.expressions.LessThanEqual;
import org.apache.doris.nereids.trees.expressions.Like;
@ -65,6 +53,7 @@ import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
import org.apache.doris.nereids.trees.expressions.WhenClause;
import org.apache.doris.nereids.trees.expressions.functions.BoundFunction;
import org.apache.doris.nereids.trees.expressions.functions.Count;
import org.apache.doris.nereids.trees.expressions.literal.Literal;
import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor;
import java.util.ArrayList;
@ -176,52 +165,8 @@ public class ExpressionTranslator extends DefaultExpressionVisitor<Expr, PlanTra
}
@Override
public Expr visitBooleanLiteral(BooleanLiteral booleanLiteral, PlanTranslatorContext context) {
return new BoolLiteral(booleanLiteral.getValue());
}
@Override
public Expr visitStringLiteral(org.apache.doris.nereids.trees.expressions.StringLiteral stringLiteral,
PlanTranslatorContext context) {
return new StringLiteral(stringLiteral.getValue());
}
@Override
public Expr visitIntegerLiteral(IntegerLiteral integerLiteral, PlanTranslatorContext context) {
return new IntLiteral(integerLiteral.getValue());
}
@Override
public Expr visitBigIntLiteral(BigIntLiteral bigIntLiteral, PlanTranslatorContext context) {
try {
return new IntLiteral(bigIntLiteral.getValue(), Type.BIGINT);
} catch (Throwable t) {
throw new IllegalStateException("Can not translate BigIntLiteral: " + bigIntLiteral.getValue(), t);
}
}
@Override
public Expr visitNullLiteral(org.apache.doris.nereids.trees.expressions.NullLiteral nullLiteral,
PlanTranslatorContext context) {
return new NullLiteral();
}
@Override
public Expr visitDoubleLiteral(DoubleLiteral doubleLiteral, PlanTranslatorContext context) {
return new FloatLiteral(doubleLiteral.getValue());
}
@Override
public Expr visitDateLiteral(DateLiteral dateLiteral, PlanTranslatorContext context) {
return new org.apache.doris.analysis.DateLiteral(dateLiteral.getYear(), dateLiteral.getMonth(),
dateLiteral.getDay(), 0, 0, 0);
}
@Override
public Expr visitDateTimeLiteral(DateTimeLiteral dateTimeLiteral, PlanTranslatorContext context) {
return new org.apache.doris.analysis.DateLiteral(dateTimeLiteral.getYear(), dateTimeLiteral.getMonth(),
dateTimeLiteral.getDay(), dateTimeLiteral.getHour(), dateTimeLiteral.getMinute(),
dateTimeLiteral.getSecond());
public Expr visitLiteral(Literal literal, PlanTranslatorContext context) {
return literal.toLegacyLiteral();
}
@Override
@ -282,7 +227,7 @@ public class ExpressionTranslator extends DefaultExpressionVisitor<Expr, PlanTra
public Expr visitCast(Cast cast, PlanTranslatorContext context) {
// left child of cast is expression, right child of cast is target type
return new CastExpr(cast.getDataType().toCatalogDataType(),
cast.left().accept(this, context));
cast.child().accept(this, context));
}
@Override
@ -312,11 +257,10 @@ public class ExpressionTranslator extends DefaultExpressionVisitor<Expr, PlanTra
}
@Override
public Expr visitArithmetic(Arithmetic arithmetic, PlanTranslatorContext context) {
Arithmetic.ArithmeticOperator arithmeticOperator = arithmetic.getArithmeticOperator();
return new ArithmeticExpr(arithmeticOperator.getStaleOp(),
arithmetic.child(0).accept(this, context),
arithmeticOperator.isBinary() ? arithmetic.child(1).accept(this, context) : null);
public Expr visitBinaryArithmetic(BinaryArithmetic binaryArithmetic, PlanTranslatorContext context) {
return new ArithmeticExpr(binaryArithmetic.getLegacyOperator(),
binaryArithmetic.child(0).accept(this, context),
binaryArithmetic.child(1).accept(this, context));
}
@Override

View File

@ -15,10 +15,21 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.types;
package org.apache.doris.nereids.jobs.batch;
import org.apache.doris.nereids.CascadesContext;
import org.apache.doris.nereids.rules.analysis.CheckAnalysis;
import com.google.common.collect.ImmutableList;
/**
* Abstract class for all integral data type in Nereids.
* Execute check analysis rules.
*/
public abstract class IntegralType extends NumericType {
public class CheckAnalysisJob extends BatchRulesJob {
public CheckAnalysisJob(CascadesContext cascadesContext) {
super(cascadesContext);
rulesJob.addAll(ImmutableList.of(
bottomUpBatch(ImmutableList.of(new CheckAnalysis()))
));
}
}

View File

@ -83,12 +83,8 @@ import org.apache.doris.nereids.trees.expressions.Add;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.And;
import org.apache.doris.nereids.trees.expressions.Between;
import org.apache.doris.nereids.trees.expressions.BigIntLiteral;
import org.apache.doris.nereids.trees.expressions.BooleanLiteral;
import org.apache.doris.nereids.trees.expressions.CaseWhen;
import org.apache.doris.nereids.trees.expressions.Cast;
import org.apache.doris.nereids.trees.expressions.DateLiteral;
import org.apache.doris.nereids.trees.expressions.DateTimeLiteral;
import org.apache.doris.nereids.trees.expressions.Divide;
import org.apache.doris.nereids.trees.expressions.EqualTo;
import org.apache.doris.nereids.trees.expressions.Exists;
@ -97,26 +93,31 @@ import org.apache.doris.nereids.trees.expressions.GreaterThan;
import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
import org.apache.doris.nereids.trees.expressions.InPredicate;
import org.apache.doris.nereids.trees.expressions.InSubquery;
import org.apache.doris.nereids.trees.expressions.IntegerLiteral;
import org.apache.doris.nereids.trees.expressions.IntervalLiteral;
import org.apache.doris.nereids.trees.expressions.LessThan;
import org.apache.doris.nereids.trees.expressions.LessThanEqual;
import org.apache.doris.nereids.trees.expressions.Like;
import org.apache.doris.nereids.trees.expressions.ListQuery;
import org.apache.doris.nereids.trees.expressions.Literal;
import org.apache.doris.nereids.trees.expressions.Mod;
import org.apache.doris.nereids.trees.expressions.Multiply;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Not;
import org.apache.doris.nereids.trees.expressions.NullLiteral;
import org.apache.doris.nereids.trees.expressions.NullSafeEqual;
import org.apache.doris.nereids.trees.expressions.Or;
import org.apache.doris.nereids.trees.expressions.Regexp;
import org.apache.doris.nereids.trees.expressions.ScalarSubquery;
import org.apache.doris.nereids.trees.expressions.StringLiteral;
import org.apache.doris.nereids.trees.expressions.Subtract;
import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
import org.apache.doris.nereids.trees.expressions.WhenClause;
import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral;
import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral;
import org.apache.doris.nereids.trees.expressions.literal.DateLiteral;
import org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral;
import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral;
import org.apache.doris.nereids.trees.expressions.literal.IntervalLiteral;
import org.apache.doris.nereids.trees.expressions.literal.LargeIntLiteral;
import org.apache.doris.nereids.trees.expressions.literal.Literal;
import org.apache.doris.nereids.trees.expressions.literal.NullLiteral;
import org.apache.doris.nereids.trees.expressions.literal.StringLiteral;
import org.apache.doris.nereids.trees.plans.JoinType;
import org.apache.doris.nereids.trees.plans.commands.Command;
import org.apache.doris.nereids.trees.plans.commands.ExplainCommand;
@ -130,6 +131,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
import org.apache.doris.nereids.trees.plans.logical.LogicalSelectHint;
import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias;
import org.apache.doris.nereids.types.DataType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
@ -492,7 +494,7 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
@Override
public Expression visitCast(DorisParser.CastContext ctx) {
return ParserUtils.withOrigin(ctx, () ->
new Cast(getExpression(ctx.expression()), ctx.identifier().getText()));
new Cast(getExpression(ctx.expression()), DataType.convertFromString(ctx.identifier().getText())));
}
@Override
@ -586,9 +588,10 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
BigInteger bigInt = new BigInteger(ctx.getText());
if (BigInteger.valueOf(bigInt.intValue()).equals(bigInt)) {
return new IntegerLiteral(bigInt.intValue());
} else {
// throw exception if out of long range
} else if (BigInteger.valueOf(bigInt.longValue()).equals(bigInt)) {
return new BigIntLiteral(bigInt.longValueExact());
} else {
return new LargeIntLiteral(bigInt);
}
}

View File

@ -40,6 +40,10 @@ public enum RuleType {
RESOLVE_PROJECT_ALIAS(RuleTypeClass.REWRITE),
RESOLVE_AGGREGATE_ALIAS(RuleTypeClass.REWRITE),
PROJECT_TO_GLOBAL_AGGREGATE(RuleTypeClass.REWRITE),
// check analysis rule
CHECK_ANALYSIS(RuleTypeClass.CHECK),
// rewrite rules
AGGREGATE_DISASSEMBLE(RuleTypeClass.REWRITE),
COLUMN_PRUNE_PROJECTION(RuleTypeClass.REWRITE),
@ -113,6 +117,7 @@ public enum RuleType {
enum RuleTypeClass {
REWRITE,
EXPLORATION,
CHECK,
IMPLEMENTATION,
SENTINEL,
;

View File

@ -0,0 +1,49 @@
// 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.doris.nereids.rules.analysis;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.typecoercion.TypeCheckResult;
import org.apache.doris.nereids.trees.plans.Plan;
import java.util.Optional;
/**
* Check analysis rule to check semantic correct after analysis by Nereids.
*/
public class CheckAnalysis extends OneAnalysisRuleFactory {
@Override
public Rule build() {
return any().then(this::checkExpressionInputTypes).toRule(RuleType.CHECK_ANALYSIS);
}
private Plan checkExpressionInputTypes(Plan plan) {
final Optional<TypeCheckResult> firstFailed = plan.getExpressions().stream()
.map(Expression::checkInputDataTypes)
.filter(TypeCheckResult::failed)
.findFirst();
if (firstFailed.isPresent()) {
throw new AnalysisException(firstFailed.get().getMessage());
}
return plan;
}
}

View File

@ -21,6 +21,7 @@ import org.apache.doris.nereids.rules.expression.rewrite.rules.BetweenToCompound
import org.apache.doris.nereids.rules.expression.rewrite.rules.NormalizeBinaryPredicatesRule;
import org.apache.doris.nereids.rules.expression.rewrite.rules.SimplifyCastRule;
import org.apache.doris.nereids.rules.expression.rewrite.rules.SimplifyNotExprRule;
import org.apache.doris.nereids.rules.expression.rewrite.rules.TypeCoercion;
import com.google.common.collect.ImmutableList;
@ -35,7 +36,8 @@ public class ExpressionNormalization extends ExpressionRewrite {
NormalizeBinaryPredicatesRule.INSTANCE,
BetweenToCompoundRule.INSTANCE,
SimplifyNotExprRule.INSTANCE,
SimplifyCastRule.INSTANCE
SimplifyCastRule.INSTANCE,
TypeCoercion.INSTANCE
);
private static final ExpressionRuleExecutor EXECUTOR = new ExpressionRuleExecutor(NORMALIZE_REWRITE_RULES);

View File

@ -40,7 +40,7 @@ public class SimplifyCastRule extends AbstractExpressionRewriteRule {
}
private Expression simplify(Cast cast) {
Expression source = cast.left();
Expression source = cast.child();
// simplify inside
if (source instanceof Cast) {
source = simplify((Cast) source);
@ -52,8 +52,8 @@ public class SimplifyCastRule extends AbstractExpressionRewriteRule {
return source;
}
if (source != cast.left()) {
return new Cast(source, cast.right());
if (source != cast.child()) {
return new Cast(source, cast.getDataType());
}
return cast;
}

View File

@ -0,0 +1,179 @@
// 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.doris.nereids.rules.expression.rewrite.rules;
import org.apache.doris.nereids.annotation.Developing;
import org.apache.doris.nereids.rules.expression.rewrite.AbstractExpressionRewriteRule;
import org.apache.doris.nereids.rules.expression.rewrite.ExpressionRewriteContext;
import org.apache.doris.nereids.trees.expressions.BinaryOperator;
import org.apache.doris.nereids.trees.expressions.CaseWhen;
import org.apache.doris.nereids.trees.expressions.Cast;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.InPredicate;
import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.util.TypeCoercionUtils;
import com.google.common.collect.Lists;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
/**
* a rule to add implicit cast for expressions.
* This class is inspired by spark's TypeCoercion.
*/
@Developing
public class TypeCoercion extends AbstractExpressionRewriteRule {
// TODO:
// 1. DecimalPrecision Process
// 2. Divide process
// 3. String promote with numeric in binary arithmetic
// 4. Date and DateTime process
public static final TypeCoercion INSTANCE = new TypeCoercion();
@Override
public Expression visit(Expression expr, ExpressionRewriteContext ctx) {
if (expr instanceof ImplicitCastInputTypes) {
return visitImplicitCastInputTypes(expr, ctx);
} else {
return super.visit(expr, ctx);
}
}
// TODO: add other expression visitor function to do type coercion if necessary.
@Override
public Expression visitBinaryOperator(BinaryOperator binaryOperator, ExpressionRewriteContext context) {
Expression left = rewrite(binaryOperator.left(), context);
Expression right = rewrite(binaryOperator.right(), context);
return Optional.of(TypeCoercionUtils.canHandleTypeCoercion(left.getDataType(), right.getDataType()))
.filter(Boolean::booleanValue)
.map(b -> TypeCoercionUtils.findTightestCommonType(left.getDataType(), right.getDataType()))
.filter(Optional::isPresent)
.map(Optional::get)
.filter(ct -> binaryOperator.inputType().acceptsType(ct))
.filter(ct -> !left.getDataType().equals(ct) || !right.getDataType().equals(ct))
.map(commonType -> {
Expression newLeft = TypeCoercionUtils.castIfNotSameType(left, commonType);
Expression newRight = TypeCoercionUtils.castIfNotSameType(right, commonType);
return binaryOperator.withChildren(newLeft, newRight);
})
.orElse(binaryOperator.withChildren(left, right));
}
@Override
public Expression visitCaseWhen(CaseWhen caseWhen, ExpressionRewriteContext context) {
List<Expression> rewrittenChildren = caseWhen.children().stream()
.map(e -> rewrite(e, context)).collect(Collectors.toList());
CaseWhen newCaseWhen = caseWhen.withChildren(rewrittenChildren);
List<DataType> dataTypesForCoercion = newCaseWhen.dataTypesForCoercion();
if (dataTypesForCoercion.size() <= 1) {
return newCaseWhen;
}
DataType first = dataTypesForCoercion.get(0);
if (dataTypesForCoercion.stream().allMatch(dataType -> dataType.equals(first))) {
return newCaseWhen;
}
Optional<DataType> optionalCommonType = TypeCoercionUtils.findWiderCommonType(dataTypesForCoercion);
return optionalCommonType
.map(commonType -> {
List<Expression> newChildren
= newCaseWhen.getWhenClauses().stream()
.map(wc -> wc.withChildren(wc.getOperand(),
TypeCoercionUtils.castIfNotSameType(wc.getResult(), commonType)))
.collect(Collectors.toList());
newCaseWhen.getDefaultValue()
.map(dv -> TypeCoercionUtils.castIfNotSameType(dv, commonType))
.ifPresent(newChildren::add);
return newCaseWhen.withChildren(newChildren);
})
.orElse(newCaseWhen);
}
@Override
public Expression visitInPredicate(InPredicate inPredicate, ExpressionRewriteContext context) {
List<Expression> rewrittenChildren = inPredicate.children().stream()
.map(e -> rewrite(e, context)).collect(Collectors.toList());
InPredicate newInPredicate = inPredicate.withChildren(rewrittenChildren);
if (newInPredicate.getOptions().stream().map(Expression::getDataType)
.allMatch(dt -> dt.equals(newInPredicate.getCompareExpr().getDataType()))) {
return newInPredicate;
}
Optional<DataType> optionalCommonType = TypeCoercionUtils.findWiderCommonType(newInPredicate.children()
.stream().map(Expression::getDataType).collect(Collectors.toList()));
return optionalCommonType
.map(commonType -> {
List<Expression> newChildren = newInPredicate.children().stream()
.map(e -> TypeCoercionUtils.castIfNotSameType(e, commonType))
.collect(Collectors.toList());
return newInPredicate.withChildren(newChildren);
})
.orElse(newInPredicate);
}
/**
* Do implicit cast for expression's children.
*/
private Expression visitImplicitCastInputTypes(Expression expr, ExpressionRewriteContext ctx) {
ImplicitCastInputTypes implicitCastInputTypes = (ImplicitCastInputTypes) expr;
List<Expression> newChildren = Lists.newArrayListWithCapacity(expr.arity());
AtomicInteger changed = new AtomicInteger(0);
for (int i = 0; i < implicitCastInputTypes.expectedInputTypes().size(); i++) {
newChildren.add(implicitCast(expr.child(i), implicitCastInputTypes.expectedInputTypes().get(i), ctx)
.map(e -> {
changed.incrementAndGet();
return e;
})
.orElse(expr.child(0))
);
}
if (changed.get() != 0) {
return expr.withChildren(newChildren);
} else {
return expr;
}
}
/**
* Return Optional.empty() if we cannot do or do not need to do implicit cast.
*/
@Developing
private Optional<Expression> implicitCast(Expression input, AbstractDataType expected,
ExpressionRewriteContext ctx) {
Expression rewrittenInput = rewrite(input, ctx);
Optional<DataType> castDataType = TypeCoercionUtils.implicitCast(rewrittenInput.getDataType(), expected);
if (castDataType.isPresent() && !castDataType.get().equals(rewrittenInput.getDataType())) {
return Optional.of(new Cast(rewrittenInput, castDataType.get()));
} else {
if (rewrittenInput == input) {
return Optional.empty();
} else {
return Optional.of(rewrittenInput);
}
}
}
}

View File

@ -20,10 +20,10 @@ package org.apache.doris.nereids.rules.rewrite.logical;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
import org.apache.doris.nereids.trees.expressions.BooleanLiteral;
import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral;
import org.apache.doris.nereids.trees.plans.GroupPlan;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;

View File

@ -21,10 +21,10 @@ import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
import org.apache.doris.nereids.trees.expressions.CompoundPredicate;
import org.apache.doris.nereids.trees.expressions.EqualTo;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Literal;
import org.apache.doris.nereids.trees.expressions.Or;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.expressions.literal.Literal;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.statistics.ColumnStats;
@ -37,7 +37,7 @@ import java.util.Map;
*/
public class FilterSelectivityCalculator extends ExpressionVisitor<Double, Void> {
private static double DEFAULT_SELECTIVITY = 0.1;
private static final double DEFAULT_SELECTIVITY = 0.1;
private final Map<Slot, ColumnStats> slotRefToStats;

View File

@ -17,8 +17,10 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.analysis.ArithmeticExpr.Operator;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.NumericType;
import com.google.common.base.Preconditions;
@ -27,15 +29,10 @@ import java.util.List;
/**
* Add Expression.
*/
public class Add extends Arithmetic implements BinaryExpression {
public Add(Expression left, Expression right) {
super(ArithmeticOperator.ADD, left, right);
}
public class Add extends BinaryArithmetic {
@Override
public String toSql() {
return left().toSql() + ' ' + getArithmeticOperator().toString()
+ ' ' + right().toSql();
public Add(Expression left, Expression right) {
super(left, right, Operator.ADD);
}
@Override
@ -44,18 +41,13 @@ public class Add extends Arithmetic implements BinaryExpression {
return new Add(children.get(0), children.get(1));
}
@Override
public boolean nullable() throws UnboundException {
return left().nullable() || right().nullable();
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitAdd(this, context);
}
public String toString() {
return left().toString() + ' ' + getArithmeticOperator().toString() + ' ' + right().toString();
@Override
public AbstractDataType inputType() {
return NumericType.INSTANCE;
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
@ -123,7 +124,7 @@ public class Alias extends NamedExpression implements UnaryExpression {
}
@Override
public Expression withChildren(List<Expression> children) {
public Alias withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 1);
return new Alias(exprId, children.get(0), name);
}

View File

@ -27,6 +27,7 @@ import java.util.List;
* And predicate expression.
*/
public class And extends CompoundPredicate {
/**
* Desc: Constructor for CompoundPredicate.
*

View File

@ -1,188 +0,0 @@
// 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.doris.nereids.trees.expressions;
import org.apache.doris.analysis.ArithmeticExpr;
import org.apache.doris.analysis.ArithmeticExpr.Operator;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
import java.util.Objects;
import java.util.function.Function;
/**
* All arithmetic operator.
*/
public abstract class Arithmetic extends Expression {
enum OperatorPosition {
BINARY_INFIX,
UNARY_PREFIX,
UNARY_POSTFIX,
}
/**
* All counts as expressions.
*/
@SuppressWarnings("checkstyle:RegexpSingleline")
public enum ArithmeticOperator {
MULTIPLY("*", "multiply",
Arithmetic.OperatorPosition.BINARY_INFIX, Operator.MULTIPLY),
DIVIDE("/", "divide",
Arithmetic.OperatorPosition.BINARY_INFIX, Operator.DIVIDE),
MOD("%", "mod",
Arithmetic.OperatorPosition.BINARY_INFIX, Operator.MOD),
ADD("+", "add",
Arithmetic.OperatorPosition.BINARY_INFIX, Operator.ADD),
SUBTRACT("-", "subtract",
Arithmetic.OperatorPosition.BINARY_INFIX, Operator.SUBTRACT),
//TODO: The following functions will be added later.
BITAND("&", "bitand",
Arithmetic.OperatorPosition.BINARY_INFIX, Operator.BITAND),
BITOR("|", "bitor",
Arithmetic.OperatorPosition.BINARY_INFIX, Operator.BITOR),
BITXOR("^", "bitxor",
Arithmetic.OperatorPosition.BINARY_INFIX, Operator.BITXOR),
BITNOT("~", "bitnot",
Arithmetic.OperatorPosition.UNARY_PREFIX, Operator.BITNOT);
private final String description;
private final String name;
private final Arithmetic.OperatorPosition pos;
private final ArithmeticExpr.Operator staleOp;
ArithmeticOperator(String description,
String name,
Arithmetic.OperatorPosition pos,
ArithmeticExpr.Operator staleOp) {
this.description = description;
this.name = name;
this.pos = pos;
this.staleOp = staleOp;
}
@Override
public String toString() {
return description;
}
public String getName() {
return name;
}
public Arithmetic.OperatorPosition getPos() {
return pos;
}
public Operator getStaleOp() {
return staleOp;
}
public boolean isUnary() {
return pos == Arithmetic.OperatorPosition.UNARY_PREFIX
|| pos == Arithmetic.OperatorPosition.UNARY_POSTFIX;
}
public boolean isBinary() {
return pos == Arithmetic.OperatorPosition.BINARY_INFIX;
}
}
private final ArithmeticOperator op;
public Arithmetic(ArithmeticOperator op, Expression... children) {
super(children);
this.op = op;
}
public ArithmeticOperator getArithmeticOperator() {
return op;
}
@Override
public DataType getDataType() {
// TODO: split Unary and Binary arithmetic
int arity = arity();
if (arity == 1) {
return child(0).getDataType();
} else if (arity == 2) {
// TODO: binary arithmetic
return child(0).getDataType();
} else {
return super.getDataType();
}
}
@Override
public boolean nullable() throws UnboundException {
if (op.isUnary()) {
return child(0).nullable();
} else {
return child(0).nullable() || child(1).nullable();
}
}
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitArithmetic(this, context);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
Arithmetic that = (Arithmetic) o;
return op == that.op && Objects.equals(this.children(), that.children());
}
@Override
public int hashCode() {
return Objects.hash(op, children());
}
@Override
public String toString() {
return stringBuilder(Object::toString);
}
@Override
public String toSql() {
return stringBuilder(Expression::toSql);
}
private String stringBuilder(Function<Expression, String> stringMapper) {
switch (op.getPos()) {
case BINARY_INFIX:
return stringMapper.apply(children.get(0)) + " " + op + " " + stringMapper.apply(children.get(1));
case UNARY_PREFIX:
return op + stringMapper.apply(children.get(0));
case UNARY_POSTFIX:
return stringMapper.apply(children.get(0)) + op;
default:
throw new IllegalStateException("Not supported operator: " + op);
}
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.shape.TernaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.BooleanType;
import org.apache.doris.nereids.types.DataType;
@ -32,9 +33,9 @@ import java.util.Objects;
*/
public class Between extends Expression implements TernaryExpression {
private Expression compareExpr;
private Expression lowerBound;
private Expression upperBound;
private final Expression compareExpr;
private final Expression lowerBound;
private final Expression upperBound;
/**
* Constructor of ComparisonPredicate.
*
@ -88,7 +89,7 @@ public class Between extends Expression implements TernaryExpression {
}
@Override
public Expression withChildren(List<Expression> children) {
public Between withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 3);
return new Between(children.get(0), children.get(1), children.get(2));
}

View File

@ -0,0 +1,54 @@
// 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.doris.nereids.trees.expressions;
import org.apache.doris.analysis.ArithmeticExpr.Operator;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
/**
* binary arithmetic operator. Such as +, -, *, /.
*/
public abstract class BinaryArithmetic extends BinaryOperator {
private final Operator legacyOperator;
public BinaryArithmetic(Expression left, Expression right, Operator legacyOperator) {
super(left, right, legacyOperator.toString());
this.legacyOperator = legacyOperator;
}
public Operator getLegacyOperator() {
return legacyOperator;
}
@Override
public DataType getDataType() throws UnboundException {
return left().getDataType();
}
@Override
public boolean nullable() throws UnboundException {
return child(0).nullable() || child(1).nullable();
}
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitBinaryArithmetic(this, context);
}
}

View File

@ -0,0 +1,75 @@
// 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.doris.nereids.trees.expressions;
import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
import org.apache.doris.nereids.trees.expressions.typecoercion.ExpectsInputTypes;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.Objects;
/**
* Abstract for all binary operator, include binary arithmetic, compound predicate, comparison predicate.
*/
public abstract class BinaryOperator extends Expression implements BinaryExpression, ExpectsInputTypes {
protected final String symbol;
public BinaryOperator(Expression left, Expression right, String symbol) {
super(left, right);
this.symbol = symbol;
}
public abstract AbstractDataType inputType();
@Override
public List<AbstractDataType> expectedInputTypes() {
return ImmutableList.of(inputType(), inputType());
}
@Override
public String toSql() {
return "(" + left().toSql() + " " + symbol + " " + right().toSql() + ")";
}
@Override
public String toString() {
return "(" + left().toString() + " " + symbol + " " + right().toString() + ")";
}
@Override
public int hashCode() {
return Objects.hash(symbol, left(), right());
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
BinaryOperator other = (BinaryOperator) o;
return Objects.equals(left(), other.left()) && Objects.equals(right(), other.right());
}
}

View File

@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;
@ -37,35 +38,34 @@ import java.util.stream.Collectors;
* If an else expr is given then it is the last child.
*/
public class CaseWhen extends Expression {
/**
* If default value exists, then defaultValueIndex is the index of the last element in children,
* otherwise it is -1
*/
private final int defaultValueIndex;
private final List<WhenClause> whenClauses;
private final Optional<Expression> defaultValue;
public CaseWhen(List<WhenClause> whenClauses) {
super(whenClauses.toArray(new Expression[0]));
defaultValueIndex = -1;
this.whenClauses = ImmutableList.copyOf(Objects.requireNonNull(whenClauses));
defaultValue = Optional.empty();
}
public CaseWhen(List<WhenClause> whenClauses, Expression defaultValue) {
super(ImmutableList.builder().addAll(whenClauses).add(defaultValue).build().toArray(new Expression[0]));
defaultValueIndex = children().size() - 1;
super(ImmutableList.<Expression>builder()
.addAll(whenClauses).add(defaultValue).build()
.toArray(new Expression[0]));
this.whenClauses = ImmutableList.copyOf(Objects.requireNonNull(whenClauses));
this.defaultValue = Optional.of(Objects.requireNonNull(defaultValue));
}
public List<WhenClause> getWhenClauses() {
List<WhenClause> whenClauses = children().stream()
.filter(e -> e instanceof WhenClause)
.map(e -> (WhenClause) e)
.collect(Collectors.toList());
return whenClauses;
}
public Optional<Expression> getDefaultValue() {
if (defaultValueIndex == -1) {
return Optional.empty();
}
return Optional.of(child(defaultValueIndex));
return defaultValue;
}
public List<DataType> dataTypesForCoercion() {
return whenClauses.stream().map(WhenClause::getDataType).collect(Collectors.toList());
}
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
@ -79,12 +79,12 @@ public class CaseWhen extends Expression {
@Override
public boolean nullable() {
for (Expression child : children()) {
if (child.nullable()) {
for (WhenClause whenClause : whenClauses) {
if (whenClause.nullable()) {
return true;
}
}
return false;
return defaultValue.map(Expression::nullable).orElse(true);
}
@Override
@ -99,7 +99,7 @@ public class CaseWhen extends Expression {
if (child instanceof WhenClause) {
output.append(child.toSql());
} else {
output.append(" ELSE " + child.toSql());
output.append(" ELSE ").append(child.toSql());
}
}
output.append(" END");
@ -107,7 +107,7 @@ public class CaseWhen extends Expression {
}
@Override
public Expression withChildren(List<Expression> children) {
public CaseWhen withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() >= 1);
List<WhenClause> whenClauseList = new ArrayList<>();
Expression defaultValue = null;
@ -117,7 +117,7 @@ public class CaseWhen extends Expression {
} else if (children.size() - 1 == i) {
defaultValue = children.get(i);
} else {
throw new IllegalArgumentException("The children format needs to be [WhenClause*, DefaultValue+]");
throw new IllegalArgumentException("The children format needs to be [WhenClause+, DefaultValue?]");
}
}
if (defaultValue == null) {

View File

@ -18,35 +18,30 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
import com.google.common.base.Preconditions;
import java.util.List;
import java.util.Objects;
/**
* cast function.
*/
public class Cast extends Expression implements BinaryExpression {
public class Cast extends Expression implements UnaryExpression {
public Cast(Expression child, String type) {
super(child, new StringLiteral(type));
}
private final DataType targetType;
public Cast(Expression child, StringLiteral type) {
super(child, type);
}
@Override
public StringLiteral right() {
return (StringLiteral) BinaryExpression.super.right();
public Cast(Expression child, DataType targetType) {
super(child);
this.targetType = targetType;
}
@Override
public DataType getDataType() {
StringLiteral type = right();
return DataType.convertFromString(type.getValue());
return targetType;
}
@Override
@ -56,23 +51,36 @@ public class Cast extends Expression implements BinaryExpression {
@Override
public boolean nullable() {
return left().nullable();
return child().nullable();
}
@Override
public Expression withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2);
Preconditions.checkArgument(children.get(1) instanceof StringLiteral);
return new Cast(children.get(0), ((StringLiteral) children.get(1)).getValue());
public Cast withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 1);
return new Cast(children.get(0), getDataType());
}
@Override
public String toSql() throws UnboundException {
return "CAST(" + left().toSql() + " AS " + right().getValue() + ")";
return "CAST(" + child().toSql() + " AS " + targetType + ")";
}
@Override
public String toString() {
return toSql();
}
@Override
public boolean equals(Object o) {
if (!super.equals(o)) {
return false;
}
Cast cast = (Cast) o;
return Objects.equals(targetType, cast.targetType);
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), targetType);
}
}

View File

@ -21,16 +21,14 @@ import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.BooleanType;
import org.apache.doris.nereids.types.DataType;
import java.util.Objects;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.AnyDataType;
/**
* Comparison predicate expression.
* Such as: "=", "<", "<=", ">", ">=", "<=>"
*/
public abstract class ComparisonPredicate extends Expression implements BinaryExpression {
protected final String symbol;
public abstract class ComparisonPredicate extends BinaryOperator {
/**
* Constructor of ComparisonPredicate.
@ -39,8 +37,7 @@ public abstract class ComparisonPredicate extends Expression implements BinaryEx
* @param right right child of comparison predicate
*/
public ComparisonPredicate(Expression left, Expression right, String symbol) {
super(left, right);
this.symbol = symbol;
super(left, right, symbol);
}
@Override
@ -48,36 +45,13 @@ public abstract class ComparisonPredicate extends Expression implements BinaryEx
return BooleanType.INSTANCE;
}
@Override
public boolean nullable() throws UnboundException {
return left().nullable() || right().nullable();
}
@Override
public String toSql() {
return "(" + left().toSql() + ' ' + symbol + ' ' + right().toSql() + ")";
}
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitComparisonPredicate(this, context);
}
@Override
public int hashCode() {
return Objects.hash(symbol, left(), right());
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
ComparisonPredicate other = (ComparisonPredicate) o;
return Objects.equals(left(), other.left())
&& Objects.equals(right(), other.right());
public AbstractDataType inputType() {
return AnyDataType.INSTANCE;
}
/**

View File

@ -19,30 +19,25 @@ package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import java.util.Objects;
import org.apache.doris.nereids.types.BooleanType;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
/**
* Compound predicate expression.
* Such as &&,||,AND,OR.
*/
public abstract class CompoundPredicate extends Expression implements BinaryExpression {
protected final String symbol;
public abstract class CompoundPredicate extends BinaryOperator {
/**
* Desc: Constructor for CompoundPredicate.
*
* @param left left child of comparison predicate
* @param right right child of comparison predicate
* @param symbol symbol used in sql
*/
public CompoundPredicate(Expression left, Expression right, String symbol) {
super(left, right);
this.symbol = symbol;
}
@Override
public String toSql() {
return "(" + left().toSql() + " " + symbol + " " + right().toSql() + ")";
super(left, right, symbol);
}
@Override
@ -50,32 +45,19 @@ public abstract class CompoundPredicate extends Expression implements BinaryExpr
return left().nullable() || right().nullable();
}
@Override
public DataType getDataType() throws UnboundException {
return BooleanType.INSTANCE;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitCompoundPredicate(this, context);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
CompoundPredicate other = (CompoundPredicate) o;
return Objects.equals(left(), other.left())
&& Objects.equals(right(), other.right());
}
@Override
public int hashCode() {
return Objects.hash(symbol, left(), right());
}
@Override
public String toString() {
return "(" + left().toString() + " " + symbol + " " + right().toString() + ")";
public AbstractDataType inputType() {
return BooleanType.INSTANCE;
}
/**

View File

@ -17,7 +17,10 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.analysis.ArithmeticExpr.Operator;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.NumericType;
import com.google.common.base.Preconditions;
@ -26,15 +29,10 @@ import java.util.List;
/**
* Divide Expression.
*/
public class Divide extends Arithmetic implements BinaryExpression {
public Divide(Expression left, Expression right) {
super(ArithmeticOperator.DIVIDE, left, right);
}
public class Divide extends BinaryArithmetic {
@Override
public String toSql() {
return left().toSql() + ' ' + getArithmeticOperator().toString()
+ ' ' + right().toSql();
public Divide(Expression left, Expression right) {
super(left, right, Operator.DIVIDE);
}
@Override
@ -47,4 +45,9 @@ public class Divide extends Arithmetic implements BinaryExpression {
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitDivide(this, context);
}
@Override
public AbstractDataType inputType() {
return NumericType.INSTANCE;
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.shape.LeafExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.types.BooleanType;

View File

@ -20,10 +20,18 @@ package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.AbstractTreeNode;
import org.apache.doris.nereids.trees.expressions.literal.Literal;
import org.apache.doris.nereids.trees.expressions.shape.LeafExpression;
import org.apache.doris.nereids.trees.expressions.typecoercion.ExpectsInputTypes;
import org.apache.doris.nereids.trees.expressions.typecoercion.TypeCheckResult;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -35,6 +43,8 @@ import java.util.Objects;
*/
public abstract class Expression extends AbstractTreeNode<Expression> {
private static final String INPUT_CHECK_ERROR_MESSAGE = "argument %d requires %s type, however '%s' is of %s type";
private final Logger logger = LoggerFactory.getLogger(this.getClass());
public Expression(Expression... children) {
@ -53,6 +63,31 @@ public abstract class Expression extends AbstractTreeNode<Expression> {
throw new UnboundException("nullable");
}
public TypeCheckResult checkInputDataTypes() {
if (this instanceof ExpectsInputTypes) {
ExpectsInputTypes expectsInputTypes = (ExpectsInputTypes) this;
return checkInputDataTypes(children, expectsInputTypes.expectedInputTypes());
}
return TypeCheckResult.SUCCESS;
}
private TypeCheckResult checkInputDataTypes(List<Expression> inputs, List<AbstractDataType> inputTypes) {
Preconditions.checkArgument(inputs.size() == inputTypes.size());
List<String> errorMessages = Lists.newArrayList();
for (int i = 0; i < inputs.size(); i++) {
Expression input = inputs.get(i);
AbstractDataType inputType = inputTypes.get(i);
if (!inputType.acceptsType(input.getDataType())) {
errorMessages.add(String.format(INPUT_CHECK_ERROR_MESSAGE,
i + 1, inputType.simpleString(), input.toSql(), input.getDataType().simpleString()));
}
}
if (!errorMessages.isEmpty()) {
return new TypeCheckResult(false, StringUtils.join(errorMessages, ", "));
}
return TypeCheckResult.SUCCESS;
}
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visit(this, context);
}

View File

@ -59,7 +59,7 @@ public class InPredicate extends Expression {
}
@Override
public Expression withChildren(List<Expression> children) {
public InPredicate withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() > 1);
return new InPredicate(children.get(0), ImmutableList.copyOf(children).subList(1, children.size()));
}

View File

@ -29,13 +29,14 @@ import java.util.Objects;
/**
* In predicate expression.
*/
public class InSubquery extends SubqueryExpr implements BinaryExpression {
private Expression compareExpr;
private ListQuery listQuery;
public class InSubquery extends SubqueryExpr {
private final Expression compareExpr;
private final ListQuery listQuery;
public InSubquery(Expression compareExpression, ListQuery listQuery) {
super(Objects.requireNonNull(listQuery.getQueryPlan(), "subquery can not be null"));
this.compareExpr = compareExpression;
this.compareExpr = Objects.requireNonNull(compareExpression);
this.listQuery = listQuery;
}
@ -68,9 +69,8 @@ public class InSubquery extends SubqueryExpr implements BinaryExpression {
}
@Override
public Expression withChildren(List<Expression> children) {
public InSubquery withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2);
Preconditions.checkArgument(children.get(0) instanceof Expression);
Preconditions.checkArgument(children.get(1) instanceof ListQuery);
return new InSubquery(children.get(0), (ListQuery) children.get(1));
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.shape.LeafExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.types.DataType;

View File

@ -17,7 +17,10 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.analysis.ArithmeticExpr.Operator;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.NumericType;
import com.google.common.base.Preconditions;
@ -26,15 +29,10 @@ import java.util.List;
/**
* Mod Expression.
*/
public class Mod extends Arithmetic implements BinaryExpression {
public Mod(Expression left, Expression right) {
super(ArithmeticOperator.MOD, left, right);
}
public class Mod extends BinaryArithmetic {
@Override
public String toSql() {
return left().toSql() + ' ' + getArithmeticOperator().toString()
+ ' ' + right().toSql();
public Mod(Expression left, Expression right) {
super(left, right, Operator.MOD);
}
@Override
@ -47,4 +45,9 @@ public class Mod extends Arithmetic implements BinaryExpression {
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitMod(this, context);
}
@Override
public AbstractDataType inputType() {
return NumericType.INSTANCE;
}
}

View File

@ -17,7 +17,10 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.analysis.ArithmeticExpr.Operator;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.NumericType;
import com.google.common.base.Preconditions;
@ -26,16 +29,16 @@ import java.util.List;
/**
* Multiply Expression.
*/
public class Multiply extends Arithmetic implements BinaryExpression {
public class Multiply extends BinaryArithmetic {
public Multiply(Expression left, Expression right) {
super(ArithmeticOperator.MULTIPLY, left, right);
super(left, right, Operator.MULTIPLY);
}
@Override
public String toSql() {
return left().toSql() + ' ' + getArithmeticOperator().toString()
+ ' ' + right().toSql();
public Expression withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2);
return new Multiply(children.get(0), children.get(1));
}
@Override
@ -44,8 +47,7 @@ public class Multiply extends Arithmetic implements BinaryExpression {
}
@Override
public Expression withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2);
return new Multiply(children.get(0), children.get(1));
public AbstractDataType inputType() {
return NumericType.INSTANCE;
}
}

View File

@ -18,9 +18,15 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.typecoercion.ExpectsInputTypes;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.BooleanType;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.Objects;
@ -28,7 +34,9 @@ import java.util.Objects;
/**
* Not expression: not a.
*/
public class Not extends Expression implements UnaryExpression {
public class Not extends Expression implements UnaryExpression, ExpectsInputTypes {
public static final List<AbstractDataType> EXPECTS_INPUT_TYPES = ImmutableList.of(BooleanType.INSTANCE);
public Not(Expression child) {
super(child);
@ -39,6 +47,11 @@ public class Not extends Expression implements UnaryExpression {
return child().nullable();
}
@Override
public DataType getDataType() throws UnboundException {
return child().getDataType();
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitNot(this, context);
@ -76,4 +89,9 @@ public class Not extends Expression implements UnaryExpression {
Preconditions.checkArgument(children.size() == 1);
return new Not(children.get(0));
}
@Override
public List<AbstractDataType> expectedInputTypes() {
return EXPECTS_INPUT_TYPES;
}
}

View File

@ -55,7 +55,7 @@ public class NullSafeEqual extends ComparisonPredicate {
}
@Override
public Expression withChildren(List<Expression> children) {
public NullSafeEqual withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2);
return new NullSafeEqual(children.get(0), children.get(1));
}

View File

@ -27,6 +27,7 @@ import java.util.List;
* Or predicate expression.
*/
public class Or extends CompoundPredicate {
/**
* Desc: Constructor for CompoundPredicate.
*

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.shape.LeafExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.types.DataType;

View File

@ -17,6 +17,8 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.trees.expressions.shape.LeafExpression;
import java.util.List;
/**

View File

@ -138,7 +138,7 @@ public class SlotReference extends Slot {
}
@Override
public Expression withChildren(List<Expression> children) {
public SlotReference withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 0);
return this;
}

View File

@ -18,17 +18,30 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.BooleanType;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.TypeCollection;
import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.Objects;
/**
* string regex expression.
* Such as: like, regexp
*/
public abstract class StringRegexPredicate extends Expression implements BinaryExpression {
public abstract class StringRegexPredicate extends Expression implements BinaryExpression, ImplicitCastInputTypes {
// used in interface expectedInputTypes to avoid new list in each time it be called
private static final List<AbstractDataType> EXPECTED_INPUT_TYPES = ImmutableList.of(
TypeCollection.CHARACTER_TYPE_COLLECTION,
TypeCollection.CHARACTER_TYPE_COLLECTION
);
/**
* like or regexp
@ -52,6 +65,11 @@ public abstract class StringRegexPredicate extends Expression implements BinaryE
return BooleanType.INSTANCE;
}
@Override
public List<AbstractDataType> expectedInputTypes() {
return EXPECTED_INPUT_TYPES;
}
@Override
public String toSql() {
return '(' + left().toSql() + ' ' + symbol + ' ' + right().toSql() + ')';

View File

@ -30,7 +30,7 @@ import java.util.Objects;
/**
* Subquery Expression.
*/
public class SubqueryExpr extends Expression {
public abstract class SubqueryExpr extends Expression {
protected LogicalPlan queryPlan;
public SubqueryExpr(LogicalPlan subquery) {

View File

@ -17,7 +17,10 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.analysis.ArithmeticExpr.Operator;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.NumericType;
import com.google.common.base.Preconditions;
@ -26,15 +29,10 @@ import java.util.List;
/**
* Subtract Expression. BinaryExpression.
*/
public class Subtract extends Arithmetic implements BinaryExpression {
public Subtract(Expression left, Expression right) {
super(ArithmeticOperator.SUBTRACT, left, right);
}
public class Subtract extends BinaryArithmetic {
@Override
public String toSql() {
return left().toSql() + ' ' + getArithmeticOperator().toString()
+ ' ' + right().toSql();
public Subtract(Expression left, Expression right) {
super(left, right, Operator.SUBTRACT);
}
@Override
@ -47,4 +45,9 @@ public class Subtract extends Arithmetic implements BinaryExpression {
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitSubtract(this, context);
}
@Override
public AbstractDataType inputType() {
return NumericType.INSTANCE;
}
}

View File

@ -18,8 +18,12 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.analysis.ArithmeticExpr.Operator;
import org.apache.doris.nereids.trees.expressions.IntervalLiteral.TimeUnit;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.literal.IntervalLiteral.TimeUnit;
import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.DateTimeType;
import com.google.common.base.Preconditions;
import org.apache.logging.log4j.LogManager;
@ -31,13 +35,14 @@ import java.util.List;
* Describes the addition and subtraction of time units from timestamps.
* Arithmetic expressions on timestamps are syntactic sugar.
* They are executed as function call exprs in the BE.
* TODO: we need to rethink this, and maybe need to add a new type of Interval then implement IntervalLiteral as others
*/
public class TimestampArithmetic extends Expression implements BinaryExpression {
private static final Logger LOG = LogManager.getLogger(TimestampArithmetic.class);
private final String funcName;
private final boolean intervalFirst;
private Operator op;
private TimeUnit timeUnit;
private final Operator op;
private final TimeUnit timeUnit;
public TimestampArithmetic(String funcName, Expression e1, Expression e2, TimeUnit timeUnit) {
this(funcName, null, e1, e2, timeUnit, false);
@ -45,7 +50,6 @@ public class TimestampArithmetic extends Expression implements BinaryExpression
public TimestampArithmetic(Operator op, Expression e1, Expression e2, TimeUnit timeUnit, boolean intervalFirst) {
this(null, op, e1, e2, timeUnit, intervalFirst);
}
/**
@ -67,7 +71,7 @@ public class TimestampArithmetic extends Expression implements BinaryExpression
}
@Override
public Expression withChildren(List<Expression> children) {
public TimestampArithmetic withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2);
return new TimestampArithmetic(this.funcName, this.op, children.get(0), children.get(1),
this.timeUnit, this.intervalFirst);
@ -78,6 +82,11 @@ public class TimestampArithmetic extends Expression implements BinaryExpression
this.intervalFirst);
}
@Override
public DataType getDataType() throws UnboundException {
return DateTimeType.INSTANCE;
}
public String getFuncName() {
return funcName;
}
@ -115,16 +124,16 @@ public class TimestampArithmetic extends Expression implements BinaryExpression
if (intervalFirst) {
// Non-function-call like version with interval as first operand.
strBuilder.append("INTERVAL ");
strBuilder.append(child(1).toSql() + " ");
strBuilder.append(child(1).toSql()).append(" ");
strBuilder.append(timeUnit);
strBuilder.append(" ").append(op.toString()).append(" ");
strBuilder.append(child(0).toSql());
} else {
// Non-function-call like version with interval as second operand.
strBuilder.append(child(0).toSql());
strBuilder.append(" " + op.toString() + " ");
strBuilder.append(" ").append(op.toString()).append(" ");
strBuilder.append("INTERVAL ");
strBuilder.append(child(1).toSql() + " ");
strBuilder.append(child(1).toSql()).append(" ");
strBuilder.append(timeUnit);
}
return strBuilder.toString();

View File

@ -18,10 +18,16 @@
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
import org.apache.doris.nereids.trees.expressions.typecoercion.ExpectsInputTypes;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.BooleanType;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.AnyDataType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.Objects;
@ -29,18 +35,30 @@ import java.util.Objects;
/**
* captures info of a single WHEN expr THEN expr clause.
*/
public class WhenClause extends Expression implements BinaryExpression {
public class WhenClause extends Expression implements BinaryExpression, ExpectsInputTypes {
public static final List<AbstractDataType> EXPECTS_INPUT_TYPES
= ImmutableList.of(BooleanType.INSTANCE, AnyDataType.INSTANCE);
public WhenClause(Expression operand, Expression result) {
super(operand, result);
}
public Expression getOperand() {
return left();
}
public Expression getResult() {
return right();
}
@Override
public String toSql() {
return "WHEN " + left().toSql() + " THEN " + right().toSql();
}
@Override
public Expression withChildren(List<Expression> children) {
public WhenClause withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2);
return new WhenClause(children.get(0), children.get(1));
}
@ -50,7 +68,6 @@ public class WhenClause extends Expression implements BinaryExpression {
return visitor.visitWhenClause(this, context);
}
@Override
public DataType getDataType() {
// when left() then right()
@ -64,6 +81,11 @@ public class WhenClause extends Expression implements BinaryExpression {
return right().nullable();
}
@Override
public List<AbstractDataType> expectedInputTypes() {
return EXPECTS_INPUT_TYPES;
}
@Override
public boolean equals(Object o) {
if (this == o) {

View File

@ -18,17 +18,30 @@
package org.apache.doris.nereids.trees.expressions.functions;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.DateTimeType;
import org.apache.doris.nereids.types.DateType;
import org.apache.doris.nereids.types.DecimalType;
import org.apache.doris.nereids.types.DoubleType;
import org.apache.doris.nereids.types.VarcharType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.NumericType;
import org.apache.doris.nereids.types.coercion.TypeCollection;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
/** avg agg function. */
public class Avg extends AggregateFunction implements UnaryExpression {
public class Avg extends AggregateFunction implements UnaryExpression, ImplicitCastInputTypes {
// used in interface expectedInputTypes to avoid new list in each time it be called
private static final List<AbstractDataType> EXPECTED_INPUT_TYPES = ImmutableList.of(
new TypeCollection(NumericType.INSTANCE, DateTimeType.INSTANCE, DateType.INSTANCE)
);
public Avg(Expression child) {
super("avg", child);
@ -36,7 +49,15 @@ public class Avg extends AggregateFunction implements UnaryExpression {
@Override
public DataType getDataType() {
return DoubleType.INSTANCE;
if (child().getDataType() instanceof DecimalType) {
return child().getDataType();
} else if (child().getDataType().isDate()) {
return DateType.INSTANCE;
} else if (child().getDataType().isDateTime()) {
return DateTimeType.INSTANCE;
} else {
return DoubleType.INSTANCE;
}
}
@Override
@ -54,4 +75,9 @@ public class Avg extends AggregateFunction implements UnaryExpression {
public DataType getIntermediateType() {
return VarcharType.createVarcharType(-1);
}
@Override
public List<AbstractDataType> expectedInputTypes() {
return EXPECTED_INPUT_TYPES;
}
}

View File

@ -18,7 +18,7 @@
package org.apache.doris.nereids.trees.expressions.functions;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.types.DataType;
import com.google.common.base.Preconditions;

View File

@ -18,19 +18,31 @@
package org.apache.doris.nereids.trees.expressions.functions;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.IntegerLiteral;
import org.apache.doris.nereids.trees.expressions.TernaryExpression;
import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral;
import org.apache.doris.nereids.trees.expressions.shape.TernaryExpression;
import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.IntegerType;
import org.apache.doris.nereids.types.StringType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.TypeCollection;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* substring function.
*/
public class Substring extends BoundFunction implements TernaryExpression {
public class Substring extends BoundFunction implements TernaryExpression, ImplicitCastInputTypes {
// used in interface expectedInputTypes to avoid new list in each time it be called
private static final List<AbstractDataType> EXPECTED_INPUT_TYPES = ImmutableList.of(
TypeCollection.CHARACTER_TYPE_COLLECTION,
IntegerType.INSTANCE,
IntegerType.INSTANCE
);
public Substring(Expression str, Expression pos, Expression len) {
super("substring", str, pos, len);
@ -51,11 +63,16 @@ public class Substring extends BoundFunction implements TernaryExpression {
}
@Override
public Expression withChildren(List<Expression> children) {
public Substring withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2 || children.size() == 3);
if (children.size() == 2) {
return new Substring(children.get(0), children.get(1));
}
return new Substring(children.get(0), children.get(1), children.get(2));
}
@Override
public List<AbstractDataType> expectedInputTypes() {
return EXPECTED_INPUT_TYPES;
}
}

View File

@ -18,19 +18,27 @@
package org.apache.doris.nereids.trees.expressions.functions;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes;
import org.apache.doris.nereids.types.BigIntType;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.DoubleType;
import org.apache.doris.nereids.types.FractionalType;
import org.apache.doris.nereids.types.IntegralType;
import org.apache.doris.nereids.types.LargeIntType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.FractionalType;
import org.apache.doris.nereids.types.coercion.IntegralType;
import org.apache.doris.nereids.types.coercion.NumericType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
/** sum agg function. */
public class Sum extends AggregateFunction implements UnaryExpression {
public class Sum extends AggregateFunction implements UnaryExpression, ImplicitCastInputTypes {
// used in interface expectedInputTypes to avoid new list in each time it be called
private static final List<AbstractDataType> EXPECTED_INPUT_TYPES = ImmutableList.of(NumericType.INSTANCE);
public Sum(Expression child) {
super("sum", child);
@ -39,7 +47,9 @@ public class Sum extends AggregateFunction implements UnaryExpression {
@Override
public DataType getDataType() {
DataType dataType = child().getDataType();
if (dataType instanceof IntegralType) {
if (dataType instanceof LargeIntType) {
return dataType;
} else if (dataType instanceof IntegralType) {
return BigIntType.INSTANCE;
} else if (dataType instanceof FractionalType) {
// TODO: precision + 10
@ -54,6 +64,11 @@ public class Sum extends AggregateFunction implements UnaryExpression {
return child().nullable();
}
@Override
public List<AbstractDataType> expectedInputTypes() {
return EXPECTED_INPUT_TYPES;
}
@Override
public Expression withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 1);

View File

@ -18,18 +18,29 @@
package org.apache.doris.nereids.trees.expressions.functions;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.DateTimeType;
import org.apache.doris.nereids.types.DateType;
import org.apache.doris.nereids.types.IntegerType;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.TypeCollection;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* year function.
*/
public class Year extends BoundFunction implements UnaryExpression {
public class Year extends BoundFunction implements UnaryExpression, ImplicitCastInputTypes {
// used in interface expectedInputTypes to avoid new list in each time it be called
private static final List<AbstractDataType> EXPECTED_INPUT_TYPES = ImmutableList.of(
new TypeCollection(DateType.INSTANCE, DateTimeType.INSTANCE)
);
public Year(Expression child) {
super("year", child);
@ -46,8 +57,13 @@ public class Year extends BoundFunction implements UnaryExpression {
}
@Override
public Expression withChildren(List<Expression> children) {
public Year withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 1);
return new Year(children.get(0));
}
@Override
public List<AbstractDataType> expectedInputTypes() {
return EXPECTED_INPUT_TYPES;
}
}

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.IntLiteral;
import org.apache.doris.analysis.LiteralExpr;

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.BoolLiteral;
import org.apache.doris.analysis.LiteralExpr;

View File

@ -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.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.analysis.StringLiteral;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.CharType;
import com.google.common.base.Preconditions;
import java.util.Objects;
/**
* char type literal
*/
public class CharLiteral extends Literal {
private final String value;
public CharLiteral(String value, int len) {
super(CharType.createCharType(len));
this.value = Objects.requireNonNull(value);
Preconditions.checkArgument(value.length() <= len);
}
@Override
public String getValue() {
return value;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitCharLiteral(this, context);
}
@Override
public LiteralExpr toLegacyLiteral() {
return new StringLiteral(value);
}
}

View File

@ -15,10 +15,12 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.DateTimeType;
import org.apache.doris.nereids.types.DateType;
@ -131,6 +133,11 @@ public class DateLiteral extends Literal {
return new DateLiteral(this, type);
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitDateLiteral(this, context);
}
@Override
public Long getValue() {
return (year * 10000 + month * 100 + day) * 1000000L;

View File

@ -15,10 +15,12 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.DateTimeType;
import org.apache.doris.nereids.types.DateType;
@ -123,6 +125,11 @@ public class DateTimeLiteral extends DateLiteral {
return this;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitDateTimeLiteral(this, context);
}
@Override
public Long getValue() {
return (year * 10000 + month * 100 + day) * 1000000L + hour * 10000 + minute * 100 + second;

View File

@ -0,0 +1,53 @@
// 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.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DecimalType;
import java.math.BigDecimal;
import java.util.Objects;
/**
* decimal type literal
*/
public class DecimalLiteral extends Literal {
private final BigDecimal value;
public DecimalLiteral(BigDecimal value) {
super(DecimalType.createDecimalType(value));
this.value = Objects.requireNonNull(value);
}
@Override
public BigDecimal getValue() {
return value;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitDecimalLiteral(this, context);
}
@Override
public LiteralExpr toLegacyLiteral() {
return new org.apache.doris.analysis.DecimalLiteral(value);
}
}

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.FloatLiteral;
import org.apache.doris.analysis.LiteralExpr;

View File

@ -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.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.FloatType;
/**
* float type literal
*/
public class FloatLiteral extends Literal {
private final float value;
public FloatLiteral(float value) {
super(FloatType.INSTANCE);
this.value = value;
}
@Override
public Float getValue() {
return value;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitFloatLiteral(this, context);
}
@Override
public LiteralExpr toLegacyLiteral() {
return new org.apache.doris.analysis.FloatLiteral((double) value, Type.FLOAT);
}
}

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.IntLiteral;
import org.apache.doris.analysis.LiteralExpr;

View File

@ -15,7 +15,12 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.DateType;
/**
* Interval for timestamp calculation.
@ -29,6 +34,11 @@ public class IntervalLiteral extends Expression {
this.timeUnit = TimeUnit.valueOf(desc.toUpperCase());
}
@Override
public DataType getDataType() throws UnboundException {
return DateType.INSTANCE;
}
public Expression value() {
return value;
}

View File

@ -0,0 +1,59 @@
// 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.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.LargeIntType;
import java.math.BigInteger;
import java.util.Objects;
/**
* large int type literal
*/
public class LargeIntLiteral extends Literal {
private final BigInteger value;
public LargeIntLiteral(BigInteger value) {
super(LargeIntType.INSTANCE);
this.value = Objects.requireNonNull(value);
}
@Override
public BigInteger getValue() {
return value;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitLargeIntLiteral(this, context);
}
@Override
public LiteralExpr toLegacyLiteral() {
try {
return new org.apache.doris.analysis.LargeIntLiteral(value.toString());
} catch (AnalysisException e) {
throw new org.apache.doris.nereids.exceptions.AnalysisException(
"Can not convert to legacy literal: " + value, e);
}
}
}

View File

@ -15,10 +15,12 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.nereids.exceptions.UnboundException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.shape.LeafExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
@ -26,6 +26,8 @@ import org.apache.doris.nereids.types.NullType;
*/
public class NullLiteral extends Literal {
public static final NullLiteral INSTANCE = new NullLiteral();
public NullLiteral() {
super(NullType.INSTANCE);
}

View File

@ -0,0 +1,58 @@
// 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.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.IntLiteral;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.catalog.Type;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.SmallIntType;
/**
* small int type literal
*/
public class SmallIntLiteral extends Literal {
private final short value;
public SmallIntLiteral(short value) {
super(SmallIntType.INSTANCE);
this.value = value;
}
@Override
public Short getValue() {
return value;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitSmallIntLiteral(this, context);
}
@Override
public LiteralExpr toLegacyLiteral() {
try {
return new IntLiteral(value, Type.SMALLINT);
} catch (AnalysisException e) {
throw new org.apache.doris.nereids.exceptions.AnalysisException(
"Can not convert to legacy literal: " + value, e);
}
}
}

View File

@ -15,10 +15,11 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.StringType;
@ -80,7 +81,7 @@ public class StringLiteral extends Literal {
}
@Override
public String toSql() {
public String toString() {
return "'" + value + "'";
}
}

View File

@ -0,0 +1,58 @@
// 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.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.IntLiteral;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.catalog.Type;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.TinyIntType;
/**
* tiny int type literal
*/
public class TinyIntLiteral extends Literal {
private final byte value;
public TinyIntLiteral(byte value) {
super(TinyIntType.INSTANCE);
this.value = value;
}
@Override
public Byte getValue() {
return value;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitTinyIntLiteral(this, context);
}
@Override
public LiteralExpr toLegacyLiteral() {
try {
return new IntLiteral(value, Type.TINYINT);
} catch (AnalysisException e) {
throw new org.apache.doris.nereids.exceptions.AnalysisException(
"Can not convert to legacy literal: " + value, e);
}
}
}

View File

@ -0,0 +1,61 @@
// 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.doris.nereids.trees.expressions.literal;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.analysis.StringLiteral;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.VarcharType;
import com.google.common.base.Preconditions;
import java.util.Objects;
/**
* varchar type literal
*/
public class VarcharLiteral extends Literal {
private final String value;
public VarcharLiteral(String value) {
super(VarcharType.SYSTEM_DEFAULT);
this.value = Objects.requireNonNull(value);
}
public VarcharLiteral(String value, int len) {
super(VarcharType.createVarcharType(len));
this.value = Objects.requireNonNull(value);
Preconditions.checkArgument(value.length() <= len);
}
@Override
public String getValue() {
return value;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitVarcharLiteral(this, context);
}
@Override
public LiteralExpr toLegacyLiteral() {
return new StringLiteral(value);
}
}

View File

@ -15,9 +15,10 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.shape;
import org.apache.doris.nereids.trees.BinaryNode;
import org.apache.doris.nereids.trees.expressions.Expression;
/**
* Interface for all expression that have two children.

View File

@ -15,9 +15,10 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.shape;
import org.apache.doris.nereids.trees.LeafNode;
import org.apache.doris.nereids.trees.expressions.Expression;
/**
* Interface for all expression that have no child.

View File

@ -15,9 +15,10 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.shape;
import org.apache.doris.nereids.trees.TernaryNode;
import org.apache.doris.nereids.trees.expressions.Expression;
/**
* Interface for all expression that have three children.

View File

@ -15,9 +15,10 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
package org.apache.doris.nereids.trees.expressions.shape;
import org.apache.doris.nereids.trees.UnaryNode;
import org.apache.doris.nereids.trees.expressions.Expression;
/**
* Abstract class for all expression that have one child.

View File

@ -0,0 +1,37 @@
// 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.doris.nereids.trees.expressions.typecoercion;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import java.util.List;
/**
* An interface to define the expected input types of an expression.
*
* This interface is typically used by operator expressions
* (e.g. {@link org.apache.doris.nereids.trees.expressions.Add})
* to define expected input types without any implicit casting.
*
* Most function expressions (e.g. {@link org.apache.doris.nereids.trees.expressions.functions.Substring}
* should extend {@link ImplicitCastInputTypes}) instead.
*/
public interface ExpectsInputTypes {
List<AbstractDataType> expectedInputTypes();
}

View File

@ -15,10 +15,10 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.types;
package org.apache.doris.nereids.trees.expressions.typecoercion;
/**
* Abstract for all fractional type in Nereids.
* an interface used to identify whether the current expression can be implicitly cast input types.
*/
public abstract class FractionalType extends NumericType {
public interface ImplicitCastInputTypes extends ExpectsInputTypes {
}

View File

@ -0,0 +1,49 @@
// 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.doris.nereids.trees.expressions.typecoercion;
import java.util.Optional;
/**
* Save type check result.
*/
public class TypeCheckResult {
public static TypeCheckResult SUCCESS = new TypeCheckResult(true, null);
final boolean success;
final Optional<String> message;
public TypeCheckResult(boolean success, String message) {
this.success = success;
this.message = Optional.ofNullable(message);
}
public boolean success() {
return success;
}
public boolean failed() {
return !success;
}
public String getMessage() {
return message.orElseGet(String::new);
}
}

View File

@ -21,6 +21,7 @@ import org.apache.doris.nereids.CascadesContext;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.memo.Memo;
import org.apache.doris.nereids.rules.analysis.Scope;
import org.apache.doris.nereids.trees.expressions.Exists;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.InSubquery;
import org.apache.doris.nereids.trees.expressions.ListQuery;
@ -43,8 +44,8 @@ public class DefaultSubExprRewriter<C> extends DefaultExpressionRewriter<C> {
}
@Override
public Expression visitSubqueryExpr(SubqueryExpr expr, C context) {
return new SubqueryExpr(analyzeSubquery(expr));
public Expression visitExistsSubquery(Exists exists, C context) {
return new Exists(analyzeSubquery(exists));
}
@Override

View File

@ -24,18 +24,14 @@ import org.apache.doris.nereids.analyzer.UnboundStar;
import org.apache.doris.nereids.trees.expressions.Add;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.And;
import org.apache.doris.nereids.trees.expressions.Arithmetic;
import org.apache.doris.nereids.trees.expressions.Between;
import org.apache.doris.nereids.trees.expressions.BigIntLiteral;
import org.apache.doris.nereids.trees.expressions.BooleanLiteral;
import org.apache.doris.nereids.trees.expressions.BinaryArithmetic;
import org.apache.doris.nereids.trees.expressions.BinaryOperator;
import org.apache.doris.nereids.trees.expressions.CaseWhen;
import org.apache.doris.nereids.trees.expressions.Cast;
import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
import org.apache.doris.nereids.trees.expressions.CompoundPredicate;
import org.apache.doris.nereids.trees.expressions.DateLiteral;
import org.apache.doris.nereids.trees.expressions.DateTimeLiteral;
import org.apache.doris.nereids.trees.expressions.Divide;
import org.apache.doris.nereids.trees.expressions.DoubleLiteral;
import org.apache.doris.nereids.trees.expressions.EqualTo;
import org.apache.doris.nereids.trees.expressions.Exists;
import org.apache.doris.nereids.trees.expressions.Expression;
@ -43,24 +39,20 @@ import org.apache.doris.nereids.trees.expressions.GreaterThan;
import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
import org.apache.doris.nereids.trees.expressions.InPredicate;
import org.apache.doris.nereids.trees.expressions.InSubquery;
import org.apache.doris.nereids.trees.expressions.IntegerLiteral;
import org.apache.doris.nereids.trees.expressions.LessThan;
import org.apache.doris.nereids.trees.expressions.LessThanEqual;
import org.apache.doris.nereids.trees.expressions.Like;
import org.apache.doris.nereids.trees.expressions.ListQuery;
import org.apache.doris.nereids.trees.expressions.Literal;
import org.apache.doris.nereids.trees.expressions.Mod;
import org.apache.doris.nereids.trees.expressions.Multiply;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Not;
import org.apache.doris.nereids.trees.expressions.NullLiteral;
import org.apache.doris.nereids.trees.expressions.NullSafeEqual;
import org.apache.doris.nereids.trees.expressions.Or;
import org.apache.doris.nereids.trees.expressions.Regexp;
import org.apache.doris.nereids.trees.expressions.ScalarSubquery;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.expressions.StringLiteral;
import org.apache.doris.nereids.trees.expressions.StringRegexPredicate;
import org.apache.doris.nereids.trees.expressions.SubqueryExpr;
import org.apache.doris.nereids.trees.expressions.Subtract;
@ -68,6 +60,22 @@ import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
import org.apache.doris.nereids.trees.expressions.WhenClause;
import org.apache.doris.nereids.trees.expressions.functions.AggregateFunction;
import org.apache.doris.nereids.trees.expressions.functions.BoundFunction;
import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral;
import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral;
import org.apache.doris.nereids.trees.expressions.literal.CharLiteral;
import org.apache.doris.nereids.trees.expressions.literal.DateLiteral;
import org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral;
import org.apache.doris.nereids.trees.expressions.literal.DecimalLiteral;
import org.apache.doris.nereids.trees.expressions.literal.DoubleLiteral;
import org.apache.doris.nereids.trees.expressions.literal.FloatLiteral;
import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral;
import org.apache.doris.nereids.trees.expressions.literal.LargeIntLiteral;
import org.apache.doris.nereids.trees.expressions.literal.Literal;
import org.apache.doris.nereids.trees.expressions.literal.NullLiteral;
import org.apache.doris.nereids.trees.expressions.literal.SmallIntLiteral;
import org.apache.doris.nereids.trees.expressions.literal.StringLiteral;
import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral;
import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral;
/**
* Use the visitor to visit expression and forward to unified method(visitExpression).
@ -80,8 +88,12 @@ public abstract class ExpressionVisitor<R, C> {
return visitNamedExpression(alias, context);
}
public R visitBinaryOperator(BinaryOperator binaryOperator, C context) {
return visit(binaryOperator, context);
}
public R visitComparisonPredicate(ComparisonPredicate cp, C context) {
return visit(cp, context);
return visitBinaryOperator(cp, context);
}
public R visitEqualTo(EqualTo equalTo, C context) {
@ -128,14 +140,34 @@ public abstract class ExpressionVisitor<R, C> {
return visit(literal, context);
}
public R visitNullLiteral(NullLiteral nullLiteral, C context) {
return visitLiteral(nullLiteral, context);
}
public R visitBooleanLiteral(BooleanLiteral booleanLiteral, C context) {
return visitLiteral(booleanLiteral, context);
}
public R visitCharLiteral(CharLiteral charLiteral, C context) {
return visitLiteral(charLiteral, context);
}
public R visitVarcharLiteral(VarcharLiteral varcharLiteral, C context) {
return visitLiteral(varcharLiteral, context);
}
public R visitStringLiteral(StringLiteral stringLiteral, C context) {
return visitLiteral(stringLiteral, context);
}
public R visitTinyIntLiteral(TinyIntLiteral tinyIntLiteral, C context) {
return visitLiteral(tinyIntLiteral, context);
}
public R visitSmallIntLiteral(SmallIntLiteral smallIntLiteral, C context) {
return visitLiteral(smallIntLiteral, context);
}
public R visitIntegerLiteral(IntegerLiteral integerLiteral, C context) {
return visitLiteral(integerLiteral, context);
}
@ -144,8 +176,16 @@ public abstract class ExpressionVisitor<R, C> {
return visitLiteral(bigIntLiteral, context);
}
public R visitNullLiteral(NullLiteral nullLiteral, C context) {
return visitLiteral(nullLiteral, context);
public R visitLargeIntLiteral(LargeIntLiteral largeIntLiteral, C context) {
return visitLiteral(largeIntLiteral, context);
}
public R visitDecimalLiteral(DecimalLiteral decimalLiteral, C context) {
return visitLiteral(decimalLiteral, context);
}
public R visitFloatLiteral(FloatLiteral floatLiteral, C context) {
return visitLiteral(floatLiteral, context);
}
public R visitDoubleLiteral(DoubleLiteral doubleLiteral, C context) {
@ -165,7 +205,7 @@ public abstract class ExpressionVisitor<R, C> {
}
public R visitCompoundPredicate(CompoundPredicate compoundPredicate, C context) {
return visit(compoundPredicate, context);
return visitBinaryOperator(compoundPredicate, context);
}
public R visitAnd(And and, C context) {
@ -200,28 +240,28 @@ public abstract class ExpressionVisitor<R, C> {
return visitBoundFunction(aggregateFunction, context);
}
public R visitArithmetic(Arithmetic arithmetic, C context) {
return visit(arithmetic, context);
public R visitBinaryArithmetic(BinaryArithmetic binaryArithmetic, C context) {
return visitBinaryOperator(binaryArithmetic, context);
}
public R visitAdd(Add add, C context) {
return visitArithmetic(add, context);
return visitBinaryArithmetic(add, context);
}
public R visitSubtract(Subtract subtract, C context) {
return visitArithmetic(subtract, context);
return visitBinaryArithmetic(subtract, context);
}
public R visitMultiply(Multiply multiply, C context) {
return visitArithmetic(multiply, context);
return visitBinaryArithmetic(multiply, context);
}
public R visitDivide(Divide divide, C context) {
return visitArithmetic(divide, context);
return visitBinaryArithmetic(divide, context);
}
public R visitMod(Mod mod, C context) {
return visitArithmetic(mod, context);
return visitBinaryArithmetic(mod, context);
}
public R visitWhenClause(WhenClause whenClause, C context) {

View File

@ -20,8 +20,8 @@ package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.IntegerLiteral;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.algebra.Limit;

View File

@ -20,7 +20,7 @@ package org.apache.doris.nereids.trees.plans.physical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.IntegerLiteral;
import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.algebra.Limit;

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.IntegralType;
/**
* BigInt data type in Nereids.
@ -29,5 +30,19 @@ public class BigIntType extends IntegralType {
public Type toCatalogDataType() {
return Type.BIGINT;
}
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof BigIntType;
}
@Override
public String simpleString() {
return "bigint";
}
@Override
public DataType defaultConcreteType() {
return this;
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.PrimitiveType;
/**
* Boolean type in Nereids.
@ -29,4 +30,9 @@ public class BooleanType extends PrimitiveType {
public Type toCatalogDataType() {
return Type.BOOLEAN;
}
@Override
public String simpleString() {
return "boolean";
}
}

View File

@ -0,0 +1,79 @@
// 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.doris.nereids.types;
import org.apache.doris.catalog.ScalarType;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.CharacterType;
import java.util.Objects;
/**
* Char type in Nereids.
*/
public class CharType extends CharacterType {
public static final CharType SYSTEM_DEFAULT = new CharType(-1);
public CharType(int len) {
super(len);
}
public static CharType createCharType(int len) {
return new CharType(len);
}
@Override
public Type toCatalogDataType() {
return ScalarType.createChar(len);
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof CharType;
}
@Override
public String simpleString() {
return "char";
}
@Override
public DataType defaultConcreteType() {
return this;
}
@Override
public boolean equals(Object o) {
if (!super.equals(o)) {
return false;
}
CharType charType = (CharType) o;
return len == charType.len;
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), len);
}
@Override
public String toSql() {
return "CHAR(" + len + ")";
}
}

View File

@ -24,11 +24,31 @@ import org.apache.doris.catalog.ScalarType;
import org.apache.doris.catalog.StructType;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.types.coercion.AbstractDataType;
import org.apache.doris.nereids.types.coercion.CharacterType;
import org.apache.doris.nereids.types.coercion.NumericType;
import org.apache.doris.nereids.types.coercion.PrimitiveType;
import com.google.common.collect.ImmutableMap;
import java.util.Locale;
import java.util.Map;
import java.util.function.Supplier;
/**
* Abstract class for all data type in Nereids.
*/
public abstract class DataType {
public abstract class DataType implements AbstractDataType {
// use class and supplier here to avoid class load deadlock.
private static final Map<Class<? extends NumericType>, Supplier<DataType>> PROMOTION_MAP
= ImmutableMap.<Class<? extends NumericType>, Supplier<DataType>>builder()
.put(TinyIntType.class, () -> SmallIntType.INSTANCE)
.put(SmallIntType.class, () -> IntegerType.INSTANCE)
.put(IntegerType.class, () -> BigIntType.INSTANCE)
.put(FloatType.class, () -> DoubleType.INSTANCE)
.build();
/**
* Convert data type in Doris catalog to data type in Nereids.
* TODO: throw exception when cannot convert catalog type to Nereids type
@ -42,12 +62,22 @@ public abstract class DataType {
switch (scalarType.getPrimitiveType()) {
case BOOLEAN:
return BooleanType.INSTANCE;
case TINYINT:
return TinyIntType.INSTANCE;
case SMALLINT:
return SmallIntType.INSTANCE;
case INT:
return IntegerType.INSTANCE;
case BIGINT:
return BigIntType.INSTANCE;
case LARGEINT:
return LargeIntType.INSTANCE;
case FLOAT:
return FloatType.INSTANCE;
case DOUBLE:
return DoubleType.INSTANCE;
case CHAR:
return CharType.createCharType(scalarType.getLength());
case VARCHAR:
return VarcharType.createVarcharType(scalarType.getLength());
case STRING:
@ -85,6 +115,7 @@ public abstract class DataType {
*/
public static DataType convertFromString(String type) {
// TODO: use a better way to resolve types
// TODO: support varchar, char, decimal
switch (type.toLowerCase()) {
case "bool":
case "boolean":
@ -108,6 +139,39 @@ public abstract class DataType {
public abstract Type toCatalogDataType();
public abstract String toSql();
@Override
public String toString() {
return toSql();
}
public String typeName() {
return this.getClass().getSimpleName().replace("Type", "").toLowerCase(Locale.ROOT);
}
@Override
public DataType defaultConcreteType() {
return this;
}
@Override
public boolean acceptsType(DataType other) {
return sameType(other);
}
/**
* this and other is same type.
*/
private boolean sameType(DataType other) {
return this.equals(other);
}
@Override
public String simpleString() {
return typeName();
}
@Override
public boolean equals(Object o) {
if (this == o) {
@ -139,4 +203,28 @@ public abstract class DataType {
public boolean isDateType() {
return isDate() || isDateTime();
}
public boolean isNullType() {
return this instanceof NullType;
}
public boolean isNumericType() {
return this instanceof NumericType;
}
public boolean isStringType() {
return this instanceof CharacterType;
}
public boolean isPrimitive() {
return this instanceof PrimitiveType;
}
public DataType promotion() {
if (PROMOTION_MAP.containsKey(this.getClass())) {
return PROMOTION_MAP.get(this.getClass()).get();
} else {
return this;
}
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.PrimitiveType;
/**
* Datetime type in Nereids.

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.PrimitiveType;
/**
* Date type in Nereids.

View File

@ -18,22 +18,80 @@
package org.apache.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.FractionalType;
import org.apache.doris.nereids.types.coercion.IntegralType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import java.math.BigDecimal;
import java.util.Map;
import java.util.Objects;
/**
* Decimal type in Nereids.
*/
public class DecimalType extends FractionalType {
private int precision;
private int scale;
public static int MAX_PRECISION = 38;
public static int MAX_SCALE = 38;
public static DecimalType SYSTEM_DEFAULT = new DecimalType(MAX_PRECISION, 18);
private static final DecimalType BOOLEAN_DECIMAL = new DecimalType(1, 0);
private static final DecimalType TINYINT_DECIMAL = new DecimalType(3, 0);
private static final DecimalType SMALLINT_DECIMAL = new DecimalType(5, 0);
private static final DecimalType INTEGER_DECIMAL = new DecimalType(10, 0);
private static final DecimalType BIGINT_DECIMAL = new DecimalType(20, 0);
private static final DecimalType LARGEINT_DECIMAL = new DecimalType(38, 0);
private static final DecimalType FLOAT_DECIMAL = new DecimalType(14, 7);
private static final DecimalType DOUBLE_DECIMAL = new DecimalType(30, 15);
private static final Map<DataType, DecimalType> FOR_TYPE_MAP = ImmutableMap.<DataType, DecimalType>builder()
.put(TinyIntType.INSTANCE, TINYINT_DECIMAL)
.put(SmallIntType.INSTANCE, SMALLINT_DECIMAL)
.put(IntegerType.INSTANCE, INTEGER_DECIMAL)
.put(BigIntType.INSTANCE, BIGINT_DECIMAL)
.put(LargeIntType.INSTANCE, LARGEINT_DECIMAL)
.put(FloatType.INSTANCE, FLOAT_DECIMAL)
.put(DoubleType.INSTANCE, DOUBLE_DECIMAL)
.build();
private final int precision;
private final int scale;
public DecimalType(int precision, int scale) {
Preconditions.checkArgument(precision >= scale);
Preconditions.checkArgument(precision > 0 && precision <= MAX_PRECISION);
Preconditions.checkArgument(scale >= 0 && scale <= MAX_SCALE);
this.precision = precision;
this.scale = scale;
}
public static DecimalType createDecimalType(int precision, int scale) {
return new DecimalType(precision, scale);
return new DecimalType(Math.min(precision, MAX_PRECISION), Math.min(scale, MAX_SCALE));
}
public static DecimalType createDecimalType(BigDecimal bigDecimal) {
int precision = org.apache.doris.analysis.DecimalLiteral.getBigDecimalPrecision(bigDecimal);
int scale = org.apache.doris.analysis.DecimalLiteral.getBigDecimalScale(bigDecimal);
return createDecimalType(precision, scale);
}
public static DecimalType forType(DataType dataType) {
if (FOR_TYPE_MAP.containsKey(dataType)) {
return FOR_TYPE_MAP.get(dataType);
}
throw new RuntimeException("Could not create decimal for type " + dataType);
}
public static DecimalType widerDecimalType(DecimalType left, DecimalType right) {
return widerDecimalType(left.getPrecision(), right.getPrecision(), left.getScale(), right.getScale());
}
private static DecimalType widerDecimalType(int leftPrecision, int rightPrecision, int leftScale, int rightScale) {
int scale = Math.max(leftScale, rightScale);
int range = Math.max(leftPrecision - leftScale, rightPrecision - rightScale);
return DecimalType.createDecimalType(range + scale, scale);
}
@Override
@ -48,5 +106,59 @@ public class DecimalType extends FractionalType {
public int getScale() {
return scale;
}
public boolean isWiderThan(DataType other) {
return isWiderThanInternal(other);
}
private boolean isWiderThanInternal(DataType other) {
if (other instanceof DecimalType) {
DecimalType dt = (DecimalType) other;
return this.precision - this.scale >= dt.precision - dt.scale && this.scale >= dt.scale;
} else if (other instanceof IntegralType) {
return isWiderThanInternal(forType(other));
}
return false;
}
@Override
public DataType defaultConcreteType() {
return SYSTEM_DEFAULT;
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof DecimalType;
}
@Override
public String simpleString() {
return "decimal";
}
@Override
public String toSql() {
return "DECIMAL(" + precision + ", " + scale + ")";
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
DecimalType that = (DecimalType) o;
return precision == that.precision && scale == that.scale;
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), precision, scale);
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.FractionalType;
/**
* Double data type in Nereids.
@ -29,4 +30,24 @@ public class DoubleType extends FractionalType {
public Type toCatalogDataType() {
return Type.DOUBLE;
}
@Override
public boolean equals(Object o) {
return o instanceof DoubleType;
}
@Override
public String simpleString() {
return "double";
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof DoubleType;
}
@Override
public DataType defaultConcreteType() {
return this;
}
}

View File

@ -0,0 +1,53 @@
// 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.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.FractionalType;
/**
* Float type in Nereids.
*/
public class FloatType extends FractionalType {
public static FloatType INSTANCE = new FloatType();
@Override
public Type toCatalogDataType() {
return Type.FLOAT;
}
@Override
public boolean equals(Object o) {
return o instanceof FloatType;
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof FloatType;
}
@Override
public DataType defaultConcreteType() {
return this;
}
@Override
public String simpleString() {
return "float";
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.IntegralType;
/**
* Integer data type in Nereids.
@ -34,4 +35,19 @@ public class IntegerType extends IntegralType {
public boolean equals(Object o) {
return o instanceof IntegerType;
}
@Override
public String simpleString() {
return "int";
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof IntegerType;
}
@Override
public DataType defaultConcreteType() {
return this;
}
}

View File

@ -0,0 +1,53 @@
// 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.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.IntegralType;
/**
* LargeInt type in Nereids.
*/
public class LargeIntType extends IntegralType {
public static LargeIntType INSTANCE = new LargeIntType();
@Override
public Type toCatalogDataType() {
return Type.LARGEINT;
}
@Override
public boolean equals(Object o) {
return o instanceof LargeIntType;
}
@Override
public String simpleString() {
return "largeint";
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof LargeIntType;
}
@Override
public DataType defaultConcreteType() {
return this;
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.PrimitiveType;
/**
* Null data type in Nereids.

View File

@ -0,0 +1,53 @@
// 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.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.IntegralType;
/**
* SmallInt type in Nereids.
*/
public class SmallIntType extends IntegralType {
public static SmallIntType INSTANCE = new SmallIntType();
@Override
public Type toCatalogDataType() {
return Type.SMALLINT;
}
@Override
public boolean equals(Object o) {
return o instanceof SmallIntType;
}
@Override
public String simpleString() {
return "smallint";
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof SmallIntType;
}
@Override
public DataType defaultConcreteType() {
return this;
}
}

View File

@ -18,15 +18,36 @@
package org.apache.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.CharacterType;
/**
* String data type in Nereids.
*/
public class StringType extends DataType {
public class StringType extends CharacterType {
public static StringType INSTANCE = new StringType();
public StringType() {
super(-1);
}
@Override
public Type toCatalogDataType() {
return Type.STRING;
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof StringType;
}
@Override
public String simpleString() {
return "string";
}
@Override
public DataType defaultConcreteType() {
return this;
}
}

View File

@ -0,0 +1,53 @@
// 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.doris.nereids.types;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.IntegralType;
/**
* TinyInt type in Nereids.
*/
public class TinyIntType extends IntegralType {
public static TinyIntType INSTANCE = new TinyIntType();
@Override
public Type toCatalogDataType() {
return Type.TINYINT;
}
@Override
public boolean equals(Object o) {
return o instanceof TinyIntType;
}
@Override
public String simpleString() {
return "tinyint";
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof TinyIntType;
}
@Override
public DataType defaultConcreteType() {
return this;
}
}

View File

@ -19,17 +19,19 @@ package org.apache.doris.nereids.types;
import org.apache.doris.catalog.ScalarType;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.coercion.CharacterType;
import java.util.Objects;
/**
* Varchar type in Nereids.
*/
public class VarcharType extends PrimitiveType {
private final int len;
public class VarcharType extends CharacterType {
public static VarcharType SYSTEM_DEFAULT = new VarcharType(-1);
public VarcharType(int len) {
this.len = len;
super(len);
}
public static VarcharType createVarcharType(int len) {
@ -41,6 +43,26 @@ public class VarcharType extends PrimitiveType {
return ScalarType.createVarcharType(len);
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof VarcharType;
}
@Override
public String simpleString() {
return "varchar";
}
@Override
public DataType defaultConcreteType() {
return this;
}
@Override
public String toSql() {
return "VARCHAR(" + len + ")";
}
@Override
public boolean equals(Object o) {
if (!super.equals(o)) {

View File

@ -15,10 +15,27 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.types;
package org.apache.doris.nereids.types.coercion;
import org.apache.doris.nereids.types.DataType;
/**
* Abstract class for all numeric type in Nereids.
* Represent a set of equality concrete data type.
*/
public abstract class NumericType extends PrimitiveType {
public interface AbstractDataType {
/**
* the default concrete type cast to when do implicit cast
*/
DataType defaultConcreteType();
/**
* This AbstractDataType could accept other without implicit cast
*/
boolean acceptsType(DataType other);
/**
* simple string used to print error message
*/
String simpleString();
}

View File

@ -0,0 +1,43 @@
// 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.doris.nereids.types.coercion;
import org.apache.doris.nereids.types.DataType;
/**
* Represent any datatype in type coercion.
*/
public class AnyDataType implements AbstractDataType {
public static final AnyDataType INSTANCE = new AnyDataType();
@Override
public DataType defaultConcreteType() {
throw new RuntimeException("Unsupported operation.");
}
@Override
public boolean acceptsType(DataType other) {
return true;
}
@Override
public String simpleString() {
return "any";
}
}

View File

@ -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.doris.nereids.types.coercion;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.StringType;
/**
* Abstract type for all characters type in Nereids.
*/
public class CharacterType extends PrimitiveType {
public static final CharacterType INSTANCE = new CharacterType(-1);
protected final int len;
public CharacterType(int len) {
this.len = len;
}
public int getLen() {
return len;
}
@Override
public Type toCatalogDataType() {
throw new RuntimeException("CharacterType is only used for implicit cast.");
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof CharacterType;
}
@Override
public DataType defaultConcreteType() {
return StringType.INSTANCE;
}
}

View File

@ -0,0 +1,43 @@
// 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.doris.nereids.types.coercion;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.DoubleType;
/**
* Abstract for all fractional type in Nereids.
*/
public class FractionalType extends NumericType {
public static final FractionalType INSTANCE = new FractionalType();
@Override
public DataType defaultConcreteType() {
return DoubleType.INSTANCE;
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof FractionalType;
}
@Override
public String simpleString() {
return "fractional";
}
}

View File

@ -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.doris.nereids.types.coercion;
import org.apache.doris.nereids.types.BigIntType;
import org.apache.doris.nereids.types.DataType;
/**
* Abstract class for all integral data type in Nereids.
*/
public class IntegralType extends NumericType {
public static final IntegralType INSTANCE = new IntegralType();
@Override
public DataType defaultConcreteType() {
return BigIntType.INSTANCE;
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof IntegralType;
}
@Override
public String simpleString() {
return "integral";
}
}

View File

@ -0,0 +1,50 @@
// 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.doris.nereids.types.coercion;
import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.nereids.types.DoubleType;
/**
* Abstract class for all numeric type in Nereids.
*/
public class NumericType extends PrimitiveType {
public static final NumericType INSTANCE = new NumericType();
@Override
public Type toCatalogDataType() {
throw new RuntimeException("NumericType is only used for implicit cast.");
}
@Override
public DataType defaultConcreteType() {
return DoubleType.INSTANCE;
}
@Override
public boolean acceptsType(DataType other) {
return other instanceof NumericType;
}
@Override
public String simpleString() {
return "numeric";
}
}

View File

@ -15,10 +15,18 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.types;
package org.apache.doris.nereids.types.coercion;
import org.apache.doris.nereids.types.DataType;
import java.util.Locale;
/**
* Primitive data type in Nereids.
*/
public abstract class PrimitiveType extends DataType {
@Override
public String toSql() {
return simpleString().toUpperCase(Locale.ROOT);
}
}

Some files were not shown because too many files have changed in this diff Show More