This is an automated email from the ASF dual-hosted git repository. huajianlan pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/master by this push: new 61fb39a5ca [Enhancement](Nereids) add some basic data structure definitions. (#10981) 61fb39a5ca is described below commit 61fb39a5ca5a52ec6686ef9443c8dd4032f81d52 Author: shee <13843187+qz...@users.noreply.github.com> AuthorDate: Tue Jul 26 17:51:59 2022 +0800 [Enhancement](Nereids) add some basic data structure definitions. (#10981) add decimal 、date、datetime type add DateLiteral add Interval for timestamp calculation --- .../antlr4/org/apache/doris/nereids/DorisLexer.g4 | 1 + .../antlr4/org/apache/doris/nereids/DorisParser.g4 | 14 +- .../doris/analysis/TimestampArithmeticExpr.java | 12 ++ .../apache/doris/nereids/analyzer/UnboundSlot.java | 5 + .../glue/translator/ExpressionTranslator.java | 23 +++ .../doris/nereids/parser/LogicalPlanBuilder.java | 57 +++++++- .../org/apache/doris/nereids/rules/RuleType.java | 2 + .../doris/nereids/rules/analysis/BindFunction.java | 33 +++++ .../rewrite/logical/PushPredicateThroughJoin.java | 1 + .../nereids/trees/expressions/DateLiteral.java | 161 +++++++++++++++++++++ .../nereids/trees/expressions/DateTimeLiteral.java | 151 +++++++++++++++++++ .../nereids/trees/expressions/Expression.java | 10 +- .../nereids/trees/expressions/IntervalLiteral.java | 64 ++++++++ .../nereids/trees/expressions/NamedExpression.java | 5 - .../nereids/trees/expressions/StringLiteral.java | 26 ++++ .../trees/expressions/TimestampArithmetic.java | 132 +++++++++++++++++ .../expressions/visitor/ExpressionVisitor.java | 15 ++ .../apache/doris/nereids/types/BooleanType.java | 2 +- .../org/apache/doris/nereids/types/DataType.java | 22 +++ .../types/{NullType.java => DateTimeType.java} | 14 +- .../nereids/types/{NullType.java => DateType.java} | 10 +- .../types/{NullType.java => DecimalType.java} | 28 +++- .../apache/doris/nereids/types/IntegerType.java | 5 + .../org/apache/doris/nereids/types/NullType.java | 2 +- .../apache/doris/nereids/types/VarcharType.java | 2 +- .../org/apache/doris/nereids/util/DateUtils.java | 146 +++++++++++++++++++ .../expression/rewrite/ExpressionRewriteTest.java | 13 +- .../org/apache/doris/nereids/ssb/SSBUtils.java | 1 + .../trees/expressions/ExpressionParserTest.java | 30 ++++ 29 files changed, 957 insertions(+), 30 deletions(-) diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 index 1f44cc1686..ff20819e56 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 @@ -360,6 +360,7 @@ VALUES: 'VALUES'; VERSION: 'VERSION'; VIEW: 'VIEW'; VIEWS: 'VIEWS'; +WEEK: 'WEEK'; WHEN: 'WHEN'; WHERE: 'WHERE'; WINDOW: 'WINDOW'; diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index c3086488cc..18ad0089af 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -199,8 +199,8 @@ primaryExpression | constant #constantDefault | ASTERISK #star | qualifiedName DOT ASTERISK #star - | identifier LEFT_PAREN DISTINCT? arguments+=expression - (COMMA arguments+=expression)* RIGHT_PAREN #functionCall + | identifier LEFT_PAREN (DISTINCT? arguments+=expression + (COMMA arguments+=expression)*)? RIGHT_PAREN #functionCall | LEFT_PAREN query RIGHT_PAREN #subqueryExpression | identifier #columnReference | base=primaryExpression DOT fieldName=identifier #dereference @@ -213,6 +213,8 @@ qualifiedName constant : NULL #nullLiteral + | interval #intervalLiteral + | identifier STRING #typeConstructor | number #numericLiteral | booleanValue #booleanLiteral | STRING+ #stringLiteral @@ -230,6 +232,14 @@ whenClause : WHEN condition=expression THEN result=expression ; +interval + : INTERVAL value=expression unit=unitIdentifier + ; + +unitIdentifier + : YEAR | MONTH | WEEK | DAY | HOUR | MINUTE | SECOND + ; + // this rule is used for explicitly capturing wrong identifiers such as test-table, which should actually be `test-table` // replace identifier with errorCapturingIdentifier where the immediate follow symbol is not an expression, otherwise // valid expressions such as "a-b" can be recognized as an identifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TimestampArithmeticExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TimestampArithmeticExpr.java index 16f6fc4e74..36b34616a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TimestampArithmeticExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TimestampArithmeticExpr.java @@ -31,6 +31,7 @@ import org.apache.doris.thrift.TExprNodeType; import org.apache.doris.thrift.TExprOpcode; import com.google.common.base.Preconditions; +import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -417,4 +418,15 @@ public class TimestampArithmeticExpr extends Expr { return description; } } + + @Override + public void finalizeImplForNereids() throws AnalysisException { + if (StringUtils.isEmpty(funcName)) { + throw new AnalysisException("function name is null"); + } + type = getChild(0).getType(); + opcode = getOpCode(); + fn = getBuiltinFunction(funcName.toLowerCase(), collectChildReturnTypes(), + Function.CompareMode.IS_NONSTRICT_SUPERTYPE_OF); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java index 60b662865b..e248264ff7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java @@ -51,6 +51,11 @@ public class UnboundSlot extends Slot implements Unbound { }).reduce((left, right) -> left + "." + right).orElse(""); } + @Override + public boolean isConstant() { + return false; + } + @Override public String toSql() { return nameParts.stream().map(Utils::quoteIfNeeded).reduce((left, right) -> left + "." + right).orElse(""); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java index 72cdb45f31..9146380e7f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java @@ -30,6 +30,7 @@ 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.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.And; @@ -37,6 +38,8 @@ import org.apache.doris.nereids.trees.expressions.Arithmetic; import org.apache.doris.nereids.trees.expressions.Between; import org.apache.doris.nereids.trees.expressions.BooleanLiteral; import org.apache.doris.nereids.trees.expressions.CaseWhen; +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; @@ -51,6 +54,7 @@ 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.SlotReference; +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.visitor.DefaultExpressionVisitor; @@ -173,6 +177,19 @@ public class ExpressionTranslator extends DefaultExpressionVisitor<Expr, PlanTra 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()); + } + @Override public Expr visitBetween(Between between, PlanTranslatorContext context) { throw new RuntimeException("Unexpected invocation"); @@ -244,4 +261,10 @@ public class ExpressionTranslator extends DefaultExpressionVisitor<Expr, PlanTra arithmetic.child(0).accept(this, context), arithmeticOperator.isBinary() ? arithmetic.child(1).accept(this, context) : null); } + + @Override + public Expr visitTimestampArithmetic(TimestampArithmetic arithmetic, PlanTranslatorContext context) { + return new TimestampArithmeticExpr(arithmetic.getFuncName(), arithmetic.left().accept(this, context), + arithmetic.right().accept(this, context), arithmetic.getTimeUnit().toString()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 2dcad23211..b5d7c2faf6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.parser; +import org.apache.doris.analysis.ArithmeticExpr.Operator; import org.apache.doris.nereids.DorisParser; import org.apache.doris.nereids.DorisParser.AggClauseContext; import org.apache.doris.nereids.DorisParser.ArithmeticBinaryContext; @@ -31,6 +32,7 @@ import org.apache.doris.nereids.DorisParser.FromClauseContext; import org.apache.doris.nereids.DorisParser.IdentifierListContext; import org.apache.doris.nereids.DorisParser.IdentifierSeqContext; import org.apache.doris.nereids.DorisParser.IntegerLiteralContext; +import org.apache.doris.nereids.DorisParser.IntervalContext; import org.apache.doris.nereids.DorisParser.JoinCriteriaContext; import org.apache.doris.nereids.DorisParser.JoinRelationContext; import org.apache.doris.nereids.DorisParser.LogicalBinaryContext; @@ -55,6 +57,8 @@ import org.apache.doris.nereids.DorisParser.StarContext; import org.apache.doris.nereids.DorisParser.StringLiteralContext; import org.apache.doris.nereids.DorisParser.SubqueryExpressionContext; import org.apache.doris.nereids.DorisParser.TableNameContext; +import org.apache.doris.nereids.DorisParser.TypeConstructorContext; +import org.apache.doris.nereids.DorisParser.UnitIdentifierContext; import org.apache.doris.nereids.DorisParser.WhereClauseContext; import org.apache.doris.nereids.DorisParserBaseVisitor; import org.apache.doris.nereids.analyzer.UnboundAlias; @@ -69,6 +73,8 @@ import org.apache.doris.nereids.trees.expressions.And; import org.apache.doris.nereids.trees.expressions.Between; import org.apache.doris.nereids.trees.expressions.BooleanLiteral; import org.apache.doris.nereids.trees.expressions.CaseWhen; +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; @@ -77,6 +83,7 @@ import org.apache.doris.nereids.trees.expressions.GreaterThan; import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; 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; @@ -92,6 +99,7 @@ import org.apache.doris.nereids.trees.expressions.Regexp; import org.apache.doris.nereids.trees.expressions.StringLiteral; import org.apache.doris.nereids.trees.expressions.SubqueryExpr; 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.plans.JoinType; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; @@ -329,8 +337,30 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { Expression left = getExpression(ctx.left); Expression right = getExpression(ctx.right); + int type = ctx.operator.getType(); + if (left instanceof IntervalLiteral) { + if (type != DorisParser.PLUS) { + throw new IllegalArgumentException("Only supported: " + Operator.ADD); + } + IntervalLiteral interval = (IntervalLiteral) left; + return new TimestampArithmetic(Operator.ADD, right, interval.value(), interval.timeUnit(), true); + } + + if (right instanceof IntervalLiteral) { + Operator op; + if (type == DorisParser.PLUS) { + op = Operator.ADD; + } else if (type == DorisParser.MINUS) { + op = Operator.SUBTRACT; + } else { + throw new IllegalArgumentException("Only supported: " + Operator.ADD + " and " + Operator.SUBTRACT); + } + IntervalLiteral interval = (IntervalLiteral) right; + return new TimestampArithmetic(op, left, interval.value(), interval.timeUnit(), false); + } + return ParserUtils.withOrigin(ctx, () -> { - switch (ctx.operator.getType()) { + switch (type) { case DorisParser.ASTERISK: return new Multiply(left, right); case DorisParser.SLASH: @@ -406,6 +436,31 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { }); } + @Override + public Expression visitInterval(IntervalContext ctx) { + return new IntervalLiteral(getExpression(ctx.value), visitUnitIdentifier(ctx.unit)); + } + + @Override + public String visitUnitIdentifier(UnitIdentifierContext ctx) { + return ctx.getText(); + } + + @Override + public Expression visitTypeConstructor(TypeConstructorContext ctx) { + String value = ctx.STRING().getText(); + value = value.substring(1, value.length() - 1); + String type = ctx.identifier().getText().toUpperCase(); + switch (type) { + case "DATE": + return new DateLiteral(value); + case "DATETIME": + return new DateTimeLiteral(value); + default: + throw new IllegalStateException("Unsupported data type : " + type); + } + } + @Override public Expression visitDereference(DereferenceContext ctx) { return ParserUtils.withOrigin(ctx, () -> { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index 6d1f55ea32..2485fad8a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -33,6 +33,8 @@ public enum RuleType { BINDING_SORT_SLOT(RuleTypeClass.REWRITE), BINDING_PROJECT_FUNCTION(RuleTypeClass.REWRITE), BINDING_AGGREGATE_FUNCTION(RuleTypeClass.REWRITE), + BINDING_FILTER_FUNCTION(RuleTypeClass.REWRITE), + RESOLVE_PROJECT_ALIAS(RuleTypeClass.REWRITE), RESOLVE_AGGREGATE_ALIAS(RuleTypeClass.REWRITE), PROJECT_TO_GLOBAL_AGGREGATE(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindFunction.java index 2aafcf6254..3a5b2bbeeb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindFunction.java @@ -17,15 +17,20 @@ package org.apache.doris.nereids.rules.analysis; +import org.apache.doris.analysis.ArithmeticExpr.Operator; import org.apache.doris.nereids.analyzer.UnboundFunction; 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.NamedExpression; +import org.apache.doris.nereids.trees.expressions.TimestampArithmetic; import org.apache.doris.nereids.trees.expressions.functions.Substring; import org.apache.doris.nereids.trees.expressions.functions.Sum; import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.types.DateTimeType; +import org.apache.doris.nereids.types.IntegerType; import com.google.common.collect.ImmutableList; @@ -51,6 +56,12 @@ public class BindFunction implements AnalysisRuleFactory { List<NamedExpression> output = bind(agg.getOutputExpressions()); return agg.withGroupByAndOutput(groupBy, output); }) + ), + RuleType.BINDING_FILTER_FUNCTION.build( + logicalFilter().then(filter -> { + List<Expression> predicates = bind(filter.getExpressions()); + return new LogicalFilter<>(predicates.get(0), filter.child()); + }) ) ); } @@ -92,5 +103,27 @@ public class BindFunction implements AnalysisRuleFactory { } return unboundFunction; } + + @Override + public Expression visitTimestampArithmetic(TimestampArithmetic arithmetic, Void context) { + String funcOpName = null; + if (arithmetic.getFuncName() == null) { + funcOpName = String.format("%sS_%s", arithmetic.getTimeUnit(), + (arithmetic.getOp() == Operator.ADD) ? "ADD" : "SUB"); + } else { + funcOpName = arithmetic.getFuncName(); + } + + Expression left = arithmetic.left(); + Expression right = arithmetic.right(); + + if (!arithmetic.left().getDataType().isDateType()) { + left = arithmetic.left().castTo(DateTimeType.INSTANCE); + } + if (!arithmetic.right().getDataType().isIntType()) { + right = arithmetic.right().castTo(IntegerType.INSTANCE); + } + return arithmetic.withFuncName(funcOpName).withChildren(ImmutableList.of(left, right)); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushPredicateThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushPredicateThroughJoin.java index 84103b0da7..0b79ea062c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushPredicateThroughJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushPredicateThroughJoin.java @@ -97,6 +97,7 @@ public class PushPredicateThroughJoin extends OneRewriteRuleFactory { if (slots.isEmpty()) { leftPredicates.add(p); rightPredicates.add(p); + continue; } if (leftInput.containsAll(slots)) { leftPredicates.add(p); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateLiteral.java new file mode 100644 index 0000000000..889ab30003 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateLiteral.java @@ -0,0 +1,161 @@ +// 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.exceptions.AnalysisException; +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.util.DateUtils; + +import com.google.common.base.Preconditions; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.joda.time.LocalDateTime; +import org.joda.time.format.DateTimeFormatter; + +/** + * Date literal in Nereids. + */ +public class DateLiteral extends Literal { + + private static final Logger LOG = LogManager.getLogger(DateLiteral.class); + + private static final int DATEKEY_LENGTH = 8; + + private static DateTimeFormatter DATE_FORMATTER = null; + private static DateTimeFormatter DATE_FORMATTER_TWO_DIGIT = null; + private static DateTimeFormatter DATEKEY_FORMATTER = null; + + protected long year; + protected long month; + protected long day; + + static { + try { + DATE_FORMATTER = DateUtils.formatBuilder("%Y-%m-%d").toFormatter(); + DATEKEY_FORMATTER = DateUtils.formatBuilder("%Y%m%d").toFormatter(); + DATE_FORMATTER_TWO_DIGIT = DateUtils.formatBuilder("%y-%m-%d").toFormatter(); + } catch (AnalysisException e) { + LOG.error("invalid date format", e); + System.exit(-1); + } + } + + public DateLiteral(String s) throws AnalysisException { + super(DateType.INSTANCE); + init(s); + } + + public DateLiteral(DataType type) throws AnalysisException { + super(type); + } + + /** + * C'tor for date type. + */ + public DateLiteral(long year, long month, long day) { + super(DateType.INSTANCE); + this.year = year; + this.month = month; + this.day = day; + } + + /** + * C'tor for type conversion. + */ + public DateLiteral(DateLiteral other, DataType type) { + super(type); + this.year = other.year; + this.month = other.month; + this.day = other.day; + } + + private void init(String s) throws AnalysisException { + try { + LocalDateTime dateTime; + if (s.split("-")[0].length() == 2) { + dateTime = DATE_FORMATTER_TWO_DIGIT.parseLocalDateTime(s); + } else if (s.length() == DATEKEY_LENGTH && !s.contains("-")) { + dateTime = DATEKEY_FORMATTER.parseLocalDateTime(s); + } else { + dateTime = DATE_FORMATTER.parseLocalDateTime(s); + } + year = dateTime.getYear(); + month = dateTime.getMonthOfYear(); + day = dateTime.getDayOfMonth(); + } catch (Exception ex) { + throw new AnalysisException("date literal [" + s + "] is invalid"); + } + } + + @Override + protected Expression uncheckedCastTo(DataType targetType) throws AnalysisException { + if (getDataType().equals(targetType)) { + return this; + } + if (targetType.isDate()) { + if (getDataType().equals(targetType)) { + return this; + } + if (targetType.equals(DateType.INSTANCE)) { + return new DateLiteral(this.year, this.month, this.day); + } else if (targetType.equals(DateTimeType.INSTANCE)) { + return new DateTimeLiteral(this.year, this.month, this.day, 0, 0, 0); + } else { + throw new AnalysisException("Error date literal type"); + } + } + //todo other target type cast + return this; + } + + public DateLiteral withDataType(DataType type) { + Preconditions.checkArgument(type.isDate() || type.isDateTime()); + return new DateLiteral(this, type); + } + + @Override + public Long getValue() { + return (year * 10000 + month * 100 + day) * 1000000L; + } + + @Override + public String toSql() { + return toString(); + } + + @Override + public String toString() { + return String.format("%04d-%02d-%02d", year, month, day); + } + + public long getYear() { + return year; + } + + public long getMonth() { + return month; + } + + public long getDay() { + return day; + } +} + + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateTimeLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateTimeLiteral.java new file mode 100644 index 0000000000..130a4e93a0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateTimeLiteral.java @@ -0,0 +1,151 @@ +// 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.exceptions.AnalysisException; +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.util.DateUtils; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.joda.time.LocalDateTime; +import org.joda.time.format.DateTimeFormatter; + +/** + * date time literal. + */ +public class DateTimeLiteral extends DateLiteral { + private static final Logger LOG = LogManager.getLogger(DateTimeLiteral.class); + + private static final int DATETIME_TO_MINUTE_STRING_LENGTH = 16; + private static final int DATETIME_TO_HOUR_STRING_LENGTH = 13; + + private static DateTimeFormatter DATE_TIME_FORMATTER = null; + private static DateTimeFormatter DATE_TIME_FORMATTER_TO_HOUR = null; + private static DateTimeFormatter DATE_TIME_FORMATTER_TO_MINUTE = null; + private static DateTimeFormatter DATE_TIME_FORMATTER_TWO_DIGIT = null; + + private long hour; + private long minute; + private long second; + + static { + try { + DATE_TIME_FORMATTER = DateUtils.formatBuilder("%Y-%m-%d %H:%i:%s").toFormatter(); + DATE_TIME_FORMATTER_TO_HOUR = DateUtils.formatBuilder("%Y-%m-%d %H").toFormatter(); + DATE_TIME_FORMATTER_TO_MINUTE = DateUtils.formatBuilder("%Y-%m-%d %H:%i").toFormatter(); + DATE_TIME_FORMATTER_TWO_DIGIT = DateUtils.formatBuilder("%y-%m-%d %H:%i:%s").toFormatter(); + } catch (AnalysisException e) { + LOG.error("invalid date format", e); + System.exit(-1); + } + } + + public DateTimeLiteral(String s) { + super(DateTimeType.INSTANCE); + init(s); + } + + /** + * C'tor data time literal. + */ + public DateTimeLiteral(long year, long month, long day, long hour, long minute, long second) { + super(DateTimeType.INSTANCE); + this.hour = hour; + this.minute = minute; + this.second = second; + this.year = year; + this.month = month; + this.day = day; + } + + private void init(String s) throws AnalysisException { + try { + LocalDateTime dateTime; + if (s.split("-")[0].length() == 2) { + dateTime = DATE_TIME_FORMATTER_TWO_DIGIT.parseLocalDateTime(s); + } else { + if (s.length() == DATETIME_TO_MINUTE_STRING_LENGTH) { + dateTime = DATE_TIME_FORMATTER_TO_MINUTE.parseLocalDateTime(s); + } else if (s.length() == DATETIME_TO_HOUR_STRING_LENGTH) { + dateTime = DATE_TIME_FORMATTER_TO_HOUR.parseLocalDateTime(s); + } else { + dateTime = DATE_TIME_FORMATTER.parseLocalDateTime(s); + } + } + year = dateTime.getYear(); + month = dateTime.getMonthOfYear(); + day = dateTime.getDayOfMonth(); + hour = dateTime.getHourOfDay(); + minute = dateTime.getMinuteOfHour(); + second = dateTime.getSecondOfMinute(); + } catch (Exception ex) { + throw new AnalysisException("date time literal [" + s + "] is invalid"); + } + } + + @Override + protected Expression uncheckedCastTo(DataType targetType) throws AnalysisException { + if (getDataType().equals(targetType)) { + return this; + } + if (targetType.isDate()) { + if (getDataType().equals(targetType)) { + return this; + } + if (targetType.equals(DateType.INSTANCE)) { + return new DateLiteral(this.year, this.month, this.day); + } else if (targetType.equals(DateTimeType.INSTANCE)) { + return new DateTimeLiteral(this.year, this.month, this.day, this.hour, this.minute, this.second); + } else { + throw new AnalysisException("Error date literal type"); + } + } + //todo other target type cast + return this; + } + + @Override + public Long getValue() { + return (year * 10000 + month * 100 + day) * 1000000L + hour * 10000 + minute * 100 + second; + } + + @Override + public String toSql() { + return toString(); + } + + @Override + public String toString() { + return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second); + } + + public long getHour() { + return hour; + } + + public long getMinute() { + return minute; + } + + public long getSecond() { + return second; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java index 98b0808e51..e40544303f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java @@ -17,6 +17,7 @@ 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.visitor.ExpressionVisitor; @@ -79,9 +80,16 @@ public abstract class Expression extends AbstractTreeNode<Expression> { * Whether the expression is a constant. */ public boolean isConstant() { - return children().stream().anyMatch(Expression::isConstant); + return children().stream().allMatch(Expression::isConstant); } + public final Expression castTo(DataType targetType) throws AnalysisException { + return uncheckedCastTo(targetType); + } + + protected Expression uncheckedCastTo(DataType targetType) throws AnalysisException { + throw new RuntimeException("Do not implement uncheckedCastTo"); + } @Override public boolean equals(Object o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/IntervalLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/IntervalLiteral.java new file mode 100644 index 0000000000..a11d408bcc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/IntervalLiteral.java @@ -0,0 +1,64 @@ +// 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; + +/** + * Interval for timestamp calculation. + */ +public class IntervalLiteral extends Expression { + private final Expression value; + private final TimeUnit timeUnit; + + public IntervalLiteral(Expression value, String desc) { + this.value = value; + this.timeUnit = TimeUnit.valueOf(desc.toUpperCase()); + } + + public Expression value() { + return value; + } + + public TimeUnit timeUnit() { + return timeUnit; + } + + /** + * Supported time unit. + */ + public enum TimeUnit { + YEAR("YEAR"), // YEARS + MONTH("MONTH"), // MONTHS + WEEK("WEEK"), // WEEKS + DAY("DAY"), // DAYS + HOUR("HOUR"), // HOURS + MINUTE("MINUTE"), // MINUTES + SECOND("SECOND"); // SECONDS + private final String description; + + TimeUnit(String description) { + this.description = description; + } + + @Override + public String toString() { + return description; + } + } +} + + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpression.java index ec6ebfb50b..2e376720f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpression.java @@ -47,11 +47,6 @@ public abstract class NamedExpression extends Expression { throw new UnboundException("qualifier"); } - @Override - public boolean isConstant() { - return false; - } - /** * Get qualified name of NamedExpression. * diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StringLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StringLiteral.java index 16ab722464..c8e7ee633e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StringLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StringLiteral.java @@ -17,7 +17,9 @@ package org.apache.doris.nereids.trees.expressions; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.StringType; /** @@ -47,6 +49,30 @@ public class StringLiteral extends Literal { return visitor.visitStringLiteral(this, context); } + @Override + protected Expression uncheckedCastTo(DataType targetType) throws AnalysisException { + if (getDataType().equals(targetType)) { + return this; + } + if (targetType.isDateType()) { + return convertToDate(targetType); + } else if (targetType.isIntType()) { + return new IntegerLiteral(Integer.parseInt(value)); + } + //todo other target type cast + return this; + } + + private DateLiteral convertToDate(DataType targetType) throws AnalysisException { + DateLiteral dateLiteral = null; + if (targetType.isDate()) { + dateLiteral = new DateLiteral(value); + } else if (targetType.isDateTime()) { + dateLiteral = new DateTimeLiteral(value); + } + return dateLiteral; + } + @Override public String toSql() { return "'" + value + "'"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/TimestampArithmetic.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/TimestampArithmetic.java new file mode 100644 index 0000000000..bdd4a02c0b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/TimestampArithmetic.java @@ -0,0 +1,132 @@ +// 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.trees.expressions.IntervalLiteral.TimeUnit; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; + +import com.google.common.base.Preconditions; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +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. + */ +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; + + public TimestampArithmetic(String funcName, Expression e1, Expression e2, TimeUnit timeUnit) { + this(funcName, null, e1, e2, timeUnit, false); + } + + public TimestampArithmetic(Operator op, Expression e1, Expression e2, TimeUnit timeUnit, boolean intervalFirst) { + this(null, op, e1, e2, timeUnit, intervalFirst); + + } + + /** + * Full parameter constructor. + */ + public TimestampArithmetic(String funcName, Operator op, Expression e1, Expression e2, TimeUnit timeUnit, + boolean intervalFirst) { + super(e1, e2); + Preconditions.checkState(op == Operator.ADD || op == Operator.SUBTRACT); + this.funcName = funcName; + this.op = op; + this.intervalFirst = intervalFirst; + this.timeUnit = timeUnit; + } + + @Override + public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) { + return visitor.visitTimestampArithmetic(this, context); + } + + @Override + public Expression 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); + } + + public Expression withFuncName(String funcName) { + return new TimestampArithmetic(funcName, this.op, children.get(0), children.get(1), this.timeUnit, + this.intervalFirst); + } + + public String getFuncName() { + return funcName; + } + + public boolean isIntervalFirst() { + return intervalFirst; + } + + public Operator getOp() { + return op; + } + + public TimeUnit getTimeUnit() { + return timeUnit; + } + + @Override + public String toString() { + return toSql(); + } + + @Override + public String toSql() { + StringBuilder strBuilder = new StringBuilder(); + if (funcName != null) { + // Function-call like version. + strBuilder.append(funcName).append("("); + strBuilder.append(child(0).toSql()).append(", "); + strBuilder.append("INTERVAL "); + strBuilder.append(child(1).toSql()); + strBuilder.append(" ").append(timeUnit); + strBuilder.append(")"); + return strBuilder.toString(); + } + if (intervalFirst) { + // Non-function-call like version with interval as first operand. + strBuilder.append("INTERVAL "); + strBuilder.append(child(1).toSql() + " "); + 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("INTERVAL "); + strBuilder.append(child(1).toSql() + " "); + strBuilder.append(timeUnit); + } + return strBuilder.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java index 95ae89512a..248c32ab29 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java @@ -30,6 +30,8 @@ import org.apache.doris.nereids.trees.expressions.BooleanLiteral; import org.apache.doris.nereids.trees.expressions.CaseWhen; 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; @@ -57,6 +59,7 @@ 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; +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; @@ -140,6 +143,14 @@ public abstract class ExpressionVisitor<R, C> { return visit(doubleLiteral, context); } + public R visitDateLiteral(DateLiteral dateLiteral, C context) { + return visit(dateLiteral, context); + } + + public R visitDateTimeLiteral(DateTimeLiteral dateTimeLiteral, C context) { + return visit(dateTimeLiteral, context); + } + public R visitBetween(Between between, C context) { return visit(between, context); } @@ -220,6 +231,10 @@ public abstract class ExpressionVisitor<R, C> { return visit(subqueryExpr, context); } + public R visitTimestampArithmetic(TimestampArithmetic arithmetic, C context) { + return visit(arithmetic, context); + } + /* ******************************************************************************************** * Unbound expressions * ********************************************************************************************/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/BooleanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/BooleanType.java index 9c1e5f566c..b236ba12a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/BooleanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/BooleanType.java @@ -22,7 +22,7 @@ import org.apache.doris.catalog.Type; /** * Boolean type in Nereids. */ -public class BooleanType extends DataType { +public class BooleanType extends PrimitiveType { public static BooleanType INSTANCE = new BooleanType(); @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java index 508c85dcc9..8a29991a2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java @@ -52,6 +52,12 @@ public abstract class DataType { return VarcharType.createVarcharType(scalarType.getLength()); case STRING: return StringType.INSTANCE; + case DATE: + return DateType.INSTANCE; + case DATETIME: + return DateTimeType.INSTANCE; + case DECIMALV2: + return DecimalType.createDecimalType(scalarType.decimalPrecision(), scalarType.decimalScale()); case NULL_TYPE: return NullType.INSTANCE; default: @@ -87,4 +93,20 @@ public abstract class DataType { public int hashCode() { return 0; } + + public boolean isDate() { + return this instanceof DateType; + } + + public boolean isIntType() { + return this instanceof IntegerType; + } + + public boolean isDateTime() { + return this instanceof DateTimeType; + } + + public boolean isDateType() { + return isDate() || isDateTime(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateTimeType.java similarity index 77% copy from fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java copy to fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateTimeType.java index 6da6be9859..0d05006e50 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateTimeType.java @@ -20,13 +20,19 @@ package org.apache.doris.nereids.types; import org.apache.doris.catalog.Type; /** - * Null data type in Nereids. + * Datetime type in Nereids. */ -public class NullType extends DataType { - public static NullType INSTANCE = new NullType(); +public class DateTimeType extends PrimitiveType { + + public static DateTimeType INSTANCE = new DateTimeType(); @Override public Type toCatalogDataType() { - return Type.NULL; + return Type.DATETIME; + } + + @Override + public boolean equals(Object o) { + return o instanceof DateTimeType; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateType.java similarity index 86% copy from fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java copy to fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateType.java index 6da6be9859..75187b4bab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateType.java @@ -20,13 +20,15 @@ package org.apache.doris.nereids.types; import org.apache.doris.catalog.Type; /** - * Null data type in Nereids. + * Date type in Nereids. */ -public class NullType extends DataType { - public static NullType INSTANCE = new NullType(); +public class DateType extends PrimitiveType { + + public static DateType INSTANCE = new DateType(); @Override public Type toCatalogDataType() { - return Type.NULL; + return Type.DATE; } } + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalType.java similarity index 63% copy from fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java copy to fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalType.java index 6da6be9859..f12d550698 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalType.java @@ -20,13 +20,33 @@ package org.apache.doris.nereids.types; import org.apache.doris.catalog.Type; /** - * Null data type in Nereids. + * Decimal type in Nereids. */ -public class NullType extends DataType { - public static NullType INSTANCE = new NullType(); +public class DecimalType extends FractionalType { + + private int precision; + private int scale; + + public DecimalType(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + public static DecimalType createDecimalType(int precision, int scale) { + return new DecimalType(precision, scale); + } @Override public Type toCatalogDataType() { - return Type.NULL; + return Type.DECIMALV2; + } + + public int getPrecision() { + return precision; + } + + public int getScale() { + return scale; } } + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/IntegerType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/IntegerType.java index 1a95ff7550..aff70cdd0b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/IntegerType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/IntegerType.java @@ -29,4 +29,9 @@ public class IntegerType extends IntegralType { public Type toCatalogDataType() { return Type.INT; } + + @Override + public boolean equals(Object o) { + return o instanceof IntegerType; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java index 6da6be9859..cf1cb952bf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java @@ -22,7 +22,7 @@ import org.apache.doris.catalog.Type; /** * Null data type in Nereids. */ -public class NullType extends DataType { +public class NullType extends PrimitiveType { public static NullType INSTANCE = new NullType(); @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VarcharType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VarcharType.java index df77733203..09f72ceef2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VarcharType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VarcharType.java @@ -25,7 +25,7 @@ import java.util.Objects; /** * Varchar type in Nereids. */ -public class VarcharType extends DataType { +public class VarcharType extends PrimitiveType { private final int len; public VarcharType(int len) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateUtils.java new file mode 100644 index 0000000000..6228fe8ef8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateUtils.java @@ -0,0 +1,146 @@ +// 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.util; + +import org.apache.doris.nereids.exceptions.AnalysisException; + +import org.joda.time.format.DateTimeFormatterBuilder; + +/** + * date util tools. + */ +public class DateUtils { + + /** + * format builder. + */ + public static DateTimeFormatterBuilder formatBuilder(String pattern) throws AnalysisException { + DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder(); + boolean escaped = false; + for (int i = 0; i < pattern.length(); i++) { + char character = pattern.charAt(i); + if (escaped) { + switch (character) { + case 'a': // %a Abbreviated weekday name (Sun..Sat) + builder.appendDayOfWeekShortText(); + break; + case 'b': // %b Abbreviated month name (Jan..Dec) + builder.appendMonthOfYearShortText(); + break; + case 'c': // %c Month, numeric (0..12) + builder.appendMonthOfYear(1); + break; + case 'd': // %d Day of the month, numeric (00..31) + builder.appendDayOfMonth(2); + break; + case 'e': // %e Day of the month, numeric (0..31) + builder.appendDayOfMonth(1); + break; + case 'H': // %H Hour (00..23) + builder.appendHourOfDay(2); + break; + case 'h': // %h Hour (01..12) + case 'I': // %I Hour (01..12) + builder.appendClockhourOfHalfday(2); + break; + case 'i': // %i Minutes, numeric (00..59) + builder.appendMinuteOfHour(2); + break; + case 'j': // %j Day of year (001..366) + builder.appendDayOfYear(3); + break; + case 'k': // %k Hour (0..23) + builder.appendHourOfDay(1); + break; + case 'l': // %l Hour (1..12) + builder.appendClockhourOfHalfday(1); + break; + case 'M': // %M Month name (January..December) + builder.appendMonthOfYearText(); + break; + case 'm': // %m Month, numeric (00..12) + builder.appendMonthOfYear(2); + break; + case 'p': // %p AM or PM + builder.appendHalfdayOfDayText(); + break; + case 'r': // %r Time, 12-hour (hh:mm:ss followed by AM or PM) + builder.appendClockhourOfHalfday(2) + .appendLiteral(':') + .appendMinuteOfHour(2) + .appendLiteral(':') + .appendSecondOfMinute(2) + .appendLiteral(' ') + .appendHalfdayOfDayText(); + break; + case 'S': // %S Seconds (00..59) + case 's': // %s Seconds (00..59) + builder.appendSecondOfMinute(2); + break; + case 'T': // %T Time, 24-hour (hh:mm:ss) + builder.appendHourOfDay(2) + .appendLiteral(':') + .appendMinuteOfHour(2) + .appendLiteral(':') + .appendSecondOfMinute(2); + break; + case 'v': // %v Week (01..53), where Monday is the first day of the week; used with %x + builder.appendWeekOfWeekyear(2); + break; + case 'x': + // %x Year for the week, where Monday is the first day of the week, + // numeric, four digits; used with %v + builder.appendWeekyear(4, 4); + break; + case 'W': // %W Weekday name (Sunday..Saturday) + builder.appendDayOfWeekText(); + break; + case 'Y': // %Y Year, numeric, four digits + builder.appendYear(4, 4); + break; + case 'y': // %y Year, numeric (two digits) + builder.appendTwoDigitYear(2020); + break; + // TODO(Gabriel): support microseconds in date literal + case 'f': // %f Microseconds (000000..999999) + case 'w': // %w Day of the week (0=Sunday..6=Saturday) + case 'U': // %U Week (00..53), where Sunday is the first day of the week + case 'u': // %u Week (00..53), where Monday is the first day of the week + case 'V': // %V Week (01..53), where Sunday is the first day of the week; used with %X + case 'X': // %X Year for the week where Sunday is the first day of the week, + // numeric, four digits; used with %V + case 'D': // %D Day of the month with English suffix (0th, 1st, 2nd, 3rd, …) + throw new AnalysisException(String.format("%%%s not supported in date format string", + character)); + case '%': // %% A literal "%" character + builder.appendLiteral('%'); + break; + default: // %<x> The literal character represented by <x> + builder.appendLiteral(character); + break; + } + escaped = false; + } else if (character == '%') { + escaped = true; + } else { + builder.appendLiteral(character); + } + } + return builder; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewriteTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewriteTest.java index f677c1570a..ced159b336 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewriteTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewriteTest.java @@ -26,8 +26,8 @@ import org.apache.doris.nereids.rules.expression.rewrite.rules.SimplifyNotExprRu import org.apache.doris.nereids.trees.expressions.Expression; import com.google.common.collect.ImmutableList; -import org.junit.Assert; import org.junit.Test; +import org.junit.jupiter.api.Assertions; /** * all expr rewrite rule test case. @@ -64,11 +64,12 @@ public class ExpressionRewriteTest { public void testNormalizeExpressionRewrite() { executor = new ExpressionRuleExecutor(NormalizeBinaryPredicatesRule.INSTANCE); + assertRewrite("1 = 1", "1 = 1"); assertRewrite("2 > x", "x < 2"); - assertRewrite("2 >= x", "x <= 2"); - assertRewrite("2 < x", "x > 2"); - assertRewrite("2 <= x", "x >= 2"); - assertRewrite("2 = x", "x = 2"); + assertRewrite("y > x", "y > x"); + assertRewrite("1 + 2 > x", "x < 1 + 2"); + assertRewrite("1 + 2 > x + 1", "x + 1 < 1 + 2"); + assertRewrite("y + 2 > x + 1", "y + 2 > x + 1"); } @Test @@ -152,6 +153,6 @@ public class ExpressionRewriteTest { Expression needRewriteExpression = PARSER.parseExpression(expression); Expression expectedExpression = PARSER.parseExpression(expected); Expression rewrittenExpression = executor.rewrite(needRewriteExpression); - Assert.assertEquals(expectedExpression, rewrittenExpression); + Assertions.assertEquals(expectedExpression, rewrittenExpression); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/ssb/SSBUtils.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/ssb/SSBUtils.java index 3433cbefd3..97de88afd3 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/ssb/SSBUtils.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/ssb/SSBUtils.java @@ -349,3 +349,4 @@ public class SSBUtils { + ")"); } } + diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java index fafee04bc6..d02a76c0ee 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java @@ -176,4 +176,34 @@ public class ExpressionParserTest { String notExistExpr = "not exists (select * from test where b = 1)"; assertExpr(notExistExpr); } + + @Test + public void testInterval() { + String interval = "tt > date '1991-05-01' + interval '1' day"; + assertExpr(interval); + + interval = "tt > '1991-05-01' + interval '1' day"; + assertExpr(interval); + + interval = "tt > '1991-05-01' + interval 1 day"; + assertExpr(interval); + + interval = "tt > '1991-05-01' - interval 1 day"; + assertExpr(interval); + + interval = "tt > date '1991-05-01' - interval '1' day"; + assertExpr(interval); + + interval = "tt > interval '1' day + '1991-05-01'"; + assertExpr(interval); + + interval = "tt > interval '1' day + date '1991-05-01'"; + assertExpr(interval); + + interval = "tt > '1991-05-01' - interval 2*1 day"; + assertExpr(interval); + + interval = "tt > now() - interval 1+1 day"; + assertExpr(interval); + } } --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org