Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
86236: tracing: re-use childrenMetadata map across trace spans r=nvanbenschoten a=nvanbenschoten

This commit pools the `childrenMetadata` hash map across trace spans,
to avoid a source of per-span heap allocations. This hash map was
added in 4ddc350, after prior optimization passes were made over the
`util/tracing` package.

This should help close the performance gap between v22.1 and v22.2.

```
name                        old time/op    new time/op    delta
KV/Insert/Native/rows=1-10    41.7µs ± 2%    41.4µs ± 1%  -0.72%  (p=0.026 n=20+18)
KV/Insert/SQL/rows=1-10        123µs ± 2%     124µs ± 2%    ~     (p=0.665 n=19+19)
KV/Update/Native/rows=1-10    66.0µs ± 2%    65.8µs ± 1%    ~     (p=0.258 n=20+19)
KV/Update/SQL/rows=1-10        170µs ± 3%     170µs ± 4%    ~     (p=0.851 n=18+20)
KV/Delete/Native/rows=1-10    41.5µs ± 2%    41.4µs ± 1%    ~     (p=0.740 n=20+18)
KV/Delete/SQL/rows=1-10        137µs ± 2%     137µs ± 2%    ~     (p=0.377 n=20+18)
KV/Scan/Native/rows=1-10      17.2µs ± 2%    17.2µs ± 3%    ~     (p=0.920 n=20+20)
KV/Scan/SQL/rows=1-10         92.2µs ± 1%    92.2µs ± 2%    ~     (p=0.667 n=20+19)

name                        old alloc/op   new alloc/op   delta
KV/Scan/SQL/rows=1-10         24.8kB ± 0%    24.3kB ± 0%  -2.00%  (p=0.000 n=19+19)
KV/Scan/Native/rows=1-10      7.80kB ± 0%    7.65kB ± 0%  -1.86%  (p=0.000 n=20+18)
KV/Delete/Native/rows=1-10    15.9kB ± 1%    15.6kB ± 0%  -1.48%  (p=0.000 n=20+17)
KV/Update/Native/rows=1-10    22.8kB ± 0%    22.5kB ± 0%  -1.44%  (p=0.000 n=17+19)
KV/Update/SQL/rows=1-10       52.4kB ± 0%    51.8kB ± 0%  -1.16%  (p=0.000 n=20+20)
KV/Insert/Native/rows=1-10    16.1kB ± 0%    15.9kB ± 0%  -1.15%  (p=0.000 n=20+20)
KV/Insert/SQL/rows=1-10       45.2kB ± 0%    44.8kB ± 0%  -0.85%  (p=0.000 n=20+20)
KV/Delete/SQL/rows=1-10       52.2kB ± 0%    51.8kB ± 0%  -0.72%  (p=0.000 n=19+19)

name                        old allocs/op  new allocs/op  delta
KV/Scan/Native/rows=1-10        57.0 ± 0%      54.0 ± 0%  -5.26%  (p=0.000 n=20+18)
KV/Update/Native/rows=1-10       189 ± 0%       182 ± 0%  -3.66%  (p=0.000 n=20+19)
KV/Scan/SQL/rows=1-10            289 ± 0%       279 ± 0%  -3.46%  (p=0.000 n=17+19)
KV/Delete/Native/rows=1-10       131 ± 0%       127 ± 0%  -3.05%  (p=0.000 n=20+20)
KV/Insert/Native/rows=1-10       132 ± 0%       128 ± 0%  -3.03%  (p=0.000 n=20+20)
KV/Insert/SQL/rows=1-10          367 ± 0%       359 ± 0%  -2.32%  (p=0.000 n=16+20)
KV/Update/SQL/rows=1-10          533 ± 0%       520 ± 0%  -2.30%  (p=0.000 n=20+20)
KV/Delete/SQL/rows=1-10          394 ± 0%       386 ± 0%  -2.03%  (p=0.000 n=20+18)
```

Release justification: avoids performance regression.

cc. `@erikgrinaker` 

86257: externalconn: register azure as a supported External Connection r=benbardin a=adityamaru

`azure` URIs can now be represented using External Connections.

Release note (sql change): Users can now
`CREATE EXTERNAL CONNECTION` to represent an `azure` URI.

Release justification: low risk change to new functionality that registers azure as a supported External Connection provider

86273: roachtest: update django version r=ecwall a=ZhouXing19

fixes #86010

Release justification: bug fix
Release note: none

Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Jane Xing <[email protected]>
  • Loading branch information
4 people committed Aug 17, 2022
4 parents 3b84166 + 82e4af4 + 8c7cf17 + e543dc9 commit 2aeed01
Show file tree
Hide file tree
Showing 18 changed files with 334 additions and 11 deletions.
3 changes: 3 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ ALL_TESTS = [
"//pkg/ccl/changefeedccl:changefeedccl_test",
"//pkg/ccl/cliccl:cliccl_test",
"//pkg/ccl/cloudccl/amazon:amazon_test",
"//pkg/ccl/cloudccl/azure:azure_test",
"//pkg/ccl/cloudccl/externalconn:externalconn_test",
"//pkg/ccl/cloudccl/gcp:gcp_test",
"//pkg/ccl/importerccl:importerccl_test",
Expand Down Expand Up @@ -666,6 +667,7 @@ GO_TARGETS = [
"//pkg/ccl/cliccl:cliccl",
"//pkg/ccl/cliccl:cliccl_test",
"//pkg/ccl/cloudccl/amazon:amazon_test",
"//pkg/ccl/cloudccl/azure:azure_test",
"//pkg/ccl/cloudccl/externalconn:externalconn_test",
"//pkg/ccl/cloudccl/gcp:gcp_test",
"//pkg/ccl/cmdccl/enc_utils:enc_utils",
Expand Down Expand Up @@ -2092,6 +2094,7 @@ GET_X_DATA_TARGETS = [
"//pkg/ccl/cliccl:get_x_data",
"//pkg/ccl/cliccl/cliflagsccl:get_x_data",
"//pkg/ccl/cloudccl/amazon:get_x_data",
"//pkg/ccl/cloudccl/azure:get_x_data",
"//pkg/ccl/cloudccl/externalconn:get_x_data",
"//pkg/ccl/cloudccl/gcp:get_x_data",
"//pkg/ccl/cmdccl/enc_utils:get_x_data",
Expand Down
32 changes: 32 additions & 0 deletions pkg/ccl/cloudccl/azure/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "azure_test",
srcs = [
"azure_connection_test.go",
"main_test.go",
],
deps = [
"//pkg/base",
"//pkg/ccl",
"//pkg/ccl/kvccl/kvtenantccl",
"//pkg/ccl/utilccl",
"//pkg/cloud/azure",
"//pkg/cloud/externalconn/providers",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/randutil",
"@com_github_azure_go_autorest_autorest//azure",
],
)

get_x_data(name = "get_x_data")
103 changes: 103 additions & 0 deletions pkg/ccl/cloudccl/azure/azure_connection_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
// 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 azure

import (
"context"
"errors"
"fmt"
"net/url"
"os"
"testing"

az "github.com/Azure/go-autorest/autorest/azure"
"github.com/cockroachdb/cockroach/pkg/base"
_ "github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/cloud/azure"
_ "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/providers" // import External Connection providers.
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
)

func (a azureConfig) URI(file string) string {
return fmt.Sprintf("azure-storage://%s/%s?%s=%s&%s=%s&%s=%s",
a.bucket, file,
azure.AzureAccountKeyParam, url.QueryEscape(a.key),
azure.AzureAccountNameParam, url.QueryEscape(a.account),
azure.AzureEnvironmentKeyParam, url.QueryEscape(a.environment))
}

type azureConfig struct {
account, key, bucket, environment string
}

func getAzureConfig() (azureConfig, error) {
cfg := azureConfig{
account: os.Getenv("AZURE_ACCOUNT_NAME"),
key: os.Getenv("AZURE_ACCOUNT_KEY"),
bucket: os.Getenv("AZURE_CONTAINER"),
environment: az.PublicCloud.Name,
}
if cfg.account == "" || cfg.key == "" || cfg.bucket == "" {
return azureConfig{}, errors.New("AZURE_ACCOUNT_NAME, AZURE_ACCOUNT_KEY, AZURE_CONTAINER must all be set")
}
if v, ok := os.LookupEnv(azure.AzureEnvironmentKeyParam); ok {
cfg.environment = v
}
return cfg, nil
}

func TestExternalConnections(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

dir, dirCleanupFn := testutils.TempDir(t)
defer dirCleanupFn()

params := base.TestClusterArgs{}
params.ServerArgs.ExternalIODir = dir

tc := testcluster.StartTestCluster(t, 1, params)
defer tc.Stopper().Stop(context.Background())

tc.WaitForNodeLiveness(t)
sqlDB := sqlutils.MakeSQLRunner(tc.Conns[0])

// Setup some dummy data.
sqlDB.Exec(t, `CREATE DATABASE foo`)
sqlDB.Exec(t, `USE foo`)
sqlDB.Exec(t, `CREATE TABLE foo (id INT PRIMARY KEY)`)
sqlDB.Exec(t, `INSERT INTO foo VALUES (1), (2), (3)`)

createExternalConnection := func(externalConnectionName, uri string) {
sqlDB.Exec(t, fmt.Sprintf(`CREATE EXTERNAL CONNECTION '%s' AS '%s'`, externalConnectionName, uri))
}
backupAndRestoreFromExternalConnection := func(backupExternalConnectionName string) {
backupURI := fmt.Sprintf("external://%s", backupExternalConnectionName)
sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE foo INTO '%s'`, backupURI))
sqlDB.Exec(t, fmt.Sprintf(`RESTORE DATABASE foo FROM LATEST IN '%s' WITH new_db_name = bar`, backupURI))
sqlDB.CheckQueryResults(t, `SELECT * FROM bar.foo`, [][]string{{"1"}, {"2"}, {"3"}})
sqlDB.CheckQueryResults(t, `SELECT * FROM crdb_internal.invalid_objects`, [][]string{})
sqlDB.Exec(t, `DROP DATABASE bar CASCADE`)
}

cfg, err := getAzureConfig()
if err != nil {
skip.IgnoreLint(t, "TestExternalConnections not configured for Azure")
return
}

ecName := "azure-ec"
createExternalConnection(ecName, cfg.URI("backup-ec"))
backupAndRestoreFromExternalConnection(ecName)
}
35 changes: 35 additions & 0 deletions pkg/ccl/cloudccl/azure/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
// 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 azure_test

import (
"os"
"testing"

_ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

func TestMain(m *testing.M) {
defer utilccl.TestingEnableEnterprise()()

securityassets.SetLoader(securitytest.EmbeddedAssets)
randutil.SeedForTests()
serverutils.InitTestServerFactory(server.TestServerFactory)
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
os.Exit(m.Run())
}

//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -302,3 +302,41 @@ enable-check-external-storage
----

subtest end

subtest basic-azure

disable-check-external-storage
----

exec-sql
CREATE EXTERNAL CONNECTION "foo-azure" AS 'azure-storage://bucket/path?AZURE_ACCOUNT_NAME=foo&AZURE_ACCOUNT_KEY=Zm9vCg==&AZURE_ENVIRONMENT=AzureUSGovernmentCloud'
----

# Reject invalid azure external connections.
exec-sql
CREATE EXTERNAL CONNECTION "invalid-param-azure" AS 'azure-storage://bucket/path?INVALIDPARAM=baz'
----
pq: failed to construct External Connection details: failed to create azure external connection: unknown azure query parameters: INVALIDPARAM

exec-sql
CREATE EXTERNAL CONNECTION "foo-azure" AS 'azure-storage://bucket/path?&AZURE_ACCOUNT_KEY=Zm9vCg==&AZURE_ENVIRONMENT=AzureUSGovernmentCloud'
----
pq: failed to construct External Connection details: failed to create azure external connection: azure uri missing "AZURE_ACCOUNT_NAME" parameter

exec-sql
CREATE EXTERNAL CONNECTION "foo-azure" AS 'azure-storage://bucket/path?&AZURE_ACCOUNT_NAME=foo&AZURE_ACCOUNT_KEY=Zm9vCg==&AZURE_ENVIRONMENT=random-env'
----
pq: failed to construct External Connection details: failed to create azure external connection: azure environment: autorest/azure: There is no cloud environment matching the name "RANDOM-ENV"

inspect-system-table
----
foo-azure STORAGE {"provider": "azure_storage", "simpleUri": {"uri": "azure-storage://bucket/path?AZURE_ACCOUNT_NAME=foo&AZURE_ACCOUNT_KEY=Zm9vCg==&AZURE_ENVIRONMENT=AzureUSGovernmentCloud"}}

exec-sql
DROP EXTERNAL CONNECTION "foo-azure";
----

enable-check-external-storage
----

subtest end
Original file line number Diff line number Diff line change
Expand Up @@ -261,3 +261,41 @@ DROP EXTERNAL CONNECTION "foo-userfile";
----

subtest end

subtest basic-azure

disable-check-external-storage
----

exec-sql
CREATE EXTERNAL CONNECTION "foo-azure" AS 'azure-storage://bucket/path?AZURE_ACCOUNT_NAME=foo&AZURE_ACCOUNT_KEY=Zm9vCg==&AZURE_ENVIRONMENT=AzureUSGovernmentCloud'
----

# Reject invalid azure external connections.
exec-sql
CREATE EXTERNAL CONNECTION "invalid-param-azure" AS 'azure-storage://bucket/path?INVALIDPARAM=baz'
----
pq: failed to construct External Connection details: failed to create azure external connection: unknown azure query parameters: INVALIDPARAM

exec-sql
CREATE EXTERNAL CONNECTION "foo-azure" AS 'azure-storage://bucket/path?&AZURE_ACCOUNT_KEY=Zm9vCg==&AZURE_ENVIRONMENT=AzureUSGovernmentCloud'
----
pq: failed to construct External Connection details: failed to create azure external connection: azure uri missing "AZURE_ACCOUNT_NAME" parameter

exec-sql
CREATE EXTERNAL CONNECTION "foo-azure" AS 'azure-storage://bucket/path?&AZURE_ACCOUNT_NAME=foo&AZURE_ACCOUNT_KEY=Zm9vCg==&AZURE_ENVIRONMENT=random-env'
----
pq: failed to construct External Connection details: failed to create azure external connection: azure environment: autorest/azure: There is no cloud environment matching the name "RANDOM-ENV"

inspect-system-table
----
foo-azure STORAGE {"provider": "azure_storage", "simpleUri": {"uri": "azure-storage://bucket/path?AZURE_ACCOUNT_NAME=foo&AZURE_ACCOUNT_KEY=Zm9vCg==&AZURE_ENVIRONMENT=AzureUSGovernmentCloud"}}

exec-sql
DROP EXTERNAL CONNECTION "foo-azure";
----

enable-check-external-storage
----

subtest end
2 changes: 1 addition & 1 deletion pkg/ccl/cloudccl/gcp/gcp_connection_test.go
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
// Copyright 2020 The Cockroach Authors.
// 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
Expand Down
9 changes: 8 additions & 1 deletion pkg/cloud/azure/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -3,13 +3,20 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "azure",
srcs = ["azure_storage.go"],
srcs = [
"azure_connection.go",
"azure_storage.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/cloud/azure",
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/cloud",
"//pkg/cloud/cloudpb",
"//pkg/cloud/externalconn",
"//pkg/cloud/externalconn/connectionpb",
"//pkg/cloud/externalconn/utils",
"//pkg/security/username",
"//pkg/server/telemetry",
"//pkg/settings/cluster",
"//pkg/util/contextutil",
Expand Down
47 changes: 47 additions & 0 deletions pkg/cloud/azure/azure_connection.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package azure

import (
"context"
"net/url"

"github.com/cockroachdb/cockroach/pkg/cloud/externalconn"
"github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb"
"github.com/cockroachdb/cockroach/pkg/cloud/externalconn/utils"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/errors"
)

func parseAndValidateAzureConnectionURI(
ctx context.Context, execCfg interface{}, user username.SQLUsername, uri *url.URL,
) (externalconn.ExternalConnection, error) {
if err := utils.CheckExternalStorageConnection(ctx, execCfg, user, uri.String()); err != nil {
return nil, errors.Wrap(err, "failed to create azure external connection")
}

connDetails := connectionpb.ConnectionDetails{
Provider: connectionpb.ConnectionProvider_azure_storage,
Details: &connectionpb.ConnectionDetails_SimpleURI{
SimpleURI: &connectionpb.SimpleURI{
URI: uri.String(),
},
},
}
return externalconn.NewExternalConnection(connDetails), nil
}

func init() {
externalconn.RegisterConnectionDetailsFromURIFactory(
externalConnectionScheme,
parseAndValidateAzureConnectionURI,
)
}
19 changes: 15 additions & 4 deletions pkg/cloud/azure/azure_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,20 +39,31 @@ const (
AzureAccountKeyParam = "AZURE_ACCOUNT_KEY"
// AzureEnvironmentKeyParam is the query parameter for the environment name in an azure URI.
AzureEnvironmentKeyParam = "AZURE_ENVIRONMENT"

scheme = "azure"
externalConnectionScheme = "azure-storage"
)

func parseAzureURL(
_ cloud.ExternalStorageURIContext, uri *url.URL,
) (cloudpb.ExternalStorage, error) {
azureURL := cloud.ConsumeURL{URL: uri}
conf := cloudpb.ExternalStorage{}
conf.Provider = cloudpb.ExternalStorageProvider_azure
conf.AzureConfig = &cloudpb.ExternalStorage_Azure{
Container: uri.Host,
Prefix: uri.Path,
AccountName: uri.Query().Get(AzureAccountNameParam),
AccountKey: uri.Query().Get(AzureAccountKeyParam),
Environment: uri.Query().Get(AzureEnvironmentKeyParam),
AccountName: azureURL.ConsumeParam(AzureAccountNameParam),
AccountKey: azureURL.ConsumeParam(AzureAccountKeyParam),
Environment: azureURL.ConsumeParam(AzureEnvironmentKeyParam),
}

// Validate that all the passed in parameters are supported.
if unknownParams := azureURL.RemainingQueryParams(); len(unknownParams) > 0 {
return cloudpb.ExternalStorage{}, errors.Errorf(
`unknown azure query parameters: %s`, strings.Join(unknownParams, ", "))
}

if conf.AzureConfig.AccountName == "" {
return conf, errors.Errorf("azure uri missing %q parameter", AzureAccountNameParam)
}
Expand Down Expand Up @@ -256,5 +267,5 @@ func (s *azureStorage) Close() error {

func init() {
cloud.RegisterExternalStorageProvider(cloudpb.ExternalStorageProvider_azure,
parseAzureURL, makeAzureStorage, cloud.RedactedParams(AzureAccountKeyParam), "azure")
parseAzureURL, makeAzureStorage, cloud.RedactedParams(AzureAccountKeyParam), scheme, externalConnectionScheme)
}
Loading

0 comments on commit 2aeed01

Please sign in to comment.