Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
110735: dev: avoid re-generating logictest files if unnecessary r=rickystewart a=liamgillies

Beforehand, running `dev testlogic` would always run all
tests regardless if they were modified, which wastes time.
This PR will check to see if logic test folders were modified 
and only re-generates them if they are modified. It also 
introduces a new flag, `--force-gen`, which will force 
generate the logic tests.

Fixes: #94845
Release note: None

110867: build/release: let `go_version` check be more flexible r=healthy-pod a=healthy-pod

This code change lets the check confirm that
`go_version` contains `fips`. Previously, it required that `fips` is a suffix of `go_version`.

Epic: none
Release note: None

110872: kv: ignore {exclusive,shared} locks in QueryResolvedTimestamp r=nvanbenschoten a=nvanbenschoten

Informs #100193.

The resolved timestamp is a function of the intents in a range, but not of the other locks in a range. This commit updates the QueryResolvedTimestamp evaluation method to use a LockTableIterator configured to ignore Exclusive or Shared locks.

Release note: None

Co-authored-by: Liam Gillies <[email protected]>
Co-authored-by: healthy-pod <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
4 people committed Sep 19, 2023
4 parents 33623e3 + 9e653ae + dd82dd5 + 72cd8ce commit 3071cac
Show file tree
Hide file tree
Showing 6 changed files with 77 additions and 36 deletions.
2 changes: 1 addition & 1 deletion build/release/teamcity-support.sh
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ verify_docker_image(){
error=1
fi
if [[ $fips_build == true ]]; then
if [[ "$go_version" != *"fips" ]]; then
if [[ "$go_version" != *"fips"* ]]; then
echo "ERROR: Go version '$go_version' does not contain 'fips'"
error=1
fi
Expand Down
20 changes: 1 addition & 19 deletions pkg/cmd/dev/test.go
Original file line number Diff line number Diff line change
Expand Up @@ -477,28 +477,10 @@ func getDirectoryFromTarget(target string) string {
}

func (d *dev) determineAffectedTargets(ctx context.Context) ([]string, error) {
// List files changed against `master`.
remotes, err := d.exec.CommandContextSilent(ctx, "git", "remote", "-v")
base, err := d.getMergeBaseHash(ctx)
if err != nil {
return nil, err
}
var upstream string
for _, remote := range strings.Split(strings.TrimSpace(string(remotes)), "\n") {
if (strings.Contains(remote, "github.com/cockroachdb/cockroach") || strings.Contains(remote, "github.com:cockroachdb/cockroach")) && strings.HasSuffix(remote, "(fetch)") {
upstream = strings.Fields(remote)[0]
break
}
}
if upstream == "" {
return nil, fmt.Errorf("could not find git upstream")
}

baseBytes, err := d.exec.CommandContextSilent(ctx, "git", "merge-base", fmt.Sprintf("%s/master", upstream), "HEAD")
if err != nil {
return nil, err
}
base := strings.TrimSpace(string(baseBytes))

changedFiles, err := d.exec.CommandContextSilent(ctx, "git", "diff", "--no-ext-diff", "--name-only", base, "--", "*.go", "**/testdata/**")
if err != nil {
return nil, err
Expand Down
22 changes: 21 additions & 1 deletion pkg/cmd/dev/testlogic.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package main

import (
"context"
"errors"
"fmt"
"log"
Expand All @@ -19,6 +20,7 @@ import (
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/spf13/cobra"
)

Expand All @@ -29,6 +31,7 @@ const (
configsFlag = "config"
showSQLFlag = "show-sql"
noGenFlag = "no-gen"
forceGenFlag = "force-gen"
flexTypesFlag = "flex-types"
workmemFlag = "default-workmem"
)
Expand Down Expand Up @@ -58,6 +61,7 @@ func makeTestLogicCmd(runE func(cmd *cobra.Command, args []string) error) *cobra
testLogicCmd.Flags().Bool(showSQLFlag, false, "show SQL statements/queries immediately before they are tested")
testLogicCmd.Flags().Bool(rewriteFlag, false, "rewrite test files using results from test run")
testLogicCmd.Flags().Bool(noGenFlag, false, "skip generating logic test files before running logic tests")
testLogicCmd.Flags().Bool(forceGenFlag, false, "force generating logic test files before running logic tests")
testLogicCmd.Flags().Bool(streamOutputFlag, false, "stream test output during run")
testLogicCmd.Flags().Bool(stressFlag, false, "run tests under stress")
testLogicCmd.Flags().String(testArgsFlag, "", "additional arguments to pass to go test binary")
Expand Down Expand Up @@ -85,6 +89,7 @@ func (d *dev) testlogic(cmd *cobra.Command, commandLine []string) error {
timeout = mustGetFlagDuration(cmd, timeoutFlag)
verbose = mustGetFlagBool(cmd, vFlag)
noGen = mustGetFlagBool(cmd, noGenFlag)
forceGen = mustGetFlagBool(cmd, forceGenFlag)
showSQL = mustGetFlagBool(cmd, showSQLFlag)
count = mustGetFlagInt(cmd, countFlag)
stress = mustGetFlagBool(cmd, stressFlag)
Expand Down Expand Up @@ -127,7 +132,7 @@ func (d *dev) testlogic(cmd *cobra.Command, commandLine []string) error {
return err
}

if !noGen {
if !noGen && (forceGen || d.shouldGenerateLogicTests(ctx)) {
err := d.generateLogicTest(cmd)
if err != nil {
return err
Expand Down Expand Up @@ -297,6 +302,21 @@ func (d *dev) testlogic(cmd *cobra.Command, commandLine []string) error {
return nil
}

// This function determines if any test_logic or execbuilder/testdata files were
// modified in the current branch, and if so, determines if we should re-generate logic tests.
func (d *dev) shouldGenerateLogicTests(ctx context.Context) bool {
if buildutil.CrdbTestBuild {
return true
}
base, _ := d.getMergeBaseHash(ctx)
// Generate logic tests if the merge base hash isn't found
if base == "" {
return true
}
changedFiles, _ := d.exec.CommandContextSilent(ctx, "git", "diff", "--no-ext-diff", "--name-only", base, "--", "pkg/sql/logictest/logictestbase/** ", "pkg/sql/logictest/testdata/**", "pkg/sql/sqlitelogictest/BUILD.bazel", "pkg/sql/sqlitelogictest/sqlitelogictest.go", "pkg/ccl/logictestccl/testdata/**", "pkg/sql/opt/exec/execbuilder/testdata/**")
return strings.TrimSpace(string(changedFiles)) != ""
}

// We know that the regular expressions for files should not contain whitespace
// because the file names do not contain whitespace. We support users passing
// whitespace separated regexps for multiple files.
Expand Down
24 changes: 24 additions & 0 deletions pkg/cmd/dev/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -249,3 +249,27 @@ func (d *dev) warnAboutChangeInStressBehavior(timeout time.Duration) {
log.Printf("Set DEV_I_UNDERSTAND_ABOUT_STRESS=1 to squelch this message")
}
}

// This function retrieves the merge-base hash between the current branch and master
func (d *dev) getMergeBaseHash(ctx context.Context) (string, error) {
// List files changed against `master`
remotes, err := d.exec.CommandContextSilent(ctx, "git", "remote", "-v")
if err != nil {
return "", err
}
var upstream string
for _, remote := range strings.Split(strings.TrimSpace(string(remotes)), "\n") {
if (strings.Contains(remote, "github.com/cockroachdb/cockroach") || strings.Contains(remote, "github.com:cockroachdb/cockroach")) && strings.HasSuffix(remote, "(fetch)") {
upstream = strings.Fields(remote)[0]
break
}
}
if upstream == "" {
return "", fmt.Errorf("could not find git upstream, run `git remote add upstream [email protected]:cockroachdb/cockroach.git`")
}
baseBytes, err := d.exec.CommandContextSilent(ctx, "git", "merge-base", fmt.Sprintf("%s/master", upstream), "HEAD")
if err != nil {
return "", err
}
return strings.TrimSpace(string(baseBytes)), nil
}
34 changes: 19 additions & 15 deletions pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,13 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -104,7 +104,13 @@ func computeMinIntentTimestamp(
) (hlc.Timestamp, []roachpb.Intent, error) {
ltStart, _ := keys.LockTableSingleKey(span.Key, nil)
ltEnd, _ := keys.LockTableSingleKey(span.EndKey, nil)
iter, err := reader.NewEngineIterator(storage.IterOptions{LowerBound: ltStart, UpperBound: ltEnd})
opts := storage.LockTableIteratorOptions{
LowerBound: ltStart,
UpperBound: ltEnd,
// Ignore Exclusive and Shared locks. We only care about intents.
MatchMinStr: lock.Intent,
}
iter, err := storage.NewLockTableIterator(reader, opts)
if err != nil {
return hlc.Timestamp{}, nil, err
}
Expand All @@ -124,22 +130,20 @@ func computeMinIntentTimestamp(
if err != nil {
return hlc.Timestamp{}, nil, err
}
lockedKey, err := keys.DecodeLockTableSingleKey(engineKey.Key)
ltKey, err := engineKey.ToLockTableKey()
if err != nil {
return hlc.Timestamp{}, nil, errors.Wrapf(err, "decoding LockTable key: %v", lockedKey)
return hlc.Timestamp{}, nil, errors.Wrapf(err, "decoding LockTable key: %v", ltKey)
}
// Unmarshal.
v, err := iter.UnsafeValue()
if err != nil {
return hlc.Timestamp{}, nil, err
if ltKey.Strength != lock.Intent {
return hlc.Timestamp{}, nil, errors.AssertionFailedf(
"unexpected strength for LockTableKey %s: %v", ltKey.Strength, ltKey)
}
if err := protoutil.Unmarshal(v, &meta); err != nil {
return hlc.Timestamp{}, nil, errors.Wrapf(err, "unmarshaling mvcc meta: %v", lockedKey)
// Unmarshal.
if err := iter.ValueProto(&meta); err != nil {
return hlc.Timestamp{}, nil, errors.Wrapf(err, "unmarshaling mvcc meta: %v", ltKey)
}
if meta.Txn == nil {
return hlc.Timestamp{}, nil,
errors.AssertionFailedf("nil transaction in LockTable. Key: %v,"+"mvcc meta: %v",
lockedKey, meta)
return hlc.Timestamp{}, nil, errors.AssertionFailedf("nil transaction in LockTable: %v", ltKey)
}

if minTS.IsEmpty() {
Expand All @@ -155,8 +159,8 @@ func computeMinIntentTimestamp(
intentFitsByCount := int64(len(encountered)) < maxEncounteredIntents
intentFitsByBytes := encounteredKeyBytes < maxEncounteredIntentKeyBytes
if oldEnough && intentFitsByCount && intentFitsByBytes {
encountered = append(encountered, roachpb.MakeIntent(meta.Txn, lockedKey))
encounteredKeyBytes += int64(len(lockedKey))
encountered = append(encountered, roachpb.MakeIntent(meta.Txn, ltKey.Key))
encounteredKeyBytes += int64(len(ltKey.Key))
}
}
return minTS, encountered, nil
Expand Down
11 changes: 11 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,28 +56,39 @@ func TestQueryResolvedTimestamp(t *testing.T) {
_, err := storage.MVCCDelete(ctx, db, roachpb.Key(k), hlc.Timestamp{}, storage.MVCCWriteOptions{})
require.NoError(t, err)
}
writeLock := func(k string, str lock.Strength) {
txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, 0, makeTS(1), 0, 1, 0)
err := storage.MVCCAcquireLock(ctx, db, &txn, str, roachpb.Key(k), nil, 0)
require.NoError(t, err)
}

// Setup: (with separated intents the actual key layout in the store is not what is listed below.)
//
// a: intent @ 5
// a: value @ 3
// b: inline value
// c: shared lock #1
// c: shared lock #2
// c: value @ 6
// c: value @ 4
// c: value @ 1
// d: intent @ 2
// e: intent @ 7
// f: inline value
// g: exclusive lock
//
// NB: must write each key in increasing timestamp order.
writeValue("a", 3)
writeIntent("a", 5)
writeInline("b")
writeValue("c", 1)
writeValue("c", 4)
writeLock("c", lock.Shared)
writeLock("c", lock.Shared)
writeIntent("d", 2)
writeIntent("e", 7)
writeInline("f")
writeLock("g", lock.Exclusive)

for _, cfg := range []struct {
name string
Expand Down

0 comments on commit 3071cac

Please sign in to comment.