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

sql: enables distributed distsql queries for multi-tenant #80353

Merged

Conversation

rharding6373
Copy link
Collaborator

@rharding6373 rharding6373 commented Apr 22, 2022

sql: enables distributed distsql queries for multi-tenant

This change allows SQL queries to be distributed in multi-tenant
environments. The distribution algorithm randomly assigns spans to SQL
instances, but if only one instance is used the spans are assigned
instead to the gateway instance. Distribution does not take locality
into account, which will be implemented in a future PR.

This change also supports running execbuilder tests with the
3node-tenant configuration, which is under CCL. These tests can be run
in the following manner:

make test PKG=./pkg/ccl/logictestccl TESTS=TestTenantExecBuild
./dev test pkg/ccl/logictestccl -f=TestTenantExecBuild

Fixes: #80680

Release note: None

@rharding6373 rharding6373 added the do-not-merge bors won't merge a PR with this label. label Apr 22, 2022
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch 3 times, most recently from 3028d75 to 64f6830 Compare April 28, 2022 00:29
@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch 2 times, most recently from 7fc0f2b to 7ab8e53 Compare April 30, 2022 00:14
@rharding6373 rharding6373 removed the do-not-merge bors won't merge a PR with this label. label May 2, 2022
@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch from 7ab8e53 to a046b80 Compare May 2, 2022 16:38
@rharding6373 rharding6373 marked this pull request as ready for review May 2, 2022 16:40
@rharding6373 rharding6373 requested review from a team as code owners May 2, 2022 16:40
Copy link
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.

Nice!

Reviewed 12 of 14 files at r1, 2 of 2 files at r2, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @cucaroach and @rharding6373)


-- commits line 13 at r2:
nit: I feel like we shouldn't be including release notes for multi-tenancy since this is not a user visible change. Unless we have a separate Serverless change label or something.


pkg/ccl/logictestccl/BUILD.bazel line 19 at r2 (raw file):

    data = glob(["testdata/**"]) + [
        "//pkg/sql/logictest:testdata",
        "//c-deps:libgeos",

nit: not sure why it is not automatically so, but these should probably be lexicographically ordered.


pkg/keys/keys.go line 902 at r2 (raw file):

	}
	// Validate that there is a table and index in the new safe key.
	sqlKey, _, err := DecodeTenantPrefix(safeKey)

If DecodeTenantPrefix returns an error, then we probably would have already encountered it above in EnsureSafeSplitKey since it calls GetRowPrefixLength which calls DecodeTenantPrefix, and it looks like we would swallow the error above.

Let me spell out what currently is happening:

  • GetRowPrefixLength via EnsureSafeSplitKey:
    • strip tenant prefix - DecodeTenantPrefix
    • check that the first varlen encoded datum is an int - I think this is a sanity check for the table ID
    • peek at the last byte of the key to find the length of column family ID encoding, make
  • EnsureSafeSplitKey
    • remove the suffix of the key containing the column family ID and a single byte for the length
  • strip tenant prefix
  • decode table ID and index ID.

Maybe a cleaner approach would be:

  • strip tenant prefix
  • strip column family ID suffix if found (using a new helper used by the new method and GetRowPrefixLength)
  • decode table ID and index ID.

But maybe I'm overthinking it, and it's ok that we have some redundancy here.


pkg/sql/logictest/testdata/logic_test/distsql_tenant line 1 at r2 (raw file):

# LogicTest: 3node-tenant

nit: it might be worth re-auditing all logic test files that have !3node-tenant directive to see whether we can run those in the multi-tenant setup. I would guess probably not, but I'd probably know better.


pkg/sql/logictest/testdata/logic_test/distsql_tenant line 16 at r2 (raw file):

# pod, showcase actual distribution.
query T
EXPLAIN SELECT * FROM t WHERE k < 10 OR (k > 20 AND k < 29) OR k > 40

nit: EXPLAIN tests should go into execbuilder folder (it is ok if the actual query execution stays here). This is so that in case EXPLAIN output changes, it is much easier to rewrite it.


pkg/ccl/logictestccl/logic_test.go line 65 at r2 (raw file):

func TestTenantExecBuild(t *testing.T) {
	defer leaktest.AfterTest(t)()
	testdataDir := "../../sql/opt/exec/execbuilder/testdata/"

nit: I'd move it to be after both defers and the skip - it's nice to see all defers and the skip right away together rather than scattered.

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch from a046b80 to c3805b1 Compare May 2, 2022 22:13
Copy link
Collaborator Author

@rharding6373 rharding6373 left a comment

Choose a reason for hiding this comment

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

TFTR!

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


-- commits line 13 at r2:

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: I feel like we shouldn't be including release notes for multi-tenancy since this is not a user visible change. Unless we have a separate Serverless change label or something.

Done. I'm eagerly awaiting the lunch and learn on release notes...


pkg/keys/keys.go line 902 at r2 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

If DecodeTenantPrefix returns an error, then we probably would have already encountered it above in EnsureSafeSplitKey since it calls GetRowPrefixLength which calls DecodeTenantPrefix, and it looks like we would swallow the error above.

Let me spell out what currently is happening:

  • GetRowPrefixLength via EnsureSafeSplitKey:
    • strip tenant prefix - DecodeTenantPrefix
    • check that the first varlen encoded datum is an int - I think this is a sanity check for the table ID
    • peek at the last byte of the key to find the length of column family ID encoding, make
  • EnsureSafeSplitKey
    • remove the suffix of the key containing the column family ID and a single byte for the length
  • strip tenant prefix
  • decode table ID and index ID.

Maybe a cleaner approach would be:

  • strip tenant prefix
  • strip column family ID suffix if found (using a new helper used by the new method and GetRowPrefixLength)
  • decode table ID and index ID.

But maybe I'm overthinking it, and it's ok that we have some redundancy here.

I did some refactoring so that GetRowPrefixLength also checks for a valid index, and that allows us to remove some redundancy and hand-wavy key encoding checks. It changes the error message in some situations, but I haven't seen any other fallout yet.


pkg/sql/logictest/testdata/logic_test/distsql_tenant line 1 at r2 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: it might be worth re-auditing all logic test files that have !3node-tenant directive to see whether we can run those in the multi-tenant setup. I would guess probably not, but I'd probably know better.

Good idea. There's only one test where we could enable it (mostly), due to a single flaky statement.


pkg/sql/logictest/testdata/logic_test/distsql_tenant line 16 at r2 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: EXPLAIN tests should go into execbuilder folder (it is ok if the actual query execution stays here). This is so that in case EXPLAIN output changes, it is much easier to rewrite it.

Removed these and added the second test to execbuilder instead.

Copy link
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 8 of 8 files at r3, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @cucaroach and @rharding6373)


-- commits line 13 at r2:

Previously, rharding6373 (Rachael Harding) wrote…

Done. I'm eagerly awaiting the lunch and learn on release notes...

Don't forget to update the PR description.


pkg/keys/keys.go line 844 at r3 (raw file):

	// Check that the table key also has a valid index ID.
	if _, _, _, err := DecodeTableIDIndexID(sqlKey); err != nil {

Hm, I wonder whether we can (and should?) be more efficient here. Currently, we have the PeekType check above that ensures that the first varlen encoded datum is an int, and then we actually decode two varlen ints. We don't really care about the values of those integers (i.e. table ID and index ID), so maybe we should just peek the type of two varlen datums and that's it?

I'm thinking of something like:

diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go
index 8e768cc8ea..2f683656c2 100644
--- a/pkg/keys/keys.go
+++ b/pkg/keys/keys.go
@@ -835,10 +835,21 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
        }
        sqlN := len(sqlKey)
 
+       // Check that the valid TableID is in the prefix.
        if encoding.PeekType(sqlKey) != encoding.Int {
                // Not a table key, so the row prefix is the entire key.
                return n, nil
        }
+       tableIDLen, err := encoding.GetUvarintLen(sqlKey)
+       if err != nil {
+               // Not a table key, so the row prefix is the entire key.
+               return n, nil
+       }
+       // Check that the valid IndexID follows the TableID in the prefix.
+       if encoding.PeekType(sqlKey[tableIDLen:]) != encoding.Int {
+               // Not a table key, so the row prefix is the entire key.
+               return n, nil
+       }
        // The column family ID length is encoded as a varint and we take advantage
        // of the fact that the column family ID itself will be encoded in 0-9 bytes
        // and thus the length of the column family ID data will fit in a single
diff --git a/pkg/util/encoding/encoding.go b/pkg/util/encoding/encoding.go
index ebe39b1784..dc0fcfa11c 100644
--- a/pkg/util/encoding/encoding.go
+++ b/pkg/util/encoding/encoding.go
@@ -471,6 +471,25 @@ func EncLenUvarintDescending(v uint64) int {
        return 2 + highestByteIndex(v)
 }
 
+// GetUvarintLen returns the length of the prefix that encodes a uint64 value
+// without actually decoding the value.
+func GetUvarintLen(b []byte) (int, error) {
+       if len(b) == 0 {
+               return 0, errors.Errorf("insufficient bytes to decode uvarint value")
+       }
+       length := int(b[0]) - intZero
+       if length <= intSmall {
+               return 1, nil
+       }
+       length -= intSmall
+       if length < 0 || length > 8 {
+               return 0, errors.Errorf("invalid uvarint length of %d", length)
+       } else if len(b) < length {
+               return 0, errors.Errorf("insufficient bytes to decode uvarint value: %q", b)
+       }
+       return 1 + length, nil
+}
+
 // DecodeUvarintAscending decodes a uint64 encoded uint64 from the input
 // buffer. The remainder of the input buffer and the decoded uint64
 // are returned.

I think this code is somewhat performance-sensitive, so it might be worth optimizing it. In particular, it would be good to look at a "macro" benchmark that stresses this code.

Looking at the callsites, GetRowPrefixLength is called by the cFetcher on every first KV of row when we can skip decoding the index key. I think this is the case with SELECT count(*) FROM t queries. We already have the corresponding BenchmarkCount and BenchmarkCountTwoCF in pkg/bench.

I did a quick check of this diff (old):

diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go
index 8e768cc8ea..e8c2559610 100644
--- a/pkg/keys/keys.go
+++ b/pkg/keys/keys.go
@@ -835,7 +835,7 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
        }
        sqlN := len(sqlKey)
 
-       if encoding.PeekType(sqlKey) != encoding.Int {
+       if _, _, _, err = DecodeTableIDIndexID(sqlKey); err != nil {
                // Not a table key, so the row prefix is the entire key.
                return n, nil
        }

against the one I posted above (new):

Count/Cockroach-24             26.4ms ± 1%    25.8ms ± 1%  -2.18%  (p=0.000 n=10+10)
Count/MultinodeCockroach-24    26.4ms ± 0%    25.8ms ± 1%  -2.15%  (p=0.000 n=9+10)
...
CountTwoCF/Cockroach-24             50.7ms ± 1%    49.5ms ± 1%  -2.25%  (p=0.000 n=10+10)
CountTwoCF/MultinodeCockroach-24    50.5ms ± 1%    49.6ms ± 1%  -1.78%  (p=0.000 n=10+10)

So it does seem noticeable but very minor.

Thoughts?


pkg/ccl/logictestccl/logic_test.go line 12 at r3 (raw file):

import (
	"github.com/cockroachdb/cockroach/pkg/sql"

nit: this seems odd.

Copy link
Contributor

@knz knz left a comment

Choose a reason for hiding this comment

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

The release note, if any, should be inside the commit message not the PR description.

Also, it may be worth adding a sentence in the release note that "at this time, no attempt is made to make query processing run close to the data".

Reviewed 7 of 14 files at r1, 1 of 2 files at r2, 8 of 8 files at r3, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @cucaroach and @rharding6373)


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

	}
	if dsp.sqlInstanceProvider == nil {
		return nil, errors.New("sql instance provider not available in multi-tenant environment")

errors.AssertionFailedf - this should never happen


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

	}
	// If spans were only assigned to one SQL instance, then assign them all to
	// the gateway instance.

I'm curious, what is the reasoning for this choice?

If, say, we have a query like SELECT onecolumn FROM table_with_many_columns WHERE ...,
then isn't it beneficial to do the filtering close to the data and only copy the data after the projection across the network?

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch from c3805b1 to 5f5cb2e Compare May 4, 2022 01:15
Copy link
Collaborator Author

@rharding6373 rharding6373 left a comment

Choose a reason for hiding this comment

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

TFTRs! Updated the commit message and PR description.

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


-- commits line 13 at r2:

Previously, yuzefovich (Yahor Yuzefovich) wrote…

Don't forget to update the PR description.

Done.


pkg/keys/keys.go line 844 at r3 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

Hm, I wonder whether we can (and should?) be more efficient here. Currently, we have the PeekType check above that ensures that the first varlen encoded datum is an int, and then we actually decode two varlen ints. We don't really care about the values of those integers (i.e. table ID and index ID), so maybe we should just peek the type of two varlen datums and that's it?

I'm thinking of something like:

diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go
index 8e768cc8ea..2f683656c2 100644
--- a/pkg/keys/keys.go
+++ b/pkg/keys/keys.go
@@ -835,10 +835,21 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
        }
        sqlN := len(sqlKey)
 
+       // Check that the valid TableID is in the prefix.
        if encoding.PeekType(sqlKey) != encoding.Int {
                // Not a table key, so the row prefix is the entire key.
                return n, nil
        }
+       tableIDLen, err := encoding.GetUvarintLen(sqlKey)
+       if err != nil {
+               // Not a table key, so the row prefix is the entire key.
+               return n, nil
+       }
+       // Check that the valid IndexID follows the TableID in the prefix.
+       if encoding.PeekType(sqlKey[tableIDLen:]) != encoding.Int {
+               // Not a table key, so the row prefix is the entire key.
+               return n, nil
+       }
        // The column family ID length is encoded as a varint and we take advantage
        // of the fact that the column family ID itself will be encoded in 0-9 bytes
        // and thus the length of the column family ID data will fit in a single
diff --git a/pkg/util/encoding/encoding.go b/pkg/util/encoding/encoding.go
index ebe39b1784..dc0fcfa11c 100644
--- a/pkg/util/encoding/encoding.go
+++ b/pkg/util/encoding/encoding.go
@@ -471,6 +471,25 @@ func EncLenUvarintDescending(v uint64) int {
        return 2 + highestByteIndex(v)
 }
 
+// GetUvarintLen returns the length of the prefix that encodes a uint64 value
+// without actually decoding the value.
+func GetUvarintLen(b []byte) (int, error) {
+       if len(b) == 0 {
+               return 0, errors.Errorf("insufficient bytes to decode uvarint value")
+       }
+       length := int(b[0]) - intZero
+       if length <= intSmall {
+               return 1, nil
+       }
+       length -= intSmall
+       if length < 0 || length > 8 {
+               return 0, errors.Errorf("invalid uvarint length of %d", length)
+       } else if len(b) < length {
+               return 0, errors.Errorf("insufficient bytes to decode uvarint value: %q", b)
+       }
+       return 1 + length, nil
+}
+
 // DecodeUvarintAscending decodes a uint64 encoded uint64 from the input
 // buffer. The remainder of the input buffer and the decoded uint64
 // are returned.

I think this code is somewhat performance-sensitive, so it might be worth optimizing it. In particular, it would be good to look at a "macro" benchmark that stresses this code.

Looking at the callsites, GetRowPrefixLength is called by the cFetcher on every first KV of row when we can skip decoding the index key. I think this is the case with SELECT count(*) FROM t queries. We already have the corresponding BenchmarkCount and BenchmarkCountTwoCF in pkg/bench.

I did a quick check of this diff (old):

diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go
index 8e768cc8ea..e8c2559610 100644
--- a/pkg/keys/keys.go
+++ b/pkg/keys/keys.go
@@ -835,7 +835,7 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
        }
        sqlN := len(sqlKey)
 
-       if encoding.PeekType(sqlKey) != encoding.Int {
+       if _, _, _, err = DecodeTableIDIndexID(sqlKey); err != nil {
                // Not a table key, so the row prefix is the entire key.
                return n, nil
        }

against the one I posted above (new):

Count/Cockroach-24             26.4ms ± 1%    25.8ms ± 1%  -2.18%  (p=0.000 n=10+10)
Count/MultinodeCockroach-24    26.4ms ± 0%    25.8ms ± 1%  -2.15%  (p=0.000 n=9+10)
...
CountTwoCF/Cockroach-24             50.7ms ± 1%    49.5ms ± 1%  -2.25%  (p=0.000 n=10+10)
CountTwoCF/MultinodeCockroach-24    50.5ms ± 1%    49.6ms ± 1%  -1.78%  (p=0.000 n=10+10)

So it does seem noticeable but very minor.

Thoughts?

Optimizing this seems like a good idea. I've adopted your suggestions with some adjustments. Thank you!


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

Previously, knz (kena) wrote…

errors.AssertionFailedf - this should never happen

Done.


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

Previously, knz (kena) wrote…

I'm curious, what is the reasoning for this choice?

If, say, we have a query like SELECT onecolumn FROM table_with_many_columns WHERE ...,
then isn't it beneficial to do the filtering close to the data and only copy the data after the projection across the network?

This will probably go away once we support locality-aware distribution, at least in this form. There's two reasons why I did this right now:

  1. It avoids an extra hop (KV->assignee->gateway->client vs KV->gateway->client), and this is an optimization we sometimes do in the non-MT code path though at a later stage,
  2. It makes the assignments and distribution type deterministic in testing when we expect to assign to only a single pod.

I added a comment and TODO to remind myself to change this once we support locality-based distribution.


pkg/ccl/logictestccl/logic_test.go line 12 at r3 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: this seems odd.

This was because of TestingOverrideExplainEnvVersion, which is called for ExecBuilder tests. Doesn't look like it's actually used as part of the explain output in any of those tests, though, so I removed it.

Copy link
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 6 of 6 files at r4, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @cucaroach, @knz, and @rharding6373)


pkg/util/encoding/encoding.go line 56 at r4 (raw file):

	// The gap between floatNaNDesc and bytesMarker was left for
	// compatibility reasons.
	bytesMarker     byte = 0x12

nit: indentation is off.


pkg/util/encoding/encoding.go line 476 at r4 (raw file):

// GetUvarintLen returns the length of the prefix that encodes a uint64 value
// without actually decoding the value.
func GetUvarintLen(b []byte) (int, error) {

It'd be good to add some unit tests for this.


pkg/ccl/logictestccl/logic_test.go line 12 at r3 (raw file):

Previously, rharding6373 (Rachael Harding) wrote…

This was because of TestingOverrideExplainEnvVersion, which is called for ExecBuilder tests. Doesn't look like it's actually used as part of the explain output in any of those tests, though, so I removed it.

(Sorry, I meant the order of imports seemed odd, should have been more specific.)

Copy link
Contributor

@cucaroach cucaroach left a comment

Choose a reason for hiding this comment

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

This is first jira link I've come across, what's the deal with that? (Seinfeld voice) I thought there was a 1:1 correspondence with jiras and GH issues?

Reviewed 7 of 14 files at r1, 1 of 2 files at r2, 6 of 8 files at r3, 4 of 6 files at r4, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @knz and @rharding6373)

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch from 5f5cb2e to 7a1d2a2 Compare May 4, 2022 21:47
Copy link
Collaborator Author

@rharding6373 rharding6373 left a comment

Choose a reason for hiding this comment

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

Removed the link to the epic from the PR description and added the related github issue instead. I don't know if there's a normally a 1:1 mapping.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @cucaroach, @knz, and @yuzefovich)


pkg/util/encoding/encoding.go line 476 at r4 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

It'd be good to add some unit tests for this.

Done.


pkg/ccl/logictestccl/logic_test.go line 12 at r3 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

(Sorry, I meant the order of imports seemed odd, should have been more specific.)

The ordering should be fixed now, too.

Copy link
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 2 of 2 files at r5, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @knz and @rharding6373)


pkg/util/encoding/encoding.go line 475 at r5 (raw file):

// GetUvarintLen returns the length of the prefix that encodes a uint64 value
// without actually decoding the value.

nit: we can explicitly mention in the comment that an error is returned IFF b does not contain a valid encoding of an unsigned int datum (this already happens in the code). Then we can drop tableIDLen >= sqlN check in GetRowPrefixLength.

Hm, I guess there is a possibility that DecodeUvarintAscending is modified without the corresponding changes to GetUvarintLen. Maybe to add a sanity check, we can modify DecodeUvarintAscending to insert a call to GetUvarintLen behind buildutil.CrdbTestBuild while asserting that the error return argument of DecodeUvarintAscending is exactly the same as of GetUvarintLen. Thoughts?

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch 2 times, most recently from 8d29f1b to 57804dd Compare May 17, 2022 17:58
Copy link
Collaborator Author

@rharding6373 rharding6373 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 1 stale) (waiting on @cucaroach, @knz, and @yuzefovich)


pkg/util/encoding/encoding.go line 475 at r5 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: we can explicitly mention in the comment that an error is returned IFF b does not contain a valid encoding of an unsigned int datum (this already happens in the code). Then we can drop tableIDLen >= sqlN check in GetRowPrefixLength.

Hm, I guess there is a possibility that DecodeUvarintAscending is modified without the corresponding changes to GetUvarintLen. Maybe to add a sanity check, we can modify DecodeUvarintAscending to insert a call to GetUvarintLen behind buildutil.CrdbTestBuild while asserting that the error return argument of DecodeUvarintAscending is exactly the same as of GetUvarintLen. Thoughts?

Done.

Instead of adding a check to the code itself, I added a decode step to the unit test TestGetUvarintLen, which should catch this kind of error without extra overhead in test builds.

Copy link
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 9 of 9 files at r6, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @knz and @rharding6373)


pkg/keys/keys.go line 851 at r6 (raw file):

	// all keys contain an index ID.
	if encoding.PeekType(sqlKey[tableIDLen:]) == encoding.Int {
		indexIDLen, err := encoding.GetUvarintLen(sqlKey[tableIDLen:])

I think err != nil check is missing.


pkg/keys/keys_test.go line 635 at r6 (raw file):

		// Same test cases, but for tenant 5.
		{e5(), e5()},                     // Not a table key
		{e5(1, 2), e5(1, 2)},             // /Tenant5/Table/1/2 -> /Tenant5/Table/1/2

nit: Tenant5 -> Tenant/5 for consistency with below.


pkg/keys/keys_test.go line 687 at r6 (raw file):

		// Exercises a former overflow bug. We decode a uint(18446744073709551610) which, if cast
		// to int carelessly, results in -6.
		{encoding.EncodeVarintAscending(encoding.EncodeVarintAscending(tenSysCodec.TablePrefix(999), 2), 322434), "malformed table key"},

Hm, I'm puzzled by this change - can you provide some context why we need double encoding?

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch from 57804dd to e305611 Compare May 17, 2022 21:47
Copy link
Collaborator Author

@rharding6373 rharding6373 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 1 stale) (waiting on @knz and @yuzefovich)


pkg/keys/keys.go line 851 at r6 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

I think err != nil check is missing.

Done.


pkg/keys/keys_test.go line 687 at r6 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

Hm, I'm puzzled by this change - can you provide some context why we need double encoding?

Since adding an optional check for the index id, the old test doesn't trigger the intended case. I simplified this a bit and expanded the comment, PTAL.

Copy link
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 3 of 3 files at r7, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @knz and @rharding6373)


pkg/keys/keys.go line 850 at r7 (raw file):

	// Check whether the prefix contains a valid IndexID after the TableID. Not
	// all keys contain an index ID.
	if encoding.PeekType(sqlKey[tableIDLen:]) == encoding.Int {

Sorry to be annoying about this, but if this check is false, shouldn't we just return n, nil? The reasoning being is that if a key has a TableID but doesn't have an IndexID, then we know it's not a table key, so the row prefix is the entire key (similar to the check above). Currently, we'll try to decode the column family suffix from the key and might encounter an error. I think it should be possible to add a test case that verifies the behavior in such a scenario.


pkg/util/encoding/encoding.go line 488 at r7 (raw file):

	if length < 0 || length > 8 {
		return 0, errors.Errorf("invalid uvarint length of %d", length)
	} else if len(b) <= length {

This change is suspicious, what is the reasoning behind it?

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch from e305611 to c1086e5 Compare May 18, 2022 00:00
Copy link
Collaborator Author

@rharding6373 rharding6373 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 1 stale) (waiting on @knz and @yuzefovich)


pkg/keys/keys.go line 850 at r7 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

Sorry to be annoying about this, but if this check is false, shouldn't we just return n, nil? The reasoning being is that if a key has a TableID but doesn't have an IndexID, then we know it's not a table key, so the row prefix is the entire key (similar to the check above). Currently, we'll try to decode the column family suffix from the key and might encounter an error. I think it should be possible to add a test case that verifies the behavior in such a scenario.

Not annoying at all. This library is used in several parts of the code and getting changes right for all use cases has been a challenge, so I appreciate all the feedback as this undergoes so many iterations!

I implemented your suggestion and added more test cases to the unit test to try to capture more expected behavior. For my own education, it is possible to have a non-table key that has a table ID? In one of the test failures I was debugging, it seemed like there was a key passed through this function that had a table ID, no index, but several other bytes in the encoding. Maybe the appearance of the table ID was due to the String function, though...


pkg/util/encoding/encoding.go line 488 at r7 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

This change is suspicious, what is the reasoning behind it?

There was a bug here which I only found when I no longer checked the length returned against the length of the sqlKey in GetRowPrefixLength. If you compare this code to DecodeUvarintAscending, DecodeUvarintAscending removes a byte from the front of b before doing this check, whereas here we do not, so we need to account for the extra byte when comparing the lengths.

Copy link
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 6 of 6 files at r8, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @knz and @rharding6373)


pkg/keys/keys.go line 850 at r7 (raw file):

it is possible to have a non-table key that has a table ID?

I think it's unlikely (and I'm pretty sure it won't happen in the SQL land), but in other parts of the database any key might be plausible - like when ranges split, in theory they could get split at any key in the middle of the original range, but my understanding there is hazy. We should be calling EnsureSafeSplitKey there which makes sure that the split point doesn't happen to be between KVs for the same row, but all other keys seem like a fair game.


pkg/util/encoding/encoding.go line 488 at r7 (raw file):

Previously, rharding6373 (Rachael Harding) wrote…

There was a bug here which I only found when I no longer checked the length returned against the length of the sqlKey in GetRowPrefixLength. If you compare this code to DecodeUvarintAscending, DecodeUvarintAscending removes a byte from the front of b before doing this check, whereas here we do not, so we need to account for the extra byte when comparing the lengths.

Indeed, nice catch! Maybe add this note as a comment (linking to DecodeUvarintAscending and explaining the difference)?

This change allows SQL queries to be distributed in multi-tenant
environments. The distribution algorithm randomly assigns spans to SQL
instances, but if only one instance is used the spans are assigned
instead to the gateway instance. Distribution does not take locality
into account, which will be implemented in a future PR.

This change also supports running execbuilder tests with the
3node-tenant configuration, which is under CCL. These tests can be run
in the following manner:

```
make test PKG=./pkg/ccl/logictestccl TESTS=TestTenantExecBuild
./dev test pkg/ccl/logictestccl -f=TestTenantExecBuild
```

Fixes: cockroachdb#80680

Release note: None
@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_partition_spans branch from c1086e5 to 9e4364a Compare May 18, 2022 17:45
Copy link
Collaborator Author

@rharding6373 rharding6373 left a comment

Choose a reason for hiding this comment

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

TFTRs!

bors r+

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


pkg/keys/keys.go line 850 at r7 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

it is possible to have a non-table key that has a table ID?

I think it's unlikely (and I'm pretty sure it won't happen in the SQL land), but in other parts of the database any key might be plausible - like when ranges split, in theory they could get split at any key in the middle of the original range, but my understanding there is hazy. We should be calling EnsureSafeSplitKey there which makes sure that the split point doesn't happen to be between KVs for the same row, but all other keys seem like a fair game.

Done.


pkg/util/encoding/encoding.go line 488 at r7 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

Indeed, nice catch! Maybe add this note as a comment (linking to DecodeUvarintAscending and explaining the difference)?

Done.

@craig
Copy link
Contributor

craig bot commented May 18, 2022

Build failed (retrying...):

@craig
Copy link
Contributor

craig bot commented May 18, 2022

Build succeeded:

@craig craig bot merged commit d2b5acf into cockroachdb:master May 18, 2022
@rharding6373 rharding6373 deleted the distsql_multi_tenant_partition_spans branch June 3, 2022 18:04
yuzefovich added a commit to yuzefovich/cockroach that referenced this pull request Jun 23, 2023
This commit removes the logic that might reassign some spans to the
gateway that can be applied in the multi-tenant deployment mode. This
logic was introduced in cockroachdb#80353 with the following [rationale](cockroachdb#80353 (review)):
```
This will probably go away once we support locality-aware distribution,
at least in this form. There's two reasons why I did this right now:
1. It avoids an extra hop (KV->assignee->gateway->client vs
KV->gateway->client), and this is an optimization we sometimes do in
the non-MT code path though at a later stage,
2. It makes the assignments and distribution type deterministic in
testing when we expect to assign to only a single pod.
```
Since then the locality-aware planning has been implemented (addressing
point 1.). Also, point 2. (`maybeMoveSingleFlowToGateway`) applies both
to single-tenant and multi-tenant modes. Additionally, running TPCH
queries experimentally has shown that this reassigning of single
TableReader can make the query latency significantly different
(sometimes reduce it, sometimes increase it) (see [here](cockroachdb#104379 (comment))).

All of these reasons suggest that we should just get rid off this logic
to unify the DistSQL physical planner more between single-tenant and
multi-tenant modes.

Release note: None
yuzefovich added a commit to yuzefovich/cockroach that referenced this pull request Jun 23, 2023
This commit removes the logic that might reassign some spans to the
gateway that can be applied in the multi-tenant deployment mode. This
logic was introduced in cockroachdb#80353 with the following [rationale](cockroachdb#80353 (review)):
```
This will probably go away once we support locality-aware distribution,
at least in this form. There's two reasons why I did this right now:
1. It avoids an extra hop (KV->assignee->gateway->client vs
KV->gateway->client), and this is an optimization we sometimes do in
the non-MT code path though at a later stage,
2. It makes the assignments and distribution type deterministic in
testing when we expect to assign to only a single pod.
```
Since then the locality-aware planning has been implemented (addressing
first half of point 1.). Also, the second half of point 1. applies both
to single-tenant and multi-tenant modes (`maybeMoveSingleFlowToGateway`).
Point 2. is a bit unfortunate and I'm not sure what to do about it yet
(we'll need to figure it out if we ever make separate-process
multi-tenant the default mode for running tests that check DistSQL
planning). For now only a single test needed an adjustment to make it
deterministic.

Additionally, running TPCH queries experimentally has shown that this
reassigning of single TableReader can make the query latency
significantly different (sometimes reduce it, sometimes increase it)
(see [here](cockroachdb#104379 (comment))).

All of these reasons suggest that we should just get rid off this logic
to unify the DistSQL physical planner more between single-tenant and
multi-tenant modes.

Release note: None
yuzefovich added a commit to yuzefovich/cockroach that referenced this pull request Jun 28, 2023
This commit removes the logic that might reassign some spans to the
gateway that can be applied in the multi-tenant deployment mode. This
logic was introduced in cockroachdb#80353 with the following [rationale](cockroachdb#80353 (review)):
```
This will probably go away once we support locality-aware distribution,
at least in this form. There's two reasons why I did this right now:
1. It avoids an extra hop (KV->assignee->gateway->client vs
KV->gateway->client), and this is an optimization we sometimes do in
the non-MT code path though at a later stage,
2. It makes the assignments and distribution type deterministic in
testing when we expect to assign to only a single pod.
```
Since then the locality-aware planning has been implemented (addressing
first half of point 1.). Also, the second half of point 1. applies both
to single-tenant and multi-tenant modes (`maybeMoveSingleFlowToGateway`).
Point 2. is a bit unfortunate and I'm not sure what to do about it yet
(we'll need to figure it out if we ever make separate-process
multi-tenant the default mode for running tests that check DistSQL
planning). For now only a single test needed an adjustment to make it
deterministic.

Additionally, running TPCH queries experimentally has shown that this
reassigning of single TableReader can make the query latency
significantly different (sometimes reduce it, sometimes increase it)
(see [here](cockroachdb#104379 (comment))).

All of these reasons suggest that we should just get rid off this logic
to unify the DistSQL physical planner more between single-tenant and
multi-tenant modes.

Release note: None
craig bot pushed a commit that referenced this pull request Jun 29, 2023
105256: sql: adjust DistSQL physical planning in multi-tenant mode r=yuzefovich a=yuzefovich

This commit removes the logic that might reassign some spans to the
gateway that can be applied in the multi-tenant deployment mode. This
logic was introduced in #80353 with the following [rationale](#80353 (review)):
```
This will probably go away once we support locality-aware distribution,
at least in this form. There's two reasons why I did this right now:
1. It avoids an extra hop (KV->assignee->gateway->client vs
KV->gateway->client), and this is an optimization we sometimes do in
the non-MT code path though at a later stage,
2. It makes the assignments and distribution type deterministic in
testing when we expect to assign to only a single pod.
```
Since then the locality-aware planning has been implemented (addressing
first half of point 1.). Also, the second half of point 1. applies both
to single-tenant and multi-tenant modes (`maybeMoveSingleFlowToGateway`).
Point 2. is a bit unfortunate and I'm not sure what to do about it yet
(we'll need to figure it out if we ever make separate-process
multi-tenant the default mode for running tests that check DistSQL
planning). For now only a single test needed an adjustment to make it
deterministic.

Additionally, running TPCH queries experimentally has shown that this
reassigning of single TableReader can make the query latency
significantly different (sometimes reduce it, sometimes increase it)
(see [here](#104379 (comment))).

All of these reasons suggest that we should just get rid off this logic
to unify the DistSQL physical planner more between single-tenant and
multi-tenant modes.

Addresses: #104379.
Epic: CRDB-26687

Release note: None

105569: roachtest: harden and extend cancel roachtest r=yuzefovich a=yuzefovich

This commit hardens `cancel` roachtest. In particular, this test
involves two goroutines: the runner that is executing longer running
TPCH query and the main goroutine that cancels that query. Previously,
in order to ensure that the main goroutine attempts to cancel the query
at the right moment, we slept for 250ms. Then, we would cancel all
running non-internal queries other than `SHOW CLUSTER QUERIES` itself.
This was problematic for a couple of reasons:
- the TPCH query might not have started yet (due some Go scheduling
delays)
- we could actually try to cancel one of the setup queries (the runner
does `USE tpch;` and `SET distsql = off;` before running the TPCH
query).

In order to address the first reason, this commit adjusts the runner to
notify the main goroutine only after the setup queries are done and
introduces the polling loop to wait until the TPCH query shows up. That
polling loop will now randomly sleep for a random duration up to 1000ms
(in order to improve the test coverage of both the optimizer and the
execution engine). Note that we only check that the cancellation
occurred within 3s (used to be 5s before this commit), so we don't
sufficiently exercise the optimizer cancellation (which isn't the
primary goal of this test anyway).

The second reason is addressed by blocking the main goroutine until the
setup queries are done.

Fixes: #105528.

Release note: None

105817: docs-issue-generation: add related PRs link to product change issue body r=nickvigilante a=nickvigilante

Fixes DEVINF-447

Release note: None

Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Nick Vigilante <[email protected]>
adityamaru pushed a commit to adityamaru/cockroach that referenced this pull request Oct 26, 2023
This commit removes the logic that might reassign some spans to the
gateway that can be applied in the multi-tenant deployment mode. This
logic was introduced in cockroachdb#80353 with the following [rationale](cockroachdb#80353 (review)):
```
This will probably go away once we support locality-aware distribution,
at least in this form. There's two reasons why I did this right now:
1. It avoids an extra hop (KV->assignee->gateway->client vs
KV->gateway->client), and this is an optimization we sometimes do in
the non-MT code path though at a later stage,
2. It makes the assignments and distribution type deterministic in
testing when we expect to assign to only a single pod.
```
Since then the locality-aware planning has been implemented (addressing
first half of point 1.). Also, the second half of point 1. applies both
to single-tenant and multi-tenant modes (`maybeMoveSingleFlowToGateway`).
Point 2. is a bit unfortunate and I'm not sure what to do about it yet
(we'll need to figure it out if we ever make separate-process
multi-tenant the default mode for running tests that check DistSQL
planning). For now only a single test needed an adjustment to make it
deterministic.

Additionally, running TPCH queries experimentally has shown that this
reassigning of single TableReader can make the query latency
significantly different (sometimes reduce it, sometimes increase it)
(see [here](cockroachdb#104379 (comment))).

All of these reasons suggest that we should just get rid off this logic
to unify the DistSQL physical planner more between single-tenant and
multi-tenant modes.

Release note: None
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.

distsql: add support for distributed sql queries in multi-tenant environments
5 participants