Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
…achdb#111214 cockroachdb#111217 cockroachdb#111223 cockroachdb#111230 cockroachdb#111231 cockroachdb#111232

110967: asim: enable random zone config event generation r=kvoli a=wenyihu6

Previously, zone config event generation used hardcoded span configurations.
This limits our ability to test the allocator more thoroughly.

To improve this, this patch enables random span configs to be generated and
applied as part of the simulation. These configurations are generated by
randomly selecting the primary region, region V.S. zone survival goal, and
leaseholder preference.

```
The following command is now supported:
"rand_events" [cycle_via_random_survival_goals]
```

Part of: cockroachdb#106192
Release Note: none
Epic: none

111192: bulk: allow caller-configurable priority in SSTBatcher r=adityamaru a=stevendanna

This adds the ability for some callers to use a higher admission priority in SSTBatcher. This is helpful for streaming where we want to run at a priority that isn't subject to the elastic admission regime.

Epic: none

Release note: None

111206: kv: fix (store|node) not found err checking r=erikgrinaker a=kvoli

`StoreNotFoundError` and `NodeNotFoundError` errors were moved to the `kvpb` pkg in cockroachdb#110374. As part of the move, `crdb_internal` functions which checked if the error were `DescNotFoundError` were also updated so that node/store not found errors would be recognized e.g.

```
errors.Is(kvpb.NewNodeNotFoundError(nodeID), &kvpb.DescNotFoundError{})
```

This didn't work, because the error doesn't match the reference error variable being given. It does match the type. Update these error assertions to use `HasType` instead.

Resolves: cockroachdb#111084
Epic: none
Release note: None

111214: release: fix roachtest artifacts name r=srosenberg a=rail

This fixes the roachtest artifacts directory name.

Epic: none
Release note: None

111217: cloud/azure: Fix azure schemes r=benbardin a=benbardin

Part of: https://cockroachlabs.atlassian.net/browse/CRDB-31120

Release note (bug fix): Fixes azure schemes in storage, kms and external conns.

111223: storage: populate timestamp field in lock table values r=nvanbenschoten a=nvanbenschoten

Informs cockroachdb#109645.

This commit starts writing the `Timestamp` field in lock table `MVCCMetadata` values for shared and exclusive locks. This mirrors the behavior of intent locks. This is not strictly needed, as the timestamp is always equal to `Txn.WriteTimestamp`, but it is cheap to do and helps unify some stats logic, which uses this field to compute "lock age".

Maybe we'll get rid of this for all lock strengths one day...

Release note: None

111230: authors: add xhesikam to authors r=xhesikam a=xhesikam

Release note: None
Epic: None

111231: backupccl: add missing ctx cancel check r=msbutler a=adityamaru

In cockroachdb#111159 we deduced from the stacks a
situation in which the goroutine draining `spanCh` had exited due to a context cancelation but the
writer was not listening for a ctx cancelation.
This manifests as a stuck restore when using
the non-default make simple import spans implementation.

Fixes: cockroachdb#111159
Release note: None

111232: kv: deflake TestLeaseExpirationBasedDrainTransferWithExtension r=nvanbenschoten a=nvanbenschoten

Informs cockroachdb#110715, which will be fixed by a non-clean backport (see 42e45b4) of this commit.

This commit deflakes `TestLeaseExpirationBasedDrainTransferWithExtension` by disabling the node suspect timer in leaseTransferTest tests. These tests manually control the clock and have a habit of inducing destabilizing clock jumps. In this case, if n2 looked at liveness immediately after one of these manual clock jumps, it would mark n1 as suspect and refuse to transfer it the lease for the 30 second `server.time_after_store_suspect`, which is longer than the 5 second `server.shutdown.lease_transfer_wait`. This would cause the test to fail.

Before this patch, the test would fail under stress race in about 8 minutes. Since the patch, it hasn't failed in over 30 minutes.

Release note: None

Co-authored-by: wenyihu6 <[email protected]>
Co-authored-by: Steven Danna <[email protected]>
Co-authored-by: Austen McClernon <[email protected]>
Co-authored-by: Rail Aliiev <[email protected]>
Co-authored-by: Ben Bardin <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: xhesikam <[email protected]>
Co-authored-by: adityamaru <[email protected]>
  • Loading branch information
9 people committed Sep 25, 2023
10 parents b3691a2 + 91adebd + a177578 + 49d0ffc + b541722 + 5cd4c7b + 5fbbfcf + 5be3ddf + 6fabaf8 + 8edaf08 commit 46cb4e3
Show file tree
Hide file tree
Showing 32 changed files with 669 additions and 207 deletions.
1 change: 1 addition & 0 deletions AUTHORS
Original file line number Diff line number Diff line change
Expand Up @@ -493,6 +493,7 @@ Yulei Xiao <[email protected]>
YZ Chin <[email protected]>
Rafael Yim <[email protected]> yznming <[email protected]>
Ryan Kuo <[email protected]> taroface <[email protected]> <[email protected]>
Xhesika Malecaj <[email protected]> <[email protected]>
Zach Brock <[email protected]> <[email protected]>
Zach Lite <[email protected]>
Zachary Smith <[email protected]> Zachary.smith <[email protected]>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ if [[ ! -f ~/.ssh/id_rsa.pub ]]; then
ssh-keygen -q -N "" -f ~/.ssh/id_rsa
fi

artifacts=$PWD/artifacts/$(date +"%%Y%%m%%d")-${TC_BUILD_ID}
artifacts=$PWD/artifacts/$(date +"%Y%m%d")-${TC_BUILD_ID}
mkdir -p "$artifacts"

if [[ ${FIPS_ENABLED:-0} == 1 ]]; then
Expand Down
6 changes: 5 additions & 1 deletion pkg/ccl/backupccl/restore_span_covering.go
Original file line number Diff line number Diff line change
Expand Up @@ -519,7 +519,11 @@ func generateAndSendImportSpans(
return err
}
for _, sp := range importSpans {
spanCh <- sp
select {
case <-ctx.Done():
return ctx.Err()
case spanCh <- sp:
}
}
return nil
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/cloudccl/externalconn/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,8 @@ SELECT connection_name,
crdb_internal.pb_to_json('cockroach.cloud.externalconn.connectionpb.ConnectionDetails', connection_details),
owner,
owner_id
FROM system.external_connections;
FROM system.external_connections
ORDER BY connection_name;
`)
output, err := sqlutils.RowsToDataDrivenOutput(rows)
require.NoError(t, err)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -229,6 +229,14 @@ exec-sql
CREATE EXTERNAL CONNECTION "foo-azure" AS 'azure-storage://bucket/path?AZURE_ACCOUNT_NAME=foo&AZURE_ACCOUNT_KEY=Zm9vCg==&AZURE_ENVIRONMENT=AzureUSGovernmentCloud'
----

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

exec-sql
CREATE EXTERNAL CONNECTION "baz-azure" AS 'azure://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'
Expand All @@ -247,13 +255,55 @@ pq: failed to construct External Connection details: failed to create azure exte

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

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

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

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

enable-check-external-storage
----

disable-check-kms
----

exec-sql
CREATE EXTERNAL CONNECTION "foo-kms" AS 'azure-kms:///cmk/id?AUTH=specified&AZURE_VAULT_NAME=vault&AZURE_CLIENT_ID=client&AZURE_CLIENT_SECRET=secret&AZURE_TENANT_ID=tenant';
----

# Reject invalid KMS URIs.
exec-sql
CREATE EXTERNAL CONNECTION "missing-cmk-kms" AS 'azure-kms:///?AUTH=implicit&CREDENTIALS=baz&ASSUME_ROLE=ronaldo,rashford,bruno&BEARER_TOKEN=foo';
----
pq: failed to construct External Connection details: failed to create Azure KMS external connection: path component of the KMS cannot be empty; must contain the Customer Managed Key

exec-sql
CREATE EXTERNAL CONNECTION "invalid-params-kms" AS 'azure-kms:///cmk?AUTH=implicit&INVALIDPARAM=baz';
----
pq: failed to construct External Connection details: failed to create Azure KMS external connection: unknown KMS query parameters: INVALIDPARAM

inspect-system-table
----
foo-kms KMS {"provider": "azure_kms", "simpleUri": {"uri": "azure-kms:///cmk/id?AUTH=specified&AZURE_VAULT_NAME=vault&AZURE_CLIENT_ID=client&AZURE_CLIENT_SECRET=secret&AZURE_TENANT_ID=tenant"}} root 1

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

inspect-system-table
----

enable-check-kms
----

subtest end
4 changes: 4 additions & 0 deletions pkg/cloud/azure/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,9 @@ go_library(
go_test(
name = "azure_test",
srcs = [
"azure_connection_test.go",
"azure_file_credentials_test.go",
"azure_kms_connection_test.go",
"azure_kms_test.go",
"azure_storage_test.go",
],
Expand All @@ -57,6 +59,8 @@ go_test(
"//pkg/cloud",
"//pkg/cloud/cloudpb",
"//pkg/cloud/cloudtestutils",
"//pkg/cloud/externalconn",
"//pkg/cloud/externalconn/connectionpb",
"//pkg/security/username",
"//pkg/settings/cluster",
"//pkg/testutils",
Expand Down
17 changes: 9 additions & 8 deletions pkg/cloud/azure/azure_connection.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,12 +30,13 @@ func validateAzureConnectionURI(
}

func init() {
externalconn.RegisterConnectionDetailsFromURIFactory(
deprecatedExternalConnectionScheme,
connectionpb.ConnectionProvider_azure_storage,
externalconn.SimpleURIFactory,
)

externalconn.RegisterDefaultValidation(deprecatedExternalConnectionScheme, validateAzureConnectionURI)

for _, s := range []string{scheme, deprecatedScheme, deprecatedExternalConnectionScheme} {
externalconn.RegisterConnectionDetailsFromURIFactory(
s,
connectionpb.ConnectionProvider_azure_storage,
externalconn.SimpleURIFactory,
)

externalconn.RegisterDefaultValidation(s, validateAzureConnectionURI)
}
}
25 changes: 25 additions & 0 deletions pkg/cloud/azure/azure_connection_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
// Copyright 2023 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 (
"testing"

"github.com/cockroachdb/cockroach/pkg/cloud/externalconn"
"github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb"
"github.com/stretchr/testify/require"
)

func TestAzureStorageConnection(t *testing.T) {
require.Equal(t, connectionpb.ConnectionProvider_azure_storage, externalconn.ProviderForURI("azure://test"))
require.Equal(t, connectionpb.ConnectionProvider_azure_storage, externalconn.ProviderForURI("azure-storage://test"))
require.Equal(t, connectionpb.ConnectionProvider_azure_storage, externalconn.ProviderForURI("azure-blob://test"))
}
4 changes: 2 additions & 2 deletions pkg/cloud/azure/azure_kms_connection.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,12 +31,12 @@ func validateAzureKMSConnectionURI(

func init() {
externalconn.RegisterConnectionDetailsFromURIFactory(
scheme,
kmsScheme,
connectionpb.ConnectionProvider_azure_kms,
externalconn.SimpleURIFactory,
)
externalconn.RegisterDefaultValidation(
scheme,
kmsScheme,
validateAzureKMSConnectionURI,
)
}
23 changes: 23 additions & 0 deletions pkg/cloud/azure/azure_kms_connection_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
// Copyright 2023 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 (
"testing"

"github.com/cockroachdb/cockroach/pkg/cloud/externalconn"
"github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb"
"github.com/stretchr/testify/require"
)

func TestAzureKMSConnection(t *testing.T) {
require.Equal(t, connectionpb.ConnectionProvider_azure_kms, externalconn.ProviderForURI("azure-kms://test"))
}
18 changes: 18 additions & 0 deletions pkg/cloud/externalconn/impl_registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,24 @@ func ExternalConnectionFromURI(
return parseAndValidateFn.parseAndValidateURI(ctx, env, externalConnectionURI, defaultValidation)
}

// ProviderForURI returns the provider associated with the scheme of a given URI,
// or UNKNOWN if none found.
// This is useful for testing.
func ProviderForURI(uri string) connectionpb.ConnectionProvider {
externalConnectionURI, err := url.Parse(uri)
if err != nil {
return connectionpb.ConnectionProvider_Unknown
}

// Find the parseAndValidateFn method for the ExternalConnection provider.
parseAndValidateFn, registered := parseAndValidateFns[externalConnectionURI.Scheme]
if !registered {
return connectionpb.ConnectionProvider_Unknown
}

return parseAndValidateFn.ConnectionProvider
}

// ExternalConnEnv contains parameters to be used to validate an external
// connection.
type ExternalConnEnv struct {
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/bulk/buffering_adder.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/limit"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -107,6 +108,7 @@ func MakeBulkAdder(
writeAtBatchTS: opts.WriteAtBatchTimestamp,
mem: bulkMon.MakeConcurrentBoundAccount(),
limiter: sendLimiter,
priority: admissionpb.BulkNormalPri,
},
timestamp: timestamp,
maxBufferLimit: opts.MaxBufferSize,
Expand Down
15 changes: 14 additions & 1 deletion pkg/kv/bulk/sst_batcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,10 @@ type SSTBatcher struct {
mem *mon.ConcurrentBoundAccount
limiter limit.ConcurrentRequestLimiter

// priority is the admission priority used for AddSSTable
// requests.
priority admissionpb.WorkPriority

// disallowShadowingBelow is described on kvpb.AddSSTableRequest.
disallowShadowingBelow hlc.Timestamp

Expand Down Expand Up @@ -229,6 +233,7 @@ func MakeSSTBatcher(
disableScatters: !scatterSplitRanges,
mem: mem,
limiter: sendLimiter,
priority: admissionpb.BulkNormalPri,
}
b.mu.lastFlush = timeutil.Now()
b.mu.tracingSpan = tracing.SpanFromContext(ctx)
Expand All @@ -254,6 +259,13 @@ func MakeStreamSSTBatcher(
ingestAll: true,
mem: mem,
limiter: sendLimiter,
// We use NormalPri since anything lower than normal
// priority is assumed to be able to handle reduced
// throughput. We are OK witht his for now since the
// consuming cluster of a replication stream does not
// have a latency sensitive workload running against
// it.
priority: admissionpb.NormalPri,
}
b.mu.lastFlush = timeutil.Now()
b.mu.tracingSpan = tracing.SpanFromContext(ctx)
Expand All @@ -280,6 +292,7 @@ func MakeTestingSSTBatcher(
ingestAll: ingestAll,
mem: mem,
limiter: sendLimiter,
priority: admissionpb.BulkNormalPri,
}
b.Reset(ctx)
return b, nil
Expand Down Expand Up @@ -815,7 +828,7 @@ func (b *SSTBatcher) addSSTable(
ba := &kvpb.BatchRequest{
Header: kvpb.Header{Timestamp: batchTS, ClientRangeInfo: roachpb.ClientRangeInfo{ExplicitlyRequested: true}},
AdmissionHeader: kvpb.AdmissionHeader{
Priority: int32(admissionpb.BulkNormalPri),
Priority: int32(b.priority),
CreateTime: timeutil.Now().UnixNano(),
Source: kvpb.AdmissionHeader_FROM_SQL,
NoMemoryReservedAtSource: true,
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvclient/kvtenant/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,7 @@ go_test(
"//pkg/util/stop",
"//pkg/util/tracing/tracingpb",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvclient/kvtenant/connector_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"google.golang.org/grpc/codes"
Expand Down Expand Up @@ -567,6 +568,7 @@ func TestConnectorRetriesUnreachable(t *testing.T) {
require.NoError(t, err)
desc, err = c.GetNodeDescriptor(3)
require.Nil(t, desc)
require.True(t, errors.HasType(err, &kvpb.DescNotFoundError{}))
require.Regexp(t, "node descriptor with node ID 3 was not found", err)
}

Expand Down
13 changes: 13 additions & 0 deletions pkg/kv/kvpb/errors_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -411,3 +411,16 @@ func TestNotLeaseholderError(t *testing.T) {
})
}
}

func TestDescNotFoundError(t *testing.T) {
t.Run("store not found", func(t *testing.T) {
err := NewStoreDescNotFoundError(42)
require.Equal(t, `store descriptor with store ID 42 was not found`, err.Error())
require.True(t, errors.HasType(err, &DescNotFoundError{}))
})
t.Run("node not found", func(t *testing.T) {
err := NewNodeDescNotFoundError(42)
require.Equal(t, `node descriptor with node ID 42 was not found`, err.Error())
require.True(t, errors.HasType(err, &DescNotFoundError{}))
})
}
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/asim/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ go_library(
"output.go",
"rand_framework.go",
"rand_gen.go",
"rand_util.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/tests",
visibility = ["//visibility:public"],
Expand All @@ -20,6 +21,10 @@ go_library(
"//pkg/kv/kvserver/asim/scheduled",
"//pkg/kv/kvserver/asim/state",
"//pkg/roachpb",
"//pkg/sql",
"//pkg/sql/catalog/catpb",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/multiregion",
],
)

Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/asim/tests/rand_framework.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,7 +269,9 @@ func (f randTestingFramework) randomEventSeriesGen(
) gen.StaticEvents {
switch eventsType := f.s.eventGen.eventsType; eventsType {
case cycleViaHardcodedSurvivalGoals:
return generateSurvivalGoalsEvents(cluster.Regions(), settings.Settings.StartTime, f.s.eventGen.durationToAssertOnEvent)
return generateHardcodedSurvivalGoalsEvents(cluster.Regions(), settings.Settings.StartTime, f.s.eventGen.durationToAssertOnEvent)
case cycleViaRandomSurvivalGoals:
return generateRandomSurvivalGoalsEvents(cluster.Regions(), settings.Settings.StartTime, f.s.eventGen.durationToAssertOnEvent, f.s.duration, f.s.randSource)
default:
panic("unknown event series type")
}
Expand Down
Loading

0 comments on commit 46cb4e3

Please sign in to comment.