Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
104089: testutils: add fingerprint utility package r=stevendanna a=msbutler

This patch adds a couple helper methods that make it easier to: fingerprint a table, database or cluster, compare fingerprints, and identify a mismatch on the table level. All fingerprinting methods call crdb_internal.fingerprint() on the table span level with the user provided options.

This patch also integrates the new methods in the backupccl and streamingccl codebases.

A future patch could add a helper method that identifies mismatched keys within a mismatched table.

Informs #103072

Release note: None

104828: base: remove OptionalNodeIDErr r=yuzefovich a=yuzefovich

This method was only used in one place where we need to get the SQL instance ID of the gateway. That place has been refactored to pass that ID explicitly from the DistSQLPlanner.

Addresses: #100826.
Epic: None

Release note: None

104888: kvserver: clarify `kv.raft_log.disable_synchronization_unsafe` r=erikgrinaker a=erikgrinaker

This setting not only disables fsync, it also disables flushing writes to the OS, so it will lose data even on process crashes.

Epic: none
Release note: None

Co-authored-by: Michael Butler <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
4 people committed Jun 14, 2023
4 parents 5741b95 + 3f05393 + 90c22c1 + a25200a commit ae25f4d
Show file tree
Hide file tree
Showing 23 changed files with 542 additions and 73 deletions.
4 changes: 4 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -576,6 +576,7 @@ ALL_TESTS = [
"//pkg/storage:storage_test",
"//pkg/testutils/docker:docker_test",
"//pkg/testutils/echotest:echotest_test",
"//pkg/testutils/fingerprintutils:fingerprintutils_test",
"//pkg/testutils/floatcmp:floatcmp_test",
"//pkg/testutils/keysutils:keysutils_test",
"//pkg/testutils/lint/passes/errcmp:errcmp_test",
Expand Down Expand Up @@ -2087,6 +2088,8 @@ GO_TARGETS = [
"//pkg/testutils/docker:testutils_docker",
"//pkg/testutils/echotest:echotest",
"//pkg/testutils/echotest:echotest_test",
"//pkg/testutils/fingerprintutils:fingerprintutils",
"//pkg/testutils/fingerprintutils:fingerprintutils_test",
"//pkg/testutils/floatcmp:floatcmp",
"//pkg/testutils/floatcmp:floatcmp_test",
"//pkg/testutils/gossiputil:gossiputil",
Expand Down Expand Up @@ -3291,6 +3294,7 @@ GET_X_DATA_TARGETS = [
"//pkg/testutils/docker:get_x_data",
"//pkg/testutils/docker/docker-fsnotify:get_x_data",
"//pkg/testutils/echotest:get_x_data",
"//pkg/testutils/fingerprintutils:get_x_data",
"//pkg/testutils/floatcmp:get_x_data",
"//pkg/testutils/gossiputil:get_x_data",
"//pkg/testutils/grpcutils:get_x_data",
Expand Down
1 change: 0 additions & 1 deletion pkg/base/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ go_library(
"//pkg/settings/cluster",
"//pkg/util",
"//pkg/util/envutil",
"//pkg/util/errorutil",
"//pkg/util/humanizeutil",
"//pkg/util/metric",
"//pkg/util/mon",
Expand Down
10 changes: 0 additions & 10 deletions pkg/base/node_id.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)
Expand Down Expand Up @@ -233,15 +232,6 @@ func (c *SQLIDContainer) OptionalNodeID() (roachpb.NodeID, bool) {
return (*NodeIDContainer)(c).Get(), true
}

// OptionalNodeIDErr is like OptionalNodeID, but returns an error (referring to
// the optionally supplied GitHub issues) if the ID is not present.
func (c *SQLIDContainer) OptionalNodeIDErr(issue int) (roachpb.NodeID, error) {
if (*NodeIDContainer)(c).standaloneSQLInstance {
return 0, errorutil.UnsupportedWithMultiTenancy(issue)
}
return (*NodeIDContainer)(c).Get(), nil
}

// SQLInstanceID returns the wrapped SQLInstanceID.
func (c *SQLIDContainer) SQLInstanceID() SQLInstanceID {
return SQLInstanceID((*NodeIDContainer)(c).Get())
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -289,6 +289,7 @@ go_test(
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/testutils/distsqlutils",
"//pkg/testutils/fingerprintutils",
"//pkg/testutils/jobutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
Expand Down
18 changes: 14 additions & 4 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/fingerprintutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
Expand Down Expand Up @@ -1082,7 +1083,9 @@ SELECT payload FROM "".crdb_internal.system_jobs ORDER BY created DESC LIMIT 10
sqlDB.Exec(t, incBackupQuery, queryArgs...)
}
bankTableID := sqlutils.QueryTableID(t, conn, "data", "public", "bank")
backupTableFingerprint := sqlutils.FingerprintTable(t, sqlDB, bankTableID)
backupTableFingerprint, err := fingerprintutils.FingerprintTable(ctx, conn, bankTableID,
fingerprintutils.Stripped())
require.NoError(t, err)

sqlDB.Exec(t, `DROP DATABASE data CASCADE`)

Expand All @@ -1103,18 +1106,22 @@ SELECT payload FROM "".crdb_internal.system_jobs ORDER BY created DESC LIMIT 10
restoreQuery = fmt.Sprintf("%s WITH kms = %s", restoreQuery, kmsURIFmtString)
}
queryArgs := append(restoreURIArgs, kmsURIArgs...)
verifyRestoreData(t, sqlDB, storageSQLDB, restoreQuery, queryArgs, numAccounts, backupTableFingerprint)
verifyRestoreData(ctx, t, conn, sqlDB, storageSQLDB, restoreQuery, queryArgs, numAccounts,
backupTableFingerprint)
}

func verifyRestoreData(
ctx context.Context,
t *testing.T,
conn *gosql.DB,
sqlDB *sqlutils.SQLRunner,
storageSQLDB *sqlutils.SQLRunner,
restoreQuery string,
restoreURIArgs []interface{},
numAccounts int,
bankStrippedFingerprint int,
bankStrippedFingerprint int64,
) {

var unused string
var restored struct {
rows, idx, bytes int64
Expand Down Expand Up @@ -1165,7 +1172,10 @@ func verifyRestoreData(
}
}
restorebankID := sqlutils.QueryTableID(t, sqlDB.DB, "data", "public", "bank")
require.Equal(t, bankStrippedFingerprint, sqlutils.FingerprintTable(t, sqlDB, restorebankID))
fingerprint, err := fingerprintutils.FingerprintTable(ctx, conn, restorebankID,
fingerprintutils.Stripped())
require.NoError(t, err)
require.Equal(t, bankStrippedFingerprint, fingerprint)
}

func TestBackupRestoreSystemTables(t *testing.T) {
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/backuprand/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ go_test(
"//pkg/sql/randgen",
"//pkg/sql/sem/tree",
"//pkg/testutils",
"//pkg/testutils/fingerprintutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
Expand Down
13 changes: 10 additions & 3 deletions pkg/ccl/backupccl/backuprand/backup_rand_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/fingerprintutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
Expand Down Expand Up @@ -98,13 +99,16 @@ database_name = 'rand' AND schema_name = 'public'`)
}

expectedCreateTableStmt := make(map[string]string)
expectedData := make(map[string]int)
expectedData := make(map[string]int64)
for _, tableName := range tableNames {
expectedCreateTableStmt[tableName] = sqlDB.QueryStr(t,
fmt.Sprintf(`SELECT create_statement FROM [SHOW CREATE TABLE %s]`, tree.NameString(tableName)))[0][0]
if runSchemaOnlyExtension == "" {
var err error
tableID := sqlutils.QueryTableID(t, sqlDB.DB, "rand", "public", tableName)
expectedData[tableName] = sqlutils.FingerprintTable(t, sqlDB, tableID)
expectedData[tableName], err = fingerprintutils.FingerprintTable(ctx, tc.Conns[0], tableID,
fingerprintutils.Stripped())
require.NoError(t, err)
}
}

Expand Down Expand Up @@ -138,7 +142,10 @@ database_name = 'rand' AND schema_name = 'public'`)
"SHOW CREATE %s not equal after RESTORE", tableName)
if runSchemaOnlyExtension == "" {
tableID := sqlutils.QueryTableID(t, sqlDB.DB, "restoredb", "public", tableName)
require.Equal(t, expectedData[tableName], sqlutils.FingerprintTable(t, sqlDB, tableID))
fingerpint, err := fingerprintutils.FingerprintTable(ctx, tc.Conns[0], tableID,
fingerprintutils.Stripped())
require.NoError(t, err)
require.Equal(t, expectedData[tableName], fingerpint)
} else {
sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT count(*) FROM %s`, restoreTable),
[][]string{{"0"}})
Expand Down
10 changes: 6 additions & 4 deletions pkg/ccl/streamingccl/replicationtestutils/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,10 +99,11 @@ type TenantStreamingClusters struct {
SrcURL url.URL
SrcCleanup func()

DestCluster *testcluster.TestCluster
DestSysServer serverutils.TestServerInterface
DestSysSQL *sqlutils.SQLRunner
DestTenantSQL *sqlutils.SQLRunner
DestCluster *testcluster.TestCluster
DestSysServer serverutils.TestServerInterface
DestSysSQL *sqlutils.SQLRunner
DestTenantConn *gosql.DB
DestTenantSQL *sqlutils.SQLRunner
}

// CreateDestTenantSQL creates a dest tenant SQL runner and returns a cleanup
Expand All @@ -112,6 +113,7 @@ type TenantStreamingClusters struct {
func (c *TenantStreamingClusters) CreateDestTenantSQL(ctx context.Context) func() error {
testTenant, destTenantConn := serverutils.StartTenant(c.T, c.DestSysServer,
base.TestTenantArgs{TenantID: c.Args.DestTenantID, DisableCreateTenant: true, SkipTenantCheck: true})
c.DestTenantConn = destTenantConn
c.DestTenantSQL = sqlutils.MakeSQLRunner(destTenantConn)
return func() error {
if err := destTenantConn.Close(); err != nil {
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/streamingccl/replicationutils/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,10 @@ go_library(
"//pkg/roachpb",
"//pkg/sql/isql",
"//pkg/storage",
"//pkg/testutils/fingerprintutils",
"//pkg/testutils/jobutils",
"//pkg/testutils/sqlutils",
"//pkg/util/ctxgroup",
"//pkg/util/hlc",
"//pkg/util/protoutil",
"//pkg/util/timeutil",
Expand Down
64 changes: 64 additions & 0 deletions pkg/ccl/streamingccl/replicationutils/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ package replicationutils

import (
"context"
gosql "database/sql"
"fmt"
"testing"

Expand All @@ -21,8 +22,10 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils/fingerprintutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -201,6 +204,67 @@ func LoadIngestionProgress(
return sp.StreamIngest, nil
}

// InvestigateFingerprints checks that the src and dst cluster data match, table
// by table. It first computes and compares their stripped fingerprints to check
// that all the latest data matches; then it computes and compares their
// revision history fingerprints.
func InvestigateFingerprints(
ctx context.Context, srcConn, dstConn *gosql.DB, startTime,
cutoverTime hlc.Timestamp,
) error {
strippedOpts := []func(*fingerprintutils.FingerprintOption){
fingerprintutils.Stripped(),
fingerprintutils.AOST(cutoverTime),
}
if err := fingerprintClustersByTable(ctx, srcConn, dstConn, strippedOpts...); err != nil {
return fmt.Errorf("failed stripped fingerprint: %w", err)
}

opts := []func(*fingerprintutils.FingerprintOption){
fingerprintutils.RevisionHistory(),
fingerprintutils.StartTime(startTime),
fingerprintutils.AOST(cutoverTime),
}
if err := fingerprintClustersByTable(ctx, srcConn, dstConn, opts...); err != nil {
return fmt.Errorf("failed revision history fingerprint: %w", err)
}
return nil
}

func fingerprintClustersByTable(
ctx context.Context,
srcConn, dstConn *gosql.DB,
optFuncs ...func(*fingerprintutils.FingerprintOption),
) error {
g := ctxgroup.WithContext(ctx)
var (
srcFingerprints, dstFingerprints map[string]map[string]int64
)
g.Go(func() error {
var err error
srcFingerprints, err = fingerprintutils.FingerprintAllDatabases(ctx, srcConn, true,
optFuncs...)
if err != nil {
return fmt.Errorf("failed getting src fingerprint: %w", err)
}
return nil
})
g.Go(func() error {
var err error
dstFingerprints, err = fingerprintutils.FingerprintAllDatabases(ctx, dstConn, true,
optFuncs...)
if err != nil {
return fmt.Errorf("failed getting dst fingerprint: %w", err)
}
return nil
})
if err := g.Wait(); err != nil {
return err
}
return fingerprintutils.CompareMultipleDatabaseFingerprints(srcFingerprints,
dstFingerprints)
}

func GetStreamIngestionStats(
ctx context.Context,
streamIngestionDetails jobspb.StreamIngestionDetails,
Expand Down
28 changes: 24 additions & 4 deletions pkg/ccl/streamingccl/streamingest/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/replicationtestutils"
"github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/replicationutils"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
Expand Down Expand Up @@ -55,11 +56,14 @@ import (
// - compare-replication-results: runs the specified SQL query on both the
// "source" and "destination" tenants and asserts that the results are equal
//
// - compare-tenant-fingerprints from=<start-time> to=<end-time> [with_revisions]
// - compare-tenant-fingerprints from=<start-time> to=<end-time> [with_revisions,table_fingerprints]
// Runs `crdb_internal.fingerprint` on both the "source" and "destination"
// tenants with the provided options and asserts that the generated fingerprints
// are equal.
//
// - the table_fingerprints option conducts another round of fingerprinting over each table in the
// clusters. (Primarily used to test fingerprint helper functions).
//
// - sleep ms=TIME
// Sleep for TIME milliseconds.
//
Expand Down Expand Up @@ -126,9 +130,7 @@ func TestDataDriven(t *testing.T) {
if ok {
replicatedTimeTarget = varValue
}
timestamp, _, err := tree.ParseDTimestamp(nil, replicatedTimeTarget, time.Microsecond)
require.NoError(t, err)
ds.replicationClusters.WaitUntilReplicatedTime(hlc.Timestamp{WallTime: timestamp.UnixNano()},
ds.replicationClusters.WaitUntilReplicatedTime(stringToHLC(t, replicatedTimeTarget),
jobspb.JobID(ds.replicationJobID))
case "start-replicated-tenant":
cleanupTenant := ds.replicationClusters.CreateDestTenantSQL(ctx)
Expand Down Expand Up @@ -213,8 +215,20 @@ func TestDataDriven(t *testing.T) {
ds.replicationClusters.DestSysSQL.QueryRow(t, fmt.Sprintf(fingerprintQuery,
ds.replicationClusters.Args.DestTenantName, from, allRevisions, to)).Scan(&fingerprintDestTenant)
require.NotZero(t, fingerprintDestTenant)
if fingerprintSrcTenant != fingerprintDestTenant {
require.NoError(t, replicationutils.InvestigateFingerprints(ctx,
ds.replicationClusters.SrcTenantConn,
ds.replicationClusters.DestTenantConn, stringToHLC(t, from), stringToHLC(t, to)))
t.Fatalf("tenant level fingerpint mismatch, but table level fingerprints match")
}
require.Equal(t, fingerprintSrcTenant, fingerprintDestTenant)

if d.HasArg("table_fingerprints") {
require.NoError(t, replicationutils.InvestigateFingerprints(ctx,
ds.replicationClusters.SrcTenantConn,
ds.replicationClusters.DestTenantConn, stringToHLC(t, from), stringToHLC(t, to)))
}

case "sleep":
var msStr string
if d.HasArg("ms") {
Expand Down Expand Up @@ -278,6 +292,12 @@ func TestDataDriven(t *testing.T) {
})
}

func stringToHLC(t *testing.T, timestamp string) hlc.Timestamp {
parsedTimestamp, _, err := tree.ParseDTimestamp(nil, timestamp, time.Microsecond)
require.NoError(t, err)
return hlc.Timestamp{WallTime: parsedTimestamp.UnixNano()}
}

type datadrivenTestState struct {
producerJobID, replicationJobID int
replicationClusters *replicationtestutils.TenantStreamingClusters
Expand Down
12 changes: 5 additions & 7 deletions pkg/ccl/streamingccl/streamingest/stream_ingestion_dist.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,9 @@ func startDistIngestion(
client,
replicatedTime,
streamProgress.Checkpoint,
initialScanTimestamp)(ctx, dsp)
initialScanTimestamp,
dsp.GatewayID(),
)(ctx, dsp)
if err != nil {
return err
}
Expand Down Expand Up @@ -128,6 +130,7 @@ func makePlan(
previousReplicatedTime hlc.Timestamp,
checkpoint jobspb.StreamIngestionCheckpoint,
initialScanTimestamp hlc.Timestamp,
gatewayID base.SQLInstanceID,
) func(context.Context, *sql.DistSQLPlanner) (*sql.PhysicalPlan, *sql.PlanningCtx, error) {
return func(ctx context.Context, dsp *sql.DistSQLPlanner) (*sql.PhysicalPlan, *sql.PlanningCtx, error) {
jobID := ingestionJob.ID()
Expand Down Expand Up @@ -185,14 +188,9 @@ func makePlan(
execinfrapb.Ordering{},
)

gatewayNodeID, err := execCtx.ExecCfg().NodeInfo.NodeID.OptionalNodeIDErr(48274)
if err != nil {
return nil, nil, err
}

// The ResultRouters from the previous stage will feed in to the
// StreamIngestionFrontier processor.
p.AddSingleGroupStage(ctx, base.SQLInstanceID(gatewayNodeID),
p.AddSingleGroupStage(ctx, gatewayID,
execinfrapb.ProcessorCoreUnion{StreamIngestionFrontier: streamIngestionFrontierSpec},
execinfrapb.PostProcessSpec{}, streamIngestionResultTypes)

Expand Down
Loading

0 comments on commit ae25f4d

Please sign in to comment.