Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

planner: Add HashJoin<-Receiver specific physicalPlan column pruner #38536

Merged
merged 23 commits into from Nov 2, 2022
Merged

planner: Add HashJoin<-Receiver specific physicalPlan column pruner #38536

merged 23 commits into from Nov 2, 2022

Conversation

yibin87
Copy link
Contributor

@yibin87 yibin87 commented Oct 19, 2022

Signed-off-by: yibin huyibin@pingcap.com

What problem does this PR solve?

Issue Number: close #38511

Problem Summary:
In some situations, current ColumnPrune optimizer for logical plan can't eliminate useless columns in exchange sender/receivers whose parent node's HashJoin. And such useless columns consume significant CPU utilities and network bandwith.
For excample, a HashJoin node may has following plan, the ExchangeSender/Receiver has useless cols: col1, col2:
// HashJoin[col0, col3; col0==col3] <- ExchangeReceiver[col0, col1, col2] <- ExchangeSender[col0, col1, col2] <- Selection[col0, col1, col2; col1 < col2] <- TableScan[col0, col1, col2]
//<- ExchangeReceiver1[col3] <- ExchangeSender1[col3] <- TableScan1[col3]

Although it is hard to figure out the exact root cause of these cases, it is easy to add a simple physical column pruner to eliminate useless columns.
In this PR, choose to add a physical projection under PhysicalExchangeSender to do the column pruning. It is considered safer than directly pruning the useless columns. And it costs a little, because the projection will be executed in memory.
And it can be considered a double check even if the root cause is completed fixed in future.

What is changed and how it works?

Check List

Tests

  • Unit test
  • Integration test
  • Manual test (add detailed scripts or steps below)
    TPCH_100, before and after this PR, query results match.
  • No code

Side effects

  • Performance regression: Consumes more CPU
  • Performance regression: Consumes more Memory
  • Breaking backward compatibility

Documentation

  • Affects user behaviors
  • Contains syntax changes
  • Contains variable changes
  • Contains experimental features
  • Changes MySQL compatibility

Release note

Please refer to Release Notes Language Style Guide to write a quality release note.

None

Signed-off-by: yibin <huyibin@pingcap.com>
@ti-chi-bot
Copy link
Member

ti-chi-bot commented Oct 19, 2022

[REVIEW NOTIFICATION]

This pull request has been approved by:

  • winoros
  • xuyifangreeneyes

To complete the pull request process, please ask the reviewers in the list to review by filling /cc @reviewer in the comment.
After your PR has acquired the required number of LGTMs, you can assign this pull request to the committer in the list by filling /assign @committer in the comment to help you merge this pull request.

The full list of commands accepted by this bot can be found here.

Reviewer can indicate their review by submitting an approval review.
Reviewer can cancel approval by submitting a request changes review.

@ti-chi-bot ti-chi-bot added release-note-none size/L Denotes a PR that changes 100-499 lines, ignoring generated files. labels Oct 19, 2022
@yibin87
Copy link
Contributor Author

yibin87 commented Oct 19, 2022

/cc @windtalker @winoros

@yibin87
Copy link
Contributor Author

yibin87 commented Oct 19, 2022

/cc @chrysan

@yibin87
Copy link
Contributor Author

yibin87 commented Oct 19, 2022

/rebuild

@yibin87
Copy link
Contributor Author

yibin87 commented Oct 19, 2022

/re-build

Copy link
Member

@winoros winoros left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, you need to change the struct from

type PhysicalExchangeSender struct {
	basePhysicalPlan

	...
}

to

type PhysicalExchangeSender struct {
	physicalSchemaProducer

	...
}

The first one means that it doesn't hold its own schema(output columns), just uses the child's.

planner/core/optimizer.go Outdated Show resolved Hide resolved
planner/core/optimizer.go Outdated Show resolved Hide resolved
@winoros
Copy link
Member

winoros commented Oct 24, 2022

And the possible place that we would meet the same issue is the place where the parent doesn't need the full schema of the child.
When the child is a scan, selection, or top-n. When the father is a join/limit/agg.

A clear way may be to pass the parent col to the physical property. And check the parent col when you create a sender.

The above solution is based on the fact that we don't project the operator's columns which are not needed by their parent(A inline projection inside the operator).

If we support the inline project inside the operator. We don't need these things.

Signed-off-by: yibin <huyibin@pingcap.com>
Signed-off-by: yibin <huyibin@pingcap.com>
@yibin87 yibin87 requested review from winoros and removed request for windtalker and chrysan October 28, 2022 08:47
Comment on lines 18 to 19
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/parser/ast"
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will fail the static check

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

planner/core/optimizer.go Show resolved Hide resolved
Signed-off-by: yibin <huyibin@pingcap.com>
@yibin87 yibin87 requested a review from winoros October 28, 2022 09:56
@yibin87
Copy link
Contributor Author

yibin87 commented Oct 31, 2022

/cc @Yisaer

@yibin87
Copy link
Contributor Author

yibin87 commented Oct 31, 2022

/run-unit-tests

@ti-chi-bot ti-chi-bot added the size/XL Denotes a PR that changes 500-999 lines, ignoring generated files. label Oct 31, 2022
Signed-off-by: yibin <huyibin@pingcap.com>
Signed-off-by: yibin <huyibin@pingcap.com>
Signed-off-by: yibin <huyibin@pingcap.com>
Signed-off-by: yibin <huyibin@pingcap.com>
Signed-off-by: yibin <huyibin@pingcap.com>
Signed-off-by: yibin <huyibin@pingcap.com>
Signed-off-by: yibin <huyibin@pingcap.com>
@yibin87
Copy link
Contributor Author

yibin87 commented Nov 1, 2022

/run-unit-tests

Signed-off-by: yibin <huyibin@pingcap.com>
@yibin87
Copy link
Contributor Author

yibin87 commented Nov 1, 2022

/run-unit-test

@yibin87 yibin87 requested review from xuyifangreeneyes and removed request for winoros and Yisaer November 1, 2022 08:18
Copy link
Contributor

@xuyifangreeneyes xuyifangreeneyes left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is possible to add a FT for the case?(check the result of explain select ...) rest LGTM

for i, mppCol := range sender.HashCols {
exprCols[i] = mppCol.Col
}
exprUsed := expression.GetUsedList(exprCols, sender.Schema())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe hashCols and hashUsed are more clear than exprCols and exprUsed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice suggestion, done.

ch := sender.children[0]
proj := PhysicalProjection{
Exprs: usedExprs,
}.Init(sctx, ch.statsInfo(), 0)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to set selectBlockOffset here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Set to child's selectBlockOffset now.

@ti-chi-bot ti-chi-bot added the status/LGT1 Indicates that a PR has LGTM 1. label Nov 1, 2022
@ti-chi-bot ti-chi-bot added status/LGT2 Indicates that a PR has LGTM 2. and removed status/LGT1 Indicates that a PR has LGTM 1. labels Nov 1, 2022
Signed-off-by: yibin <huyibin@pingcap.com>
@yibin87
Copy link
Contributor Author

yibin87 commented Nov 2, 2022

/run-mysql-test

@windtalker
Copy link
Contributor

/merge

@ti-chi-bot
Copy link
Member

This pull request has been accepted and is ready to merge.

Commit hash: 0e036e4

@ti-chi-bot ti-chi-bot added the status/can-merge Indicates a PR has been approved by a committer. label Nov 2, 2022
@ti-chi-bot ti-chi-bot merged commit e245b84 into pingcap:master Nov 2, 2022
@sre-bot
Copy link
Contributor

sre-bot commented Nov 2, 2022

TiDB MergeCI notify

🔴 Bad News! New failing [2] after this pr merged.
These new failed integration tests seem to be caused by the current PR, please try to fix these new failed integration tests, thanks!

CI Name Result Duration Compare with Parent commit
idc-jenkins-ci-tidb/common-test 🟥 failed 1, success 10, total 11 16 min New failing
idc-jenkins-ci-tidb/integration-common-test 🟥 failed 3, success 14, total 17 16 min New failing
idc-jenkins-ci-tidb/integration-ddl-test 🔴 failed 1, success 5, total 6 25 min Existing failure
idc-jenkins-ci/integration-cdc-test 🟢 all 38 tests passed 19 min Existing passed
idc-jenkins-ci-tidb/tics-test 🟢 all 1 tests passed 8 min 50 sec Existing passed
idc-jenkins-ci-tidb/integration-compatibility-test 🟢 all 1 tests passed 8 min 24 sec Existing passed
idc-jenkins-ci-tidb/sqllogic-test-2 🟢 all 28 tests passed 5 min 11 sec Existing passed
idc-jenkins-ci-tidb/sqllogic-test-1 🟢 all 26 tests passed 4 min 14 sec Existing passed
idc-jenkins-ci-tidb/mybatis-test 🟢 all 1 tests passed 3 min 3 sec Existing passed
idc-jenkins-ci-tidb/plugin-test 🟢 build success, plugin test success 4min Existing passed

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
release-note-none size/XL Denotes a PR that changes 500-999 lines, ignoring generated files. status/can-merge Indicates a PR has been approved by a committer. status/LGT2 Indicates that a PR has LGTM 2.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Column prune should remove useless filter columns in ExchangeSender/Receiver
6 participants