Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
30405: roachtest: mark acceptance as stable r=petermattis a=tschottdorf

all of its subtests are already stable, but in running a test locally I
noticed that the top-level test was marked as passing as unstable. I'm
not sure, but this might mean that the top-level test would actually not
fail? Either way, better to mark it as stable explicitly.

We should also spend some thought on how diverging notions of Stable in
sub vs top level test are treated, not sure that this is well-defined.

Release note: None

30446: opt: fix panic when srf used with GROUP BY r=rytaft a=rytaft

Instead of panicking, we now throw an appropriate error.

Fixes #30412

Release note (bug fix): Fixed a panic that occurred when a
generator function such as unnest was used in the SELECT list
in the presence of GROUP BY.

30450: roachtest: remove now-unnecessary hack r=petermattis a=tschottdorf

Closes #27717.

Release note: None

30451: storage: give TestReplicateRemovedNodeDisruptiveElection more time r=petermattis a=tschottdorf

Perhaps:

Fixes #27253.

Release note: None

30452: storage: de-flake TestReplicaIDChangePending r=petermattis a=tschottdorf

setReplicaID refreshes the proposal and was thus synchronously writing
to the commandProposed chan. This channel could have filled up due to
an earlier reproposal already, deadlocking the test.

Fixes #28132.

Release note: None

30455: testcluster: improve AddReplicas check r=petermattis a=tschottdorf

AddReplicas was verifying that a replica had indeed been added, but
there's no guarantee that the replicate queue wouldn't have removed
it in the meantime. Attempt to work around this somewhat. The real
solution is not to provide that guarantee, but some tests likely
rely on it (and the failure is extremely rare, i.e. the new for
loop basically never runs).

Observed in #28368.

Release note: None

30456: storage: unskip TestClosedTimestampCanServe for non-race r=petermattis a=tschottdorf

Fixes #28607.

Release note: None

Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
Co-authored-by: Rebecca Taft <becca@cockroachlabs.com>
  • Loading branch information
3 people committed Sep 20, 2018
8 parents fbd747c + db1eb7d + 066c9db + 081eddd + 8757765 + db68ad9 + 4b20a07 + fe0d16b commit 3d304bc
Show file tree
Hide file tree
Showing 9 changed files with 89 additions and 52 deletions.
5 changes: 3 additions & 2 deletions pkg/cmd/roachtest/acceptance.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,9 @@ func registerAcceptance(r *registry) {
// local mode the acceptance tests should be configured to run within a
// minute or so as these tests are run on every merge to master.
spec := testSpec{
Name: "acceptance",
Nodes: nodes(4),
Name: "acceptance",
Nodes: nodes(4),
Stable: true, // DO NOT COPY to new tests
}

testCases := []struct {
Expand Down
14 changes: 2 additions & 12 deletions pkg/cmd/roachtest/upgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,17 +48,7 @@ func registerUpgrade(r *registry) {
c.Put(ctx, b, "./cockroach", c.Range(1, nodes))
// Force disable encryption.
// TODO(mberhault): allow it once oldVersion >= 2.1.
start := func() {
c.Start(ctx, c.Range(1, nodes), startArgsDontEncrypt)
}
start()
time.Sleep(5 * time.Second)

// TODO(tschottdorf): this is a hack similar to the one in the mixed version
// test. Remove it when we have a 2.0.x binary that has #27639 fixed.
c.Stop(ctx, c.Range(1, nodes))
start()
time.Sleep(5 * time.Second)
c.Start(ctx, c.Range(1, nodes), startArgsDontEncrypt)

const stageDuration = 30 * time.Second
const timeUntilStoreDead = 90 * time.Second
Expand Down Expand Up @@ -266,7 +256,7 @@ func registerUpgrade(r *registry) {
}
}

const oldVersion = "v2.0.0"
const oldVersion = "v2.0.5"
for _, n := range []int{5} {
r.Add(testSpec{
Name: fmt.Sprintf("upgrade/oldVersion=%s/nodes=%d", oldVersion, n),
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/opt/optbuilder/groupby.go
Original file line number Diff line number Diff line change
Expand Up @@ -515,3 +515,10 @@ var aggOpLookup = map[string]opt.Operator{
"json_agg": opt.JsonAggOp,
"jsonb_agg": opt.JsonbAggOp,
}

func newGroupingError(name *tree.Name) error {
return pgerror.NewErrorf(pgerror.CodeGroupingError,
"column \"%s\" must appear in the GROUP BY clause or be used in an aggregate function",
tree.ErrString(name),
)
}
11 changes: 7 additions & 4 deletions pkg/sql/opt/optbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,10 +141,7 @@ func (b *Builder) buildScalar(
// Non-grouping column was referenced. Note that a column that is part
// of a larger grouping expression would have been detected by the
// groupStrs checking code above.
panic(builderError{pgerror.NewErrorf(pgerror.CodeGroupingError,
"column \"%s\" must appear in the GROUP BY clause or be used in an aggregate function",
tree.ErrString(&t.name),
)})
panic(builderError{newGroupingError(&t.name)})
}

return b.finishBuildScalarRef(t, inScope, outScope, outCol, colRefs)
Expand Down Expand Up @@ -407,6 +404,12 @@ func (b *Builder) buildScalar(

case *srf:
if len(t.cols) == 1 {
if inGroupingContext {
// Non-grouping column was referenced. Note that a column that is part
// of a larger grouping expression would have been detected by the
// groupStrs checking code above.
panic(builderError{newGroupingError(&t.cols[0].name)})
}
return b.finishBuildScalarRef(&t.cols[0], inScope, outScope, outCol, colRefs)
}
list := make([]memo.GroupID, len(t.cols))
Expand Down
16 changes: 16 additions & 0 deletions pkg/sql/opt/optbuilder/testdata/srfs
Original file line number Diff line number Diff line change
Expand Up @@ -908,3 +908,19 @@ project
│ │ └── true [type=bool]
│ └── const: 1000 [type=int]
└── true [type=bool]

# Regression test for #30412.
build
SELECT 0, unnest(ARRAY[0]) GROUP BY 1
----
error (42803): column "unnest" must appear in the GROUP BY clause or be used in an aggregate function

build
SELECT 0, unnest(ARRAY[0]) GROUP BY 1, 2
----
error: unnest(): generator functions are not allowed in GROUP BY

build
SELECT 0, information_schema._pg_expandarray(ARRAY[0]) GROUP BY 1
----
error (42803): column "x" must appear in the GROUP BY clause or be used in an aggregate function
2 changes: 1 addition & 1 deletion pkg/storage/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3193,7 +3193,7 @@ func TestReplicateRemovedNodeDisruptiveElection(t *testing.T) {
default:
t.Fatalf("unexpected error type %T: %s", pErr.GetDetail(), pErr)
}
case <-time.After(5 * time.Second):
case <-time.After(45 * time.Second):
t.Fatal("did not get expected error")
}

Expand Down
21 changes: 16 additions & 5 deletions pkg/storage/closed_timestamp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package storage_test

import (
"context"
"fmt"
"testing"
"time"

Expand All @@ -28,14 +29,21 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)

func TestClosedTimestampCanServe(t *testing.T) {
defer leaktest.AfterTest(t)()
t.Skip("https://github.com/cockroachdb/cockroach/issues/28607")

if util.RaceEnabled {
// Limiting how long transactions can run does not work
// well with race unless we're extremely lenient, which
// drives up the test duration.
t.Skip("skipping under race")
}

ctx := context.Background()
const numNodes = 3
Expand All @@ -47,13 +55,16 @@ func TestClosedTimestampCanServe(t *testing.T) {
// Every 0.1s=100ms, try close out a timestamp ~300ms in the past.
// We don't want to be more aggressive than that since it's also
// a limit on how long transactions can run.
if _, err := db0.Exec(`
SET CLUSTER SETTING kv.closed_timestamp.target_duration = '300ms';
SET CLUSTER SETTING kv.closed_timestamp.close_fraction = 0.1/0.3;
targetDuration := 300 * time.Millisecond
closeFraction := 0.3

if _, err := db0.Exec(fmt.Sprintf(`
SET CLUSTER SETTING kv.closed_timestamp.target_duration = '%s';
SET CLUSTER SETTING kv.closed_timestamp.close_fraction = %.3f;
SET CLUSTER SETTING kv.closed_timestamp.follower_reads_enabled = true;
CREATE DATABASE cttest;
CREATE TABLE cttest.kv (id INT PRIMARY KEY, value STRING);
`); err != nil {
`, targetDuration, closeFraction)); err != nil {
t.Fatal(err)
}

Expand Down
5 changes: 4 additions & 1 deletion pkg/storage/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7861,7 +7861,10 @@ func TestReplicaIDChangePending(t *testing.T) {
repl.mu.Lock()
repl.mu.submitProposalFn = func(p *ProposalData) error {
if p.Request.Timestamp == magicTS {
commandProposed <- struct{}{}
select {
case commandProposed <- struct{}{}:
default:
}
}
return nil
}
Expand Down
60 changes: 33 additions & 27 deletions pkg/testutils/testcluster/testcluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -348,37 +348,43 @@ func (tc *TestCluster) AddReplicas(
startKey roachpb.Key, targets ...roachpb.ReplicationTarget,
) (roachpb.RangeDescriptor, error) {
rKey := keys.MustAddr(startKey)
rangeDesc, err := tc.changeReplicas(
roachpb.ADD_REPLICA, rKey, targets...,
)
if err != nil {
return roachpb.RangeDescriptor{}, err
}
errRetry := errors.Errorf("target not found")
for {
rangeDesc, err := tc.changeReplicas(
roachpb.ADD_REPLICA, rKey, targets...,
)
if err != nil {
return roachpb.RangeDescriptor{}, err
}

// Wait for the replication to complete on all destination nodes.
if err := retry.ForDuration(time.Second*5, func() error {
for _, target := range targets {
// Use LookupReplica(keys) instead of GetRange(rangeID) to ensure that the
// snapshot has been transferred and the descriptor initialized.
store, err := tc.findMemberStore(target.StoreID)
if err != nil {
log.Errorf(context.TODO(), "unexpected error: %s", err)
return err
}
repl := store.LookupReplica(rKey)
if repl == nil {
return errors.Errorf("range not found on store %d", target)
}
desc := repl.Desc()
if _, ok := desc.GetReplicaDescriptor(target.StoreID); !ok {
return errors.Errorf("target store %d not yet in range descriptor %v", target.StoreID, desc)
// Wait for the replication to complete on all destination nodes.
if err := retry.ForDuration(time.Second*25, func() error {
for _, target := range targets {
// Use LookupReplica(keys) instead of GetRange(rangeID) to ensure that the
// snapshot has been transferred and the descriptor initialized.
store, err := tc.findMemberStore(target.StoreID)
if err != nil {
log.Errorf(context.TODO(), "unexpected error: %s", err)
return err
}
repl := store.LookupReplica(rKey)
if repl == nil {
return errors.Wrapf(errRetry, "for target %s", target)
}
desc := repl.Desc()
if _, ok := desc.GetReplicaDescriptor(target.StoreID); !ok {
return errors.Errorf("target store %d not yet in range descriptor %v", target.StoreID, desc)
}
}
return nil
}); errors.Cause(err) == errRetry {
log.Warningf(context.Background(), "target was likely downreplicated again; retrying after %s", err)
continue
} else if err != nil {
return roachpb.RangeDescriptor{}, err
}
return nil
}); err != nil {
return roachpb.RangeDescriptor{}, err
return rangeDesc, nil
}
return rangeDesc, nil
}

// RemoveReplicas is part of the TestServerInterface.
Expand Down

0 comments on commit 3d304bc

Please sign in to comment.