[GitHub] [doris] zhengshiJ commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
zhengshiJ commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929593583 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java: ## @@ -0,0 +1,249 @@ +// 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.stats; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Table; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.trees.Filter; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Aggregate; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.Project; +import org.apache.doris.nereids.trees.plans.Scan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribution; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHeapSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; +import org.apache.doris.statistics.TableStats; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Used to calculate the stats for each operator + */ +public class StatsCalculator extends DefaultPlanVisitor { + +private final GroupExpression groupExpression; + +public StatsCalculator(GroupExpression groupExpression) { +this.groupExpression = groupExpression; +} + +/** + * Do estimate. + */ +public void estimate() { +StatsDeriveResult stats = groupExpression.getPlan().accept(this, null); +groupExpression.getOwnerGroup().setStatistics(stats); +Plan plan = groupExpression.getPlan(); +long limit = plan.getLimit(); +if (limit != -1) { +stats.setRowCount(Math.min(limit, stats.getRowCount())); +} +groupExpression.setStatDerived(true); +} + +@Override +public StatsDeriveResult visitLogicalAggregate(LogicalAggregate agg, Void context) { +return computeAggregate(agg); +} + +@Override +public StatsDeriveResult visitLogicalFilter(LogicalFilter filter, Void context) { +return computeFilter(filter); +} + +@Override +public StatsDeriveResult visitLogicalOlapScan(LogicalOlapScan olapScan, Void context) { +olapScan.getExpressions(); +return computeScan(olapScan); +} + +@Override +public StatsDeriveResult visitLogicalProject(LogicalProject project, Void context) { +return computeProject(project); +} + +@Override +public StatsDeriveResult visitLogicalSort(LogicalSort sort, Void context) { +return groupExpression.getCopyOfChildStats(0); +} + +@Override +public StatsDeriveResult visitLogicalJoin(LogicalJoin join, Void context) { +return HashJoinEstimation.estimate(groupExpression.getCopyOfChildStats(0), +groupExpression.getCopyOfChildStats(1), +join.getCondi
[GitHub] [doris] wangshuo128 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
wangshuo128 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929593812 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java: ## @@ -0,0 +1,249 @@ +// 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.stats; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Table; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.trees.Filter; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Aggregate; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.Project; +import org.apache.doris.nereids.trees.plans.Scan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribution; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHeapSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; +import org.apache.doris.statistics.TableStats; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Used to calculate the stats for each operator + */ +public class StatsCalculator extends DefaultPlanVisitor { + +private final GroupExpression groupExpression; + +public StatsCalculator(GroupExpression groupExpression) { +this.groupExpression = groupExpression; +} + +/** + * Do estimate. + */ +public void estimate() { +StatsDeriveResult stats = groupExpression.getPlan().accept(this, null); +groupExpression.getOwnerGroup().setStatistics(stats); +Plan plan = groupExpression.getPlan(); +long limit = plan.getLimit(); +if (limit != -1) { +stats.setRowCount(Math.min(limit, stats.getRowCount())); +} +groupExpression.setStatDerived(true); +} + +@Override +public StatsDeriveResult visitLogicalAggregate(LogicalAggregate agg, Void context) { +return computeAggregate(agg); +} + +@Override +public StatsDeriveResult visitLogicalFilter(LogicalFilter filter, Void context) { +return computeFilter(filter); +} + +@Override +public StatsDeriveResult visitLogicalOlapScan(LogicalOlapScan olapScan, Void context) { +olapScan.getExpressions(); +return computeScan(olapScan); +} + +@Override +public StatsDeriveResult visitLogicalProject(LogicalProject project, Void context) { +return computeProject(project); +} + +@Override +public StatsDeriveResult visitLogicalSort(LogicalSort sort, Void context) { +return groupExpression.getCopyOfChildStats(0); +} + +@Override +public StatsDeriveResult visitLogicalJoin(LogicalJoin join, Void context) { +return HashJoinEstimation.estimate(groupExpression.getCopyOfChildStats(0), +groupExpression.getCopyOfChildStats(1), +join.getCon
[GitHub] [doris] adonis0147 opened a new pull request, #11203: [enhancement](nereids) Don't normalize expressions when pushing down predicates
adonis0147 opened a new pull request, #11203: URL: https://github.com/apache/doris/pull/11203 # Proposed changes ~~Issue Number: close #xxx~~ ## Problem Summary: Don't normalize expressions when pushing down predicates ## Checklist(Required) 1. Does it affect the original behavior: No 2. Has unit tests been added: No Need 3. Has document been added or modified: No Need 4. Does it need to update dependencies: No 5. Are there any changes that cannot be rolled back: No ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] wangshuo128 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
wangshuo128 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929594835 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/HashJoinEstimation.java: ## @@ -0,0 +1,155 @@ +// 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.stats; + +import org.apache.doris.common.CheckedMath; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.JoinType; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; + +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Map; + +/** + * Estimate hash join stats. + * TODO: Update other props in the ColumnStats properly. + */ +public class HashJoinEstimation { + +/** + * Do estimate. + */ +public static StatsDeriveResult estimate(StatsDeriveResult leftStats, StatsDeriveResult rightStats, +Expression eqCondition, JoinType joinType) { +StatsDeriveResult statsDeriveResult = new StatsDeriveResult(leftStats); +statsDeriveResult.merge(rightStats); +List eqConjunctList = ExpressionUtils.extractConjunct(eqCondition); +long rowCount = -1; +if (joinType.isSemiOrAntiJoin()) { Review Comment: ok -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] wangshuo128 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
wangshuo128 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929593812 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java: ## @@ -0,0 +1,249 @@ +// 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.stats; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Table; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.trees.Filter; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Aggregate; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.Project; +import org.apache.doris.nereids.trees.plans.Scan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribution; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHeapSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; +import org.apache.doris.statistics.TableStats; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Used to calculate the stats for each operator + */ +public class StatsCalculator extends DefaultPlanVisitor { + +private final GroupExpression groupExpression; + +public StatsCalculator(GroupExpression groupExpression) { +this.groupExpression = groupExpression; +} + +/** + * Do estimate. + */ +public void estimate() { +StatsDeriveResult stats = groupExpression.getPlan().accept(this, null); +groupExpression.getOwnerGroup().setStatistics(stats); +Plan plan = groupExpression.getPlan(); +long limit = plan.getLimit(); +if (limit != -1) { +stats.setRowCount(Math.min(limit, stats.getRowCount())); +} +groupExpression.setStatDerived(true); +} + +@Override +public StatsDeriveResult visitLogicalAggregate(LogicalAggregate agg, Void context) { +return computeAggregate(agg); +} + +@Override +public StatsDeriveResult visitLogicalFilter(LogicalFilter filter, Void context) { +return computeFilter(filter); +} + +@Override +public StatsDeriveResult visitLogicalOlapScan(LogicalOlapScan olapScan, Void context) { +olapScan.getExpressions(); +return computeScan(olapScan); +} + +@Override +public StatsDeriveResult visitLogicalProject(LogicalProject project, Void context) { +return computeProject(project); +} + +@Override +public StatsDeriveResult visitLogicalSort(LogicalSort sort, Void context) { +return groupExpression.getCopyOfChildStats(0); +} + +@Override +public StatsDeriveResult visitLogicalJoin(LogicalJoin join, Void context) { +return HashJoinEstimation.estimate(groupExpression.getCopyOfChildStats(0), +groupExpression.getCopyOfChildStats(1), +join.getCon
[GitHub] [doris] cambyzju commented on pull request #11200: [Bug] Fix Be core while passing constant array type arguments to some array functions
cambyzju commented on PR #11200: URL: https://github.com/apache/doris/pull/11200#issuecomment-1195093555 LGTM -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] spaces-X opened a new pull request, #11204: [fix](be): fix stack overflow in unhex function
spaces-X opened a new pull request, #11204: URL: https://github.com/apache/doris/pull/11204 # Proposed changes Fix be crushed due to stack overflow in the `unhex` function. Located and fixed by @924060929 ## Problem Summary: Describe the overview of changes. ## Checklist(Required) 1. Does it affect the original behavior: (Yes/No/I Don't know) 2. Has unit tests been added: (Yes/No/No Need) 3. Has document been added or modified: (Yes/No/No Need) 4. Does it need to update dependencies: (Yes/No) 5. Are there any changes that cannot be rolled back: (Yes/No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] jackwener commented on a diff in pull request #11035: [Feature](nereids) support sub-query and alias for TPC-H
jackwener commented on code in PR #11035: URL: https://github.com/apache/doris/pull/11035#discussion_r929609084 ## fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java: ## @@ -118,6 +121,38 @@ private Plan groupToTreeNode(Group group) { return result.withChildren(childrenNode); } +private static class GroupExpressionAdapter { +private final GroupExpression groupExpr; + +public GroupExpressionAdapter(GroupExpression groupExpr) { +this.groupExpr = groupExpr; +} + +public GroupExpression getGroupExpr() { +return groupExpr; +} + +@Override +public boolean equals(Object o) { +if (this == o) { +return true; +} +if (o == null || getClass() != o.getClass()) { +return false; +} +GroupExpressionAdapter that = (GroupExpressionAdapter) o; +if (that.groupExpr.getPlan() instanceof LogicalOlapScan) { Review Comment: why special judgment for `OlapScan`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] github-actions[bot] commented on pull request #11177: [Bug](fe-ut) Fix UT when test cases in package
github-actions[bot] commented on PR #11177: URL: https://github.com/apache/doris/pull/11177#issuecomment-1195103374 PR approved by at least one committer and no changes requested. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] github-actions[bot] commented on pull request #11177: [Bug](fe-ut) Fix UT when test cases in package
github-actions[bot] commented on PR #11177: URL: https://github.com/apache/doris/pull/11177#issuecomment-1195103409 PR approved by anyone and no changes requested. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] github-actions[bot] commented on pull request #11201: [fix](image) fix bug that latestValidatedImageSeq may not be the second largest image id
github-actions[bot] commented on PR #11201: URL: https://github.com/apache/doris/pull/11201#issuecomment-1195105335 PR approved by at least one committer and no changes requested. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] github-actions[bot] commented on pull request #11201: [fix](image) fix bug that latestValidatedImageSeq may not be the second largest image id
github-actions[bot] commented on PR #11201: URL: https://github.com/apache/doris/pull/11201#issuecomment-1195105375 PR approved by anyone and no changes requested. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] jackwener commented on pull request #11035: [Feature](nereids) support sub-query and alias for TPC-H
jackwener commented on PR #11035: URL: https://github.com/apache/doris/pull/11035#issuecomment-1195107352 I recommend that you can learn use ` git rebase `. Use `git pull upstream master --rebase`. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] github-actions[bot] commented on pull request #11164: [Bug]Fix select command denied for user for specified table
github-actions[bot] commented on PR #11164: URL: https://github.com/apache/doris/pull/11164#issuecomment-1195109919 PR approved by at least one committer and no changes requested. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] jackwener commented on a diff in pull request #11035: [Feature](nereids) support sub-query and alias for TPC-H
jackwener commented on code in PR #11035: URL: https://github.com/apache/doris/pull/11035#discussion_r929618914 ## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java: ## @@ -51,64 +51,64 @@ public class BindSlotReference implements AnalysisRuleFactory { @Override public List buildRules() { return ImmutableList.of( -RuleType.BINDING_PROJECT_SLOT.build( -logicalProject().then(project -> { -List boundSlots = -bind(project.getProjects(), project.children(), project); -return new LogicalProject<>(flatBoundStar(boundSlots), project.child()); -}) -), -RuleType.BINDING_FILTER_SLOT.build( -logicalFilter().then(filter -> { -Expression boundPredicates = bind(filter.getPredicates(), filter.children(), filter); -return new LogicalFilter<>(boundPredicates, filter.child()); -}) -), -RuleType.BINDING_JOIN_SLOT.build( -logicalJoin().then(join -> { -Optional cond = join.getCondition() -.map(expr -> bind(expr, join.children(), join)); -return new LogicalJoin<>(join.getJoinType(), cond, join.left(), join.right()); -}) -), -RuleType.BINDING_AGGREGATE_SLOT.build( -logicalAggregate().then(agg -> { -List groupBy = bind(agg.getGroupByExpressions(), agg.children(), agg); -List output = bind(agg.getOutputExpressions(), agg.children(), agg); -return agg.withGroupByAndOutput(groupBy, output); -}) -), -RuleType.BINDING_SORT_SLOT.build( -logicalSort().then(sort -> { -List sortItemList = sort.getOrderKeys() -.stream() -.map(orderKey -> { -Expression item = bind(orderKey.getExpr(), sort.children(), sort); -return new OrderKey(item, orderKey.isAsc(), orderKey.isNullFirst()); -}).collect(Collectors.toList()); +RuleType.BINDING_PROJECT_SLOT.build( Review Comment: Reduce the `code-style-change`. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [enhance](*): improve code about optional (#11153)
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 a3df2e5982 [enhance](*): improve code about optional (#11153) a3df2e5982 is described below commit a3df2e59827cedfaa1cdaeed78c329cb24b99b8c Author: jakevin AuthorDate: Tue Jul 26 15:32:42 2022 +0800 [enhance](*): improve code about optional (#11153) improve code about optional --- .../main/java/org/apache/doris/analysis/AlterPolicyStmt.java | 7 +++ .../java/org/apache/doris/load/loadv2/SparkRepository.java | 12 ++-- .../org/apache/doris/load/routineload/RoutineLoadJob.java| 11 --- .../apache/doris/load/routineload/RoutineLoadManager.java| 4 +--- .../nereids/pattern/generator/PatternGeneratorAnalyzer.java | 7 +++ .../nereids/pattern/generator/javaast/ClassDeclaration.java | 12 +++- .../nereids/pattern/generator/javaast/EnumDeclaration.java | 4 +--- .../pattern/generator/javaast/IdentifyTypeArgumentsPair.java | 8 +++- .../pattern/generator/javaast/InterfaceDeclaration.java | 8 ++-- .../nereids/pattern/generator/javaast/TypeDeclaration.java | 6 ++ .../nereids/pattern/generator/javaast/TypeParameter.java | 7 ++- .../doris/nereids/pattern/generator/javaast/TypeType.java| 7 ++- .../rules/rewrite/logical/PushPredicateThroughJoin.java | 6 +- .../doris/nereids/trees/plans/logical/LogicalPlan.java | 6 +- .../doris/nereids/trees/plans/physical/PhysicalHashJoin.java | 6 +++--- .../src/main/java/org/apache/doris/policy/PolicyMgr.java | 8 +++- .../src/main/java/org/apache/doris/qe/ConnectContext.java| 2 +- 17 files changed, 37 insertions(+), 84 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterPolicyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterPolicyStmt.java index ab94a30830..3b145cd55e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterPolicyStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterPolicyStmt.java @@ -70,10 +70,9 @@ public class AlterPolicyStmt extends DdlStmt { .getPolicyMgr().getPoliciesByType(PolicyTypeEnum.STORAGE); Optional hasPolicy = policiesByType.stream() .filter(policy -> policy.getPolicyName().equals(this.policyName)).findAny(); -if (!hasPolicy.isPresent()) { -throw new AnalysisException("Unknown storage policy: " + this.policyName); -} -StoragePolicy storagePolicy = (StoragePolicy) hasPolicy.get(); +StoragePolicy storagePolicy = (StoragePolicy) hasPolicy.orElseThrow( +() -> new AnalysisException("Unknown storage policy: " + this.policyName) +); // default storage policy use alter storage policy to add s3 resource. if (!policyName.equalsIgnoreCase(StoragePolicy.DEFAULT_STORAGE_POLICY_NAME) && properties.containsKey( diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java index 753a1fc192..4eb9ab881b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java @@ -342,23 +342,15 @@ public class SparkRepository { } public SparkLibrary getDppLibrary() { -SparkLibrary result = null; Optional library = libraries.stream() .filter(lib -> lib.libType == SparkLibrary.LibType.DPP).findFirst(); -if (library.isPresent()) { -result = library.get(); -} -return result; +return library.orElse(null); } public SparkLibrary getSpark2xLibrary() { -SparkLibrary result = null; Optional library = libraries.stream() .filter(lib -> lib.libType == SparkLibrary.LibType.SPARK2X).findFirst(); -if (library.isPresent()) { -result = library.get(); -} -return result; +return library.orElse(null); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index c8e7d3b532..3c3234a15e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -854,13 +854,10 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl routineLoadTaskInfoList.stream() .filter(entity -> entity.getTxnId() == txnState
[GitHub] [doris] 924060929 merged pull request #11153: [enhance](*): improve code about optional
924060929 merged PR #11153: URL: https://github.com/apache/doris/pull/11153 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] 924060929 merged pull request #10479: [feature](nereids): join reorder
924060929 merged PR #10479: URL: https://github.com/apache/doris/pull/10479 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [feature](nereids): join reorder (#10479)
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 846716ac10 [feature](nereids): join reorder (#10479) 846716ac10 is described below commit 846716ac10b701646f8f2b1c73708a4f97dd1c87 Author: jakevin AuthorDate: Tue Jul 26 15:35:00 2022 +0800 [feature](nereids): join reorder (#10479) Enhance join reorder. Add LAsscom (include with project). Add Commute. Add UT for join reorder --- .../org/apache/doris/analysis/PredicateUtils.java | 2 +- .../glue/translator/PhysicalPlanTranslator.java| 4 +- .../nereids/properties/LogicalProperties.java | 2 +- .../org/apache/doris/nereids/rules/RuleSet.java| 6 +- .../rules/exploration/JoinReorderContext.java | 68 ++-- .../rules/exploration/join/JoinCommutative.java| 59 --- .../rules/exploration/join/JoinCommute.java| 102 .../rules/exploration/join/JoinExchange.java | 2 + .../rules/exploration/join/JoinLAsscom.java| 123 -- .../exploration/join/JoinLeftAssociative.java | 54 -- .../rules/exploration/join/JoinProjectLAsscom.java | 185 + .../rewrite/logical/PushPredicateThroughJoin.java | 54 +++--- .../nereids/rules/rewrite/logical/ReorderJoin.java | 4 +- .../nereids/trees/expressions/SlotReference.java | 1 + .../nereids/trees/plans/logical/LogicalJoin.java | 6 + .../apache/doris/nereids/util/ExpressionUtils.java | 61 ++- .../rules/exploration/join/JoinCommuteTest.java| 70 .../rules/exploration/join/JoinLAsscomTest.java| 176 .../exploration/join/JoinProjectLAsscomTest.java | 148 + .../doris/nereids/util/ExpressionUtilsTest.java| 12 +- 20 files changed, 961 insertions(+), 178 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PredicateUtils.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PredicateUtils.java index f6d058d450..7d8b10e22a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PredicateUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PredicateUtils.java @@ -30,7 +30,7 @@ public class PredicateUtils { * Some examples: * a or b -> a, b * a or b or c -> a, b, c - * (a and b) or (c or d) -> (a and b), (c and d) + * (a and b) or (c or d) -> (a and b), c, d * (a or b) and c -> (a or b) and c * a -> a */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index b314138116..47f9748263 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -348,7 +348,7 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor execEqConjunctList = ExpressionUtils.extractConjunct(eqJoinExpression).stream() +List execEqConjunctList = ExpressionUtils.extractConjunctive(eqJoinExpression).stream() .map(EqualTo.class::cast) .map(e -> swapEqualToForChildrenOrder(e, hashJoin.left().getOutput())) .map(e -> ExpressionTranslator.translate(e, context)) @@ -400,7 +400,7 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor expressionList = ExpressionUtils.extractConjunct(expression); +List expressionList = ExpressionUtils.extractConjunctive(expression); expressionList.stream().map(e -> ExpressionTranslator.translate(e, context)).forEach(planNode::addConjunct); return inputFragment; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java index c7fd2b66b3..443c8448ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java @@ -39,7 +39,7 @@ public class LogicalProperties { */ public LogicalProperties(Supplier> outputSupplier) { this.outputSupplier = Suppliers.memoize( -Objects.requireNonNull(outputSupplier, "outputSupplier can not be null") +Objects.requireNonNull(outputSupplier, "outputSupplier can not be null") ); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index f9827431df..0ca1eb8ea4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-cor
[GitHub] [doris] Kikyou1997 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
Kikyou1997 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929623972 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java: ## @@ -0,0 +1,249 @@ +// 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.stats; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Table; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.trees.Filter; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Aggregate; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.Project; +import org.apache.doris.nereids.trees.plans.Scan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribution; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHeapSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; +import org.apache.doris.statistics.TableStats; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Used to calculate the stats for each operator + */ +public class StatsCalculator extends DefaultPlanVisitor { + +private final GroupExpression groupExpression; + +public StatsCalculator(GroupExpression groupExpression) { +this.groupExpression = groupExpression; +} + +/** + * Do estimate. + */ +public void estimate() { +StatsDeriveResult stats = groupExpression.getPlan().accept(this, null); +groupExpression.getOwnerGroup().setStatistics(stats); +Plan plan = groupExpression.getPlan(); +long limit = plan.getLimit(); +if (limit != -1) { +stats.setRowCount(Math.min(limit, stats.getRowCount())); +} +groupExpression.setStatDerived(true); +} + +@Override +public StatsDeriveResult visitLogicalAggregate(LogicalAggregate agg, Void context) { +return computeAggregate(agg); +} + +@Override +public StatsDeriveResult visitLogicalFilter(LogicalFilter filter, Void context) { +return computeFilter(filter); +} + +@Override +public StatsDeriveResult visitLogicalOlapScan(LogicalOlapScan olapScan, Void context) { +olapScan.getExpressions(); +return computeScan(olapScan); +} + +@Override +public StatsDeriveResult visitLogicalProject(LogicalProject project, Void context) { +return computeProject(project); +} + +@Override +public StatsDeriveResult visitLogicalSort(LogicalSort sort, Void context) { +return groupExpression.getCopyOfChildStats(0); +} + +@Override +public StatsDeriveResult visitLogicalJoin(LogicalJoin join, Void context) { +return HashJoinEstimation.estimate(groupExpression.getCopyOfChildStats(0), +groupExpression.getCopyOfChildStats(1), +join.getCond
[GitHub] [doris] yiguolei merged pull request #10702: [refactor] Rename Catalog to Env
yiguolei merged PR #10702: URL: https://github.com/apache/doris/pull/10702 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] Kikyou1997 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
Kikyou1997 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929637321 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/HashJoinEstimation.java: ## @@ -0,0 +1,155 @@ +// 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.stats; + +import org.apache.doris.common.CheckedMath; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.JoinType; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; + +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Map; + +/** + * Estimate hash join stats. + * TODO: Update other props in the ColumnStats properly. + */ +public class HashJoinEstimation { + +/** + * Do estimate. + */ +public static StatsDeriveResult estimate(StatsDeriveResult leftStats, StatsDeriveResult rightStats, +Expression eqCondition, JoinType joinType) { +StatsDeriveResult statsDeriveResult = new StatsDeriveResult(leftStats); +statsDeriveResult.merge(rightStats); +List eqConjunctList = ExpressionUtils.extractConjunct(eqCondition); +long rowCount = -1; +if (joinType.isSemiOrAntiJoin()) { +rowCount = getSemiJoinRowCount(leftStats, rightStats, eqConjunctList, joinType); +} else if (joinType.isInnerJoin() || joinType.isOuterJoin()) { +rowCount = getJoinRowCount(leftStats, rightStats, eqConjunctList, joinType); +} else { +throw new RuntimeException("joinType is not supported"); +} +statsDeriveResult.setRowCount(rowCount); +return statsDeriveResult; +} + +private static long getSemiJoinRowCount(StatsDeriveResult leftStats, StatsDeriveResult rightStats, +List eqConjunctList, JoinType joinType) { +long rowCount; +if (JoinType.RIGHT_SEMI_JOIN.equals(joinType) || JoinType.RIGHT_ANTI_JOIN.equals(joinType)) { +if (rightStats.getRowCount() == -1) { +return -1; +} +rowCount = rightStats.getRowCount(); +} else { +if (leftStats.getRowCount() == -1) { +return -1; +} +rowCount = leftStats.getRowCount(); +} +Map leftSlotToColStats = leftStats.getSlotRefToColumnStatsMap(); +Map rightSlotToColStats = rightStats.getSlotRefToColumnStatsMap(); +double minSelectivity = 1.0; +for (Expression eqJoinPredicate : eqConjunctList) { +long lhsNdv = leftSlotToColStats.get(eqJoinPredicate.child(0)).getNdv(); +lhsNdv = Math.min(lhsNdv, leftStats.getRowCount()); +long rhsNdv = rightSlotToColStats.get(eqJoinPredicate.child(1)).getNdv(); +rhsNdv = Math.min(rhsNdv, rightStats.getRowCount()); +// Skip conjuncts with unknown NDV on either side. +if (lhsNdv == -1 || rhsNdv == -1) { +continue; +} +// TODO: Do we need NULL_AWARE_LEFT_ANTI_JOIN type as stale optimizer? +double selectivity = 1.0; +switch (joinType) { +case LEFT_SEMI_JOIN: { +selectivity = (double) Math.min(lhsNdv, rhsNdv) / (double) (lhsNdv); +break; +} +case RIGHT_SEMI_JOIN: { +selectivity = (double) Math.min(lhsNdv, rhsNdv) / (double) (rhsNdv); +break; +} +case LEFT_ANTI_JOIN: +selectivity = (double) (lhsNdv > rhsNdv ? (lhsNdv - rhsNdv) : lhsNdv) / (double) lhsNdv; +break; +case RIGHT_ANTI_JOIN: { +selectivity = (double) (rhsNdv > lhsNdv ? (rhsNdv - lhsNdv) : rhsNdv) / (double) rhsNdv; +break; +} +default: +throw new RuntimeException("joinType
[GitHub] [doris] zhengshiJ commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
zhengshiJ commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929645398 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java: ## @@ -0,0 +1,249 @@ +// 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.stats; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Table; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.trees.Filter; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Aggregate; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.Project; +import org.apache.doris.nereids.trees.plans.Scan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribution; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHeapSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; +import org.apache.doris.statistics.TableStats; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Used to calculate the stats for each operator + */ +public class StatsCalculator extends DefaultPlanVisitor { + +private final GroupExpression groupExpression; + +public StatsCalculator(GroupExpression groupExpression) { +this.groupExpression = groupExpression; +} + +/** + * Do estimate. + */ +public void estimate() { +StatsDeriveResult stats = groupExpression.getPlan().accept(this, null); +groupExpression.getOwnerGroup().setStatistics(stats); +Plan plan = groupExpression.getPlan(); +long limit = plan.getLimit(); +if (limit != -1) { +stats.setRowCount(Math.min(limit, stats.getRowCount())); +} +groupExpression.setStatDerived(true); +} + +@Override +public StatsDeriveResult visitLogicalAggregate(LogicalAggregate agg, Void context) { +return computeAggregate(agg); +} + +@Override +public StatsDeriveResult visitLogicalFilter(LogicalFilter filter, Void context) { +return computeFilter(filter); +} + +@Override +public StatsDeriveResult visitLogicalOlapScan(LogicalOlapScan olapScan, Void context) { +olapScan.getExpressions(); +return computeScan(olapScan); +} + +@Override +public StatsDeriveResult visitLogicalProject(LogicalProject project, Void context) { +return computeProject(project); +} + +@Override +public StatsDeriveResult visitLogicalSort(LogicalSort sort, Void context) { +return groupExpression.getCopyOfChildStats(0); +} + +@Override +public StatsDeriveResult visitLogicalJoin(LogicalJoin join, Void context) { +return HashJoinEstimation.estimate(groupExpression.getCopyOfChildStats(0), +groupExpression.getCopyOfChildStats(1), +join.getCondi
[GitHub] [doris] starocean999 commented on pull request #11197: [Bug][Vectorized] Fix the bug of memtable shrink
starocean999 commented on PR #11197: URL: https://github.com/apache/doris/pull/11197#issuecomment-1195147655 LGTM -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] dataroaring merged pull request #11197: [Bug][Vectorized] Fix the bug of memtable shrink
dataroaring merged PR #11197: URL: https://github.com/apache/doris/pull/11197 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] dataroaring closed issue #11115: [Bug] vectorized load use after free on master
dataroaring closed issue #5: [Bug] vectorized load use after free on master URL: https://github.com/apache/doris/issues/5 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [Bug][Vectorized] Fix use-after-free bug of memtable shrink (#11197)
This is an automated email from the ASF dual-hosted git repository. dataroaring 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 8551ceaa1b [Bug][Vectorized] Fix use-after-free bug of memtable shrink (#11197) 8551ceaa1b is described below commit 8551ceaa1b0f181abe861297c331646fd5c4cede Author: HappenLee AuthorDate: Tue Jul 26 16:10:44 2022 +0800 [Bug][Vectorized] Fix use-after-free bug of memtable shrink (#11197) Co-authored-by: lihaopeng --- be/src/olap/memtable.cpp | 9 +++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp index 2c3ee73357..a44c54c552 100644 --- a/be/src/olap/memtable.cpp +++ b/be/src/olap/memtable.cpp @@ -312,10 +312,15 @@ void MemTable::_collect_vskiplist_results() { for (size_t i = _schema->num_key_columns(); i < _schema->num_columns(); ++i) { auto function = _agg_functions[i]; auto agg_place = it.key()->agg_places(i); -function->insert_result_into(agg_place, - *(_output_mutable_block.get_column_by_position(i))); +auto col_ptr = _output_mutable_block.get_column_by_position(i).get(); +function->insert_result_into(agg_place, *col_ptr); if constexpr (is_final) { function->destroy(agg_place); +} else { +function->reset(agg_place); +function->add(agg_place, + const_cast(&col_ptr), idx, + nullptr); } } if constexpr (!is_final) { - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] Kikyou1997 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
Kikyou1997 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929658090 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java: ## @@ -0,0 +1,249 @@ +// 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.stats; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Table; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.trees.Filter; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Aggregate; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.Project; +import org.apache.doris.nereids.trees.plans.Scan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribution; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHeapSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; +import org.apache.doris.statistics.TableStats; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Used to calculate the stats for each operator + */ +public class StatsCalculator extends DefaultPlanVisitor { + +private final GroupExpression groupExpression; + +public StatsCalculator(GroupExpression groupExpression) { +this.groupExpression = groupExpression; +} + +/** + * Do estimate. + */ +public void estimate() { +StatsDeriveResult stats = groupExpression.getPlan().accept(this, null); +groupExpression.getOwnerGroup().setStatistics(stats); Review Comment: Agree, will do this later. ## fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java: ## @@ -0,0 +1,191 @@ +// 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.stats; + +import org.apache.doris.nereids.memo.Group; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.
[GitHub] [doris] Kikyou1997 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
Kikyou1997 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929658805 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java: ## @@ -0,0 +1,249 @@ +// 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.stats; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Table; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.trees.Filter; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Aggregate; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.Project; +import org.apache.doris.nereids.trees.plans.Scan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribution; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHeapSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; +import org.apache.doris.statistics.TableStats; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Used to calculate the stats for each operator + */ +public class StatsCalculator extends DefaultPlanVisitor { + +private final GroupExpression groupExpression; + +public StatsCalculator(GroupExpression groupExpression) { +this.groupExpression = groupExpression; +} + +/** + * Do estimate. + */ +public void estimate() { +StatsDeriveResult stats = groupExpression.getPlan().accept(this, null); +groupExpression.getOwnerGroup().setStatistics(stats); +Plan plan = groupExpression.getPlan(); +long limit = plan.getLimit(); +if (limit != -1) { +stats.setRowCount(Math.min(limit, stats.getRowCount())); +} +groupExpression.setStatDerived(true); +} + +@Override +public StatsDeriveResult visitLogicalAggregate(LogicalAggregate agg, Void context) { +return computeAggregate(agg); +} + +@Override +public StatsDeriveResult visitLogicalFilter(LogicalFilter filter, Void context) { +return computeFilter(filter); +} + +@Override +public StatsDeriveResult visitLogicalOlapScan(LogicalOlapScan olapScan, Void context) { +olapScan.getExpressions(); +return computeScan(olapScan); +} + +@Override +public StatsDeriveResult visitLogicalProject(LogicalProject project, Void context) { +return computeProject(project); +} + +@Override +public StatsDeriveResult visitLogicalSort(LogicalSort sort, Void context) { +return groupExpression.getCopyOfChildStats(0); +} + +@Override +public StatsDeriveResult visitLogicalJoin(LogicalJoin join, Void context) { +return HashJoinEstimation.estimate(groupExpression.getCopyOfChildStats(0), +groupExpression.getCopyOfChildStats(1), +join.getCond
[GitHub] [doris] Kikyou1997 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
Kikyou1997 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929668127 ## fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java: ## @@ -0,0 +1,249 @@ +// 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.stats; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Table; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.trees.Filter; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Aggregate; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.Project; +import org.apache.doris.nereids.trees.plans.Scan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribution; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHeapSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; +import org.apache.doris.statistics.TableStats; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Used to calculate the stats for each operator + */ +public class StatsCalculator extends DefaultPlanVisitor { + +private final GroupExpression groupExpression; + +public StatsCalculator(GroupExpression groupExpression) { +this.groupExpression = groupExpression; +} + +/** + * Do estimate. + */ +public void estimate() { +StatsDeriveResult stats = groupExpression.getPlan().accept(this, null); +groupExpression.getOwnerGroup().setStatistics(stats); +Plan plan = groupExpression.getPlan(); +long limit = plan.getLimit(); +if (limit != -1) { +stats.setRowCount(Math.min(limit, stats.getRowCount())); +} +groupExpression.setStatDerived(true); +} + +@Override +public StatsDeriveResult visitLogicalAggregate(LogicalAggregate agg, Void context) { +return computeAggregate(agg); +} + +@Override +public StatsDeriveResult visitLogicalFilter(LogicalFilter filter, Void context) { +return computeFilter(filter); +} + +@Override +public StatsDeriveResult visitLogicalOlapScan(LogicalOlapScan olapScan, Void context) { +olapScan.getExpressions(); +return computeScan(olapScan); +} + +@Override +public StatsDeriveResult visitLogicalProject(LogicalProject project, Void context) { +return computeProject(project); +} + +@Override +public StatsDeriveResult visitLogicalSort(LogicalSort sort, Void context) { +return groupExpression.getCopyOfChildStats(0); +} + +@Override +public StatsDeriveResult visitLogicalJoin(LogicalJoin join, Void context) { +return HashJoinEstimation.estimate(groupExpression.getCopyOfChildStats(0), +groupExpression.getCopyOfChildStats(1), +join.getCond
[GitHub] [doris] zhannngchen opened a new pull request, #11205: [feature-wip](unique-key-merge-on-write) remove AggType on unique table with MoW, enable preAggreation, DSIP-018[5/2]
zhannngchen opened a new pull request, #11205: URL: https://github.com/apache/doris/pull/11205 # Proposed changes Issue Number: close #xxx ## Problem Summary: see DSIP-018:https://cwiki.apache.org/confluence/display/DORIS/DSIP-018%3A+Support+Merge-On-Write+implementation+for+UNIQUE+KEY+data+model This is the patch for stage 5-2 ## Checklist(Required) 1. Does it affect the original behavior: (Yes/No/I Don't know) 2. Has unit tests been added: (Yes/No/No Need) 3. Has document been added or modified: (Yes/No/No Need) 4. Does it need to update dependencies: (Yes/No) 5. Are there any changes that cannot be rolled back: (Yes/No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] morrySnow opened a new pull request, #11206: [refactor](Nereids) ExprId count from 1 in each query
morrySnow opened a new pull request, #11206: URL: https://github.com/apache/doris/pull/11206 # Proposed changes Issue Number: close #xxx ## Problem Summary: Currently, ExprId is global across query. It is hard to reproduce same exprid for same query. it lead to hard to debug. This PR change this behavior by count from 1 in each query. ## Checklist(Required) 1. Does it affect the original behavior: No 2. Has unit tests been added: No Need 3. Has document been added or modified: No Need 4. Does it need to update dependencies:No 5. Are there any changes that cannot be rolled back: No ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] sohardforaname commented on a diff in pull request #11035: [Feature](nereids) support sub-query and alias for TPC-H
sohardforaname commented on code in PR #11035: URL: https://github.com/apache/doris/pull/11035#discussion_r929680269 ## fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java: ## @@ -118,6 +121,38 @@ private Plan groupToTreeNode(Group group) { return result.withChildren(childrenNode); } +private static class GroupExpressionAdapter { +private final GroupExpression groupExpr; + +public GroupExpressionAdapter(GroupExpression groupExpr) { +this.groupExpr = groupExpr; +} + +public GroupExpression getGroupExpr() { +return groupExpr; +} + +@Override +public boolean equals(Object o) { +if (this == o) { +return true; +} +if (o == null || getClass() != o.getClass()) { +return false; +} +GroupExpressionAdapter that = (GroupExpressionAdapter) o; +if (that.groupExpr.getPlan() instanceof LogicalOlapScan) { Review Comment: The class is a package class for GroupExpression because the equals() of the latter can not divide UnboundRelation from LogicalScanOlap as their logical properties are the same, but actually they are not the same. ## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java: ## @@ -51,64 +51,64 @@ public class BindSlotReference implements AnalysisRuleFactory { @Override public List buildRules() { return ImmutableList.of( -RuleType.BINDING_PROJECT_SLOT.build( -logicalProject().then(project -> { -List boundSlots = -bind(project.getProjects(), project.children(), project); -return new LogicalProject<>(flatBoundStar(boundSlots), project.child()); -}) -), -RuleType.BINDING_FILTER_SLOT.build( -logicalFilter().then(filter -> { -Expression boundPredicates = bind(filter.getPredicates(), filter.children(), filter); -return new LogicalFilter<>(boundPredicates, filter.child()); -}) -), -RuleType.BINDING_JOIN_SLOT.build( -logicalJoin().then(join -> { -Optional cond = join.getCondition() -.map(expr -> bind(expr, join.children(), join)); -return new LogicalJoin<>(join.getJoinType(), cond, join.left(), join.right()); -}) -), -RuleType.BINDING_AGGREGATE_SLOT.build( -logicalAggregate().then(agg -> { -List groupBy = bind(agg.getGroupByExpressions(), agg.children(), agg); -List output = bind(agg.getOutputExpressions(), agg.children(), agg); -return agg.withGroupByAndOutput(groupBy, output); -}) -), -RuleType.BINDING_SORT_SLOT.build( -logicalSort().then(sort -> { -List sortItemList = sort.getOrderKeys() -.stream() -.map(orderKey -> { -Expression item = bind(orderKey.getExpr(), sort.children(), sort); -return new OrderKey(item, orderKey.isAsc(), orderKey.isNullFirst()); -}).collect(Collectors.toList()); +RuleType.BINDING_PROJECT_SLOT.build( Review Comment: ok, thx -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] wangshuo128 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
wangshuo128 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929687958 ## fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java: ## @@ -39,8 +39,49 @@ public static String quoteIfNeeded(String part) { } /** - * Fully qualified identifier name parts, i.e., concat qualifier and name into a list. Review Comment: You deleted the comments of `qualifiedNameParts` by mistake... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] adonis0147 opened a new pull request, #11207: [enhancement](nereids) Fix the wrong order of arguments of Assert.assertEquals
adonis0147 opened a new pull request, #11207: URL: https://github.com/apache/doris/pull/11207 # Proposed changes ~~Issue Number: close #xxx~~ ## Problem Summary: The first argument of method `Assert.assertEquals` should be an expected value. Wrong order may confuse us when cases fail. ## Checklist(Required) 1. Does it affect the original behavior: No 2. Has unit tests been added: No Need 3. Has document been added or modified: No Need 4. Does it need to update dependencies: No 5. Are there any changes that cannot be rolled back: No ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] dataroaring opened a new pull request, #11208: grade datatype cases to p0 to test p0 pipeline
dataroaring opened a new pull request, #11208: URL: https://github.com/apache/doris/pull/11208 # Proposed changes Issue Number: close #xxx ## Problem Summary: Describe the overview of changes. ## Checklist(Required) 1. Does it affect the original behavior: (Yes/No/I Don't know) 2. Has unit tests been added: (Yes/No/No Need) 3. Has document been added or modified: (Yes/No/No Need) 4. Does it need to update dependencies: (Yes/No) 5. Are there any changes that cannot be rolled back: (Yes/No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] englefly opened a new pull request, #11209: [feature] (Nereids) support limit clause
englefly opened a new pull request, #11209: URL: https://github.com/apache/doris/pull/11209 # Proposed changes including: - limit clause parser - implementation rule to transform LogicalLimit to PhysicalLimit Issue Number: close #xxx ## Problem Summary: Describe the overview of changes. ## Checklist(Required) 1. Does it affect the original behavior: (Yes/No/I Don't know) 2. Has unit tests been added: (Yes/No/No Need) 3. Has document been added or modified: (Yes/No/No Need) 4. Does it need to update dependencies: (Yes/No) 5. Are there any changes that cannot be rolled back: (Yes/No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] zhannngchen commented on a diff in pull request #11195: [feature-wip](unique-key-merge-on-write) update delete bitmap while p…
zhannngchen commented on code in PR #11195: URL: https://github.com/apache/doris/pull/11195#discussion_r929709212 ## be/src/olap/task/engine_publish_version_task.cpp: ## @@ -86,39 +115,44 @@ Status EnginePublishVersionTask::finish() { res = Status::OLAPInternalError(OLAP_ERR_PUSH_TABLE_NOT_EXIST); continue; } - -publish_status = StorageEngine::instance()->txn_manager()->publish_txn( -partition_id, tablet, transaction_id, version); -if (publish_status != Status::OK()) { -LOG(WARNING) << "failed to publish version. rowset_id=" << rowset->rowset_id() - << ", tablet_id=" << tablet_info.tablet_id - << ", txn_id=" << transaction_id; -_error_tablet_ids->push_back(tablet_info.tablet_id); -res = publish_status; +Version max_version = tablet->max_version(); +// in uniq key model with aux index, we should see all Review Comment: in uniq key model with merge-on-write ## be/src/olap/txn_manager.cpp: ## @@ -308,8 +309,108 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, _clear_txn_partition_map_unlocked(transaction_id, partition_id); } } +} +auto tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id); +#ifdef BE_TEST +if (tablet == nullptr) { +return Status::OK(); +} +#endif +// Check if have to build extra delete bitmap for table of UNIQUE_KEY model +if (!tablet->enable_unique_key_merge_on_write() || +tablet->tablet_meta()->preferred_rowset_type() != RowsetTypePB::BETA_ROWSET || +rowset_ptr->keys_type() != KeysType::UNIQUE_KEYS) { return Status::OK(); } +CHECK(version.first == version.second) << "impossible: " << version; + +// For each key in current set, check if it overwrites any previously +// written keys +OlapStopWatch watch; +std::vector segments; +std::vector pre_segments; +auto beta_rowset = reinterpret_cast(rowset_ptr.get()); +Status st = beta_rowset->load_segments(&segments); +if (!st.ok()) return st; +// lock tablet meta to modify delete bitmap +std::lock_guard meta_wrlock(tablet->get_header_lock()); +for (auto& seg : segments) { +seg->load_index(); // We need index blocks to iterate +auto pk_idx = seg->get_primary_key_index(); +int cnt = 0; +int total = pk_idx->num_rows(); +int32_t remaining = total; +bool exact_match = false; +std::string last_key; +int batch_size = 1024; +MemPool pool; +while (remaining > 0) { +std::unique_ptr iter; +RETURN_IF_ERROR(pk_idx->new_iterator(&iter)); + +size_t num_to_read = std::min(batch_size, remaining); +std::unique_ptr cvb; +RETURN_IF_ERROR(ColumnVectorBatch::create(num_to_read, false, pk_idx->type_info(), + nullptr, &cvb)); +ColumnBlock block(cvb.get(), &pool); +ColumnBlockView column_block_view(&block); +Slice last_key_slice(last_key); +RETURN_IF_ERROR(iter->seek_at_or_after(&last_key_slice, &exact_match)); + +size_t num_read = num_to_read; +RETURN_IF_ERROR(iter->next_batch(&num_read, &column_block_view)); +DCHECK(num_to_read == num_read); +last_key = (reinterpret_cast(cvb->cell_ptr(num_read - 1)))->to_string(); + +// exclude last_key, last_key will be read in next batch. +if (num_read == batch_size && num_read != remaining) { +num_read -= 1; +} +for (size_t i = 0; i < num_read; i++) { +const Slice* key = reinterpret_cast(cvb->cell_ptr(i)); +// first check if exist in pre segment +bool find = _check_pk_in_pre_segments(pre_segments, *key, tablet, version); +if (find) { +cnt++; Review Comment: we should `continue` here? ## be/src/olap/task/engine_publish_version_task.cpp: ## @@ -86,39 +115,44 @@ Status EnginePublishVersionTask::finish() { res = Status::OLAPInternalError(OLAP_ERR_PUSH_TABLE_NOT_EXIST); continue; } - -publish_status = StorageEngine::instance()->txn_manager()->publish_txn( -partition_id, tablet, transaction_id, version); -if (publish_status != Status::OK()) { -LOG(WARNING) << "failed to publish version. rowset_id=" << rowset->rowset_id() - << ", tablet_id=" << tablet_info.tablet_id - << ", txn_id=" << transaction_id; -_error_tablet_ids->push_back(tablet_info.tablet_id)
[GitHub] [doris] pengxiangyu commented on a diff in pull request #11205: [feature-wip](unique-key-merge-on-write) remove AggType on unique table with MoW, enable preAggreation, DSIP-018[5/2]
pengxiangyu commented on code in PR #11205: URL: https://github.com/apache/doris/pull/11205#discussion_r929722858 ## be/src/olap/memtable.cpp: ## @@ -64,8 +64,8 @@ MemTable::MemTable(int64_t tablet_id, Schema* schema, const TabletSchema* tablet } else { _insert_fn = &MemTable::_insert_agg; } -if (_tablet_schema->has_sequence_col()) { -_aggregate_two_row_fn = &MemTable::_aggregate_two_row_with_sequence; Review Comment: Is sequence_col in tablet_schema not needed?what abort old meta info? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] luwei16 commented on a diff in pull request #11006: (performance)[scanner] Isolate local and remote queries using different scanner…
luwei16 commented on code in PR #11006: URL: https://github.com/apache/doris/pull/11006#discussion_r929723974 ## be/src/common/config.h: ## @@ -798,6 +798,11 @@ CONF_Int32(s3_transfer_executor_pool_size, "2"); CONF_Bool(enable_time_lut, "true"); +// number of s3 scanner thread pool size +CONF_Int32(doris_s3_scanner_thread_pool_thread_num, "48"); Review Comment: changed to 16 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] luwei16 commented on a diff in pull request #11006: (performance)[scanner] Isolate local and remote queries using different scanner…
luwei16 commented on code in PR #11006: URL: https://github.com/apache/doris/pull/11006#discussion_r929724365 ## be/src/common/config.h: ## @@ -798,6 +798,11 @@ CONF_Int32(s3_transfer_executor_pool_size, "2"); CONF_Bool(enable_time_lut, "true"); +// number of s3 scanner thread pool size +CONF_Int32(doris_s3_scanner_thread_pool_thread_num, "48"); +// number of s3 scanner thread pool queue size +CONF_Int32(doris_s3_scanner_thread_pool_queue_size, "102400"); Review Comment: changed to 10240 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] luwei16 commented on a diff in pull request #11006: (performance)[scanner] Isolate local and remote queries using different scanner…
luwei16 commented on code in PR #11006: URL: https://github.com/apache/doris/pull/11006#discussion_r929724579 ## be/src/exec/olap_scan_node.cpp: ## @@ -1503,6 +1503,7 @@ void OlapScanNode::transfer_thread(RuntimeState* state) { * 4. Regularly increase the priority of the remaining tasks in the queue to avoid starvation for large queries */ PriorityThreadPool* thread_pool = state->exec_env()->scan_thread_pool(); +PriorityThreadPool* s3_thread_pool = state->exec_env()->s3_scan_thread_pool(); Review Comment: done ## be/src/runtime/exec_env.h: ## @@ -200,6 +201,7 @@ class ExecEnv { // TODO(cmy): find a better way to unify these 2 pools. PriorityThreadPool* _scan_thread_pool = nullptr; +PriorityThreadPool* _s3_scan_thread_pool = nullptr; Review Comment: done -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] zhannngchen commented on a diff in pull request #11205: [feature-wip](unique-key-merge-on-write) remove AggType on unique table with MoW, enable preAggreation, DSIP-018[5/2]
zhannngchen commented on code in PR #11205: URL: https://github.com/apache/doris/pull/11205#discussion_r929746562 ## be/src/olap/memtable.cpp: ## @@ -64,8 +64,8 @@ MemTable::MemTable(int64_t tablet_id, Schema* schema, const TabletSchema* tablet } else { _insert_fn = &MemTable::_insert_agg; } -if (_tablet_schema->has_sequence_col()) { -_aggregate_two_row_fn = &MemTable::_aggregate_two_row_with_sequence; Review Comment: This part of code is not removed, just move to `_aggregate_two_row`, it works quite similar like the method `_aggregate_two_row_in_block`. For the unique table without MoW(merge-on-write), they all proceeded by the method `_aggregate_two_row`, no necessary to separate to 2 methods. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] morrySnow opened a new pull request, #11210: [enhancement](Nereids)enable explain query for nereids planner
morrySnow opened a new pull request, #11210: URL: https://github.com/apache/doris/pull/11210 ## Problem Summary: Support explain syntax. And generate explain tree string by nereids' planner. ## Checklist(Required) 1. Does it affect the original behavior: No 2. Has unit tests been added: Yes 3. Has document been added or modified: No Need 4. Does it need to update dependencies: No 5. Are there any changes that cannot be rolled back: No -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] morrySnow commented on pull request #11210: [enhancement](Nereids)enable explain query for nereids planner
morrySnow commented on PR #11210: URL: https://github.com/apache/doris/pull/11210#issuecomment-1195252286 @924060929 PTAL -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei commented on pull request #11154: [FOLLOW-UP] (datetimev2) complete date function ut and built-in function declaration
yiguolei commented on PR #11154: URL: https://github.com/apache/doris/pull/11154#issuecomment-1195263274 Fast merge it since it is under developement -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei merged pull request #11154: [FOLLOW-UP] (datetimev2) complete date function ut and built-in function declaration
yiguolei merged PR #11154: URL: https://github.com/apache/doris/pull/11154 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] 924060929 merged pull request #10981: [Enhancement](Nereids) add some basic data structure definitions.
924060929 merged PR #10981: URL: https://github.com/apache/doris/pull/10981 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [Enhancement](Nereids) add some basic data structure definitions. (#10981)
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
[GitHub] [doris] muyizi commented on pull request #11040: fix_arena_push_size
muyizi commented on PR #11040: URL: https://github.com/apache/doris/pull/11040#issuecomment-1195266784 > 如果 ChunkAllocator 中的 chunk 支持不定长内存的话,那预期存在内存浪费问题, > > 按之前的逻辑,申请1624k会实际申请2048k,free时放回2048k的free list中,在之后满足1024k到2048k的内存申请 > > chunk支持不定长内存的话,第一次申请1624k,free时放回1024k的free list中,在后续当成1024k来用,相比之前一直多浪费400k内存,再次申请1600k时也无法复用上次申请的内存。 > > 所以我建议 ChunkAllocator 还是保持仅定长内存,否则为了不浪费内存还要加更复杂的比较逻辑,如我上面的patch,对代码健壮性做一些完善,若认可这个思路,可以参考我的patch重新提交下代码~ @muyizi ,若有其他见解欢迎随时沟通~ 对应您说的,我认为只支持定长内存其实本质上就不可避免造成内存浪费,用户需要1624k只能去申请2048k,所以我觉得内存浪费这个是一直存在的; 把allocate不定长的方式不暴露给使用者确实是目前比较简单的方法,但是后面如需使用不定长申请,还是需要改那块逻辑的 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei merged pull request #11196: [FIX]string pad function should be always nullable for both string an…
yiguolei merged PR #11196: URL: https://github.com/apache/doris/pull/11196 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [FIX]string pad function should be always nullable for both string and varchar type (#11196)
This is an automated email from the ASF dual-hosted git repository. yiguolei 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 f46a801b1b [FIX]string pad function should be always nullable for both string and varchar type (#11196) f46a801b1b is described below commit f46a801b1b66b3ba8b3cdac2dab9e529b32122d4 Author: starocean999 <40539150+starocean...@users.noreply.github.com> AuthorDate: Tue Jul 26 17:55:06 2022 +0800 [FIX]string pad function should be always nullable for both string and varchar type (#11196) --- gensrc/script/doris_builtins_functions.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 041fcb4e97..8753edd7f1 100755 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -2015,10 +2015,10 @@ visible_functions = [ '15FunctionContextERKNS1_9StringValERKNS1_6IntValE', '', '', 'vec', ''], [['lpad'], 'STRING', ['STRING', 'INT', 'STRING'], '_ZN5doris15StringFunctions4lpadEPN9doris_udf' -'15FunctionContextERKNS1_9StringValERKNS1_6IntValES6_', '', '', 'vec', ''], +'15FunctionContextERKNS1_9StringValERKNS1_6IntValES6_', '', '', 'vec', 'ALWAYS_NULLABLE'], [['rpad'], 'STRING', ['STRING', 'INT', 'STRING'], '_ZN5doris15StringFunctions4rpadEPN9doris_udf' -'15FunctionContextERKNS1_9StringValERKNS1_6IntValES6_', '', '', 'vec', ''], +'15FunctionContextERKNS1_9StringValERKNS1_6IntValES6_', '', '', 'vec', 'ALWAYS_NULLABLE'], [['append_trailing_char_if_absent'], 'STRING', ['STRING', 'STRING'], '_ZN5doris15StringFunctions30append_trailing_char_if_absentEPN9doris_udf15FunctionContextERKNS1_9StringValES6_', '', '', 'vec', 'ALWAYS_NULLABLE'], - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei merged pull request #11177: [Bug](fe-ut) Fix UT when test cases in package
yiguolei merged PR #11177: URL: https://github.com/apache/doris/pull/11177 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei closed issue #11176: [Bug](fe-ut) UT failed when test cases in package
yiguolei closed issue #11176: [Bug](fe-ut) UT failed when test cases in package URL: https://github.com/apache/doris/issues/11176 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated (f46a801b1b -> 0f6ec629ab)
This is an automated email from the ASF dual-hosted git repository. yiguolei pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/doris.git from f46a801b1b [FIX]string pad function should be always nullable for both string and varchar type (#11196) add 0f6ec629ab [Bug](fe-ut) Fix UT when test cases in package (#11177) No new revisions were added by this update. Summary of changes: .../test/java/org/apache/doris/utframe/MockedFrontend.java| 11 +-- .../test/java/org/apache/doris/utframe/TestWithFeService.java | 3 ++- .../src/test/java/org/apache/doris/utframe/UtFrameUtils.java | 2 +- 3 files changed, 4 insertions(+), 12 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei merged pull request #11160: [regression] optimization the result check for test_json_load
yiguolei merged PR #11160: URL: https://github.com/apache/doris/pull/11160 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: optimization the result check for test_json_load (#11160)
This is an automated email from the ASF dual-hosted git repository. yiguolei 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 2c34b6cf2b optimization the result check for test_json_load (#11160) 2c34b6cf2b is described below commit 2c34b6cf2bc145f0afb9e9e72e04e1cb121cb9ed Author: carlvinhust2012 AuthorDate: Tue Jul 26 17:58:30 2022 +0800 optimization the result check for test_json_load (#11160) Co-authored-by: hucheng01 --- .../data/load/stream_load/test_json_load.out | 117 + .../suites/load/stream_load/test_json_load.groovy | 142 + 2 files changed, 148 insertions(+), 111 deletions(-) diff --git a/regression-test/data/load/stream_load/test_json_load.out b/regression-test/data/load/stream_load/test_json_load.out new file mode 100644 index 00..d53bd5bd1b --- /dev/null +++ b/regression-test/data/load/stream_load/test_json_load.out @@ -0,0 +1,117 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +1 beijing 2345671 +2 shanghai2345672 +3 guangzhou 2345673 +4 shenzhen2345674 +5 hangzhou2345675 +6 nanjing 2345676 +7 wuhan 2345677 +8 chengdu 2345678 +9 xian2345679 +10 hefei 23456710 +200changsha3456789 + +-- !select -- +10 beijing 2345671 +20 shanghai2345672 +30 guangzhou 2345673 +40 shenzhen2345674 +50 hangzhou2345675 +60 nanjing 2345676 +70 wuhan 2345677 +80 chengdu 2345678 +90 xian2345679 +100hefei 23456710 +200changsha3456789 + +-- !select -- +1 2345671 +2 2345672 +3 2345673 +4 2345674 +5 2345675 +6 2345676 +7 2345677 +8 2345678 +9 2345679 +10 23456710 +200755 + +-- !select -- +1 210 +2 220 +3 230 +4 240 +5 250 +6 260 +7 270 +8 280 +9 290 +10 300 +200755 + +-- !select -- +1 1454547 +2 1244264 +3 528369 +4 594201 +5 594201 +6 2345672 +7 2345673 +8 2345674 +9 2345675 +10 2345676 +200755 + +-- !select -- +10 1454547 +20 1244264 +30 528369 +40 594201 +50 594201 +60 2345672 +70 2345673 +80 2345674 +90 2345675 +1002345676 +200755 + +-- !select -- +60 2345672 +70 2345673 +80 2345674 +90 2345675 +1002345676 +200755 + +-- !select -- +60 2345672 +70 2345673 +80 2345674 +90 2345675 +1002345676 +200755 + +-- !select -- +10 beijing 2345671 +20 shanghai2345672 +200changsha3456789 + +-- !select -- +2 shanghai2345672 +3 guangzhou 2345673 +4 shenzhen2345674 +5 hangzhou2345675 +6 nanjing 2345676 +7 wuhan 2345677 +8 chengdu 2345678 +9 xian2345679 +200changsha3456789 + +-- !select -- +2 shanghai2345672 +3 guangzhou 2345673 +4 shenzhen2345674 +200changsha3456789 + diff --git a/regression-test/suites/load/stream_load/test_json_load.groovy b/regression-test/suites/load/stream_load/test_json_load.groovy index 305de78eac..914f1c227f 100644 --- a/regression-test/suites/load/stream_load/test_json_load.groovy +++ b/regression-test/suites/load/stream_load/test_json_load.groovy @@ -130,6 +130,23 @@ suite("test_json_load", "load") { assertTrue(result1[0][0] == 0, "Query OK, 0 rows affected") } +def check_load_result = {checklabel, testTablex -> +max_try_milli_secs = 1 +while(max_try_milli_secs) { +result = sql "show load where label = '${checklabel}'" +if(result[0][2] == "FINISHED") { +qt_select "select * from ${testTablex} order by id" +break +} else { +sleep(1000) // wait 1 second every time +max_try_milli_secs -= 1000 +if(max_try_milli_secs <= 0) { +assertEquals(1, 2) +} +} +} +} + // case1: import simple json try { sql "DROP TABLE IF EXISTS ${testTable}" @@ -138,17 +155,7 @@ suite("test_json_load", "load") { load_json_data.call('true', '', 'json', '', '', '', '', '', 'simple_json.json') -// select the table and check whether the data is correct -def result3 = sql "select * from test_json_load order by id" -assertTrue(result3.size() == 11) -assertTrue(result3[0].size() == 3) -assertTrue(result3[0][0] == 1) -assertTrue(result3[0][1] == "beijing") -assertTrue(result3[0][2] == 2345671) -assertTrue(result3[9].size() == 3) -assertTrue(result3[9][0] == 10) -assertTrue(
[GitHub] [doris] yiguolei merged pull request #11164: [Bug]Fix select command denied for user for specified table
yiguolei merged PR #11164: URL: https://github.com/apache/doris/pull/11164 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei closed issue #11163: [Bug] Fix select command denied for specified table
yiguolei closed issue #11163: [Bug] Fix select command denied for specified table URL: https://github.com/apache/doris/issues/11163 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [Bug]Fix select command denied for user for specified table
This is an automated email from the ASF dual-hosted git repository. yiguolei 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 a1ad978960 [Bug]Fix select command denied for user for specified table a1ad978960 is described below commit a1ad978960c500e35e36717092cf61cc992e8383 Author: xy720 <22125576+xy...@users.noreply.github.com> AuthorDate: Tue Jul 26 17:59:28 2022 +0800 [Bug]Fix select command denied for user for specified table --- fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PaloAuth.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PaloAuth.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PaloAuth.java index 0cc2907f82..314375197b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PaloAuth.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PaloAuth.java @@ -465,7 +465,7 @@ public class PaloAuth implements Writable { public boolean checkTblPriv(ConnectContext ctx, TableName tableName, PrivPredicate wanted) { Preconditions.checkState(tableName.isFullyQualified()); -return checkTblPriv(ctx, tableName.getCtl(), tableName.getDb(), wanted); +return checkTblPriv(ctx, tableName.getDb(), tableName.getTbl(), wanted); } public boolean checkTblPriv(UserIdentity currentUser, String ctl, String db, String tbl, PrivPredicate wanted) { - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei merged pull request #11132: [UDAF](sample) impl rpc avg udaf
yiguolei merged PR #11132: URL: https://github.com/apache/doris/pull/11132 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [UDAF](sample) impl rpc avg udaf (#11132)
This is an automated email from the ASF dual-hosted git repository. yiguolei 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 03d466febe [UDAF](sample) impl rpc avg udaf (#11132) 03d466febe is described below commit 03d466febe774492b52a1f0e522b21e997990f5a Author: chenlinzhong <490103...@qq.com> AuthorDate: Tue Jul 26 18:00:13 2022 +0800 [UDAF](sample) impl rpc avg udaf (#11132) --- .../aggregate_functions/aggregate_function_rpc.h | 2 +- .../cpp_function_service_demo.cpp | 32 ++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/be/src/vec/aggregate_functions/aggregate_function_rpc.h b/be/src/vec/aggregate_functions/aggregate_function_rpc.h index 5612ce27fd..6a9a3894c8 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_rpc.h +++ b/be/src/vec/aggregate_functions/aggregate_function_rpc.h @@ -252,7 +252,7 @@ public: reinterpret_cast(return_type.get())->get_nested_type(); } WhichDataType which(result_type); -if (which.is_float()) { +if (which.is_float32()) { float ret = response.result(0).float_value(0); to.insert_data((char*)&ret, 0); } diff --git a/samples/doris-demo/remote-udaf-cpp-demo/cpp_function_service_demo.cpp b/samples/doris-demo/remote-udaf-cpp-demo/cpp_function_service_demo.cpp index e15fba65ab..8f8b338f5f 100644 --- a/samples/doris-demo/remote-udaf-cpp-demo/cpp_function_service_demo.cpp +++ b/samples/doris-demo/remote-udaf-cpp-demo/cpp_function_service_demo.cpp @@ -59,6 +59,38 @@ public: } result->add_int32_value(sum); } +if(fun_name=="rpc_avg_update"){ +result->mutable_type()->set_id(PGenericType::DOUBLE); +double sum=0; +int64_t size = request->args(0).int32_value_size(); +for (size_t i = 0; i < request->args(0).int32_value_size(); ++i) { +sum += request->args(0).int32_value(i); +} +if(request->has_context() && request->context().has_function_context()){ +sum += request->context().function_context().args_data(0).double_value(0); +size += request->context().function_context().args_data(0).int32_value(0); +} +result->add_double_value(sum); +result->add_int32_value(size); +} +if(fun_name=="rpc_avg_merge"){ +result->mutable_type()->set_id(PGenericType::INT32); +double sum= 0; +int32_t size = 0; +for (size_t i = 0; i < request->args_size(); ++i) { +sum += request->args(i).double_value(0); +size += request->args(i).int32_value(0); +} +result->add_double_value(sum); +result->add_int32_value(size); +} +if(fun_name=="rpc_avg_finalize"){ + result->mutable_type()->set_id(PGenericType::DOUBLE); + double sum = request->context().function_context().args_data(0).double_value(0); + int64_t size = request->context().function_context().args_data(0).int32_value(0); + double avg = sum / size; + result->add_double_value(avg); +} response->mutable_status()->set_status_code(0); } - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei merged pull request #11120: [Sample] Add Flink Connector Sample Code
yiguolei merged PR #11120: URL: https://github.com/apache/doris/pull/11120 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [Sample] Add Flink Connector Sample Code (#11120)
This is an automated email from the ASF dual-hosted git repository. yiguolei 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 f5479fa80e [Sample] Add Flink Connector Sample Code (#11120) f5479fa80e is described below commit f5479fa80eb341e15728605143ac321618df7ab0 Author: wudi <676366...@qq.com> AuthorDate: Tue Jul 26 18:00:51 2022 +0800 [Sample] Add Flink Connector Sample Code (#11120) Co-authored-by: wudi <> --- samples/doris-demo/flink-demo-v1.1/pom.xml | 1 + .../apache/doris/demo/flink/Cdc2DorisSQLDemo.java | 68 ++ .../demo/flink/DorisFlinkConnectorDemoV1.java | 2 +- ...rDemoV1.java => Kafka2DorisDataStreamDemo.java} | 62 ++-- samples/doris-demo/spark-demo/pom.xml | 4 +- 5 files changed, 103 insertions(+), 34 deletions(-) diff --git a/samples/doris-demo/flink-demo-v1.1/pom.xml b/samples/doris-demo/flink-demo-v1.1/pom.xml index 5e907fb34b..803a75d760 100644 --- a/samples/doris-demo/flink-demo-v1.1/pom.xml +++ b/samples/doris-demo/flink-demo-v1.1/pom.xml @@ -34,6 +34,7 @@ under the License. 1.2.62 2.8.3 compile +1.7.30 diff --git a/samples/doris-demo/flink-demo-v1.1/src/main/java/org/apache/doris/demo/flink/Cdc2DorisSQLDemo.java b/samples/doris-demo/flink-demo-v1.1/src/main/java/org/apache/doris/demo/flink/Cdc2DorisSQLDemo.java new file mode 100644 index 00..0d1f61fa3e --- /dev/null +++ b/samples/doris-demo/flink-demo-v1.1/src/main/java/org/apache/doris/demo/flink/Cdc2DorisSQLDemo.java @@ -0,0 +1,68 @@ +// 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.demo.flink; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +public class Cdc2DorisSQLDemo { + +public static void main(String[] args) throws Exception { + +final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); +env.enableCheckpointing(1); +env.setParallelism(1); +final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); +// register a table in the catalog +tEnv.executeSql( +"CREATE TABLE cdc_test_source (\n" + +" id INT,\n" + +" name STRING\n" + +") WITH (\n" + +" 'connector' = 'mysql-cdc',\n" + +" 'hostname' = '127.0.0.1',\n" + +" 'port' = '3306',\n" + +" 'username' = 'root',\n" + +" 'password' = '',\n" + +" 'database-name' = 'db',\n" + +" 'table-name' = 'test_source'\n" + +")"); +//doris table +tEnv.executeSql( +"CREATE TABLE doris_test_sink (" + +"id INT," + +"name STRING" + +") " + +"WITH (\n" + +" 'connector' = 'doris',\n" + +" 'fenodes' = '127.0.0.1:8030',\n" + +" 'table.identifier' = 'db.test_sink',\n" + +" 'username' = 'root',\n" + +" 'password' = '',\n" + +/* doris stream load label, In the exactly-once scenario, + the label is globally unique and must be restarted from the latest checkpoint when restarting. + Exactly-once semantics can be turned off via sink.enable-2pc. */ +" 'sink.label-prefix' = 'doris_label',\n" + +" 'sink.properties.format' = 'json',\n" + //json data format +" 'sink.properties.read_json_by_line' = 'true'\n" + +")"); + +//insert into mysql table to doris table +tEnv.executeSql("INSERT INTO doris_test_sink select id,name from cdc_test_source"); +env.execute(); +} +} diff --git a/samples/doris-demo/flink-demo-v1.1/src/main/java/org/apache/doris/demo/flink/DorisFlinkConnectorDemoV1.java b/samples/doris-demo/flink-demo-v1.1/src/m
[GitHub] [doris] 924060929 merged pull request #11171: [Refactor](Nereids) Move analysis logic to dedicated class `NereidsAnalyzer`.
924060929 merged PR #11171: URL: https://github.com/apache/doris/pull/11171 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [Refactor](Nereids) Move analysis logic to dedicated class `NereidsAnalyzer`. (#11171)
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 e99f617b21 [Refactor](Nereids) Move analysis logic to dedicated class `NereidsAnalyzer`. (#11171) e99f617b21 is described below commit e99f617b219a53d367339c85b885a9618d753f96 Author: Shuo Wang AuthorDate: Tue Jul 26 18:32:07 2022 +0800 [Refactor](Nereids) Move analysis logic to dedicated class `NereidsAnalyzer`. (#11171) This PR proposes to move the analysis logic to the dedicated class NereidsAnalyzer, which has the following benefits: Unify the analysis logic in production and test files. Facilitate analyzing subquery plans within different scopes. --- .../org/apache/doris/nereids/NereidsPlanner.java | 27 ++-- .../doris/nereids/analyzer/NereidsAnalyzer.java| 79 ++ .../rules/rewrite/logical/ColumnPruningTest.java | 9 +-- .../rules/rewrite/logical/TestAnalyzer.java| 60 .../doris/nereids/ssb/SSBJoinReorderTest.java | 4 +- .../doris/nereids/tpch/AnalyzeCheckTestBase.java | 4 +- 6 files changed, 95 insertions(+), 88 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 2b32416680..c5427c943c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -21,17 +21,16 @@ import org.apache.doris.analysis.DescriptorTable; import org.apache.doris.analysis.StatementBase; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; +import org.apache.doris.nereids.analyzer.NereidsAnalyzer; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.glue.translator.PhysicalPlanTranslator; import org.apache.doris.nereids.glue.translator.PlanTranslatorContext; -import org.apache.doris.nereids.jobs.batch.AnalyzeRulesJob; import org.apache.doris.nereids.jobs.batch.DisassembleRulesJob; import org.apache.doris.nereids.jobs.batch.JoinReorderRulesJob; import org.apache.doris.nereids.jobs.batch.OptimizeRulesJob; import org.apache.doris.nereids.jobs.batch.PredicatePushDownRulesJob; import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; -import org.apache.doris.nereids.memo.Memo; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.Plan; @@ -99,29 +98,17 @@ public class NereidsPlanner extends Planner { // TODO: refactor, just demo code here public PhysicalPlan plan(LogicalPlan plan, PhysicalProperties outputProperties, ConnectContext connectContext) throws AnalysisException { -plannerContext = new Memo(plan) -.newPlannerContext(connectContext) +plannerContext = new NereidsAnalyzer(connectContext) +.analyzeWithPlannerContext(plan) +// TODO: revisit this. What is the appropriate time to set physical properties? Maybe before enter +// cascades style optimize phase. .setJobContext(outputProperties); -// Get plan directly. Just for SSB. -return doPlan(); -} -/** - * The actual execution of the plan, including the generation and execution of the job. - * @return PhysicalPlan. - */ -private PhysicalPlan doPlan() { -analyze(); rewrite(); optimize(); -return getRoot().extractPlan(); -} -/** - * Analyze: bind references according to metadata in the catalog, perform semantic analysis, etc. - */ -private void analyze() { -new AnalyzeRulesJob(plannerContext).execute(); +// Get plan directly. Just for SSB. +return getRoot().extractPlan(); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/NereidsAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/NereidsAnalyzer.java new file mode 100644 index 00..16daf6ec40 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/NereidsAnalyzer.java @@ -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 requir
[GitHub] [doris] wangshuo128 opened a new pull request, #11211: (Git)[Nereids] ignore ANTLR4 generated files and dir.
wangshuo128 opened a new pull request, #11211: URL: https://github.com/apache/doris/pull/11211 # Proposed changes This PR proposes the following file and dir auto-generated by ANTLR4: ``` fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.tokens fe/fe-core/src/main/antlr4/org/apache/doris/nereids/gen/ ``` ## Problem Summary: Describe the overview of changes. ## Checklist(Required) 1. Does it affect the original behavior: (Yes/No/I Don't know) 2. Has unit tests been added: (Yes/No/No Need) 3. Has document been added or modified: (Yes/No/No Need) 4. Does it need to update dependencies: (Yes/No) 5. Are there any changes that cannot be rolled back: (Yes/No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] starocean999 opened a new pull request, #11212: [fix] the nullable info is lost in ifnull expr
starocean999 opened a new pull request, #11212: URL: https://github.com/apache/doris/pull/11212 # Proposed changes Issue Number: close #xxx ## Problem Summary: The ifnull(col_left, col_right) is converted to if(isnull(col_left), col_right, col_left). the col_left's nullable info is lost during converting. ## Checklist(Required) 1. Does it affect the original behavior: (Yes/No/I Don't know) 2. Has unit tests been added: (Yes/No/No Need) 3. Has document been added or modified: (Yes/No/No Need) 4. Does it need to update dependencies: (Yes/No) 5. Are there any changes that cannot be rolled back: (Yes/No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yixiutt commented on a diff in pull request #11195: [feature-wip](unique-key-merge-on-write) update delete bitmap while p…
yixiutt commented on code in PR #11195: URL: https://github.com/apache/doris/pull/11195#discussion_r929817716 ## be/src/olap/tablet.h: ## @@ -438,6 +438,9 @@ inline void Tablet::set_cumulative_layer_point(int64_t new_point) { } inline bool Tablet::enable_unique_key_merge_on_write() const { +if (_tablet_meta == nullptr) { Review Comment: actually it's for test -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yixiutt commented on a diff in pull request #11195: [feature-wip](unique-key-merge-on-write) update delete bitmap while p…
yixiutt commented on code in PR #11195: URL: https://github.com/apache/doris/pull/11195#discussion_r929819900 ## be/src/olap/txn_manager.cpp: ## @@ -308,8 +309,108 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, _clear_txn_partition_map_unlocked(transaction_id, partition_id); } } +} +auto tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id); +#ifdef BE_TEST +if (tablet == nullptr) { +return Status::OK(); +} +#endif +// Check if have to build extra delete bitmap for table of UNIQUE_KEY model +if (!tablet->enable_unique_key_merge_on_write() || +tablet->tablet_meta()->preferred_rowset_type() != RowsetTypePB::BETA_ROWSET || +rowset_ptr->keys_type() != KeysType::UNIQUE_KEYS) { return Status::OK(); } +CHECK(version.first == version.second) << "impossible: " << version; + +// For each key in current set, check if it overwrites any previously +// written keys +OlapStopWatch watch; +std::vector segments; +std::vector pre_segments; +auto beta_rowset = reinterpret_cast(rowset_ptr.get()); +Status st = beta_rowset->load_segments(&segments); +if (!st.ok()) return st; +// lock tablet meta to modify delete bitmap +std::lock_guard meta_wrlock(tablet->get_header_lock()); +for (auto& seg : segments) { +seg->load_index(); // We need index blocks to iterate +auto pk_idx = seg->get_primary_key_index(); +int cnt = 0; +int total = pk_idx->num_rows(); +int32_t remaining = total; +bool exact_match = false; +std::string last_key; +int batch_size = 1024; +MemPool pool; +while (remaining > 0) { +std::unique_ptr iter; +RETURN_IF_ERROR(pk_idx->new_iterator(&iter)); + +size_t num_to_read = std::min(batch_size, remaining); +std::unique_ptr cvb; +RETURN_IF_ERROR(ColumnVectorBatch::create(num_to_read, false, pk_idx->type_info(), + nullptr, &cvb)); +ColumnBlock block(cvb.get(), &pool); +ColumnBlockView column_block_view(&block); +Slice last_key_slice(last_key); +RETURN_IF_ERROR(iter->seek_at_or_after(&last_key_slice, &exact_match)); + +size_t num_read = num_to_read; +RETURN_IF_ERROR(iter->next_batch(&num_read, &column_block_view)); +DCHECK(num_to_read == num_read); +last_key = (reinterpret_cast(cvb->cell_ptr(num_read - 1)))->to_string(); + +// exclude last_key, last_key will be read in next batch. +if (num_read == batch_size && num_read != remaining) { +num_read -= 1; +} +for (size_t i = 0; i < num_read; i++) { +const Slice* key = reinterpret_cast(cvb->cell_ptr(i)); +// first check if exist in pre segment +bool find = _check_pk_in_pre_segments(pre_segments, *key, tablet, version); +if (find) { +cnt++; Review Comment: fixed -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] 924060929 merged pull request #11192: [refactor] (Nereids) add equals for OrderKey
924060929 merged PR #11192: URL: https://github.com/apache/doris/pull/11192 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [refactor] (Nereids) add equals for OrderKey (#11192)
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 fcdb543e17 [refactor] (Nereids) add equals for OrderKey (#11192) fcdb543e17 is described below commit fcdb543e17c8376956094eda1b6e83ea8b7d9a75 Author: minghong AuthorDate: Tue Jul 26 18:55:27 2022 +0800 [refactor] (Nereids) add equals for OrderKey (#11192) LogicalSort.equals() method depends on OrderKey.equals(), which is not defined correctly. This pr defines OrderKey.equals() to enable correctly comparing LogicalSort. --- .../org/apache/doris/nereids/properties/OrderKey.java | 19 +++ 1 file changed, 19 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/OrderKey.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/OrderKey.java index 7d6ee6eb8d..8059b2ddfa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/OrderKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/OrderKey.java @@ -19,6 +19,8 @@ package org.apache.doris.nereids.properties; import org.apache.doris.nereids.trees.expressions.Expression; +import java.util.Objects; + /** * Represents the order key of a statement. */ @@ -67,4 +69,21 @@ public class OrderKey { public String toString() { return expr.toSql(); } + +@Override +public int hashCode() { +return Objects.hash(expr, isAsc, nullFirst); +} + +@Override +public boolean equals(Object o) { +if (this == o) { +return true; +} +if (o == null || getClass() != o.getClass()) { +return false; +} +OrderKey that = (OrderKey) o; +return isAsc == that.isAsc() && nullFirst == that.isNullFirst() && expr.equals(that.getExpr()); +} } - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] github-actions[bot] commented on pull request #11204: [fix](be): fix stack overflow in unhex function
github-actions[bot] commented on PR #11204: URL: https://github.com/apache/doris/pull/11204#issuecomment-1195341320 PR approved by at least one committer and no changes requested. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] github-actions[bot] commented on pull request #11204: [fix](be): fix stack overflow in unhex function
github-actions[bot] commented on PR #11204: URL: https://github.com/apache/doris/pull/11204#issuecomment-1195341351 PR approved by anyone and no changes requested. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] zhannngchen commented on a diff in pull request #11195: [feature-wip](unique-key-merge-on-write) update delete bitmap while p…
zhannngchen commented on code in PR #11195: URL: https://github.com/apache/doris/pull/11195#discussion_r929844943 ## be/src/olap/tablet.h: ## @@ -438,6 +438,9 @@ inline void Tablet::set_cumulative_layer_point(int64_t new_point) { } inline bool Tablet::enable_unique_key_merge_on_write() const { +if (_tablet_meta == nullptr) { Review Comment: ok -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei closed issue #11199: [Bug] Be core while passing constant array type arguments to some array functions
yiguolei closed issue #11199: [Bug] Be core while passing constant array type arguments to some array functions URL: https://github.com/apache/doris/issues/11199 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yiguolei merged pull request #11200: [Bug] Fix Be core while passing constant array type arguments to some array functions
yiguolei merged PR #11200: URL: https://github.com/apache/doris/pull/11200 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [bugfix]fix constant argument coredump (#11200)
This is an automated email from the ASF dual-hosted git repository. yiguolei 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 2e210fb363 [bugfix]fix constant argument coredump (#11200) 2e210fb363 is described below commit 2e210fb36350b1c48598b4755807c545e78626b6 Author: xy720 <22125576+xy...@users.noreply.github.com> AuthorDate: Tue Jul 26 19:30:06 2022 +0800 [bugfix]fix constant argument coredump (#11200) --- .../apache/doris/analysis/FunctionCallExpr.java| 12 .../java/org/apache/doris/catalog/ArrayType.java | 4 ++ .../test_array_functions_by_literal.out| 81 ++ .../test_array_functions_by_literal.groovy | 37 ++ 4 files changed, 134 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java index d05c7adfc0..8407726208 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java @@ -1102,6 +1102,18 @@ public class FunctionCallExpr extends Expr { this.type = new ArrayType(children.get(0).getType()); } } + +if (this.type instanceof ArrayType) { +ArrayType arrayType = (ArrayType) type; +boolean containsNull = false; +for (Expr child : children) { +Type childType = child.getType(); +if (childType instanceof ArrayType) { +containsNull |= ((ArrayType) childType).getContainsNull(); +} +} +arrayType.setContainsNull(containsNull); +} } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ArrayType.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ArrayType.java index 26f4aa0aec..a2a9092b10 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ArrayType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ArrayType.java @@ -62,6 +62,10 @@ public class ArrayType extends Type { return containsNull; } +public void setContainsNull(boolean containsNull) { +this.containsNull = containsNull; +} + @Override public PrimitiveType getPrimitiveType() { return PrimitiveType.ARRAY; diff --git a/regression-test/data/query/sql_functions/array_functions/test_array_functions_by_literal.out b/regression-test/data/query/sql_functions/array_functions/test_array_functions_by_literal.out index 0a37764fb7..406ff361fa 100644 --- a/regression-test/data/query/sql_functions/array_functions/test_array_functions_by_literal.out +++ b/regression-test/data/query/sql_functions/array_functions/test_array_functions_by_literal.out @@ -109,3 +109,84 @@ false -- !sql -- 3 + +-- !sql -- +2 + +-- !sql -- +6 + +-- !sql -- +1 + +-- !sql -- +3 + +-- !sql -- +[1, 2, 3] + +-- !sql -- +[1, 2, 3, NULL] + +-- !sql -- +['a'] + +-- !sql -- +['a', NULL] + +-- !sql -- +[2, 3] + +-- !sql -- +[2, 3, NULL] + +-- !sql -- +['b', 'c'] + +-- !sql -- +['b', 'c', NULL] + +-- !sql -- +[1, 2, 3] + +-- !sql -- +[1, 2, 3] + +-- !sql -- +[NULL, 1, 2, 3] + +-- !sql -- +[NULL, 1, 2, 3] + +-- !sql -- +['a', 'b', 'c'] + +-- !sql -- +['a', 'b', 'c'] + +-- !sql -- +false + +-- !sql -- +true + +-- !sql -- +\N + +-- !sql -- +[1, 2, 3, 4] + +-- !sql -- +[1] + +-- !sql -- +[2, 3] + +-- !sql -- +[1, 2, 3, 4, NULL] + +-- !sql -- +[1] + +-- !sql -- +[2, 3] diff --git a/regression-test/suites/query/sql_functions/array_functions/test_array_functions_by_literal.groovy b/regression-test/suites/query/sql_functions/array_functions/test_array_functions_by_literal.groovy index 2fbc570e82..a7b73fd5e7 100644 --- a/regression-test/suites/query/sql_functions/array_functions/test_array_functions_by_literal.groovy +++ b/regression-test/suites/query/sql_functions/array_functions/test_array_functions_by_literal.groovy @@ -65,4 +65,41 @@ suite("test_array_functions_by_literal", "all") { qt_sql "select array_sum([1,2,3])" qt_sql "select array_min([1,2,3])" qt_sql "select array_max([1,2,3])" +qt_sql "select array_avg([1,2,3,null])" +qt_sql "select array_sum([1,2,3,null])" +qt_sql "select array_min([1,2,3,null])" +qt_sql "select array_max([1,2,3,null])" + +// array_distinct function +qt_sql "select array_distinct([1,1,2,2,3,3])" +qt_sql "select array_distinct([1,1,2,2,3,3,null])" +qt_sql "select array_distinct(['a','a','a'])" +qt_sql "select array_distinct(['a','a','a',null])" + +// array_remove function +qt_sql "select array_remove([1,2,3], 1)" +qt_sql "select array_remove([1,2,3,null], 1)" +qt_sql "select array_remove(['a','b','c'], 'a')" +qt_sql "select array_remove(['a','b','c',null], 'a')" + +// array_sort function +qt_sql "s
[doris] branch dev-1.1.2 created (now 2dbd70bf92)
This is an automated email from the ASF dual-hosted git repository. yiguolei pushed a change to branch dev-1.1.2 in repository https://gitbox.apache.org/repos/asf/doris.git at 2dbd70bf92 1.1.1 rc03 (#11161) No new revisions were added by this update. - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] carlvinhust2012 opened a new pull request, #11213: [feature-wip](array-type) add the array_reverse function
carlvinhust2012 opened a new pull request, #11213: URL: https://github.com/apache/doris/pull/11213 # Proposed changes 1. this pr is used to add the array_reverse function. Issue Number: close https://github.com/apache/doris/issues/10052 2.we can run this function in sql as follow: mysql> set enable_array_type=true; mysql> set enable_vectorized_engine=true; mysql> select k1, k2, array_reverse(k2) from array_test; +--+-+-+ | k1 | k2 | array_reverse(`k2`) | +--+-+-+ | 1 | [1, 2, 3, 4, 5] | [5, 4, 3, 2, 1] | | 2 | [6, 7, 8] | [8, 7, 6] | | 3 | [] | [] | | 4 | NULL| NULL| | 5 | [1, 2, 3, 4, 5, 4, 3, 2, 1] | [1, 2, 3, 4, 5, 4, 3, 2, 1] | | 6 | [1, 2, 3, NULL] | [NULL, 3, 2, 1] | | 7 | [4, 5, 6, NULL, NULL] | [NULL, NULL, 6, 5, 4] | +--+-+-+ mysql> select k1, k2, array_reverse(k2) from array_test01; +--+---+---+ | k1 | k2| array_reverse(`k2`) | +--+---+---+ | 1 | ['a', 'b', 'c', 'd'] | ['d', 'c', 'b', 'a'] | | 2 | ['e', 'f', 'g', 'h'] | ['h', 'g', 'f', 'e'] | | 3 | [NULL, 'a', NULL, 'b', NULL, 'c'] | ['c', NULL, 'b', NULL, 'a', NULL] | | 4 | ['d', 'e', NULL, ' '] | [' ', NULL, 'e', 'd'] | | 5 | [' ', NULL, 'f', 'g'] | ['g', 'f', NULL, ' '] | +--+---+---+ ## Problem Summary: Describe the overview of changes. ## Checklist(Required) 1. Does it affect the original behavior: (No) 2. Has unit tests been added: (No) 3. Has document been added or modified: (No) 4. Does it need to update dependencies: (No) 5. Are there any changes that cannot be rolled back: (No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] morrySnow commented on a diff in pull request #11035: [Feature](nereids) support sub-query and alias for TPC-H
morrySnow commented on code in PR #11035: URL: https://github.com/apache/doris/pull/11035#discussion_r929852592 ## fe/fe-core/pom.xml: ## @@ -668,6 +668,12 @@ under the License. maven-compiler-plugin 3.10.1 + Review Comment: revert these changes ## docs/zh-CN/developer/developer-guide/fe-idea-dev.md: ## @@ -32,6 +32,15 @@ under the License. 安装 JDK1.8+ ,使用 IntelliJ IDEA 打开 FE. + MacOS下载 Review Comment: revert these changes ## fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java: ## @@ -131,11 +172,16 @@ private Plan groupToTreeNode(Group group) { */ private GroupExpression insertOrRewriteGroupExpression(GroupExpression groupExpression, Group target, boolean rewrite, LogicalProperties logicalProperties) { -GroupExpression existedGroupExpression = groupExpressions.get(groupExpression); -if (existedGroupExpression != null) { +GroupExpressionAdapter adapter = new GroupExpressionAdapter(groupExpression); +// GroupExpression existedGroupExpression = groupExpressions.get(groupExpression); Review Comment: remove useless code directly ## fe/fe-core/src/test/java/org/apache/doris/nereids/util/AnalyzeSubQueryTest.java: ## @@ -0,0 +1,261 @@ +// 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.common.AnalysisException; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.PlannerContext; +import org.apache.doris.nereids.analyzer.Unbound; +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.jobs.batch.FinalizeAnalyzeJob; +import org.apache.doris.nereids.jobs.rewrite.RewriteBottomUpJob; +import org.apache.doris.nereids.memo.Group; +import org.apache.doris.nereids.memo.Memo; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.RuleFactory; +import org.apache.doris.nereids.rules.analysis.BindFunction; +import org.apache.doris.nereids.rules.analysis.BindRelation; +import org.apache.doris.nereids.rules.analysis.BindSlotReference; +import org.apache.doris.nereids.rules.analysis.BindSubQueryAlias; +import org.apache.doris.nereids.rules.analysis.ProjectToGlobalAggregate; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.utframe.TestWithFeService; + +import com.google.common.collect.Lists; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +public class AnalyzeSubQueryTest extends TestWithFeService { +private final NereidsParser parser = new NereidsParser(); + +private final List testSql = Lists.newArrayList( +"SELECT * FROM T1", +"SELECT * FROM T1 ORDER BY ID", +"SELECT * FROM T1 JOIN T2 ON T1.ID = T2.ID", +"SELECT * FROM T1 T", +"SELECT T.ID FROM T1 T", +"SELECT * FROM (SELECT * FROM T1 T) T2", +"SELECT T1.ID ID FROM T1", +"SELECT T.ID FROM T1 T", +"SELECT A.ID, B.SCORE FROM T1 A, T2 B WHERE A.ID = B.ID GROUP BY A.ID ORDER BY A.ID", +"SELECT X.ID FROM (SELECT * FROM T1 A JOIN (SELECT ID ID1 FROM T1) AS B ON A.ID = B.ID1) X WHERE X.SCORE < 20", +"SELECT X.ID + X.SCORE FROM (SELECT * FROM T1 A JOIN (SELECT SUM(ID + 1) ID1 FROM T1 T GROUP BY ID) AS B ON A.ID = B.ID1 ORDER BY A.ID DESC) X WHERE X.ID - X.SCORE < 20" +); + +@Override +protected void runBeforeAll() throws Exception { +createDatabase("test"); +connectContext.setDatabase("default_cluster:test"); + +createTables( +"CREATE TABLE IF NOT EXISTS T1 (\n" ++ "id bigint,\n" ++ "score bigint\n" ++ ")\n" ++ "DUPLICATE KEY(id)\n" ++ "D
[GitHub] [doris] morrySnow commented on pull request #11181: [Regression] Add tpch plan check
morrySnow commented on PR #11181: URL: https://github.com/apache/doris/pull/11181#issuecomment-1195381987 please fix license -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] morrySnow commented on pull request #11206: [refactor](Nereids) ExprId count from 1 in each query
morrySnow commented on PR #11206: URL: https://github.com/apache/doris/pull/11206#issuecomment-1195385002 because nereids generate exprid for alias in parser. and we could do multi statement parser. so count expid from 1 for each query is impossible. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] morrySnow closed pull request #11206: [refactor](Nereids) ExprId count from 1 in each query
morrySnow closed pull request #11206: [refactor](Nereids) ExprId count from 1 in each query URL: https://github.com/apache/doris/pull/11206 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] zhannngchen commented on pull request #11205: [feature-wip](unique-key-merge-on-write) remove AggType on unique table with MoW, enable preAggreation, DSIP-018[5/2]
zhannngchen commented on PR #11205: URL: https://github.com/apache/doris/pull/11205#issuecomment-1195385680 @EmmyMiao87 can you help to review the code change of Planer? Basically, when user enabled unique key merge-on-write, we don't need to merge the data with same keys to ensure which row is latest, old row is marked as deleted in DeleteBitmap. So we can enable pre-aggregation to accelerate the query execution, just like processing data in duplicate keys table. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] ccoffline opened a new issue, #11214: [Enhancement] add more metrics
ccoffline opened a new issue, #11214: URL: https://github.com/apache/doris/issues/11214 ### Search before asking - [X] I had searched in the [issues](https://github.com/apache/incubator-doris/issues?q=is%3Aissue) and found no similar issues. ### Description There are dozens of Doris metrics are used in Meituan alone but can be open-sourced. ### Solution * transaction metrics(count, latency, publish) * query rpc metrics(count, latency, size, drill down to each BE) * query metrics(drill down to each DB) * thrift rpc metrics(drill down to each BE) One concern is that when there are too many DB/BE, there may be a huge cost on these metrics. We may add switch configs on these metrics. ### Are you willing to submit PR? - [X] Yes I am willing to submit a PR! ### Code of Conduct - [X] I agree to follow this project's [Code of Conduct](https://www.apache.org/foundation/policies/conduct) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch dev-1.1.2 updated (2dbd70bf92 -> 1cde8cd819)
This is an automated email from the ASF dual-hosted git repository. yiguolei pushed a change to branch dev-1.1.2 in repository https://gitbox.apache.org/repos/asf/doris.git from 2dbd70bf92 1.1.1 rc03 (#11161) new 3fc0ba91dc [bug]string pad functions should always be nullable (#11140) new 1cde8cd819 [FIX]string pad function should be always nullable for both string and varchar type (#11196) The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: gensrc/script/doris_builtins_functions.py | 4 +-- ..._timestamp.out => test_string_pad_function.out} | 10 +++--- .../test_string_pad_function.groovy} | 37 +++--- 3 files changed, 27 insertions(+), 24 deletions(-) copy regression-test/data/correctness/{test_current_timestamp.out => test_string_pad_function.out} (63%) copy regression-test/suites/{performance/redundant_conjuncts.groovy => correctness/test_string_pad_function.groovy} (57%) - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] 01/02: [bug]string pad functions should always be nullable (#11140)
This is an automated email from the ASF dual-hosted git repository. yiguolei pushed a commit to branch dev-1.1.2 in repository https://gitbox.apache.org/repos/asf/doris.git commit 3fc0ba91dcfb5af771325a7c4ee79474a838ea9f Author: starocean999 <40539150+starocean...@users.noreply.github.com> AuthorDate: Tue Jul 26 10:20:11 2022 +0800 [bug]string pad functions should always be nullable (#11140) * string pad functions should always be nullable --- .../data/correctness/test_string_pad_function.out | 9 + .../correctness/test_string_pad_function.groovy| 46 ++ 2 files changed, 55 insertions(+) diff --git a/regression-test/data/correctness/test_string_pad_function.out b/regression-test/data/correctness/test_string_pad_function.out new file mode 100644 index 00..5fc247c7d7 --- /dev/null +++ b/regression-test/data/correctness/test_string_pad_function.out @@ -0,0 +1,9 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_lpad -- +\N +10:00 + +-- !select_rpad -- +\N +10:00 + diff --git a/regression-test/suites/correctness/test_string_pad_function.groovy b/regression-test/suites/correctness/test_string_pad_function.groovy new file mode 100644 index 00..929a71dfa5 --- /dev/null +++ b/regression-test/suites/correctness/test_string_pad_function.groovy @@ -0,0 +1,46 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_string_pad_function") { +sql """ +drop table if exists table_pad; +""" + +sql """ +create table table_pad ( +a int not null, +b varchar(10) not null +) +ENGINE=OLAP +distributed by hash(a) +properties( +'replication_num' = '1' +); +""" + +sql """ +insert into table_pad values(1,'10'), (2,'20'); +""" + +qt_select_lpad """ +select CASE WHEN table_pad.a = 1 THEN CONCAT(LPAD(b, 2, 0), ':00') END result from table_pad order by result; +""" + +qt_select_rpad """ +select CASE WHEN table_pad.a = 1 THEN CONCAT(RPAD(b, 2, 0), ':00') END result from table_pad order by result; +""" +} - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] 02/02: [FIX]string pad function should be always nullable for both string and varchar type (#11196)
This is an automated email from the ASF dual-hosted git repository. yiguolei pushed a commit to branch dev-1.1.2 in repository https://gitbox.apache.org/repos/asf/doris.git commit 1cde8cd81985f4a519541c76821aa95d2d98b5a2 Author: starocean999 <40539150+starocean...@users.noreply.github.com> AuthorDate: Tue Jul 26 17:55:06 2022 +0800 [FIX]string pad function should be always nullable for both string and varchar type (#11196) --- gensrc/script/doris_builtins_functions.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 0a3d138b63..2c44b50737 100755 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -976,10 +976,10 @@ visible_functions = [ '15FunctionContextERKNS1_9StringValERKNS1_6IntValE', '', '', 'vec', ''], [['lpad'], 'STRING', ['STRING', 'INT', 'STRING'], '_ZN5doris15StringFunctions4lpadEPN9doris_udf' -'15FunctionContextERKNS1_9StringValERKNS1_6IntValES6_', '', '', 'vec', ''], +'15FunctionContextERKNS1_9StringValERKNS1_6IntValES6_', '', '', 'vec', 'ALWAYS_NULLABLE'], [['rpad'], 'STRING', ['STRING', 'INT', 'STRING'], '_ZN5doris15StringFunctions4rpadEPN9doris_udf' -'15FunctionContextERKNS1_9StringValERKNS1_6IntValES6_', '', '', 'vec', ''], +'15FunctionContextERKNS1_9StringValERKNS1_6IntValES6_', '', '', 'vec', 'ALWAYS_NULLABLE'], [['append_trailing_char_if_absent'], 'STRING', ['STRING', 'STRING'], '_ZN5doris15StringFunctions30append_trailing_char_if_absentEPN9doris_udf15FunctionContextERKNS1_9StringValES6_', '', '', 'vec', 'ALWAYS_NULLABLE'], - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] BiteTheDDDDt commented on pull request #11212: [fix] the nullable info is lost in ifnull expr
BiteThet commented on PR #11212: URL: https://github.com/apache/doris/pull/11212#issuecomment-1195392540 Do you have any case to repeat the error? ```java if (children.get(0).isNullable()) { return children.get(1).isNullable(); } ``` The nullable here needs to be consistent with `FE`, I think we should do more change at `ifnull`. This is the modification of get_return_type_impl, we also need to implement similar logic in execute_impl  -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] carlvinhust2012 closed pull request #11213: [feature-wip](array-type) add the array_reverse function
carlvinhust2012 closed pull request #11213: [feature-wip](array-type) add the array_reverse function URL: https://github.com/apache/doris/pull/11213 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] Kikyou1997 commented on a diff in pull request #11179: [feature](nereids) Add stats derive framework for new optimizer
Kikyou1997 commented on code in PR #11179: URL: https://github.com/apache/doris/pull/11179#discussion_r929887561 ## fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java: ## @@ -0,0 +1,191 @@ +// 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.stats; + +import org.apache.doris.nereids.memo.Group; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.IntegerLiteral; +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.functions.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.Sum; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.JoinType; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.statistics.ColumnStats; +import org.apache.doris.statistics.StatsDeriveResult; + +import com.google.common.base.Supplier; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class StatsCalculatorTest { Review Comment: done -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] github-actions[bot] commented on pull request #11207: [enhancement](nereids) Fix the wrong order of arguments of Assert.assertEquals
github-actions[bot] commented on PR #11207: URL: https://github.com/apache/doris/pull/11207#issuecomment-1195406256 PR approved by anyone and no changes requested. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] morningman opened a new pull request, #11215: [fix](ut) fix FE ut
morningman opened a new pull request, #11215: URL: https://github.com/apache/doris/pull/11215 # Proposed changes Issue Number: close #xxx ## Problem Summary: Describe the overview of changes. ## Checklist(Required) 1. Does it affect the original behavior: (Yes/No/I Don't know) 2. Has unit tests been added: (Yes/No/No Need) 3. Has document been added or modified: (Yes/No/No Need) 4. Does it need to update dependencies: (Yes/No) 5. Are there any changes that cannot be rolled back: (Yes/No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] HappenLee opened a new issue, #11216: [Bug] [ColumnDictionary] Coredump in column dictionary insert data from DefaultValueColumnIterator
HappenLee opened a new issue, #11216: URL: https://github.com/apache/doris/issues/11216 ### Search before asking - [X] I had searched in the [issues](https://github.com/apache/incubator-doris/issues?q=is%3Aissue) and found no similar issues. ### Version master ### What's Wrong? #0 0x7f52487938a5 in raise () from /lib64/libc.so.6 #1 0x7f5248795085 in abort () from /lib64/libc.so.6 #2 0x7f524abf4f0c in google::DumpStackTraceAndExit () at src/utilities.cc:160 #3 0x7f524d06dd6d in google::LogMessage::Fail () at src/logging.cc:1650 #4 0x7f524d0702a9 in google::LogMessage::SendToLog (this=0x7f52181abd10) at src/logging.cc:1607 #5 0x7f524d06d8d6 in google::LogMessage::Flush (this=0x7f52181abd10) at src/logging.cc:1476 #6 0x7f524d070919 in google::LogMessageFatal::~LogMessageFatal (this=this@entry=0x7f52181abd10, __in_chrg=) at src/logging.cc:2226 #7 0x7f524b15b560 in doris::vectorized::ColumnDictionary::insert_data (this=, pos=) at /home/disk6/palo/release/baidu/bdg/doris/palo-1.1/be/src/vec/columns/column_dictionary.h:100 #8 0x7f524b8f144e in doris::vectorized::ColumnNullable::insert_data (this=0x7f55413354c0, pos=, length=) ### What You Expected? run normally ### How to Reproduce? _No response_ ### Anything Else? _No response_ ### Are you willing to submit PR? - [X] Yes I am willing to submit a PR! ### Code of Conduct - [X] I agree to follow this project's [Code of Conduct](https://www.apache.org/foundation/policies/conduct) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] HappenLee opened a new pull request, #11217: [Fix] Coredump in column dictionary insert data from DefaultValueColumnIterator
HappenLee opened a new pull request, #11217: URL: https://github.com/apache/doris/pull/11217 # Proposed changes Issue Number: close #11216 ## Problem Summary: Describe the overview of changes. ## Checklist(Required) 1. Does it affect the original behavior: (Yes/No/I Don't know) 2. Has unit tests been added: (Yes/No/No Need) 3. Has document been added or modified: (Yes/No/No Need) 4. Does it need to update dependencies: (Yes/No) 5. Are there any changes that cannot be rolled back: (Yes/No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[doris] branch master updated: [enhancement](nereids) Fix the wrong order of arguments of Assert.assertEquals (#11207)
This is an automated email from the ASF dual-hosted git repository. dataroaring 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 8a6c9d870e [enhancement](nereids) Fix the wrong order of arguments of Assert.assertEquals (#11207) 8a6c9d870e is described below commit 8a6c9d870ed3879b3221edd28a00692ec3bca5fb Author: Adonis Ling AuthorDate: Tue Jul 26 20:33:32 2022 +0800 [enhancement](nereids) Fix the wrong order of arguments of Assert.assertEquals (#11207) --- .../doris/nereids/jobs/RewriteTopDownJobTest.java | 16 - .../pattern/GroupExpressionMatchingTest.java | 6 ++-- .../rules/exploration/join/JoinCommuteTest.java| 4 +-- .../rewrite/logical/PushDownPredicateTest.java | 20 +-- .../doris/nereids/util/ExpressionUtilsTest.java| 40 +++--- 5 files changed, 43 insertions(+), 43 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/RewriteTopDownJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/RewriteTopDownJobTest.java index e28a508084..3d13777cc9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/RewriteTopDownJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/RewriteTopDownJobTest.java @@ -81,19 +81,19 @@ public class RewriteTopDownJobTest { Assertions.assertEquals(1, rootGroup.getLogicalExpressions().size()); GroupExpression rootGroupExpression = rootGroup.getLogicalExpression(); List output = rootGroup.getLogicalProperties().getOutput(); -Assertions.assertEquals(output.size(), 1); -Assertions.assertEquals(output.get(0).getName(), "name"); -Assertions.assertEquals(output.get(0).getDataType(), StringType.INSTANCE); +Assertions.assertEquals(1, output.size()); +Assertions.assertEquals("name", output.get(0).getName()); +Assertions.assertEquals(StringType.INSTANCE, output.get(0).getDataType()); Assertions.assertEquals(1, rootGroupExpression.children().size()); Assertions.assertEquals(PlanType.LOGICAL_PROJECT, rootGroupExpression.getPlan().getType()); Group leafGroup = rootGroupExpression.child(0); output = leafGroup.getLogicalProperties().getOutput(); -Assertions.assertEquals(output.size(), 2); -Assertions.assertEquals(output.get(0).getName(), "id"); -Assertions.assertEquals(output.get(0).getDataType(), IntegerType.INSTANCE); -Assertions.assertEquals(output.get(1).getName(), "name"); -Assertions.assertEquals(output.get(1).getDataType(), StringType.INSTANCE); +Assertions.assertEquals(2, output.size()); +Assertions.assertEquals("id", output.get(0).getName()); +Assertions.assertEquals(IntegerType.INSTANCE, output.get(0).getDataType()); +Assertions.assertEquals("name", output.get(1).getName()); +Assertions.assertEquals(StringType.INSTANCE, output.get(1).getDataType()); Assertions.assertEquals(1, leafGroup.getLogicalExpressions().size()); GroupExpression leafGroupExpression = leafGroup.getLogicalExpression(); Assertions.assertEquals(PlanType.LOGICAL_BOUND_RELATION, leafGroupExpression.getPlan().getType()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/pattern/GroupExpressionMatchingTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/pattern/GroupExpressionMatchingTest.java index 7c2a2b8b34..d4f9c333b2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/pattern/GroupExpressionMatchingTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/pattern/GroupExpressionMatchingTest.java @@ -265,7 +265,7 @@ public class GroupExpressionMatchingTest { case LEFT_OUTER_JOIN: Assertions.assertTrue(context.parent instanceof LogicalJoin); LogicalJoin parent = (LogicalJoin) context.parent; -Assertions.assertEquals(parent.getJoinType(), JoinType.INNER_JOIN); +Assertions.assertEquals(JoinType.INNER_JOIN, parent.getJoinType()); break; default: notExpectedPlan(join, context); @@ -287,13 +287,13 @@ public class GroupExpressionMatchingTest { case "b": { Assertions.assertTrue(context.parent instanceof LogicalJoin); LogicalJoin parent = (LogicalJoin) context.parent; -Assertions.assertEquals(parent.getJoinType(), JoinType.LEFT_OUTER_JOIN); +Assertions.assertEquals(JoinType.LEFT_OUTER_JOIN, parent.getJoinType()); break; } case "c": { Assertions.assertTrue(context.parent instanceof LogicalJoin); LogicalJoin parent = (LogicalJoin) context.
[GitHub] [doris] dataroaring merged pull request #11207: [enhancement](nereids) Fix the wrong order of arguments of Assert.assertEquals
dataroaring merged PR #11207: URL: https://github.com/apache/doris/pull/11207 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] BiteTheDDDDt closed pull request #11130: [Enhancement][Thirdparty] upgrade phmap to 1.35
BiteThet closed pull request #11130: [Enhancement][Thirdparty] upgrade phmap to 1.35 URL: https://github.com/apache/doris/pull/11130 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org
[GitHub] [doris] yangzhg opened a new pull request, #11218: [WIP][feature] support `create`,`alter`,`refresh`,`drop` stmt for multi table materialized view
yangzhg opened a new pull request, #11218: URL: https://github.com/apache/doris/pull/11218 # Proposed changes Issue Number: #7503 ## Problem Summary: support `create`,`alter`,`refresh`,`drop` stmt for multi table materialized view ### create ```sql create materialized view mv_name build [immediate | deferred] refresh [fast | complete | never]. on [commit | demand] start with start_time next interval PARTITION BY [range|list] DISTRIBUTED BY hash(cols..) BUCKETS 16 PROPERTIES() as select ...; ``` ### DESC ```sql DESCRIBE test_mv_view; ``` ### SHOW CREATE ```sql SHOW CREATE MATERIALIZED VIEW test_mv_view; ``` ### DROP ```sql DROP MATERIALIZED VIEW test_mv_view; ``` ### ALTER ```sql alter materialized view mv1 refresh [fast | complete | never] on [commit | demand] start with start_time next interval ``` ### refresh ```sql REFRESH MATERIALIZED VIEW test_mv_view [complete]; ``` ## Checklist(Required) 1. Does it affect the original behavior: (Yes/No/I Don't know) 2. Has unit tests been added: (Yes/No/No Need) 3. Has document been added or modified: (Yes/No/No Need) 4. Does it need to update dependencies: (Yes/No) 5. Are there any changes that cannot be rolled back: (Yes/No) ## Further comments If this is a relatively large or complex change, kick off the discussion at [d...@doris.apache.org](mailto:d...@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org