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

distsql: add batching for secondary lookup joins #25815

Merged
merged 1 commit into from
May 23, 2018

Conversation

solongordon
Copy link
Contributor

Lookup joins on non-covering secondary indexes were previously making a
separate primary index scan for every secondary index row. Now those
scans are grouped together into batches of up to 100 spans.

Release note: None

@solongordon solongordon requested review from a team May 22, 2018 18:55
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@solongordon
Copy link
Contributor Author

Please ignore the first commit which is from a dependent PR.

@solongordon solongordon force-pushed the batch-lookup-joins branch from 9fd2dcb to 1533557 Compare May 22, 2018 18:59
@solongordon
Copy link
Contributor Author

Review status: 0 of 5 files reviewed at latest revision, all discussions resolved.


pkg/sql/distsqlrun/joinreader.go, line 388 at r2 (raw file):

// returned, both the inputs and the output have been drained and closed, except
// if an error is returned.
func (jr *joinReader) indexJoinLookup(

I disentangled the index join and lookup join logic a bit because there was too much cognitive overhead. I think we could stand go further in this direction down the line.


Comments from Reviewable

@jordanlewis
Copy link
Member

With regard to testing, I'd definitely like to at least see a test case that has multiple results in the outer join. There's a bunch of code here that tries to preserve ordering of the results (I think) - that doesn't seem to be tested properly if there's no test case that has more than one output.


Reviewed 5 of 5 files at r1.
Review status: 4 of 5 files reviewed at latest revision, 1 unresolved discussion, some commit checks failed.


pkg/sql/distsqlrun/joinreader.go, line 388 at r2 (raw file):

Previously, solongordon (Solon) wrote…

I disentangled the index join and lookup join logic a bit because there was too much cognitive overhead. I think we could stand go further in this direction down the line.

👍 although I wish the disentangling was in its own commit - the code movement combined with the changes makes it hard to see what's new vs what's copied.


pkg/sql/distsqlrun/joinreader.go, line 485 at r2 (raw file):

			// second lookup against the primary index and replace our previous
			// results with the primary rows.
			secondaryIndexRows := make([]sqlbase.EncDatumRow, len(lookupRows))

This doesn't have to be allocated every time, does it? We could allocate a slice up front and reuse it. At least put a TODO or something here so it's easy to spot if this shows up as a hot allocations spot in profiling later.


pkg/sql/distsqlrun/joinreader.go, line 494 at r2 (raw file):

			}
			for i := range rows {
				lookupRows[i].row = rows[i]

Is it possible that primaryLookup won't always return a one to one match for all of its inputs? Do we need to check anything before replacing lookupRows values? I suppose this same code path was present in the last PR, but just noticing it now.


Comments from Reviewable

@RaduBerinde
Copy link
Member

I would try to run all the logic tests with the lookup join flag set and see if something breaks (other than EXPLAIN (DISTSQL) statements and the like)


Review status: 4 of 5 files reviewed at latest revision, 3 unresolved discussions, some commit checks failed.


pkg/sql/distsqlrun/joinreader.go, line 456 at r2 (raw file):

	// lookupRow represents an index key and the corresponding row.
	type lookupRow = struct {

Nice, TIL


pkg/sql/distsqlrun/joinreader.go, line 553 at r2 (raw file):

	batchSize := len(secondaryIndexRows)
	if batchSize == 0 {
		return secondaryIndexRows, nil

why secondaryIndexRows instead of nil?


Comments from Reviewable

@petermattis
Copy link
Collaborator

Review status: 4 of 5 files reviewed at latest revision, 5 unresolved discussions, some commit checks failed.


pkg/sql/distsqlrun/joinreader.go, line 456 at r2 (raw file):

Previously, RaduBerinde wrote…

Nice, TIL

What is the = doing? I know you an make a type that is local to a function, but that doesn't require the =. Hmm, I think that is making a type alias and I don't think it is necessary. (Not harmful, but also not necessary).


Comments from Reviewable

@solongordon
Copy link
Contributor Author

Yeah, I agree about more tests. I did just add some outer join tests with multiple results to the other PR. But really we ought to have some tests where the number of results exceeds the batch size. (I've manually tested this by setting the batch size very small and running logic tests.) I'll look into that. Also I'll see if I can try out Radu's suggestion.


Review status: 4 of 5 files reviewed at latest revision, 5 unresolved discussions, some commit checks failed.


pkg/sql/distsqlrun/joinreader.go, line 388 at r2 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

👍 although I wish the disentangling was in its own commit - the code movement combined with the changes makes it hard to see what's new vs what's copied.

Yeah, sorry about that.


pkg/sql/distsqlrun/joinreader.go, line 456 at r2 (raw file):

Previously, petermattis (Peter Mattis) wrote…

What is the = doing? I know you an make a type that is local to a function, but that doesn't require the =. Hmm, I think that is making a type alias and I don't think it is necessary. (Not harmful, but also not necessary).

Yeah, this was just my fingers misremembering the type syntax. Fixed.


pkg/sql/distsqlrun/joinreader.go, line 485 at r2 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

This doesn't have to be allocated every time, does it? We could allocate a slice up front and reuse it. At least put a TODO or something here so it's easy to spot if this shows up as a hot allocations spot in profiling later.

True. I added a TODO for now.


pkg/sql/distsqlrun/joinreader.go, line 494 at r2 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

Is it possible that primaryLookup won't always return a one to one match for all of its inputs? Do we need to check anything before replacing lookupRows values? I suppose this same code path was present in the last PR, but just noticing it now.

As far as I understand it shouldn't be possible, since each secondary index row should correspond to exactly one primary index row. But if there's some weird case I'm missing I'd love to know.

primaryLookup does check that it fetches exactly the same number of primary index rows as expected. Though it wouldn't catch if two primary rows somehow mapped onto the same secondary one. I could iterate over the output and assert that there are no nil values, which I think would do it, though not sure if that's necessary.


pkg/sql/distsqlrun/joinreader.go, line 553 at r2 (raw file):

Previously, RaduBerinde wrote…

why secondaryIndexRows instead of nil?

No good reason, fixed.


Comments from Reviewable

@solongordon solongordon force-pushed the batch-lookup-joins branch 2 times, most recently from e5b698e to 3e29991 Compare May 23, 2018 12:51
@solongordon
Copy link
Contributor Author

I ended up just making the batch size a parameter and lowering it to 2 in unit tests.


Review status: 0 of 7 files reviewed at latest revision, 5 unresolved discussions.


Comments from Reviewable

@solongordon solongordon force-pushed the batch-lookup-joins branch from 3e29991 to 6cae2f4 Compare May 23, 2018 15:11
@solongordon
Copy link
Contributor Author

The PR this depended on was merged and this was rebased. PTAL.

@RaduBerinde
Copy link
Member

Review status: 0 of 2 files reviewed at latest revision, 4 unresolved discussions.


pkg/sql/distsqlrun/joinreader.go, line 290 at r3 (raw file):

	var alloc sqlbase.DatumAlloc
	rows := make([]sqlbase.EncDatumRow, 0)

what's the point of making zero-sized slices? I'd just let it be var rows []sqlbase.EncDatumRow


pkg/sql/distsqlrun/joinreader_test.go, line 232 at r3 (raw file):

			// Set a lower batch size to force multiple batches.
			jr.batchSize = 2

[nit] could be a random value between 1 and 10, cheap way to test more cases


pkg/sql/distsqlrun/joinreader_test.go, line 350 at r3 (raw file):

			// Set a lower batch size to force multiple batches.
			jr.batchSize = 2

Same


pkg/sql/distsqlrun/processors.proto, line 261 at r3 (raw file):

  optional Expression index_filter_expr = 5 [(gogoproto.nullable) = false];

  optional sqlbase.JoinType type = 6 [(gogoproto.nullable) = false];

Whoa, why are you changing the type field id to 6? That will unnecessarily break compatibility

BTW bump the distsqlrun version and mention the change in the versions text file


pkg/sql/logictest/testdata/logic_test/lookup_join, line 395 at r3 (raw file):

10 100
10 200

It would be good to have some tests where the distsql plan is actually distributed. Maybe create a secondary table and split that and use it (perhaps in some subquery) instead of VALUES


Comments from Reviewable

Lookup joins on non-covering secondary indexes were previously making a
separate primary index scan for every secondary index row. Now those
scans are grouped together into batches of up to 100 spans.

Release note: None
@solongordon solongordon force-pushed the batch-lookup-joins branch from 6cae2f4 to 67ae2ab Compare May 23, 2018 17:25
@solongordon
Copy link
Contributor Author

Review status: 0 of 4 files reviewed at latest revision, 9 unresolved discussions.


pkg/sql/distsqlrun/joinreader.go, line 290 at r3 (raw file):

Previously, RaduBerinde wrote…

what's the point of making zero-sized slices? I'd just let it be var rows []sqlbase.EncDatumRow

Done.


pkg/sql/distsqlrun/joinreader_test.go, line 232 at r3 (raw file):

Previously, RaduBerinde wrote…

[nit] could be a random value between 1 and 10, cheap way to test more cases

I like the idea though I'm worried about introducing randomness into unit tests. Could lead to flakiness.


pkg/sql/distsqlrun/processors.proto, line 261 at r3 (raw file):

Previously, RaduBerinde wrote…

Whoa, why are you changing the type field id to 6? That will unnecessarily break compatibility

BTW bump the distsqlrun version and mention the change in the versions text file

I think you must be seeing a diff from the rebase. (There was some renumbering between revisions of the PR this depended on.) processors.proto isn't touched by this commit.

And ohhh I didn't know about or had forgotten about the distsql version. That makes sense. I thought you were referring to the cockroach version in a previous comment. Done.


pkg/sql/logictest/testdata/logic_test/lookup_join, line 395 at r3 (raw file):

Previously, RaduBerinde wrote…

It would be good to have some tests where the distsql plan is actually distributed. Maybe create a secondary table and split that and use it (perhaps in some subquery) instead of VALUES

Yeah, makes sense. I'm going to create an issue for follow-up test enhancements since there are a couple others I'd like to make too.


Comments from Reviewable

@solongordon
Copy link
Contributor Author

Review status: 0 of 4 files reviewed at latest revision, 9 unresolved discussions.


pkg/sql/logictest/testdata/logic_test/lookup_join, line 395 at r3 (raw file):

Previously, solongordon wrote…

Yeah, makes sense. I'm going to create an issue for follow-up test enhancements since there are a couple others I'd like to make too.

Created #25862


Comments from Reviewable

@solongordon
Copy link
Contributor Author

bors r+

craig bot pushed a commit that referenced this pull request May 23, 2018
25815: distsql: add batching for secondary lookup joins r=solongordon a=solongordon

Lookup joins on non-covering secondary indexes were previously making a
separate primary index scan for every secondary index row. Now those
scans are grouped together into batches of up to 100 spans.

Release note: None

Co-authored-by: Solon Gordon <[email protected]>
@RaduBerinde
Copy link
Member

Review status: 0 of 4 files reviewed at latest revision, 7 unresolved discussions, some commit checks pending.


pkg/sql/distsqlrun/joinreader_test.go, line 232 at r3 (raw file):

Previously, solongordon wrote…

I like the idea though I'm worried about introducing randomness into unit tests. Could lead to flakiness.

Many unit tests have randomness. If the test is flaky w.r.t the batch size, that's definitely a bug..


pkg/sql/distsqlrun/processors.proto, line 261 at r3 (raw file):

Previously, solongordon wrote…

I think you must be seeing a diff from the rebase. (There was some renumbering between revisions of the PR this depended on.) processors.proto isn't touched by this commit.

And ohhh I didn't know about or had forgotten about the distsql version. That makes sense. I thought you were referring to the cockroach version in a previous comment. Done.

Ah sorry I was looking at some partial diffs


Comments from Reviewable

@craig
Copy link
Contributor

craig bot commented May 23, 2018

Build succeeded

@craig craig bot merged commit 67ae2ab into cockroachdb:master May 23, 2018
windchan7 pushed a commit to windchan7/cockroach that referenced this pull request May 24, 2018
25815: distsql: add batching for secondary lookup joins r=solongordon a=solongordon

Lookup joins on non-covering secondary indexes were previously making a
separate primary index scan for every secondary index row. Now those
scans are grouped together into batches of up to 100 spans.

Encryption benchmark: cockroachdb#19783.
Release note: None

Co-authored-by: Solon Gordon <[email protected]>
@solongordon solongordon deleted the batch-lookup-joins branch June 25, 2018 17:15
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.

5 participants