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: support lookup join on secondary index #25628

Merged
merged 1 commit into from
May 22, 2018

Conversation

solongordon
Copy link
Contributor

joinReader now supports lookup joins on secondary indexes. This was a
trivial change for queries where all the output columns are included in
the secondary index. I just modified the physical planner to specify the
secondary index in the JoinReaderSpec and removed checks which prevented
secondary indexes from being used.

The more complicated situation is when we want to do a lookup join
against a non-covering index. In this case, the logical planner plans an
index join before the inner join, but we want to perform the lookup join
first. We now handle this by only planning the lookup join during
physical planning, not the index join. During execution, the joinReader
detects that there are output columns not covered by the secondary
index, and it performs primary index lookups as necessary to retrieve
the additional columns.

Fixes #25431

@solongordon solongordon requested review from RaduBerinde and a team May 17, 2018 19:30
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@solongordon
Copy link
Contributor Author

cc @asubiotto @petermattis

I'm also planning to add tests to joinreader_test.go but wanted to get some eyes on this.

@solongordon
Copy link
Contributor Author

Is it acceptable not to add a new version for this feature since lookup joins are still experimental?


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


Comments from Reviewable

@RaduBerinde
Copy link
Member

Nice, this looks easier than I though it would be!

I would add a version, but I wouldn't bump the min-accepted version.


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


pkg/sql/distsqlrun/joinreader.go, line 168 at r1 (raw file):

) bool {
	coveredCols := util.MakeFastIntSet()
	for _, i := range append(index.ColumnIDs, index.ExtraColumnIDs...) {

I don't understand what we're doing here. Aren't neededColumns ordinals (0, 1, 2, 3..) and not column IDs?

Also, don't mutate the slice that belongs to index just to save a line of code. What if it has spare capacity and we have two separate queries running this code in parallel?


pkg/sql/distsqlrun/joinreader.go, line 171 at r1 (raw file):

		coveredCols.Add(numInputCols + int(i) - 1)
	}
	for _, col := range neededColumns.Ordered() {

return neededColumns.SubsetOf(coveredCols)


pkg/sql/distsqlrun/joinreader.go, line 179 at r1 (raw file):

}

func getColumnIndexes(index sqlbase.IndexDescriptor) util.FastIntSet {

[nit] indices (we use "indexes" for table indexes)


pkg/sql/distsqlrun/joinreader.go, line 182 at r1 (raw file):

	cols := util.MakeFastIntSet()
	for _, col := range index.ColumnIDs {
		// Convert from 1-based to 0-based.

Same problem here, you are mapping Column IDs to ordinals. This perhaps happens to work because the table you are testing with didn't have schema changes, but won't work in general.


Comments from Reviewable

@solongordon
Copy link
Contributor Author

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


pkg/sql/distsqlrun/joinreader.go, line 182 at r1 (raw file):

Previously, RaduBerinde wrote…

Same problem here, you are mapping Column IDs to ordinals. This perhaps happens to work because the table you are testing with didn't have schema changes, but won't work in general.

Got it, thank you for clarifying that. What is the preferred way to map Column ID to ordinal? Iterate over TableDescriptor.Columns?


Comments from Reviewable

@RaduBerinde
Copy link
Member

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


pkg/sql/distsqlrun/joinreader.go, line 182 at r1 (raw file):

Previously, solongordon (Solon) wrote…

Got it, thank you for clarifying that. What is the preferred way to map Column ID to ordinal? Iterate over TableDescriptor.Columns?

initRowFetcher builds a colIdxMap, perhaps that should be pulled out (especially since we are now calling it twice in some cases).


Comments from Reviewable

@solongordon solongordon requested a review from a team May 17, 2018 21:13
@petermattis
Copy link
Collaborator

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


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

		return distsqlrun.TableReaderSpec{}, distsqlrun.PostProcessSpec{}, err
	}
	s.IndexIdx = indexIdx

I think you probably should be using the index ID (i.e. n.index.ID). I don't think there are any guarantees that the positional index of a SQL index within n.desc.Indexes is immutable. There is a guarantee that the index ID will not change.


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

	// and the right equality columns are a prefix of that node's index. In the
	// index join case, joinReader will first perform the secondary index lookup
	// and then do a primary index lookup on the resulting rows.

Do we need to check for the inverse scenario, where the right side of the join can be used to "lookup" in the left side? Or is that already checked for via some other means? @RaduBerinde?


pkg/sql/distsqlrun/joinreader.go, line 65 at r1 (raw file):

	primaryLookupRequired bool
	primaryFetcher        sqlbase.RowFetcher

sizeof(sqlbase.RowFetcher) == 328. I think this should be a pointer that you only initialize of a primary lookup is required. This would also allow getting rid of the primaryLookupRequired field.


Comments from Reviewable

@solongordon
Copy link
Contributor Author

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


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

Previously, petermattis (Peter Mattis) wrote…

I think you probably should be using the index ID (i.e. n.index.ID). I don't think there are any guarantees that the positional index of a SQL index within n.desc.Indexes is immutable. There is a guarantee that the index ID will not change.

Hm, this was just a refactor so I could pull out logic for getting an IndexIdx. If this behavior is dangerous than probably we should open a separate issue?


pkg/sql/distsqlrun/joinreader.go, line 65 at r1 (raw file):

Previously, petermattis (Peter Mattis) wrote…

sizeof(sqlbase.RowFetcher) == 328. I think this should be a pointer that you only initialize of a primary lookup is required. This would also allow getting rid of the primaryLookupRequired field.

Done.


pkg/sql/distsqlrun/joinreader.go, line 168 at r1 (raw file):

Previously, RaduBerinde wrote…

I don't understand what we're doing here. Aren't neededColumns ordinals (0, 1, 2, 3..) and not column IDs?

Also, don't mutate the slice that belongs to index just to save a line of code. What if it has spare capacity and we have two separate queries running this code in parallel?

Done. I was fuzzy on the ordinal vs ID distinction but now I've seen the light.


pkg/sql/distsqlrun/joinreader.go, line 171 at r1 (raw file):

Previously, RaduBerinde wrote…

return neededColumns.SubsetOf(coveredCols)

Done. I had to add in the input cols to coveredCols for this to work.


pkg/sql/distsqlrun/joinreader.go, line 179 at r1 (raw file):

Previously, RaduBerinde wrote…

[nit] indices (we use "indexes" for table indexes)

Done.


pkg/sql/distsqlrun/joinreader.go, line 182 at r1 (raw file):

Previously, RaduBerinde wrote…

initRowFetcher builds a colIdxMap, perhaps that should be pulled out (especially since we are now calling it twice in some cases).

Done.


Comments from Reviewable

@RaduBerinde
Copy link
Member

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

Previously, solongordon (Solon) wrote…

Hm, this was just a refactor so I could pull out logic for getting an IndexIdx. If this behavior is dangerous than probably we should open a separate issue?

We are putting whatever version of n.desc we have in the spec (above); distsqlrun does not look up table descriptors.


Comments from Reviewable

@RaduBerinde
Copy link
Member

LGTM modulo some comments.

Still need to add a version.


Reviewed 1 of 7 files at r2.
Review status: 1 of 9 files reviewed at latest revision, 3 unresolved discussions.


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

Previously, petermattis (Peter Mattis) wrote…

Do we need to check for the inverse scenario, where the right side of the join can be used to "lookup" in the left side? Or is that already checked for via some other means? @RaduBerinde?

I don't have all the context here, but perhaps the idea was that the query can be rewritten to flip the order if necessary. If we want to handle the symmetric case, it should be a separate PR.


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

	lookupCols columns

	primaryFetcher     *sqlbase.RowFetcher

This block could use a comment


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

	var neededCols util.FastIntSet
	if jr.index == &jr.desc.PrimaryIndex ||
		indexIsCovering(jr.index, jr.desc.ColumnIdxMap(), jr.out.neededColumns(), len(jr.inputTypes)) {

We are calculating this map multiple times (once here, and once in every call to initRowFetcher). We should just calculate it once and pass it initRowFetcher.


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

		indexIsCovering(jr.index, jr.desc.ColumnIdxMap(), jr.out.neededColumns(), len(jr.inputTypes)) {
		// jr.index includes all the needed output columns, so only need one lookup.
		neededCols = jr.rowFetcherColumns()

Add a comment for which it's clear how rowFetcherColumns() differs from neededColumns(). rowFetchecColumns() could also use a comment explaining the lookup join case


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

		coveredCols.Add(numInputCols + colIdxMap[columnID])
	}
	return neededColumns.SubsetOf(coveredCols)

What about StoreColumnIDs?


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

}

func getColumnIndices(

maybe getIndexColSet


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

) util.FastIntSet {
	cols := util.MakeFastIntSet()
	for _, columnID := range index.ColumnIDs {

What about ExtraColumnIDs, StoredColumnIDs?


pkg/sql/logictest/testdata/logic_test/lookup_join, line 275 at r2 (raw file):



####################################

These tests don't verify that what the processor is doing internally is right. For example, it might be doing secondary lookups unnecessarily.

We should add a group of tests that use the KV TRACING facility to make sure we are doing the correct lookups (like https://github.com/cockroachdb/cockroach/blob/master/pkg/sql/logictest/testdata/logic_test/insert#L46). The tests should contain cases that break if we don't consider ExtraColumnIDs and StoreColumnIDs.


Comments from Reviewable

@petermattis
Copy link
Collaborator

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


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

Previously, RaduBerinde wrote…

We are putting whatever version of n.desc we have in the spec (above); distsqlrun does not look up table descriptors.

Ah, got it. Agreed that this is safe. Carry on.


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

Previously, RaduBerinde wrote…

I don't have all the context here, but perhaps the idea was that the query can be rewritten to flip the order if necessary. If we want to handle the symmetric case, it should be a separate PR.

Do we do any join associativity in the 2.0 planner? If not, we might want to document this restriction when we add documentation for experimental_force_lookup_join.


Comments from Reviewable

@RaduBerinde
Copy link
Member

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


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

Previously, petermattis (Peter Mattis) wrote…

Do we do any join associativity in the 2.0 planner? If not, we might want to document this restriction when we add documentation for experimental_force_lookup_join.

We don't. Agreed on the documentation.


Comments from Reviewable

@petermattis
Copy link
Collaborator

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


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

Previously, RaduBerinde wrote…

We don't. Agreed on the documentation.

Cc @lhirata who I believe is documenting experimental_force_lookup_join.


Comments from Reviewable

@asubiotto
Copy link
Contributor

Thanks for taking this on, left some minor comments


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


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

Previously, RaduBerinde wrote…

What about StoreColumnIDs?

This looks sort of similar to what is in the scrub_tablereader.go, you might be able to share the logic.


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

) (sqlbase.EncDatumRow, error) {
	index := jr.desc.PrimaryIndex
	keyValues := make(sqlbase.EncDatumRow, len(index.ColumnIDs))

This might benefit from using a scratch slice instead.


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

	}
	spans := []roachpb.Span{{Key: key, EndKey: key.PrefixEnd()}}
	err = jr.primaryFetcher.StartScan(ctx, txn, spans, false, 0, false)

Add comments to arguments


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

		return nil, errors.New("expected exactly one row but found none")
	}
	nextRow, _, _, err := jr.primaryFetcher.NextRow(ctx)

I'm not sure how necessary this assertion is. If you want to keep it, you might want to consider exposing rowFetcher.kvEnd.


pkg/sql/logictest/testdata/logic_test/lookup_join, line 275 at r2 (raw file):

Previously, RaduBerinde wrote…

These tests don't verify that what the processor is doing internally is right. For example, it might be doing secondary lookups unnecessarily.

We should add a group of tests that use the KV TRACING facility to make sure we are doing the correct lookups (like https://github.com/cockroachdb/cockroach/blob/master/pkg/sql/logictest/testdata/logic_test/insert#L46). The tests should contain cases that break if we don't consider ExtraColumnIDs and StoreColumnIDs.

I don't think we plumb kv tracing down to distsql, this might need to be done in a separate PR.


Comments from Reviewable

@solongordon
Copy link
Contributor Author

Thanks all for the comments. I think I've addressed everything except for adding more granular unit testing. I also still have a TODO for batching the primary index fetches, which I'd like to do in a follow-up PR if no one objects.


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


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

Previously, RaduBerinde wrote…

This block could use a comment

Done.


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

Previously, RaduBerinde wrote…

We are calculating this map multiple times (once here, and once in every call to initRowFetcher). We should just calculate it once and pass it initRowFetcher.

Done.


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

Previously, RaduBerinde wrote…

Add a comment for which it's clear how rowFetcherColumns() differs from neededColumns(). rowFetchecColumns() could also use a comment explaining the lookup join case

Done, hopefully clearer now. I renamed rowFetcherColumns to neededRightCols which is more reflective of its purpose.


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

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

This looks sort of similar to what is in the scrub_tablereader.go, you might be able to share the logic.

Good call, done.


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

Previously, RaduBerinde wrote…

maybe getIndexColSet

Done.


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

Previously, RaduBerinde wrote…

What about ExtraColumnIDs, StoredColumnIDs?

Done.


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

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

This might benefit from using a scratch slice instead.

Could you clarify? Unless I'm misunderstanding I think this is already a scratch slice. (EncDatumRow is just []EncDatum).


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

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

Add comments to arguments

Done.


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

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

I'm not sure how necessary this assertion is. If you want to keep it, you might want to consider exposing rowFetcher.kvEnd.

Yeah it's not strictly necessary, but it's cheap and if it fails we're probably returning incorrect results, so I thought it was worth it.


pkg/sql/logictest/testdata/logic_test/lookup_join, line 275 at r2 (raw file):

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

I don't think we plumb kv tracing down to distsql, this might need to be done in a separate PR.

Maybe this can be addressed in joinreader_test.go? Going to take a look at supplementing that now.


Comments from Reviewable

@asubiotto
Copy link
Contributor

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


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

Previously, solongordon (Solon) wrote…

Could you clarify? Unless I'm misunderstanding I think this is already a scratch slice. (EncDatumRow is just []EncDatum).

On every lookup, you're creating a new keyValues slice. I'm suggesting you have one on the joinReader called scratchKeyValues that you make once then reset on each lookup similar to

ag.scratch = encoded[:0]
.


Comments from Reviewable

@solongordon
Copy link
Contributor Author

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


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

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

On every lookup, you're creating a new keyValues slice. I'm suggesting you have one on the joinReader called scratchKeyValues that you make once then reset on each lookup similar to

ag.scratch = encoded[:0]
.

Ah got it, thanks. If it's cool with you I think I'll hold off on the scratch slice for now since this should be a small array which is allocated on the stack. Also this code is likely to change when I add batching.


pkg/sql/logictest/testdata/logic_test/lookup_join, line 275 at r2 (raw file):

Previously, solongordon (Solon) wrote…

Maybe this can be addressed in joinreader_test.go? Going to take a look at supplementing that now.

I added a unit test which covers the various cases here (ColumnID, ExtraColumnID, StoreColumnID, and non-covering index.)


Comments from Reviewable

@solongordon solongordon requested a review from a team May 21, 2018 19:40
@solongordon solongordon requested a review from a team May 21, 2018 19:40
@solongordon solongordon force-pushed the secondary-lookup-join branch 2 times, most recently from d8230cf to 39d7192 Compare May 21, 2018 20:00
joinReader now supports lookup joins on secondary indexes. This was a
trivial change for queries where all the output columns are included in
the secondary index. I just modified the physical planner to specify the
secondary index in the JoinReaderSpec and removed checks which prevented
secondary indexes from being used.

The more complicated situation is when we want to do a lookup join
against a non-covering index. In this case, the logical planner plans an
index join before the inner join, but we want to perform the lookup join
first. We now handle this by only planning the lookup join during
physical planning, not the index join. During execution, the joinReader
detects that there are output columns not covered by the secondary
index, and it performs primary index lookups as necessary to retrieve
the additional columns.

Fixes cockroachdb#25431

Release note (sql change): The experimental lookup join feature now
supports secondary indexes.
@RaduBerinde
Copy link
Member

:lgtm:


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


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

Previously, solongordon (Solon) wrote…

Ah got it, thanks. If it's cool with you I think I'll hold off on the scratch slice for now since this should be a small array which is allocated on the stack. Also this code is likely to change when I add batching.

Ok to hold off, but wanted to mention that make never allocates on the stack AFAIK


Comments from Reviewable

@solongordon
Copy link
Contributor Author

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


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

Previously, RaduBerinde wrote…

Ok to hold off, but wanted to mention that make never allocates on the stack AFAIK

Hm, I came across this thread which seems to indicate otherwise, but I'd love to know more. https://groups.google.com/d/msg/golang-nuts/KdbtOqNK6JQ/ehUOmI7LaKwJ


Comments from Reviewable

@petermattis
Copy link
Collaborator

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


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

Previously, solongordon (Solon) wrote…

Hm, I came across this thread which seems to indicate otherwise, but I'd love to know more. https://groups.google.com/d/msg/golang-nuts/KdbtOqNK6JQ/ehUOmI7LaKwJ

My understanding is that Dave Cheney is wrong there and that make always results in a heap allocation. I'd have to test that to be certain. I certainly wouldn't rely on the compiler recognizing that a slice is not escaping and can be allocated on the stack.


Comments from Reviewable

@solongordon
Copy link
Contributor Author

bors r+

craig bot pushed a commit that referenced this pull request May 22, 2018
25005: ui: add top-level LoginContainer: require login before rendering anything r=couchand a=vilterp

Depends on #25057 
Touches #24939

![](https://user-images.githubusercontent.com/7341/39150096-38b4cd28-470f-11e8-9d67-e1832d35a211.gif)

Shown above:
1. go to admin UI; see login screen
2. error message when you type in the wrong password
3. you can't hit an authenticated endpoint because you don't have a valid session (this checking is turned on by #24944, only in secure mode)
4. once you login with the right password, you can see the UI (the temporary "connection lost" banner shouldn't be there)
5. now you (and the UI itself) can hit endpoints, because you have a valid session

Todos:
- [ ] redirect to login page instead of current wrapper component switching thing
- [ ] logout
    - [ ] logout button (make API call; redirect to login & blow away redux store)
    - [ ] log out other tabs (if they get a 403, redirect to login & blow away redux store)
- [ ] styling

Release note: None

25628: distsql: support lookup join on secondary index r=solongordon a=solongordon

joinReader now supports lookup joins on secondary indexes. This was a
trivial change for queries where all the output columns are included in
the secondary index. I just modified the physical planner to specify the
secondary index in the JoinReaderSpec and removed checks which prevented
secondary indexes from being used.

The more complicated situation is when we want to do a lookup join
against a non-covering index. In this case, the logical planner plans an
index join before the inner join, but we want to perform the lookup join
first. We now handle this by only planning the lookup join during
physical planning, not the index join. During execution, the joinReader
detects that there are output columns not covered by the secondary
index, and it performs primary index lookups as necessary to retrieve
the additional columns.

Fixes #25431

Co-authored-by: Pete Vilter <vilterp@cockroachlabs.com>
Co-authored-by: Andrew Couch <andrew@cockroachlabs.com>
Co-authored-by: Solon Gordon <solon@cockroachlabs.com>
@craig
Copy link
Contributor

craig bot commented May 22, 2018

Build succeeded

@craig craig bot merged commit 36c1f98 into cockroachdb:master May 22, 2018
@solongordon
Copy link
Contributor Author

Review status: 0 of 16 files reviewed at latest revision, 7 unresolved discussions, all commit checks successful.


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

Previously, petermattis (Peter Mattis) wrote…

My understanding is that Dave Cheney is wrong there and that make always results in a heap allocation. I'd have to test that to be certain. I certainly wouldn't rely on the compiler recognizing that a slice is not escaping and can be allocated on the stack.

I did an experiment which appears to back up Dave. As far as I can tell the slice is allocated on the heap when it exceeds 8192 bytes, which I suppose is the stack frame size.

solon:~/tmp/stackallocation$ cat main_test.go
package main

import "testing"

func BenchmarkSlice8191(b *testing.B) {
	for i := 0; i < b.N; i++ {
		s := make([]int, 8191)
		s[0] = 1
	}
}

func BenchmarkSlice8192(b *testing.B) {
	for i := 0; i < b.N; i++ {
		s := make([]int, 8192)
		s[0] = 1
	}
}

solon:~/tmp/stackallocation$ go tool compile -m main_test.go
main_test.go:5:28: BenchmarkSlice8191 b does not escape
main_test.go:7:12: BenchmarkSlice8191 make([]int, 8191) does not escape
main_test.go:14:12: make([]int, 8192) escapes to heap
main_test.go:12:28: BenchmarkSlice8192 b does not escape

solon:~/tmp/stackallocation$ go test -bench=.
goos: darwin
goarch: amd64
BenchmarkSlice8191-4   	 1000000	      1248 ns/op
BenchmarkSlice8192-4   	  300000	      4222 ns/op
PASS
ok  	_/Users/solon/tmp/stackallocation	2.583s

Comments from Reviewable

@jordanlewis
Copy link
Member

TIL!

@solongordon
Copy link
Contributor Author

Review status: 0 of 16 files reviewed at latest revision, 7 unresolved discussions, all commit checks successful.


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

Previously, solongordon (Solon) wrote…

I did an experiment which appears to back up Dave. As far as I can tell the slice is allocated on the heap when it exceeds 8192 bytes, which I suppose is the stack frame size.

solon:~/tmp/stackallocation$ cat main_test.go
package main

import "testing"

func BenchmarkSlice8191(b *testing.B) {
	for i := 0; i < b.N; i++ {
		s := make([]int, 8191)
		s[0] = 1
	}
}

func BenchmarkSlice8192(b *testing.B) {
	for i := 0; i < b.N; i++ {
		s := make([]int, 8192)
		s[0] = 1
	}
}

solon:~/tmp/stackallocation$ go tool compile -m main_test.go
main_test.go:5:28: BenchmarkSlice8191 b does not escape
main_test.go:7:12: BenchmarkSlice8191 make([]int, 8191) does not escape
main_test.go:14:12: make([]int, 8192) escapes to heap
main_test.go:12:28: BenchmarkSlice8192 b does not escape

solon:~/tmp/stackallocation$ go test -bench=.
goos: darwin
goarch: amd64
BenchmarkSlice8191-4   	 1000000	      1248 ns/op
BenchmarkSlice8192-4   	  300000	      4222 ns/op
PASS
ok  	_/Users/solon/tmp/stackallocation	2.583s

The plot thickens though: the compiler isn't smart enough to allocate on the stack if the length is not a constant. I think that means it will allocate on the heap in this case after all.

golang/go#20533


Comments from Reviewable

@petermattis
Copy link
Collaborator

Review status: 0 of 16 files reviewed at latest revision, 7 unresolved discussions, all commit checks successful.


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

Previously, solongordon (Solon) wrote…

The plot thickens though: the compiler isn't smart enough to allocate on the stack if the length is not a constant. I think that means it will allocate on the heap in this case after all.

golang/go#20533

Thanks for following up on this. TIL that the compiler will allocate constant length slices on the stack in some instances. That's interesting.


Comments from Reviewable

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.

None yet

6 participants