Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
80792: changefeedccl: update tests to random tenant random sink r=samiskin a=samiskin

Previously most of our tests did not run on tenants as it was not
ergonomic to do so given our helpers.  We would also manually run tests
across multiple sinks even if the test did not care about what sink it
was ran on, drastically increasing the execution time of our test suite.

This PR updates our helper infrastructure to use a shared TestServer
struct that allows access to both the system and secondary tenant
interfaces, and by default runs tests on a **random sink** every time.

Running only once per test reduces `pkg/ccl/changefeedccl` test suite
execution time to **under a minute** on a standard employee gceworker.

A couple testing bugs with certain sinks were fixed but some remain 
(that simply didn't show up earlier), resulting in some TODOs added on
tests that limit the sinks unnecessarily.

---

The new set of tools look like:
```go
// The main struct that is used by most tests, replaces prior use of the `db *gosql.DB` parameter and `f.Server()`
type TestServer struct {
        // DB and Server can either be a System or Secondary tenant, randomly determined or forced through options
	DB           *gosql.DB
	Server       serverutils.TestTenantInterface

        // SystemDB and SystemServer are always the system tenant, useful for certain tasks that require a full TestServerInterface like getting PublicTableDescriptors or kv stores 
	SystemDB     *gosql.DB
	SystemServer serverutils.TestServerInterface

        // Some values are accessed differently depending on whether we're using a system or secondary tenant, so that specific logic is done in the makeServer functions
	Codec        keys.SQLCodec
	TestingKnobs base.TestingKnobs
}

testFn := func (t *testing.T, s TestServer, f cdctest.TestFeedFactory) {
   ...  /* very similar content to before */
}

cdcTest(t, testFn, ... /* feedTestOptions like before */)
```

Test options include:
```go
feedTestNoTenants() // Some functionality did not have a straightforward way to make it work for tenants
feedTestForceTenant()
feedTestForceSink("kafka")

// Sinkless is currently by default disallowed from the random selection since many tests
// will assume the existence of a job and I don't want to have someone do their dev while
// never hitting a Sinkless test, CI happens to run on any of the other sinks, then some 
// poor non-cdc dev gets hit with a Sinkless failure.
feedTestIncludeSinkless() 

// Restrict/omit random selection to/from certain sinks (such as for features that only work on certain sinks)
feedTestRestrictSinks("kafka", "enterprise")
feedTestOmitSinks("cloudstorage")
```

The lower level helpers to make servers and feeds have also changed to make cluster tests easier:
```go
  s, serverCleanup := makeTestServer(t);
  
  // Feed factory helper is now detached from server creation to allow easily setting up a feed factory
  // on an arbitrary server, such as when manually creating a Cluster.
  f, sinkCleanup := makeFeedFactory(t, randomSinkType(), s.Server, s.DB, ... /* feedTestOptions */)
```

Release note: None

82274: backupccl: display up to 10 missing files in `SHOW BACKUP .. with check_files` r=dt a=msbutler



Previously, `SHOW BACKUP WITH check_files` displayed the first missing SST.
This patch will display up to 100 missing SSTs. Further, this renames the
misleading `approximateTablePhysicalSize` to `approximateSpanPhysicalSize`.
Below I write out how physical table size is calculated:

1. Each range we backup maps to 1 to many spans (currently in the
backup_manfest.files object).

2. 1 to many spans get written to an SST. No span will get written to multiple
SSTs.

3. When backup created these spans, it tried really hard to split spans at
table boundaries, so only one table’s data could be in a span, but a last
minute table creation makes this near impossible, due to slow range splits.
A big table will have many spans.

4. To compute the approximate logical size (called size_bytes in SHOW BACKUP)
of each table, we sum the logical bytes over all it’s spans. We identify a
table’s span by checking the table prefix of the first key in the span. See
getTableSizes method)

5. To compute the physical size (file_bytes in SHOW BACKUP) of a span, compute
the logical size of each SST by summing the logical bytes in the SST  over its
spans (see getLogicalSSTSize method), and attribute a portion of the physical
SST size (returned from cloud storage) to a span  using the formula:
(sstPhysicalSize) * (logicalSpanSize) / (logicalSSTSize) = physicalSpanSize (
the approximateSpanTableSize method implements this).

6. To compute the physical size of a table, sum over the physical sizes the
table’s spans

Release note (sql change): SHOW BACKUP WITH check_files will display up to 10
missing SSTs.

82329: kv, gossip: remove misc deprecated system config code  r=RichardJCai a=RichardJCai

Release note: None

82418: dev: various improvements to `dev generate cgo` and friends r=mari-crl a=rickystewart

1. Up until this point, `dev generate go` has not included the
   `zcgo_flags.go` sources, which has been a point of confusion for
   people who expect `dev generate go` to generate all the .go files.
   Now `dev generate go` includes `cgo` as well, and there is a new
   target `dev generate go_nocgo` that does what `dev generate go` used
   to do.
2. Now `dev generate cgo` is conscious of where `force_build_cdeps` is
   set. If it is, then we make sure not to check in one of the
   pre-archived locations. To this end we add a `test_force_build_cdeps`
   target that `dev generate cgo` builds.

Release note: None

82430: sql: use uint32 for DOid r=otan a=rafiss

refs #41904

This resolves a piece of tech debt that has caused us a few surprises
and bugs in the past. This doesn't change anything about the on-disk
representation -- it just makes it so that OIDs handled in memory are
more reliably unsigned 32 bit integers.

Release note: None

82443: storage: tweak `newMVCCIterator()` r=jbowens a=erikgrinaker

This patch tweaks `newMVCCIterator()` for use with MVCC range
tombstones, and uses it for all appropriate MVCC operations.

Release note: None

82530: cloud: bump orchestrator to v22.1.1 r=e-mbrown a=e-mbrown

Release note: None

Co-authored-by: Shiranka Miskin <[email protected]>
Co-authored-by: Michael Butler <[email protected]>
Co-authored-by: richardjcai <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: e-mbrown <[email protected]>
  • Loading branch information
8 people committed Jun 7, 2022
8 parents 7d7f2d4 + 24d490e + 509e4a0 + dc80d3c + 9847968 + dde16a8 + c28f1ee + 723818e commit 0c8f826
Show file tree
Hide file tree
Showing 111 changed files with 1,466 additions and 2,282 deletions.
10 changes: 10 additions & 0 deletions build/bazelutil/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,16 @@ analysis_test(
),
)

# The output file will be empty unless we're using the force_build_cdeps config.
genrule(
name = "test_force_build_cdeps",
outs = ["test_force_build_cdeps.txt"],
cmd = select({
"//build/toolchains:force_build_cdeps": "echo 1 > $@",
"//conditions:default": "touch $@",
}),
)

lint_binary(
name = "lint",
test = "//pkg/testutils/lint:lint_test",
Expand Down
5 changes: 4 additions & 1 deletion build/toolchains/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -463,7 +463,10 @@ config_setting(
flag_values = {
":force_build_cdeps_flag": "true",
},
visibility = ["//c-deps:__pkg__"],
visibility = [
"//build/bazelutil:__pkg__",
"//c-deps:__pkg__",
],
)

bool_flag(
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/bring-your-own-certs/client.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ spec:
serviceAccountName: cockroachdb
containers:
- name: cockroachdb-client
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
# Keep a pod open indefinitely so kubectl exec can be used to get a shell to it
# and run cockroach client commands, such as cockroach sql, cockroach node status, etc.
command:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,7 @@ spec:
topologyKey: kubernetes.io/hostname
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
# TODO: Change these to appropriate values for the hardware that you're running. You can see
# the resources that can be allocated on each of your Kubernetes nodes by running:
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/client-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ spec:
mountPath: /cockroach-certs
containers:
- name: cockroachdb-client
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
volumeMounts:
- name: client-certs
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/cluster-init-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ spec:
mountPath: /cockroach-certs
containers:
- name: cluster-init
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
volumeMounts:
- name: client-certs
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/cluster-init.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@ spec:
spec:
containers:
- name: cluster-init
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
command:
- "/cockroach/cockroach"
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/cockroachdb-statefulset-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -195,7 +195,7 @@ spec:
topologyKey: kubernetes.io/hostname
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
# TODO: Change these to appropriate values for the hardware that you're running. You can see
# the resources that can be allocated on each of your Kubernetes nodes by running:
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/cockroachdb-statefulset.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,7 @@ spec:
topologyKey: kubernetes.io/hostname
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
# TODO: Change these to appropriate values for the hardware that you're running. You can see
# the resources that can be allocated on each of your Kubernetes nodes by running:
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/multiregion/client-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ spec:
serviceAccountName: cockroachdb
containers:
- name: cockroachdb-client
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
volumeMounts:
- name: client-certs
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/multiregion/cluster-init-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ spec:
serviceAccountName: cockroachdb
containers:
- name: cluster-init
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
volumeMounts:
- name: client-certs
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ spec:
topologyKey: kubernetes.io/hostname
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
ports:
- containerPort: 26257
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ spec:
name: cockroach-env
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
# TODO: Change these to appropriate values for the hardware that you're running. You can see
# the resources that can be allocated on each of your Kubernetes nodes by running:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ spec:
hostNetwork: true
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
# TODO: If you configured taints to give CockroachDB exclusive access to nodes, feel free
# to remove the requests and limits sections. If you didn't, you'll need to change these to
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ spec:
topologyKey: kubernetes.io/hostname
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
# TODO: If you configured taints to give CockroachDB exclusive access to nodes, feel free
# to remove the requests and limits sections. If you didn't, you'll need to change these to
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,7 @@ spec:
- name: cockroachdb
# NOTE: Always use the most recent version of CockroachDB for the best
# performance and reliability.
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
# TODO: Change these to appropriate values for the hardware that you're running. You can see
# the resources that can be allocated on each of your Kubernetes nodes by running:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -232,7 +232,7 @@ spec:
- name: cockroachdb
# NOTE: Always use the most recent version of CockroachDB for the best
# performance and reliability.
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
# TODO: Change these to appropriate values for the hardware that you're running. You can see
# the resources that can be allocated on each of your Kubernetes nodes by running:
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.6/client-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ spec:
mountPath: /cockroach-certs
containers:
- name: cockroachdb-client
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
volumeMounts:
- name: client-certs
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.6/cluster-init-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ spec:
mountPath: /cockroach-certs
containers:
- name: cluster-init
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
volumeMounts:
- name: client-certs
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.6/cluster-init.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@ spec:
spec:
containers:
- name: cluster-init
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
command:
- "/cockroach/cockroach"
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.6/cockroachdb-statefulset-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ spec:
topologyKey: kubernetes.io/hostname
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
ports:
- containerPort: 26257
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.6/cockroachdb-statefulset.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ spec:
topologyKey: kubernetes.io/hostname
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
ports:
- containerPort: 26257
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.7/client-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ spec:
mountPath: /cockroach-certs
containers:
- name: cockroachdb-client
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
volumeMounts:
- name: client-certs
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.7/cluster-init-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ spec:
mountPath: /cockroach-certs
containers:
- name: cluster-init
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
volumeMounts:
- name: client-certs
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.7/cluster-init.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@ spec:
spec:
containers:
- name: cluster-init
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
command:
- "/cockroach/cockroach"
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.7/cockroachdb-statefulset-secure.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,7 @@ spec:
topologyKey: kubernetes.io/hostname
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
ports:
- containerPort: 26257
Expand Down
2 changes: 1 addition & 1 deletion cloud/kubernetes/v1.7/cockroachdb-statefulset.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ spec:
topologyKey: kubernetes.io/hostname
containers:
- name: cockroachdb
image: cockroachdb/cockroach:v22.1.0
image: cockroachdb/cockroach:v22.1.1
imagePullPolicy: IfNotPresent
ports:
- containerPort: 26257
Expand Down
45 changes: 37 additions & 8 deletions pkg/ccl/backupccl/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ package backupccl
import (
"context"
"path"
"sort"
"strings"
"time"

Expand Down Expand Up @@ -490,6 +491,8 @@ func checkBackupFiles(
storeFactory cloud.ExternalStorageFromURIFactory,
user username.SQLUsername,
) ([][]int64, error) {
const maxMissingFiles = 10
missingFiles := make(map[string]struct{}, maxMissingFiles)

checkLayer := func(layer int) ([]int64, error) {
// TODO (msbutler): Right now, checkLayer opens stores for each backup layer. In 22.2,
Expand Down Expand Up @@ -550,10 +553,19 @@ func checkBackupFiles(
}
sz, err := store.Size(ctx, f.Path)
if err != nil {
return nil, errors.Wrapf(err, "Error checking file %s in %s", f.Path, uri)
uriNoLocality := strings.Split(uri, "?")[0]
missingFile := path.Join(uriNoLocality, f.Path)
if _, ok := missingFiles[missingFile]; !ok {
missingFiles[missingFile] = struct{}{}
if maxMissingFiles == len(missingFiles) {
break
}
}
continue
}
fileSizes[i] = sz
}

return fileSizes, nil
}

Expand All @@ -563,8 +575,23 @@ func checkBackupFiles(
if err != nil {
return nil, err
}
if len(missingFiles) == maxMissingFiles {
break
}
manifestFileSizes[layer] = layerFileSizes
}
if len(missingFiles) > 0 {
filesForMsg := make([]string, 0, len(missingFiles))
for file := range missingFiles {
filesForMsg = append(filesForMsg, file)
}
errorMsgPrefix := "The following files are missing from the backup:"
if len(missingFiles) == maxMissingFiles {
errorMsgPrefix = "Multiple files cannot be read from the backup including:"
}
sort.Strings(filesForMsg)
return nil, errors.Newf("%s\n\t%s", errorMsgPrefix, strings.Join(filesForMsg, "\n\t"))
}
return manifestFileSizes, nil
}

Expand Down Expand Up @@ -832,11 +859,11 @@ func getLogicalSSTSize(files []backuppb.BackupManifest_File) map[string]int64 {
return sstDataSize
}

// approximateTablePhysicalSize approximates the number bytes written to disk for the table.
func approximateTablePhysicalSize(
logicalTableSize int64, logicalFileSize int64, sstFileSize int64,
// approximateSpanPhysicalSize approximates the number of bytes written to disk for the span.
func approximateSpanPhysicalSize(
logicalSpanSize int64, logicalSSTSize int64, physicalSSTSize int64,
) int64 {
return int64(float64(sstFileSize) * (float64(logicalTableSize) / float64(logicalFileSize)))
return int64(float64(physicalSSTSize) * (float64(logicalSpanSize) / float64(logicalSSTSize)))
}

// getTableSizes gathers row and size count for each table in the manifest
Expand Down Expand Up @@ -872,7 +899,8 @@ func getTableSizes(
s := tableSizes[descpb.ID(tableID)]
s.rowCount.Add(file.EntryCounts)
if len(fileSizes) > 0 {
s.fileSize = approximateTablePhysicalSize(s.rowCount.DataSize, logicalSSTSize[file.Path], fileSizes[i])
s.fileSize += approximateSpanPhysicalSize(file.EntryCounts.DataSize, logicalSSTSize[file.Path],
fileSizes[i])
}
tableSizes[descpb.ID(tableID)] = s
}
Expand Down Expand Up @@ -997,7 +1025,7 @@ func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower {
backupType = "incremental"
}

sstDataSize := getLogicalSSTSize(manifest.Files)
logicalSSTSize := getLogicalSSTSize(manifest.Files)
for j, file := range manifest.Files {
filePath := file.Path
if inCol != nil {
Expand All @@ -1012,7 +1040,8 @@ func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower {
}
sz := int64(-1)
if len(info.fileSizes) > 0 {
sz = approximateTablePhysicalSize(info.fileSizes[i][j], file.EntryCounts.DataSize, sstDataSize[file.Path])
sz = approximateSpanPhysicalSize(file.EntryCounts.DataSize,
logicalSSTSize[file.Path], info.fileSizes[i][j])
}
rows = append(rows, tree.Datums{
tree.NewDString(filePath),
Expand Down
Loading

0 comments on commit 0c8f826

Please sign in to comment.