Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
136738: workload/schemachanger: fix drop columns during PK swaps r=fqazi a=fqazi

Previously, if a declarative alter primary key and drop column were executed concurrently this workload did not correctly pick the correct columns. It was possible for the new primary key column to be selected for the drop operation. To address this, this patch allows potential execution errors for invalid column references when a PK swap is in progress.

Fixes: #134056
Fixes: #136661
Fixes: #136405
Fixes: #132298

Release note: None

136754: revert: move revertccl out of ccl r=dt a=dt

Release note: none.
Epic: none.

136803: scop: fix non-redaction of scop Type, Phase, and Stage r=rafiss a=rafiss

Epic: None
Release note: None

136804: builtins: fix handling of <1 seconds in make_timestamp{tz} r=rafiss a=rafiss

On some platforms, `make_timestamp(1, 1, 1, 0, 0, 0)` would return a value of `0293-09-22 00:12:43.145224 BC`. Now all platforms will compute the correct value of `0001-01-01 00:00:00`.

Also, on some platforms a fractional second value that is between 0 and 1 (exclusive), would always be interpeted as zero nanoseconds. Now it will always be converted to nanoseconds correctly.

fixes #126766
Release note (bug fix): Fixed a bug that would cause the make_timestamp and make_timestamptz builtin functions to incorrectly extract the `seconds` argument if the value was less than 1.

Co-authored-by: Faizan Qazi <[email protected]>
Co-authored-by: David Taylor <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
4 people committed Dec 5, 2024
5 parents 78ab13a + 541a3d1 + 2cc3d8f + 6732bd2 + 27b31c9 commit f9a6082
Show file tree
Hide file tree
Showing 22 changed files with 132 additions and 73 deletions.
2 changes: 1 addition & 1 deletion .github/CODEOWNERS
Validating CODEOWNERS rules …
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,7 @@
/pkg/crosscluster/ @cockroachdb/disaster-recovery
/pkg/backup/ @cockroachdb/disaster-recovery
/pkg//backup/*_job.go @cockroachdb/disaster-recovery @cockroachdb/jobs-prs
/pkg/ccl/revertccl/ @cockroachdb/disaster-recovery
/pkg/revert/ @cockroachdb/disaster-recovery
/pkg/ccl/storageccl/ @cockroachdb/disaster-recovery
/pkg/ccl/cloudccl/ @cockroachdb/disaster-recovery
/pkg/cloud/ @cockroachdb/disaster-recovery
Expand Down
6 changes: 3 additions & 3 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,6 @@ ALL_TESTS = [
"//pkg/ccl/partitionccl:partitionccl_test",
"//pkg/ccl/pgcryptoccl/pgcryptocipherccl:pgcryptocipherccl_test",
"//pkg/ccl/pgcryptoccl:pgcryptoccl_test",
"//pkg/ccl/revertccl:revertccl_test",
"//pkg/ccl/schemachangerccl:schemachangerccl_test",
"//pkg/ccl/serverccl/adminccl:adminccl_test",
"//pkg/ccl/serverccl/diagnosticsccl:diagnosticsccl_test",
Expand Down Expand Up @@ -301,6 +300,7 @@ ALL_TESTS = [
"//pkg/raft/rafttest:rafttest_test",
"//pkg/raft/tracker:tracker_test",
"//pkg/raft:raft_test",
"//pkg/revert:revert_test",
"//pkg/roachpb:roachpb_disallowed_imports_test",
"//pkg/roachpb:roachpb_test",
"//pkg/roachprod/cloud:cloud_test",
Expand Down Expand Up @@ -951,8 +951,6 @@ GO_TARGETS = [
"//pkg/ccl/pgcryptoccl/pgcryptocipherccl:pgcryptocipherccl_test",
"//pkg/ccl/pgcryptoccl:pgcryptoccl",
"//pkg/ccl/pgcryptoccl:pgcryptoccl_test",
"//pkg/ccl/revertccl:revertccl",
"//pkg/ccl/revertccl:revertccl_test",
"//pkg/ccl/schemachangerccl:schemachangerccl",
"//pkg/ccl/schemachangerccl:schemachangerccl_test",
"//pkg/ccl/securityccl/fipsccl:fipsccl",
Expand Down Expand Up @@ -1604,6 +1602,8 @@ GO_TARGETS = [
"//pkg/release:release",
"//pkg/repstream/streampb:streampb",
"//pkg/repstream:repstream",
"//pkg/revert:revert",
"//pkg/revert:revert_test",
"//pkg/roachpb:roachpb",
"//pkg/roachpb:roachpb_test",
"//pkg/roachprod/cloud:cloud",
Expand Down
2 changes: 1 addition & 1 deletion pkg/crosscluster/physical/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ go_library(
deps = [
"//pkg/backup",
"//pkg/base",
"//pkg/ccl/revertccl",
"//pkg/ccl/utilccl",
"//pkg/cloud",
"//pkg/cloud/externalconn",
Expand All @@ -45,6 +44,7 @@ go_library(
"//pkg/multitenant/mtinfopb",
"//pkg/repstream",
"//pkg/repstream/streampb",
"//pkg/revert",
"//pkg/roachpb",
"//pkg/server/telemetry",
"//pkg/settings",
Expand Down
4 changes: 2 additions & 2 deletions pkg/crosscluster/physical/stream_ingest_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,14 +8,14 @@ package physical
import (
"context"

"github.com/cockroachdb/cockroach/pkg/ccl/revertccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/crosscluster/replicationutils"
"github.com/cockroachdb/cockroach/pkg/crosscluster/streamclient"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/repstream"
"github.com/cockroachdb/cockroach/pkg/repstream/streampb"
"github.com/cockroachdb/cockroach/pkg/revert"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
Expand Down Expand Up @@ -47,7 +47,7 @@ func (r *streamIngestManagerImpl) GetReplicationStatsAndStatus(
func (r *streamIngestManagerImpl) RevertTenantToTimestamp(
ctx context.Context, tenantName roachpb.TenantName, revertTo hlc.Timestamp,
) error {
return revertccl.RevertTenantToTimestamp(ctx, r.evalCtx, tenantName, revertTo, r.sessionID)
return revert.RevertTenantToTimestamp(ctx, r.evalCtx, tenantName, revertTo, r.sessionID)
}

func newStreamIngestManagerWithPrivilegesCheck(
Expand Down
6 changes: 3 additions & 3 deletions pkg/crosscluster/physical/stream_ingestion_dist.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/backup"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/revertccl"
"github.com/cockroachdb/cockroach/pkg/crosscluster"
"github.com/cockroachdb/cockroach/pkg/crosscluster/replicationutils"
"github.com/cockroachdb/cockroach/pkg/crosscluster/streamclient"
Expand All @@ -24,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/repstream/streampb"
"github.com/cockroachdb/cockroach/pkg/revert"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
Expand Down Expand Up @@ -74,11 +74,11 @@ func startDistIngestion(
log.Infof(ctx, "reverting tenant %s to time %s (via %s) before starting replication", details.DestinationTenantID, replicatedTime, revertTo)

spanToRevert := keys.MakeTenantSpan(details.DestinationTenantID)
if err := revertccl.RevertSpansFanout(ctx, execCtx.ExecCfg().DB, execCtx,
if err := revert.RevertSpansFanout(ctx, execCtx.ExecCfg().DB, execCtx,
[]roachpb.Span{spanToRevert},
revertTo,
false, /* ignoreGCThreshold */
revertccl.RevertDefaultBatchSize,
revert.RevertDefaultBatchSize,
nil, /* onCompletedCallback */
); err != nil {
return err
Expand Down
6 changes: 3 additions & 3 deletions pkg/crosscluster/physical/stream_ingestion_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,6 @@ import (
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/ccl/revertccl"
"github.com/cockroachdb/cockroach/pkg/crosscluster/producer"
"github.com/cockroachdb/cockroach/pkg/crosscluster/replicationutils"
"github.com/cockroachdb/cockroach/pkg/crosscluster/streamclient"
Expand All @@ -21,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb"
"github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb"
"github.com/cockroachdb/cockroach/pkg/repstream/streampb"
"github.com/cockroachdb/cockroach/pkg/revert"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -501,13 +501,13 @@ func maybeRevertToCutoverTimestamp(
return cutoverTimestamp, false, err
}

batchSize := int64(revertccl.RevertDefaultBatchSize)
batchSize := int64(revert.RevertDefaultBatchSize)
if p.ExecCfg().StreamingTestingKnobs != nil && p.ExecCfg().StreamingTestingKnobs.OverrideRevertRangeBatchSize != 0 {
batchSize = p.ExecCfg().StreamingTestingKnobs.OverrideRevertRangeBatchSize
}
// On cutover, replication has stopped so therefore should set replicated time to 0
p.ExecCfg().JobRegistry.MetricsStruct().StreamIngest.(*Metrics).ReplicatedTimeSeconds.Update(0)
if err := revertccl.RevertSpansFanout(ctx,
if err := revert.RevertSpansFanout(ctx,
p.ExecCfg().DB,
p,
remainingSpansToRevert,
Expand Down
11 changes: 4 additions & 7 deletions pkg/ccl/revertccl/BUILD.bazel → pkg/revert/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,16 +1,15 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "revertccl",
name = "revert",
srcs = [
"alter_reset_tenant.go",
"revert.go",
"revert_tenant.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/revertccl",
importpath = "github.com/cockroachdb/cockroach/pkg/revert",
visibility = ["//visibility:public"],
deps = [
"//pkg/ccl/utilccl",
"//pkg/keys",
"//pkg/kv",
"//pkg/kv/kvpb",
Expand Down Expand Up @@ -41,20 +40,18 @@ go_library(
)

go_test(
name = "revertccl_test",
name = "revert_test",
srcs = [
"main_test.go",
"revert_test.go",
],
embed = [":revertccl"],
embed = [":revert"],
exec_properties = select({
"//build/toolchains:is_heavy": {"test.Pool": "large"},
"//conditions:default": {"test.Pool": "default"},
}),
deps = [
"//pkg/base",
"//pkg/ccl",
"//pkg/ccl/storageccl",
"//pkg/kv",
"//pkg/kv/kvpb",
"//pkg/roachpb",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,12 +3,11 @@
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package revertccl
package revert

import (
"context"

"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/exprutil"
Expand Down Expand Up @@ -43,9 +42,6 @@ func alterTenantResetHook(
if err := sql.CanManageTenant(ctx, p); err != nil {
return err
}
if err := utilccl.CheckEnterpriseEnabled(p.ExecCfg().Settings, alterTenantResetOp); err != nil {
return err
}

tenInfo, err := p.LookupTenantInfo(ctx, alterTenantStmt.TenantSpec, alterTenantResetOp)
if err != nil {
Expand Down
5 changes: 1 addition & 4 deletions pkg/ccl/revertccl/main_test.go → pkg/revert/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,14 +3,12 @@
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package revertccl_test
package revert_test

import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/ccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/server"
Expand All @@ -20,7 +18,6 @@ import (
)

func TestMain(m *testing.M) {
defer ccl.TestingEnableEnterprise()()
securityassets.SetLoader(securitytest.EmbeddedAssets)
randutil.SeedForTests()
serverutils.InitTestServerFactory(server.TestServerFactory)
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/revertccl/revert.go → pkg/revert/revert.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package revertccl
package revert

import (
"context"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package revertccl
package revert

import (
"context"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package revertccl
package revert

import (
"context"
Expand Down
24 changes: 24 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/datetime
Original file line number Diff line number Diff line change
Expand Up @@ -2064,6 +2064,18 @@ SELECT make_timestamp(2013, 7, 15, 8, 15, 23.5231231244234)::string
make_timestamp
2013-07-15 08:15:23.523123

# Test with seconds < 1.

query T
select make_timestamp(1, 1, 1, 0, 0, 0);
----
0001-01-01 00:00:00 +0000 +0000

query T
select make_timestamp(1, 1, 1, 0, 0, 0.1234);
----
0001-01-01 00:00:00.1234 +0000 +0000

statement error pgcode 22008 pq: make_timestamp\(\): year value of 0 is not valid
SELECT make_timestamp(0, 7, 15, 8, 15, 23.5);

Expand Down Expand Up @@ -2098,6 +2110,18 @@ SELECT make_timestamptz(2013, 7, 15, 8, 15, 23.5, 'America/New_York')
make_timestamptz
2013-07-15 07:15:23.5 -0500 EST

# Test with seconds < 1.

query T
select make_timestamptz(2020, 1, 1, 0, 0, 0, 'America/New_York');
----
2020-01-01 00:00:00 -0500 EST

query T
select make_timestamptz(2020, 1, 1, 0, 0, 0.1234, 'America/New_York');
----
2020-01-01 00:00:00.1234 -0500 EST

statement error pgcode 22008 pq: make_timestamptz\(\): year value of 0 is not valid
SELECT make_timestamptz(0, 7, 15, 8, 15, 23.5);

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scop/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ go_library(
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/sem/catid",
"//pkg/sql/sem/semenumpb",
"@com_github_cockroachdb_redact//:redact",
],
)

Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/schemachanger/scop/ops.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,8 @@
// Package scop describes ops within a schema change.
package scop

import "github.com/cockroachdb/redact"

// Op represents an action to be taken on a single descriptor.
type Op interface {
Type() Type
Expand All @@ -15,6 +17,11 @@ type Op interface {
// same Stage only if they share a type.
type Type int

var _ redact.SafeValue = Type(0)

// SafeValue implements the redact.SafeValue interface.
func (t Type) SafeValue() {}

//go:generate stringer -type=Type

const (
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/schemachanger/scop/phase.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,8 @@

package scop

import "github.com/cockroachdb/redact"

// A Phase represents the context in which an op is executed within a schema
// change. Different phases require different dependencies for the execution of
// the ops to be plumbed in.
Expand All @@ -14,6 +16,11 @@ package scop
// transition to transactional schema changes.
type Phase int

var _ redact.SafeValue = Phase(0)

// SafeValue implements the redact.SafeValue interface.
func (p Phase) SafeValue() {}

//go:generate stringer --type Phase

const (
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scplan/internal/scstage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,5 +19,6 @@ go_library(
"//pkg/sql/schemachanger/screl",
"//pkg/util/iterutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
],
)
15 changes: 15 additions & 0 deletions pkg/sql/schemachanger/scplan/internal/scstage/stage.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// A Stage is a sequence of ops to be executed "together" as part of a schema
Expand Down Expand Up @@ -43,6 +44,20 @@ type Stage struct {
Ordinal, StagesInPhase int
}

var _ redact.SafeFormatter = Stage{}

func (s Stage) SafeFormat(p redact.SafePrinter, _ rune) {
p.Printf("%s stage %d of %d with ",
s.Phase, s.Ordinal, s.StagesInPhase)
if n := len(s.Ops()); n > 1 {
p.Printf("%d %s ops", n, s.Type())
} else if n == 1 {
p.Printf("1 %s op", s.Type())
} else {
p.Printf("no ops")
}
}

// Type returns the type of the operations in this stage.
func (s Stage) Type() scop.Type {
if len(s.EdgeOps) == 0 {
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -11998,8 +11998,8 @@ func makeTimestampStatementBuiltinOverload(withOutputTZ bool, withInputTZ bool)
hour := int(tree.MustBeDInt(args[3]))
min := int(tree.MustBeDInt(args[4]))
sec := float64(tree.MustBeDFloat(args[5]))
truncatedSec := math.Floor(sec)
nsec := math.Mod(sec, truncatedSec) * float64(time.Second)
truncatedSec, remainderSec := math.Modf(sec)
nsec := remainderSec * float64(time.Second)
t := time.Date(year, month, day, hour, min, int(truncatedSec), int(nsec), location)
if withOutputTZ {
return tree.MakeDTimestampTZ(t, time.Microsecond)
Expand Down
Loading

0 comments on commit f9a6082

Please sign in to comment.