Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
60512: backupccl: update backup detached tests with retry execution under transaction r=pbardea a=Elliebababa

update backup and restore detached tests with retry execution under transaction

Previously, backup and restore detached tests do not retry if transactions fail. They could be flaky tests and fail in CI check.

With this PR, backup and restore detached tests under transaction will not be aborted by retry errors.

Release notes: None.

60594: build: explicitly add unzip as a dependency r=rickystewart a=ulfjack

The `unzip` package is not installed by default in Ubuntu. It's currently pulled in as an implicit dependency of the Bazel package, but that breaks if Bazel ever removes that dependency (or when changing the Dockerfile to not install Bazel).

60648: distsql: improve test harness for processors against operators r=yuzefovich a=yuzefovich

Previously, when comparing the output of processors and operators, if
any order is allowed, we would perform a quadratic search. This is
suboptimal, and this commit improves the situation by sorting both
outputs first and then using the ordered comparison.

Additionally, this commit fixes a rare flake when seemingly the same
outputs would result in a mismatch. I could somewhat reliably reproduce
it before the patch and cannot reproduce it after the patch. It is
likely to do something with the float comparison, but I didn't get to
the bottom of the flake, yet I don't think it's worth it given that this
commit improves the situation anyway.

Fixes: #60608.

Release note: None

60666: util/tracing: re-key activeSpans map on span ID r=irfansharif a=angelapwen

Note that the changes in this PR should be used in #60616 to more easily retrieve a span given its span ID, without having to visit all spans and check for a match on span ID. Whichever PR lands later should update the sections of code marked as #TODOs in #60616.

Previously, a tracer's activeSpans map was keyed on the memory
address of the span itself. This commit keys the map on the span ID
(which is deterministically unique) and sets the corresponding
value in the map to be the memory address of the span itself. This
allows us to continue to visit all active spans via the map, but
also easily retrieve a span from its ID using the map.

Release note: None

60679: licenses: Update CCL text r=petermattis a=bdarnell

Remove a dead link and clarify related text

Release note: None


60681: sql: fix spacing for comments around virtual schema r=irfansharif a=irfansharif

Release note: None

Co-authored-by: elliebababa <ellie24.huang@gmail.com>
Co-authored-by: Ulf Adams <ulfjack@users.noreply.github.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: angelapwen <angelaw@cockroachlabs.com>
Co-authored-by: Ben Darnell <ben@cockroachlabs.com>
Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
  • Loading branch information
7 people committed Feb 17, 2021
7 parents 889c27a + f61a102 + aa2c482 + 7989683 + 85becc0 + c572f86 + b5ac826 commit 5744288
Show file tree
Hide file tree
Showing 7 changed files with 88 additions and 103 deletions.
1 change: 1 addition & 0 deletions build/bazelbuilder/Dockerfile
Expand Up @@ -14,6 +14,7 @@ RUN apt-get update \
gnupg2 \
libncurses-dev \
make \
unzip \
&& update-alternatives --install /usr/bin/clang clang /usr/bin/clang-10 100 \
--slave /usr/bin/clang++ clang++ /usr/bin/clang++-10 \
&& apt-get clean
Expand Down
43 changes: 25 additions & 18 deletions licenses/CCL.txt
Expand Up @@ -19,7 +19,7 @@ CockroachDB Community License Agreement
Labs, including both CockroachDB Core and CockroachDB Enterprise
editions, as defined below.

(b) "CockroachDB Core" shall mean the open source version of
(b) "CockroachDB Core" shall mean the version of
CockroachDB, available free of charge at

https://github.com/cockroachdb/cockroach
Expand Down Expand Up @@ -133,16 +133,16 @@ CockroachDB Community License Agreement
granted to You under this License for that Work shall terminate as
of the date such litigation is filed.

iii License to Third Party Works: From time to time Cockroach Labs may
use, or provide You access to, Third Party Works in connection
CockroachDB Enterprise Edition. You acknowledge and agree that in
iii License to Third Party Works: From time to time Cockroach Labs may
use, or provide You access to, Third Party Works in connection with
CockroachDB Enterprise Edition. You acknowledge and agree that in
addition to this Agreement, Your use of Third Party Works is subject
to all other terms and conditions set forth in the License provided
with or contained in such Third Party Works. Some Third Party Works
with or contained in such Third Party Works. Some Third Party Works
may be licensed to You solely for use with CockroachDB Enterprise
Edition under the terms of a third party License, or as otherwise
notified by Cockroach Labs, and not under the terms of this
Agreement. You agree that the owners and third party licensors of
Agreement. You agree that the owners and third party licensors of
Third Party Works are intended third party beneficiaries to this
Agreement.

Expand All @@ -152,18 +152,25 @@ CockroachDB Community License Agreement

4. Fees for CockroachDB Enterprise Edition or CockroachDB Support.

(a) Fees. The License to CockroachDB Enterprise Edition is conditioned upon
Your payment of the fees specified on

https://cockroachlabs.com/pricing

which You agree to pay to Cockroach Labs in accordance with the payment
terms set out on that page. Any professional services or support for
CockroachDB may also be subject to Your payment of fees, which will be
specified by Cockroach Labs when you sign up to receive such
professional services or support. Cockroach Labs reserves the right to
change the fees at any time with prior written notice; for recurring
fees, any such adjustments will take effect as of the next pay period.
(a) Fees. The License to CockroachDB Enterprise Edition is conditioned
upon Your entering into a signed written agreement with Cockroach Labs
for its use (a “Paid Enterprise License”) and timely paying Cockroach
Labs for such Paid Enterprise License; provided that features of
CockroachDB Enterprise Edition that are not designated as “Enterprise
features” at

https://www.cockroachlabs.com/docs/stable/enterprise-licensing.html

and do not require a license key (e.g. the feature does not require
the use of the ‘enterprise.license’ cluster setting, and the software
has not been modified to remove such a requirement) may be used for
free under the terms of the Agreement without a Paid Enterprise
License. Any professional services or support for CockroachDB may also
be subject to Your payment of fees, which will be specified by
Cockroach Labs when you sign up to receive such professional services
or support. Cockroach Labs reserves the right to change the fees at
any time with prior written notice; for recurring fees, any such
adjustments will take effect as of the next pay period.

(b) Overdue Payments and Taxes. Overdue payments are subject to a service
charge equal to the lesser of 1.5% per month or the maximum legal
Expand Down
51 changes: 25 additions & 26 deletions pkg/ccl/backupccl/backup_test.go
Expand Up @@ -5059,39 +5059,39 @@ func TestDetachedBackup(t *testing.T) {
defer log.Scope(t).Close(t)

const numAccounts = 1
_, tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
ctx, tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
defer cleanupFn()

db := sqlDB.DB.(*gosql.DB)

// running backup under transaction requires DETACHED.
var jobID int64
tx, err := db.Begin()
require.NoError(t, err)
err = tx.QueryRow(`BACKUP DATABASE data TO $1`, LocalFoo).Scan(&jobID)
err := crdb.ExecuteTx(ctx, db, nil /* txopts */, func(tx *gosql.Tx) error {
return tx.QueryRow(`BACKUP DATABASE data TO $1`, LocalFoo).Scan(&jobID)
})
require.True(t, testutils.IsError(err,
"BACKUP cannot be used inside a transaction without DETACHED option"))
require.NoError(t, tx.Rollback())

// Okay to run DETACHED backup, even w/out explicit transaction.
sqlDB.QueryRow(t, `BACKUP DATABASE data TO $1 WITH DETACHED`, LocalFoo).Scan(&jobID)
waitForSuccessfulJob(t, tc, jobID)

// Backup again, under explicit transaction.
tx, err = db.Begin()
require.NoError(t, err)
err = tx.QueryRow(`BACKUP DATABASE data TO $1 WITH DETACHED`, LocalFoo+"/1").Scan(&jobID)
err = crdb.ExecuteTx(ctx, db, nil /* txopts */, func(tx *gosql.Tx) error {
return tx.QueryRow(`BACKUP DATABASE data TO $1 WITH DETACHED`, LocalFoo+"/1").Scan(&jobID)
})
require.NoError(t, err)
require.NoError(t, tx.Commit())
waitForSuccessfulJob(t, tc, jobID)

// Backup again under transaction, but this time abort the transaction.
// No new jobs should have been created.
allJobsQuery := "SELECT job_id FROM [SHOW JOBS]"
allJobs := sqlDB.QueryStr(t, allJobsQuery)
tx, err = db.Begin()
tx, err := db.Begin()
require.NoError(t, err)
err = tx.QueryRow(`BACKUP DATABASE data TO $1 WITH DETACHED`, LocalFoo+"/2").Scan(&jobID)
err = crdb.Execute(func() error {
return tx.QueryRow(`BACKUP DATABASE data TO $1 WITH DETACHED`, LocalFoo+"/2").Scan(&jobID)
})
require.NoError(t, err)
require.NoError(t, tx.Rollback())
sqlDB.CheckQueryResults(t, allJobsQuery, allJobs)
Expand All @@ -5106,7 +5106,7 @@ func TestDetachedRestore(t *testing.T) {
defer log.Scope(t).Close(t)

const numAccounts = 1
_, tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
ctx, tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
defer cleanupFn()

db := sqlDB.DB.(*gosql.DB)
Expand All @@ -5119,12 +5119,11 @@ func TestDetachedRestore(t *testing.T) {

// Running RESTORE under transaction requires DETACHED.
var jobID int64
tx, err := db.Begin()
require.NoError(t, err)
err = tx.QueryRow(`RESTORE TABLE t FROM $1 WITH INTO_DB=test`, LocalFoo).Scan(&jobID)
err := crdb.ExecuteTx(ctx, db, nil /* txopts */, func(tx *gosql.Tx) error {
return tx.QueryRow(`RESTORE TABLE t FROM $1 WITH INTO_DB=test`, LocalFoo).Scan(&jobID)
})
require.True(t, testutils.IsError(err,
"RESTORE cannot be used inside a transaction without DETACHED option"))
require.NoError(t, tx.Rollback())

// Okay to run DETACHED RESTORE, even w/out explicit transaction.
sqlDB.QueryRow(t, `RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`,
Expand All @@ -5133,21 +5132,22 @@ func TestDetachedRestore(t *testing.T) {
sqlDB.Exec(t, `DROP TABLE test.t`)

// RESTORE again, under explicit transaction.
tx, err = db.Begin()
require.NoError(t, err)
err = tx.QueryRow(`RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`, LocalFoo).Scan(&jobID)
err = crdb.ExecuteTx(ctx, db, nil /* txopts */, func(tx *gosql.Tx) error {
return tx.QueryRow(`RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`, LocalFoo).Scan(&jobID)
})
require.NoError(t, err)
require.NoError(t, tx.Commit())
waitForSuccessfulJob(t, tc, jobID)
sqlDB.Exec(t, `DROP TABLE test.t`)

// RESTORE again under transaction, but this time abort the transaction.
// No new jobs should have been created.
allJobsQuery := "SELECT job_id FROM [SHOW JOBS]"
allJobs := sqlDB.QueryStr(t, allJobsQuery)
tx, err = db.Begin()
tx, err := db.Begin()
require.NoError(t, err)
err = tx.QueryRow(`RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`, LocalFoo).Scan(&jobID)
err = crdb.Execute(func() error {
return tx.QueryRow(`RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`, LocalFoo).Scan(&jobID)
})
require.NoError(t, err)
require.NoError(t, tx.Rollback())
sqlDB.CheckQueryResults(t, allJobsQuery, allJobs)
Expand Down Expand Up @@ -5704,11 +5704,10 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) {
backupWithDetachedOption := query + ` WITH DETACHED`
db := sqlDB.DB.(*gosql.DB)
var jobID int64
tx, err := db.Begin()
require.NoError(t, err)
err = tx.QueryRow(backupWithDetachedOption).Scan(&jobID)
err := crdb.ExecuteTx(ctx, db, nil /* txopts */, func(tx *gosql.Tx) error {
return tx.QueryRow(backupWithDetachedOption).Scan(&jobID)
})
require.NoError(t, err)
require.NoError(t, tx.Commit())
waitForSuccessfulJob(t, tc, jobID)
},
},
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/crdb_internal.go
Expand Up @@ -1145,9 +1145,9 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans (
goroutine_id INT NOT NULL, -- The ID of the goroutine on which the span was created.
finished BOOL NOT NULL, -- True if the span has been Finish()ed, false otherwise.
start_time TIMESTAMPTZ, -- The span's start time.
duration INTERVAL, -- The span's duration, measured from start to Finish().
-- A span whose recording is collected before it's finished will
-- have the duration set as the "time of collection - start time".
duration INTERVAL, -- The span's duration, measured from start to Finish().
-- A span whose recording is collected before it's finished will
-- have the duration set as the "time of collection - start time".
operation STRING NULL -- The span's operation.
)`,
populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error {
Expand Down
79 changes: 28 additions & 51 deletions pkg/sql/distsql/columnar_utils_test.go
Expand Up @@ -12,10 +12,11 @@ package distsql

import (
"context"
"fmt"
"math"
"math/rand"
"sort"
"strconv"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
Expand Down Expand Up @@ -257,14 +258,6 @@ func verifyColOperator(t *testing.T, args verifyColOperatorArgs) error {
procRows, colOpRows, procMetas, colOpMetas)
}

printRowsOutput := func(rows [][]string) string {
res := ""
for i, row := range rows {
res = fmt.Sprintf("%s\n%d: %v", res, i, row)
}
return res
}

datumsMatch := func(expected, actual string, typ *types.T) (bool, error) {
switch typ.Family() {
case types.FloatFamily:
Expand Down Expand Up @@ -303,54 +296,38 @@ func verifyColOperator(t *testing.T, args verifyColOperatorArgs) error {
colIdxsToCheckForEquality[i] = i
}
}

if args.anyOrder {
used := make([]bool, len(colOpRows))
for i, expStrRow := range procRows {
rowMatched := false
for j, retStrRow := range colOpRows {
if used[j] {
continue
}
foundDifference := false
// The rows are allowed to be in any order, so in order to use the
// ordered comparison below we will sort rows from both the processor
// and the operator lexicographically.
getLessFn := func(rows [][]string) func(int, int) bool {
return func(i, j int) bool {
for _, colIdx := range colIdxsToCheckForEquality {
match, err := datumsMatch(expStrRow[colIdx], retStrRow[colIdx], args.pspec.ResultTypes[colIdx])
if err != nil {
return errors.Errorf("error while parsing datum in rows\n%v\n%v\n%s",
expStrRow, retStrRow, err.Error())
if cmp := strings.Compare(rows[i][colIdx], rows[j][colIdx]); cmp != 0 {
return cmp < 0
}
if !match {
foundDifference = true
break
}
}
if !foundDifference {
rowMatched = true
used[j] = true
break
}
}
if !rowMatched {
return errors.Errorf("different results: no match found for row %d of processor output\n"+
"processor output:%s\n\ncolumnar operator output:%s",
i, printRowsOutput(procRows), printRowsOutput(colOpRows))
return false
}
}
} else {
for i, expStrRow := range procRows {
retStrRow := colOpRows[i]
// anyOrder is false, so the result rows must match in the same order.
for _, colIdx := range colIdxsToCheckForEquality {
match, err := datumsMatch(expStrRow[colIdx], retStrRow[colIdx], args.pspec.ResultTypes[colIdx])
if err != nil {
return errors.Errorf("error while parsing datum in rows\n%v\n%v\n%s",
expStrRow, retStrRow, err.Error())
}
if !match {
return errors.Errorf(
"different results on row %d;\nexpected:\n%s\ngot:\n%s",
i, expStrRow, retStrRow,
)
}
sort.Slice(procRows, getLessFn(procRows))
sort.Slice(colOpRows, getLessFn(colOpRows))
}

for i, expStrRow := range procRows {
retStrRow := colOpRows[i]
for _, colIdx := range colIdxsToCheckForEquality {
match, err := datumsMatch(expStrRow[colIdx], retStrRow[colIdx], args.pspec.ResultTypes[colIdx])
if err != nil {
return errors.Errorf("error while parsing datum in rows\n%v\n%v\n%s",
expStrRow, retStrRow, err.Error())
}
if !match {
return errors.Errorf(
"different results on row %d;\nexpected:\n%s\ngot:\n%s",
i, expStrRow, retStrRow,
)
}
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/tracing/span.go
Expand Up @@ -228,7 +228,7 @@ func (s *Span) Finish() {
s.netTr.Finish()
}
s.tracer.activeSpans.Lock()
delete(s.tracer.activeSpans.m, s)
delete(s.tracer.activeSpans.m, s.crdb.spanID)
s.tracer.activeSpans.Unlock()
}

Expand Down
9 changes: 5 additions & 4 deletions pkg/util/tracing/tracer.go
Expand Up @@ -150,6 +150,7 @@ type Tracer struct {
// activeSpans is a map that references all non-Finish'ed local root spans,
// i.e. those for which no WithLocalParent(<non-nil>) option was supplied.
// It also elides spans created using WithBypassRegistry.
// The map is keyed on the span ID, which is deterministically unique.
//
// In normal operation, a local root Span is inserted on creation and
// removed on .Finish().
Expand All @@ -164,7 +165,7 @@ type Tracer struct {
// has grown to accommodate the usual number of active local root spans,
// and the critical sections of the mutex are very small.
syncutil.Mutex
m map[*Span]struct{}
m map[uint64]*Span
}

includeAsyncSpansInRecordings bool // see TestingIncludeAsyncSpansInRecordings
Expand All @@ -175,7 +176,7 @@ type Tracer struct {
// backends.
func NewTracer() *Tracer {
t := &Tracer{}
t.activeSpans.m = map[*Span]struct{}{}
t.activeSpans.m = make(map[uint64]*Span)
t.noopSpan = &Span{tracer: t}
return t
}
Expand Down Expand Up @@ -442,7 +443,7 @@ func (t *Tracer) startSpanGeneric(
// Local root span - put it into the registry of active local root
// spans. `Span.Finish` takes care of deleting it again.
t.activeSpans.Lock()
t.activeSpans.m[s] = struct{}{}
t.activeSpans.m[spanID] = s
t.activeSpans.Unlock()
}

Expand Down Expand Up @@ -680,7 +681,7 @@ func (t *Tracer) ExtractMetaFrom(carrier Carrier) (*SpanMeta, error) {
func (t *Tracer) VisitSpans(visitor func(*Span) error) error {
t.activeSpans.Lock()
sl := make([]*Span, 0, len(t.activeSpans.m))
for sp := range t.activeSpans.m {
for _, sp := range t.activeSpans.m {
sl = append(sl, sp)
}
t.activeSpans.Unlock()
Expand Down

0 comments on commit 5744288

Please sign in to comment.