Skip to content

sql,execinfrapb: DistSQLPlanner and related changes for inverted join#50709

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
sumeerbhola:ijintegrate
Jul 1, 2020
Merged

sql,execinfrapb: DistSQLPlanner and related changes for inverted join#50709
craig[bot] merged 1 commit intocockroachdb:masterfrom
sumeerbhola:ijintegrate

Conversation

@sumeerbhola
Copy link
Copy Markdown
Collaborator

Release note: None

@sumeerbhola sumeerbhola requested review from rytaft and yuzefovich June 26, 2020 19:10
@sumeerbhola sumeerbhola requested a review from a team as a code owner June 26, 2020 19:10
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

Copy link
Copy Markdown
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @rytaft and @yuzefovich)


pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial, line 30 at r1 (raw file):

  (16, 'POLYGON((1.0 1.0, 5.0 1.0, 5.0 5.0, 1.0 5.0, 1.0 1.0))')

query II

@rytaft this fails with

--- FAIL: TestLogic (0.25s)
    logic.go:2722: randomize batchSize to 3
    test_log_scope.go:77: test logs captured to: /var/folders/rm/0t5c352977v4t7rz3qzjy1x40000gn/T/logTestLogic155114881
    test_log_scope.go:58: use -show-logs to present logs inline
    --- FAIL: TestLogic/local (0.00s)
        --- FAIL: TestLogic/local/inverted_join_geospatial (0.16s)
            logic.go:2296: 
                
                testdata/logic_test/inverted_join_geospatial:30: SELECT ltable.k, rtable.k FROM ltable JOIN rtable ON ST_Intersects(ltable.geom, rtable.geom)
                expected success, but found
                (XX000) internal error: interface conversion: *memo.FiltersExpr is not memo.RelExpr: missing method Cost
                catch.go:29: in ShouldCatch()
                DETAIL: stack trace:
                github.com/cockroachdb/cockroach/pkg/util/errorutil/catch.go:29: ShouldCatch()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:193: func1()
                runtime/panic.go:679: gopanic()
                runtime/iface.go:85: getitab()
                runtime/iface.go:449: assertI2I()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:183: colStatFromChild()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:1439: colStatFromJoinRight()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:1272: colStatJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:367: colStat()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:193: colStatFromChild()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:295: colStatFromInput()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:3505: func1()
                github.com/cockroachdb/cockroach/pkg/sql/opt/colset.go:55: func1()
                github.com/cockroachdb/cockroach/pkg/util/fast_int_set.go:167: ForEach()
                github.com/cockroachdb/cockroach/pkg/sql/opt/colset.go:55: ForEach()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:3503: selectivityFromNullsRemoved()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:1083: buildJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/logical_props_builder.go:403: buildJoinProps()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/logical_props_builder.go:449: buildLookupJoinProps()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/expr.og.go:16875: MemoizeLookupJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:2526: rowsProcessed()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/memo.go:375: RowsProcessed()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/coster.go:449: computeLookupJoinCost()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/coster.go:191: ComputeCost()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:507: optimizeGroupMember()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:439: optimizeGroup()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:239: optimizeExpr()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:494: optimizeGroupMember()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:439: optimizeGroup()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:213: Optimize()
                github.com/cockroachdb/cockroach/pkg/sql/plan_opt.go:479: buildExecMemo()
                github.com/cockroachdb/cockroach/pkg/sql/plan_opt.go:172: makeOptimizerPlan()
                
                NOTE: internal errors may have more details in logs. Use -show-logs.
            logic.go:2051: 
                 pq: internal error: interface conversion: *memo.FiltersExpr is not memo.RelExpr: missing method Cost
            logic.go:2629: 
                testdata/logic_test/inverted_join_geospatial:34: error while processing
            logic.go:2629: testdata/logic_test/inverted_join_geospatial:34: too many errors encountered, skipping the rest of the input

Copy link
Copy Markdown
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

Reviewed 8 of 8 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola and @yuzefovich)


pkg/sql/distsql_physical_planner.go, line 2052 at r1 (raw file):

	}

	if n.joinType == sqlbase.LeftSemiJoin || n.joinType == sqlbase.LeftAntiJoin {

The comment in invertedJoinNode says only inner and left outer are supported -- is that incorrect?


pkg/sql/execinfrapb/flow_diagram.go, line 164 at r1 (raw file):

		details = append(details, joinTypeDetail(jr.Type))
	}
	details = append(details, fmt.Sprintf("%s@%s", index, jr.Table.Name))

I guess the table and index should be swapped here too?


pkg/sql/execinfrapb/flow_diagram.go, line 288 at r1 (raw file):

		details = append(details, joinTypeDetail(ij.Type))
	}
	details = append(details, fmt.Sprintf("%s@%s", index, ij.Table.Name))

should index and table name be swapped?


pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial, line 30 at r1 (raw file):

Previously, sumeerbhola wrote…

@rytaft this fails with

--- FAIL: TestLogic (0.25s)
    logic.go:2722: randomize batchSize to 3
    test_log_scope.go:77: test logs captured to: /var/folders/rm/0t5c352977v4t7rz3qzjy1x40000gn/T/logTestLogic155114881
    test_log_scope.go:58: use -show-logs to present logs inline
    --- FAIL: TestLogic/local (0.00s)
        --- FAIL: TestLogic/local/inverted_join_geospatial (0.16s)
            logic.go:2296: 
                
                testdata/logic_test/inverted_join_geospatial:30: SELECT ltable.k, rtable.k FROM ltable JOIN rtable ON ST_Intersects(ltable.geom, rtable.geom)
                expected success, but found
                (XX000) internal error: interface conversion: *memo.FiltersExpr is not memo.RelExpr: missing method Cost
                catch.go:29: in ShouldCatch()
                DETAIL: stack trace:
                github.com/cockroachdb/cockroach/pkg/util/errorutil/catch.go:29: ShouldCatch()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:193: func1()
                runtime/panic.go:679: gopanic()
                runtime/iface.go:85: getitab()
                runtime/iface.go:449: assertI2I()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:183: colStatFromChild()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:1439: colStatFromJoinRight()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:1272: colStatJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:367: colStat()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:193: colStatFromChild()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:295: colStatFromInput()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:3505: func1()
                github.com/cockroachdb/cockroach/pkg/sql/opt/colset.go:55: func1()
                github.com/cockroachdb/cockroach/pkg/util/fast_int_set.go:167: ForEach()
                github.com/cockroachdb/cockroach/pkg/sql/opt/colset.go:55: ForEach()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:3503: selectivityFromNullsRemoved()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:1083: buildJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/logical_props_builder.go:403: buildJoinProps()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/logical_props_builder.go:449: buildLookupJoinProps()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/expr.og.go:16875: MemoizeLookupJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/statistics_builder.go:2526: rowsProcessed()
                github.com/cockroachdb/cockroach/pkg/sql/opt/memo/memo.go:375: RowsProcessed()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/coster.go:449: computeLookupJoinCost()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/coster.go:191: ComputeCost()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:507: optimizeGroupMember()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:439: optimizeGroup()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:239: optimizeExpr()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:494: optimizeGroupMember()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:439: optimizeGroup()
                github.com/cockroachdb/cockroach/pkg/sql/opt/xform/optimizer.go:213: Optimize()
                github.com/cockroachdb/cockroach/pkg/sql/plan_opt.go:479: buildExecMemo()
                github.com/cockroachdb/cockroach/pkg/sql/plan_opt.go:172: makeOptimizerPlan()
                
                NOTE: internal errors may have more details in logs. Use -show-logs.
            logic.go:2051: 
                 pq: internal error: interface conversion: *memo.FiltersExpr is not memo.RelExpr: missing method Cost
            logic.go:2629: 
                testdata/logic_test/inverted_join_geospatial:34: error while processing
            logic.go:2629: testdata/logic_test/inverted_join_geospatial:34: too many errors encountered, skipping the rest of the input

Looking into it

Copy link
Copy Markdown
Member

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

Reviewed 6 of 8 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola)


pkg/sql/distsql_physical_planner.go, line 2001 at r1 (raw file):

	invertedJoinerSpec.LookupColumn = uint32(plan.PlanToStreamColMap[n.inputCol])

	// The n.table node can be configured with an arbitrary set of columns. Apply

This part seems like a copy/paste from createPlanForLookupJoin - we should extract a helper method to be used in both places instead.


pkg/sql/inverted_join.go, line 20 at r1 (raw file):

)

type invertedJoinNode struct {

This also seems very similar to lookupJoinNode - I wonder whether we should extract lookupJoinBase or something. I'm not convinced that we should though, curious to hear your thoughts.


pkg/sql/inverted_join.go, line 46 at r1 (raw file):

// batches of lookups that can be performed.
//
// TODO(sumeer): does this mean different input batches can be

This method in particular is used only in one place in order to add "parallel" attribute in EXPLAIN output.

Lookup join sets it to true only when the lookup columns form a key, I think, otherwise, each lookup can return multiple rows which can cause memory blowup.


pkg/sql/walk.go, line 253 at r1 (raw file):

		n.input = v.visit(n.input)

	case *lookupJoinNode:

I think we need to add entries for the new planNodes here.

Copy link
Copy Markdown
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola)


pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial, line 30 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

Looking into it

Opened #50713 to fix this.

Copy link
Copy Markdown
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @rytaft, @sumeerbhola, and @yuzefovich)


pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial, line 30 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

Opened #50713 to fix this.

One step further :)

I now get the following failure. The problem happens in buildInvertedJoin in relational.go. It has
inputCols = {1, 2}
join.Cols = {1, 2, 3}
lookupCols = {3}

So lookupColMap will only contain 3 as a key. But the inverted expression is on cols 2 and 4 (ScalarExpr Args has 2 and 4 as expected).

--- FAIL: TestLogic (0.39s)
    logic.go:2722: randomize batchSize to 3
    test_log_scope.go:77: test logs captured to: /var/folders/rm/0t5c352977v4t7rz3qzjy1x40000gn/T/logTestLogic949366495
    test_log_scope.go:58: use -show-logs to present logs inline
    --- FAIL: TestLogic/local (0.19s)
        --- FAIL: TestLogic/local/inverted_join_geospatial (0.18s)
            logic.go:2296: 
                
                testdata/logic_test/inverted_join_geospatial:30: SELECT url FROM [EXPLAIN (DISTSQL)
                SELECT lk, rk FROM ltable JOIN rtable@geom_index ON ST_Intersects(ltable.geom, rtable.geom)]
                expected success, but found
                (XX000) internal error: cannot map variable 4 to an indexed var
                scalar.go:118: in indexedVar()
                DETAIL: stack trace:
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:118: indexedVar()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:110: buildVariable()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:94: buildScalar()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:262: buildFunction()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:94: buildScalar()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:1411: buildInvertedJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:219: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:1329: buildLookupJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:216: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:584: buildProject()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:197: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/statement.go:130: buildExplain()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:273: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:584: buildProject()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:197: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/builder.go:136: build()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/builder.go:105: Build()
                github.com/cockroachdb/cockroach/pkg/sql/plan_opt.go:215: makeOptimizerPlan()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor_exec.go:838: makeExecPlan()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor_exec.go:727: dispatchToExecutionEngine()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor_exec.go:500: execStmtInOpenState()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor_exec.go:99: execStmt()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor.go:1356: execCmd()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor.go:1285: run()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor.go:490: ServeConn()
                github.com/cockroachdb/cockroach/pkg/sql/pgwire/conn.go:595: func1()
                runtime/asm_amd64.s:1357: goexit()
                
                NOTE: internal errors may have more details in logs. Use -show-logs.
            logic.go:2051: 
                 pq: internal error: cannot map variable 4 to an indexed var
            logic.go:2629: 
                testdata/logic_test/inverted_join_geospatial:36: error while processing

rytaft added a commit to rytaft/cockroach that referenced this pull request Jun 29, 2020
…oins

This commit fixes an issue identified in cockroachdb#50709 in which the execbuilder
code for building inverted joins was failing due to the lack of an indexed
var for the column indexed by the inverted index. The indexed var is
necessary since the inverted expression contains a reference to the inverted
column. This commit fixes the issue by including the column in the indexed var
helper.

This commit does not include a release note since this issue was introduced
a few days ago and has not been released.

Release note: None
@rytaft
Copy link
Copy Markdown
Collaborator

rytaft commented Jun 29, 2020


pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial, line 30 at r1 (raw file):

Previously, sumeerbhola wrote…

One step further :)

I now get the following failure. The problem happens in buildInvertedJoin in relational.go. It has
inputCols = {1, 2}
join.Cols = {1, 2, 3}
lookupCols = {3}

So lookupColMap will only contain 3 as a key. But the inverted expression is on cols 2 and 4 (ScalarExpr Args has 2 and 4 as expected).

--- FAIL: TestLogic (0.39s)
    logic.go:2722: randomize batchSize to 3
    test_log_scope.go:77: test logs captured to: /var/folders/rm/0t5c352977v4t7rz3qzjy1x40000gn/T/logTestLogic949366495
    test_log_scope.go:58: use -show-logs to present logs inline
    --- FAIL: TestLogic/local (0.19s)
        --- FAIL: TestLogic/local/inverted_join_geospatial (0.18s)
            logic.go:2296: 
                
                testdata/logic_test/inverted_join_geospatial:30: SELECT url FROM [EXPLAIN (DISTSQL)
                SELECT lk, rk FROM ltable JOIN rtable@geom_index ON ST_Intersects(ltable.geom, rtable.geom)]
                expected success, but found
                (XX000) internal error: cannot map variable 4 to an indexed var
                scalar.go:118: in indexedVar()
                DETAIL: stack trace:
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:118: indexedVar()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:110: buildVariable()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:94: buildScalar()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:262: buildFunction()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/scalar.go:94: buildScalar()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:1411: buildInvertedJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:219: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:1329: buildLookupJoin()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:216: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:584: buildProject()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:197: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/statement.go:130: buildExplain()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:273: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:584: buildProject()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/relational.go:197: buildRelational()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/builder.go:136: build()
                github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder/builder.go:105: Build()
                github.com/cockroachdb/cockroach/pkg/sql/plan_opt.go:215: makeOptimizerPlan()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor_exec.go:838: makeExecPlan()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor_exec.go:727: dispatchToExecutionEngine()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor_exec.go:500: execStmtInOpenState()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor_exec.go:99: execStmt()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor.go:1356: execCmd()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor.go:1285: run()
                github.com/cockroachdb/cockroach/pkg/sql/conn_executor.go:490: ServeConn()
                github.com/cockroachdb/cockroach/pkg/sql/pgwire/conn.go:595: func1()
                runtime/asm_amd64.s:1357: goexit()
                
                NOTE: internal errors may have more details in logs. Use -show-logs.
            logic.go:2051: 
                 pq: internal error: cannot map variable 4 to an indexed var
            logic.go:2629: 
                testdata/logic_test/inverted_join_geospatial:36: error while processing

Sorry for the hassle! Fixed in #50754.

rytaft added a commit to rytaft/cockroach that referenced this pull request Jun 29, 2020
…oins

This commit fixes an issue identified in cockroachdb#50709 in which the execbuilder
code for building inverted joins was failing due to the lack of an indexed
var for the column indexed by the inverted index. The indexed var is
necessary since the inverted expression contains a reference to the inverted
column. This commit fixes the issue by including the column in the indexed var
helper.

This commit does not include a release note since this issue was introduced
a few days ago and has not been released.

Release note: None
Copy link
Copy Markdown
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

Reviewed 7 of 7 files at r2.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola)

Copy link
Copy Markdown
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

TFTRs!

Addressed comments. And this works when I patch #50754

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @rytaft, @sumeerbhola, and @yuzefovich)


pkg/sql/distsql_physical_planner.go, line 2001 at r1 (raw file):

Previously, yuzefovich wrote…

This part seems like a copy/paste from createPlanForLookupJoin - we should extract a helper method to be used in both places instead.

Created mappingHelperForLookupJoins.


pkg/sql/distsql_physical_planner.go, line 2052 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

The comment in invertedJoinNode says only inner and left outer are supported -- is that incorrect?

That was incorrect. Fixed. Geospatial will only use inner and left outer due to false positives.


pkg/sql/inverted_join.go, line 20 at r1 (raw file):

Previously, yuzefovich wrote…

This also seems very similar to lookupJoinNode - I wonder whether we should extract lookupJoinBase or something. I'm not convinced that we should though, curious to hear your thoughts.

For such a small struct, I think it is more readable for it to be separate.


pkg/sql/inverted_join.go, line 46 at r1 (raw file):

each lookup can return multiple rows which can cause memory blowup.

For the ordering required cases of lookup join, all the looked-up rows for an input batch are going to be placed in a RowContainer so is the memory blowup concern elsewhere, say in intermediate buffering layers between TCP and the RowContainer?


pkg/sql/walk.go, line 253 at r1 (raw file):

Previously, yuzefovich wrote…

I think we need to add entries for the new planNodes here.

Done


pkg/sql/execinfrapb/flow_diagram.go, line 164 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

I guess the table and index should be swapped here too?

@yuzefovich Can you comment?


pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial, line 30 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

Sorry for the hassle! Fixed in #50754.

Thanks

Copy link
Copy Markdown
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 7 of 7 files at r3.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @sumeerbhola and @yuzefovich)


pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_explain, line 44 at r3 (raw file):

                │             table                  rtable@geom_index
                │             type                   inner
                │             ·                      st_intersects(@2, @4)

Is there a way to show the variable names instead of @2 and @4?

Copy link
Copy Markdown
Member

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @2, @4, @rytaft, and @sumeerbhola)


pkg/sql/distsql_physical_planner.go, line 1964 at r3 (raw file):

// mappingHelperForLookupJoins creates slices etc. for the columns of
// lookup-style joins (that involve an input that is used to lookup a table).

super nit: probably s/lookup a table/lookup from a table/.


pkg/sql/distsql_physical_planner.go, line 2030 at r3 (raw file):

	// var indexes 0 to numInputNodeCols-1 and to table columns with var indexes
	// starting from numInputNodeCols.
	indexVarMap := makePlanToStreamColMap(numInputNodeCols + len(n.table.cols))

nit: this indexVarMap creation could also be extracted in a separate method and reused in two places.


pkg/sql/inverted_join.go, line 20 at r1 (raw file):

Previously, sumeerbhola wrote…

For such a small struct, I think it is more readable for it to be separate.

SGTM.


pkg/sql/inverted_join.go, line 46 at r1 (raw file):

Previously, sumeerbhola wrote…

each lookup can return multiple rows which can cause memory blowup.

For the ordering required cases of lookup join, all the looked-up rows for an input batch are going to be placed in a RowContainer so is the memory blowup concern elsewhere, say in intermediate buffering layers between TCP and the RowContainer?

I think the memory blowup might occur at the KV level, before we get any rows to put in the RowContainer. The thing is that KV layer doesn't perform much memory accounting, and until recently there wasn't any accounting whatsoever. I believe in 20.1 Tobi added 10MB limit (which might be configurable, not sure about that) on the amount of data that is fetched. For example, if for a particular row on the left (i.e. row from which "lookup columns" come from), there are 100MB of data of looked rows on the right, those will be returned in 10 batches with a full stop of the fetching before the caller asks for more data - the fetcher is "limiting batches". I think that maybe if we tell the row fetcher that it should not limit batches (meaning that it can run parallel - or pipelined - fetches), that memory accounting might be turned off (this sentence is my speculation cause I never looked into that layer closely :) ).

Overall, my knowledge of these things is quite hazy, so I might be saying something wrong. Feel free to ping Alfonso or Tobi to get more details (and with more confidence :) ).


pkg/sql/walk.go, line 398 at r3 (raw file):

				v.expr(name, "onExpr", -1, n.onExpr)
			}
		}

nit: you should check CanParallelize method here and add "parallel" attribute if it is parallelizable.


pkg/sql/execinfrapb/flow_diagram.go, line 164 at r1 (raw file):

Previously, sumeerbhola wrote…

@yuzefovich Can you comment?

Hm, it's an interesting question. I think they should be swapped - we would then use notation similar to index hints, but throughout this file we use "index@table" notation for some reason. I think you should leave a todo for this (feel free to use my name).


pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_explain, line 44 at r3 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

Is there a way to show the variable names instead of @2 and @4?

Not sure, but it might be hard because invertedExpr might be serialized here.


pkg/sql/opt/exec/execbuilder/testdata/inverted_index, line 632 at r3 (raw file):

query TTTTT
EXPLAIN (VERBOSE) SELECT * FROM geo_table2 JOIN geo_table ON ST_Intersects(geo_table2.geom, geo_table.geom)
----

I this intentionally empty?

Copy link
Copy Markdown
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @2, @4, @asubiotto, @rytaft, @sumeerbhola, @tbg, and @yuzefovich)


pkg/sql/distsql_physical_planner.go, line 1964 at r3 (raw file):

Previously, yuzefovich wrote…

super nit: probably s/lookup a table/lookup from a table/.

Done.


pkg/sql/distsql_physical_planner.go, line 2030 at r3 (raw file):

Previously, yuzefovich wrote…

nit: this indexVarMap creation could also be extracted in a separate method and reused in two places.

Done


pkg/sql/inverted_join.go, line 46 at r1 (raw file):

Previously, yuzefovich wrote…

I think the memory blowup might occur at the KV level, before we get any rows to put in the RowContainer. The thing is that KV layer doesn't perform much memory accounting, and until recently there wasn't any accounting whatsoever. I believe in 20.1 Tobi added 10MB limit (which might be configurable, not sure about that) on the amount of data that is fetched. For example, if for a particular row on the left (i.e. row from which "lookup columns" come from), there are 100MB of data of looked rows on the right, those will be returned in 10 batches with a full stop of the fetching before the caller asks for more data - the fetcher is "limiting batches". I think that maybe if we tell the row fetcher that it should not limit batches (meaning that it can run parallel - or pipelined - fetches), that memory accounting might be turned off (this sentence is my speculation cause I never looked into that layer closely :) ).

Overall, my knowledge of these things is quite hazy, so I might be saying something wrong. Feel free to ping Alfonso or Tobi to get more details (and with more confidence :) ).

@tbg @asubiotto could you shed some light on this matter?


pkg/sql/walk.go, line 398 at r3 (raw file):

Previously, yuzefovich wrote…

nit: you should check CanParallelize method here and add "parallel" attribute if it is parallelizable.

Done


pkg/sql/execinfrapb/flow_diagram.go, line 164 at r1 (raw file):

Previously, yuzefovich wrote…

Hm, it's an interesting question. I think they should be swapped - we would then use notation similar to index hints, but throughout this file we use "index@table" notation for some reason. I think you should leave a todo for this (feel free to use my name).

Done


pkg/sql/execinfrapb/flow_diagram.go, line 288 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

should index and table name be swapped?

Added TODO as discussed above


pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_explain, line 44 at r3 (raw file):

Previously, yuzefovich wrote…

Not sure, but it might be hard because invertedExpr might be serialized here.

I am just calling planVisitor.expr() with the tree.Expr just like is done for the ON expr.


pkg/sql/opt/exec/execbuilder/testdata/inverted_index, line 632 at r3 (raw file):

Previously, yuzefovich wrote…

I this intentionally empty?

I'll fix this once #50754 is merged

Copy link
Copy Markdown
Member

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 3 of 3 files at r4.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @2, @4, @asubiotto, @rytaft, @sumeerbhola, @tbg, and @yuzefovich)


pkg/sql/distsql_physical_planner.go, line 2044 at r4 (raw file):

	indexVarMap := makeIndexVarMapForLookupJoins(numInputNodeCols, n.table, plan, &post)
	if invertedJoinerSpec.InvertedExpr, err = physicalplan.MakeExpression(
		n.invertedExpr, planCtx, indexVarMap); err != nil {

super nit: I think usually we would put the closing parenthesis on the new line in such case.


pkg/sql/distsql_physical_planner.go, line 2050 at r4 (raw file):

	if n.onExpr != nil {
		if invertedJoinerSpec.OnExpr, err = physicalplan.MakeExpression(
			n.onExpr, planCtx, indexVarMap); err != nil {

ditto for closing parenthesis

Copy link
Copy Markdown
Member

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @2, @4, @asubiotto, @rytaft, @sumeerbhola, and @tbg)


pkg/sql/rowexec/inverted_joiner.go, line 434 at r4 (raw file):

	log.VEventf(ij.Ctx, 1, "scanning %d spans", len(indexSpans))
	if err = ij.fetcher.StartScan(
		ij.Ctx, ij.FlowCtx.Txn, indexSpans, false /* limitBatches */, 0, /* limitHint */

To be clear, CanParallelize doesn't determine whether fetcher performs parallel scan, rather this false /* limitBatches */ argument does. Namely, "false" means that the fetcher should not limit itself and will perform parallel scans. For regular lookup join, we set this argument to false when the lookup columns form a key.

craig bot pushed a commit that referenced this pull request Jun 30, 2020
50754: opt: add indexed var for inverted column when execbuilding inverted joins r=rytaft a=rytaft

This commit fixes an issue identified in #50709 in which the `execbuilder`
code for building inverted joins was failing due to the lack of an indexed
var for the column indexed by the inverted index. The indexed var is
necessary since the inverted expression contains a reference to the inverted
column. This commit fixes the issue by including the column in the indexed var
helper.

This commit does not include a release note since this issue was introduced
a few days ago and has not been released.

Release note: None

Co-authored-by: Rebecca Taft <becca@cockroachlabs.com>
Copy link
Copy Markdown
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @2, @4, @asubiotto, @rytaft, @sumeerbhola, @tbg, and @yuzefovich)


pkg/sql/distsql_physical_planner.go, line 2044 at r4 (raw file):

Previously, yuzefovich wrote…

super nit: I think usually we would put the closing parenthesis on the new line in such case.

Done.


pkg/sql/distsql_physical_planner.go, line 2050 at r4 (raw file):

Previously, yuzefovich wrote…

ditto for closing parenthesis

Done.


pkg/sql/inverted_join.go, line 46 at r1 (raw file):

Previously, sumeerbhola wrote…

@tbg @asubiotto could you shed some light on this matter?

I've adjusted this comment to say it should be kept in sync with invertedJoiner behavior. We can revisit later whether it should set limitBatches to false as it currently does.


pkg/sql/rowexec/inverted_joiner.go, line 434 at r4 (raw file):

Previously, yuzefovich wrote…

To be clear, CanParallelize doesn't determine whether fetcher performs parallel scan, rather this false /* limitBatches */ argument does. Namely, "false" means that the fetcher should not limit itself and will perform parallel scans. For regular lookup join, we set this argument to false when the lookup columns form a key.

Ack

@sumeerbhola
Copy link
Copy Markdown
Collaborator Author

bors r+

craig bot pushed a commit that referenced this pull request Jun 30, 2020
50709: sql,execinfrapb: DistSQLPlanner and related changes for inverted join r=sumeerbhola a=sumeerbhola

Release note: None

50760: backupccl: fix formatting of backup jobs in jobs table r=pbardea a=pbardea

When adding cluster backup/restore, the method which generated the job
description was not updated to handle cluster backups. Previously, a
cluster backup job would appear in the jobs table as `BACKUP TABLE TO`
rather than `BACKUP TO`.

Release note (bug fix): Cluster backup would previously appear as
`BACKUP TABLE TO` rather than `BACKUP TO` in the jobs table.

50770: roachtest: update version map and create fixtures r=jlinder a=asubiotto

This commit adds the recently released 19.1.10, 19.2.8, and 20.1.3 to the
version map in PredecessorVersion.

Release note: None (testing change)

50815: opt: fix ResolvedType() for aggregateInfo in the optbuilder r=rytaft a=rytaft

Prior to this commit, it was possible that calling `ResolvedType()`
on an `aggregateInfo` object returned the wrong type. This was because
`aggregateInfo` did not implement `ResolvedType()`, and was therefore
passing the call to the embedded `tree.FuncExpr`, which may have been
stripped of its original type information. This commit fixes the
problem by adding an implementation of `ResolvedType()` to `aggregateInfo`,
which simply returns the type of the aggregation column represented by
the struct.

Fixes #46914

Release note (bug fix): Fixed an internal error that could happen
during planning for some queries with aggregate functions embedded in
complex scalar expressions.

Co-authored-by: sumeerbhola <sumeer@cockroachlabs.com>
Co-authored-by: Paul Bardea <pbardea@gmail.com>
Co-authored-by: Alfonso Subiotto Marques <alfonso@cockroachlabs.com>
Co-authored-by: Rebecca Taft <becca@cockroachlabs.com>
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jun 30, 2020

Build failed (retrying...)

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jun 30, 2020

Build failed (retrying...)

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jun 30, 2020

Build failed (retrying...)

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jul 1, 2020

Build succeeded

@craig craig bot merged commit 3a03f38 into cockroachdb:master Jul 1, 2020
@sumeerbhola sumeerbhola deleted the ijintegrate branch July 7, 2020 13:56
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants