Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
90776: lint: update commit message to include issue/epic refs r=nickvigilante a=nickvigilante

The standard commit message now includes some help text on including issue and epic references.

Fixes #90303 

Release note: None

91776: sql: use new zone config read/write path through collection r=chengxiong-ruan a=chengxiong-ruan

part of #88571

This is based on #91721

Replace all use cases all metadata updater with the new read/write path of zone configs.

92603: kv: allow secondary tenants to view range metadata for their ranges  r=knz a=arulajmani

This patch introduces a new GetRangeDescriptorsRPC to scan over range
descriptors. The RPC returns a consistent view of range descriptors in
the system for ranges that overlap with the requested keyspan. The RPC
is setup to allow for a streaming response in the future, but for now,
we only return a single response. Care must be taken to uphold the
consistency semantics if we were to make use of the streaming behaviour
in the future.

This RPC is then used to allow secondary tenants to view range metadata
for ranges they own. This is done by proxying scans through the
`Connector` for secondary tenants, and performing the usual keybound
authorization we do in other places.

This patch serves as the enabler for secondary tenants to do things
like `SHOW RANGES` or have scoped access to
`crdb_internal.ranges_no_leases` in the future.

Release note: None

92837: acceptance: fix TestComposeGSS r=rafiss a=rail

Previously, the TestComposeGSS was broken, because the base docker image (`postgres:11`) used APT repos, that had been moved to a different location.

This PR changes the base image to `postgres:15`. The upgrade should be relatively safe.

Fixes #91420
Epic: None
Release note: None

Co-authored-by: Nick Vigilante <[email protected]>
Co-authored-by: Chengxiong Ruan <[email protected]>
Co-authored-by: Arul Ajmani <[email protected]>
Co-authored-by: Rail Aliiev <[email protected]>
  • Loading branch information
5 people committed Dec 1, 2022
5 parents 2ac677c + 4a8b490 + ae5c0f6 + 5931f03 + cd1baf0 commit 80c4287
Show file tree
Hide file tree
Showing 81 changed files with 1,036 additions and 452 deletions.
36 changes: 35 additions & 1 deletion githooks/prepare-commit-msg
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,33 @@ $cchar Note: to disable this commit template, run: git config --global --add coc
fi
fi

ghIssuePart="(#\d+)"
ghIssueRepoPart="([\w.-]+[/][\w.-]+#\d+)"
ghURLPart="(https://github.com/[-a-z0-9]+/[-._a-z0-9/]+/issues/\d+)"
jiraIssuePart="([[:alpha:]]+-\d+)"
jiraURLPart="https://cockroachlabs.atlassian.net/browse/${jiraIssuePart}"
issueRefPart="${ghIssuePart}|${ghIssueRepoPart}|${ghURLPart}|${jiraIssuePart}|${jiraURLPart}"
afterRefPart="[,.;]?(?:[ \t\n\r]+|\$)"
fixIssueRefRE="(?i:close[sd]?|fix(?:e[sd])?|resolve[sd]?):?\s+(?:(?:${issueRefPart})${afterRefPart})+"
informIssueRefRE="(?:part of|see also|informs):?\s+(?:(?:${issueRefPart})${afterRefPart})+"
epicRefRE="epic:?\s+(?:(?:${jiraIssuePart}|${jiraURLPart})${afterRefPart})+"
epicNoneRE="epic:?\s+(?:(none)${afterRefPart})+"

# Add an issue or epic reference.
if ! grep -q -i -E "^${fixIssueRefRE}|${informIssueRefRE}|${epicRefRE}|${epicNoneRE}" "$1"; then
sed_script+="/$cchar Please enter the commit message for your changes./i\\
$cchar Please enter a valid issue or epic reference:\\
${cchar}Epic: none\\
${cchar} ^-- not related to an issue or an epic\\
${cchar}Fixes: #77376\\
${cchar}Part of: https://cockroachlabs.atlassian.net/browse/DOC-1355\\
${cchar}Informs: https://github.com/cockroachdb/cockroach/issues/33316\\
${cchar}Epic: CRDB-8035\\
;
"
fi

# Add an explicit "Release note: None" if no release note was specified.
if ! grep -q '^Release note' "$1"; then
sed_script+="/$cchar Please enter the commit message for your changes./i\\
Expand Down Expand Up @@ -115,6 +142,9 @@ $cchar <what was there before: Previously, ...>\\
$cchar <why it needed to change: This was inadequate because ...>\\
$cchar <what you did about it: To address this, this patch ...>\\
$cchar\\
$cchar Fixes <GH/Jira issue ID/URL to GH/Jira issue>\\
$cchar ---\\
$cchar\\
$cchar Release note (<category>): <what> <show> <why>\\
$cchar ---\\
$cchar\\
Expand All @@ -134,7 +164,11 @@ $cchar\\
"
fi

sed_script+="$cchar The release note must be present if your commit has user-facing\\
sed_script+="$cchar An issue or epic reference must be in the PR body or each commit message\\
$cchar if the PR or commit is part of an issue or epic. Use \\\`Epic: none\\\` otherwise.\\
$cchar See also: https://wiki.crdb.io/wiki/spaces/CRDB/pages/2009039063/\\
$cchar\\
$cchar The release note must be present if your commit has user-facing\\
$cchar or backward-incompatible changes. Use 'Release note: None' otherwise.\\
$cchar\\
$cchar Things to keep in mind for release notes:\\
Expand Down
8 changes: 4 additions & 4 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -593,7 +593,7 @@ ALL_TESTS = [
"//pkg/util/quantile:quantile_test",
"//pkg/util/quotapool:quotapool_test",
"//pkg/util/randutil:randutil_test",
"//pkg/util/rangedesciter:rangedesciter_test",
"//pkg/util/rangedesc:rangedesc_test",
"//pkg/util/retry:retry_test",
"//pkg/util/ring:ring_test",
"//pkg/util/schedulerlatency:schedulerlatency_test",
Expand Down Expand Up @@ -2054,8 +2054,8 @@ GO_TARGETS = [
"//pkg/util/quotapool:quotapool_test",
"//pkg/util/randutil:randutil",
"//pkg/util/randutil:randutil_test",
"//pkg/util/rangedesciter:rangedesciter",
"//pkg/util/rangedesciter:rangedesciter_test",
"//pkg/util/rangedesc:rangedesc",
"//pkg/util/rangedesc:rangedesc_test",
"//pkg/util/retry:retry",
"//pkg/util/retry:retry_test",
"//pkg/util/ring:ring",
Expand Down Expand Up @@ -3052,7 +3052,7 @@ GET_X_DATA_TARGETS = [
"//pkg/util/quantile:get_x_data",
"//pkg/util/quotapool:get_x_data",
"//pkg/util/randutil:get_x_data",
"//pkg/util/rangedesciter:get_x_data",
"//pkg/util/rangedesc:get_x_data",
"//pkg/util/retry:get_x_data",
"//pkg/util/ring:get_x_data",
"//pkg/util/schedulerlatency:get_x_data",
Expand Down
2 changes: 1 addition & 1 deletion pkg/acceptance/compose/gss/psql/Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ COPY . .
RUN go test -v -c -tags gss_compose -o gss.test

# Copy the test binary to an image with psql and krb installed.
FROM postgres:11
FROM postgres:15

RUN apt-get update && \
DEBIAN_FRONTEND=noninteractive apt-get install --yes --no-install-recommends \
Expand Down
2 changes: 0 additions & 2 deletions pkg/acceptance/compose_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,11 @@ import (

"github.com/cockroachdb/cockroach/pkg/acceptance/cluster"
"github.com/cockroachdb/cockroach/pkg/build/bazel"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
)

const composeDir = "compose"

func TestComposeGSS(t *testing.T) {
skip.WithIssue(t, 91420)
testCompose(t, filepath.Join("gss", "docker-compose.yml"), "psql")
}

Expand Down
12 changes: 6 additions & 6 deletions pkg/bench/rttanalysis/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
Expand Up @@ -87,12 +87,12 @@ exp,benchmark
1,SystemDatabaseQueries/select_system.users_with_empty_database_Name
1,SystemDatabaseQueries/select_system.users_with_schema_Name
2,SystemDatabaseQueries/select_system.users_without_schema_Name
15,Truncate/truncate_1_column_0_rows
15,Truncate/truncate_1_column_1_row
15,Truncate/truncate_1_column_2_rows
15,Truncate/truncate_2_column_0_rows
15,Truncate/truncate_2_column_1_rows
15,Truncate/truncate_2_column_2_rows
13,Truncate/truncate_1_column_0_rows
13,Truncate/truncate_1_column_1_row
13,Truncate/truncate_1_column_2_rows
13,Truncate/truncate_2_column_0_rows
13,Truncate/truncate_2_column_1_rows
13,Truncate/truncate_2_column_2_rows
1,VirtualTableQueries/select_crdb_internal.invalid_objects_with_1_fk
1,VirtualTableQueries/select_crdb_internal.tables_with_1_fk
4,VirtualTableQueries/virtual_table_cache_with_point_lookups
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1027,6 +1027,7 @@ func createImportingDescriptors(
txn,
p.ExecCfg(),
descsCol,
p.ExtendedEvalContext().Tracing.KVTracingEnabled(),
); err != nil {
return err
}
Expand Down Expand Up @@ -1175,6 +1176,7 @@ func createImportingDescriptors(
ctx,
txn,
p.ExecCfg(),
p.ExtendedEvalContext().Tracing.KVTracingEnabled(),
descsCol,
regionConfig,
mutTable,
Expand Down
20 changes: 10 additions & 10 deletions pkg/ccl/benchccl/rttanalysisccl/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
@@ -1,19 +1,19 @@
exp,benchmark
13,AlterPrimaryRegion/alter_empty_database_alter_primary_region
16,AlterPrimaryRegion/alter_empty_database_set_initial_primary_region
13,AlterPrimaryRegion/alter_populated_database_alter_primary_region
17,AlterPrimaryRegion/alter_populated_database_set_initial_primary_region
11,AlterPrimaryRegion/alter_empty_database_alter_primary_region
14,AlterPrimaryRegion/alter_empty_database_set_initial_primary_region
11,AlterPrimaryRegion/alter_populated_database_alter_primary_region
15,AlterPrimaryRegion/alter_populated_database_set_initial_primary_region
11,AlterRegions/alter_empty_database_add_region
13,AlterRegions/alter_empty_database_drop_region
11,AlterRegions/alter_populated_database_add_region
13,AlterRegions/alter_populated_database_drop_region
13,AlterSurvivalGoals/alter_empty_database_from_region_to_zone
13,AlterSurvivalGoals/alter_empty_database_from_zone_to_region
33,AlterSurvivalGoals/alter_populated_database_from_region_to_zone
33,AlterSurvivalGoals/alter_populated_database_from_zone_to_region
11,AlterSurvivalGoals/alter_empty_database_from_region_to_zone
11,AlterSurvivalGoals/alter_empty_database_from_zone_to_region
21,AlterSurvivalGoals/alter_populated_database_from_region_to_zone
21,AlterSurvivalGoals/alter_populated_database_from_zone_to_region
14,AlterTableLocality/alter_from_global_to_rbr
14,AlterTableLocality/alter_from_global_to_regional_by_table
12,AlterTableLocality/alter_from_global_to_regional_by_table
10,AlterTableLocality/alter_from_rbr_to_global
10,AlterTableLocality/alter_from_rbr_to_regional_by_table
14,AlterTableLocality/alter_from_regional_by_table_to_global
12,AlterTableLocality/alter_from_regional_by_table_to_global
14,AlterTableLocality/alter_from_regional_by_table_to_rbr
4 changes: 4 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
name = "kvtenantccl",
srcs = [
"connector.go",
"rangedesc_iter.go",
"setting_overrides.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl",
Expand All @@ -26,6 +27,7 @@ go_library(
"//pkg/util/grpcutil",
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/rangedesc",
"//pkg/util/retry",
"//pkg/util/syncutil",
"//pkg/util/syncutil/singleflight",
Expand All @@ -47,6 +49,7 @@ go_test(
"setting_overrides_test.go",
"tenant_kv_test.go",
"tenant_range_lookup_test.go",
"tenant_scan_range_descriptors_test.go",
"tenant_trace_test.go",
"tenant_upgrade_test.go",
],
Expand Down Expand Up @@ -91,6 +94,7 @@ go_test(
"//pkg/util/netutil",
"//pkg/util/protoutil",
"//pkg/util/randutil",
"//pkg/util/rangedesc",
"//pkg/util/retry",
"//pkg/util/stop",
"//pkg/util/tracing/tracingpb",
Expand Down
46 changes: 46 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/rangedesc"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil/singleflight"
Expand Down Expand Up @@ -437,6 +438,51 @@ func (c *Connector) FirstRange() (*roachpb.RangeDescriptor, error) {
return nil, status.Error(codes.Unauthenticated, "kvtenant.Proxy does not have access to FirstRange")
}

// NewIterator implements the rangedesc.IteratorFactory interface.
func (c *Connector) NewIterator(
ctx context.Context, span roachpb.Span,
) (rangedesc.Iterator, error) {
var rangeDescriptors []roachpb.RangeDescriptor
for ctx.Err() == nil {
rangeDescriptors = rangeDescriptors[:0] // clear out.
client, err := c.getClient(ctx)
if err != nil {
continue
}
stream, err := client.GetRangeDescriptors(ctx, &roachpb.GetRangeDescriptorsRequest{
Span: span,
})
if err != nil {
// TODO(arul): We probably don't want to treat all errors here as "soft".
// for example, it doesn't make much sense to retry the request if it fails
// the keybounds check.
// Soft RPC error. Drop client and retry.
log.Warningf(ctx, "error issuing GetRangeDescriptors RPC: %v", err)
c.tryForgetClient(ctx, client)
continue
}

for ctx.Err() == nil {
e, err := stream.Recv()
if err != nil {
if err == io.EOF {
return &rangeDescIterator{
rangeDescs: rangeDescriptors,
curIdx: 0,
}, nil
}
// TODO(arul): We probably don't want to treat all errors here as "soft".
// Soft RPC error. Drop client and retry.
log.Warningf(ctx, "error consuming GetRangeDescriptors RPC: %v", err)
c.tryForgetClient(ctx, client)
break
}
rangeDescriptors = append(rangeDescriptors, e.RangeDescriptors...)
}
}
return nil, ctx.Err()
}

// TokenBucket implements the kvtenant.TokenBucketProvider interface.
func (c *Connector) TokenBucket(
ctx context.Context, in *roachpb.TokenBucketRequest,
Expand Down
6 changes: 6 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/connector_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,12 @@ func (m *mockServer) SpanConfigConformance(
panic("unimplemented")
}

func (m *mockServer) GetRangeDescriptors(
*roachpb.GetRangeDescriptorsRequest, roachpb.Internal_GetRangeDescriptorsServer,
) error {
panic("unimplemented")
}

func gossipEventForClusterID(clusterID uuid.UUID) *roachpb.GossipSubscriptionEvent {
return &roachpb.GossipSubscriptionEvent{
Key: gossip.KeyClusterID,
Expand Down
33 changes: 33 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/rangedesc_iter.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
// Copyright 2022 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package kvtenantccl

import "github.com/cockroachdb/cockroach/pkg/roachpb"

// rangeDescIterator is the concrete (private) implementation of the
// rangedesc.Iterator interface used by the Connector.
type rangeDescIterator struct {
rangeDescs []roachpb.RangeDescriptor
curIdx int
}

// Valid implements the rangedesc.Iterator interface.
func (i *rangeDescIterator) Valid() bool {
return i.curIdx < len(i.rangeDescs)
}

// Next implements the rangedesc.Iterator interface.
func (i *rangeDescIterator) Next() {
i.curIdx++
}

// CurRangeDescriptor implements the rangedesc.Iterator interface.
func (i *rangeDescIterator) CurRangeDescriptor() roachpb.RangeDescriptor {
return i.rangeDescs[i.curIdx]
}
Loading

0 comments on commit 80c4287

Please sign in to comment.