Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
104820: backupccl: adjust a test to run for secondary tenant codec too r=yuzefovich a=yuzefovich

Fixes: #82882.

Release note: None

104823: row: remove leftover reference to TestingSQLCodec in a test r=yuzefovich a=yuzefovich

This was missed in #82890.

Addresses: #48123.
Epic: None

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

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
  • Loading branch information
craig[bot] and yuzefovich committed Jun 14, 2023
4 parents 8e3bf5b + a6e57dd + ef1ba50 + 90c22c1 commit 17fe468
Show file tree
Hide file tree
Showing 7 changed files with 47 additions and 48 deletions.
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: 0 additions & 1 deletion pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -290,7 +290,6 @@ go_test(
"//pkg/testutils/datapathutils",
"//pkg/testutils/distsqlutils",
"//pkg/testutils/jobutils",
"//pkg/testutils/keysutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
Expand Down
67 changes: 41 additions & 26 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
"github.com/cockroachdb/cockroach/pkg/testutils/keysutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
Expand Down Expand Up @@ -6177,11 +6176,7 @@ func getMockTableDesc(
// methods.
func TestPublicIndexTableSpans(t *testing.T) {
defer leaktest.AfterTest(t)()
codec := keysutils.TestingSQLCodec
execCfg := &sql.ExecutorConfig{
Codec: codec,
}
unusedMap := make(map[tableAndIndex]bool)

testCases := []struct {
name string
tableID descpb.ID
Expand Down Expand Up @@ -6268,29 +6263,49 @@ func TestPublicIndexTableSpans(t *testing.T) {
},
}

for _, test := range testCases {
tableDesc := getMockTableDesc(test.tableID, test.pkIndex,
test.indexes, test.addingIndexes, test.droppingIndexes)
t.Run(fmt.Sprintf("%s:%s", "forEachPublicIndexTableSpan", test.name), func(t *testing.T) {
var spans []roachpb.Span
forEachPublicIndexTableSpan(tableDesc.TableDesc(), unusedMap, codec, func(sp roachpb.Span) {
spans = append(spans, sp)
})
var unmergedSpans []string
for _, span := range spans {
unmergedSpans = append(unmergedSpans, span.String())
for _, useSecondaryTenant := range []bool{false, true} {
name, codec := "system", keys.SystemSQLCodec
if useSecondaryTenant {
const tenantID = 42
name, codec = "secondary", keys.MakeSQLCodec(roachpb.MustMakeTenantID(tenantID))
for _, tc := range testCases {
for i, sp := range tc.expectedSpans {
tc.expectedSpans[i] = fmt.Sprintf("/Tenant/%d%s", tenantID, sp)
}
for i, sp := range tc.expectedMergedSpans {
tc.expectedMergedSpans[i] = fmt.Sprintf("/Tenant/%d%s", tenantID, sp)
}
}
require.Equal(t, test.expectedSpans, unmergedSpans)
})
}
execCfg := &sql.ExecutorConfig{Codec: codec}
unusedMap := make(map[tableAndIndex]bool)

t.Run(fmt.Sprintf("%s:%s", "spansForAllTableIndexes", test.name), func(t *testing.T) {
mergedSpans, err := spansForAllTableIndexes(execCfg, []catalog.TableDescriptor{tableDesc}, nil /* revs */)
require.NoError(t, err)
var mergedSpanStrings []string
for _, mSpan := range mergedSpans {
mergedSpanStrings = append(mergedSpanStrings, mSpan.String())
t.Run(name, func(t *testing.T) {
for _, test := range testCases {
tableDesc := getMockTableDesc(test.tableID, test.pkIndex,
test.indexes, test.addingIndexes, test.droppingIndexes)
t.Run(fmt.Sprintf("%s:%s", "forEachPublicIndexTableSpan", test.name), func(t *testing.T) {
var spans []roachpb.Span
forEachPublicIndexTableSpan(tableDesc.TableDesc(), unusedMap, codec, func(sp roachpb.Span) {
spans = append(spans, sp)
})
var unmergedSpans []string
for _, span := range spans {
unmergedSpans = append(unmergedSpans, span.String())
}
require.Equal(t, test.expectedSpans, unmergedSpans)
})

t.Run(fmt.Sprintf("%s:%s", "spansForAllTableIndexes", test.name), func(t *testing.T) {
mergedSpans, err := spansForAllTableIndexes(execCfg, []catalog.TableDescriptor{tableDesc}, nil /* revs */)
require.NoError(t, err)
var mergedSpanStrings []string
for _, mSpan := range mergedSpans {
mergedSpanStrings = append(mergedSpanStrings, mSpan.String())
}
require.Equal(t, test.expectedMergedSpans, mergedSpanStrings)
})
}
require.Equal(t, test.expectedMergedSpans, mergedSpanStrings)
})
}
}
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
1 change: 0 additions & 1 deletion pkg/sql/row/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,6 @@ go_test(
"//pkg/sql/sem/tree",
"//pkg/storage",
"//pkg/testutils",
"//pkg/testutils/keysutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/util/encoding",
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/row/expr_walker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/testutils/keysutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -202,7 +201,7 @@ func TestJobBackedSeqChunkProvider(t *testing.T) {
}

for id, val := range test.seqIDToExpectedVal {
seqDesc := createAndIncrementSeqDescriptor(ctx, t, id, keysutils.TestingSQLCodec,
seqDesc := createAndIncrementSeqDescriptor(ctx, t, id, evalCtx.Codec,
test.incrementBy, test.seqIDToOpts[id], kvDB)
seqMetadata := &row.SequenceMetadata{
SeqDesc: seqDesc,
Expand Down

0 comments on commit 17fe468

Please sign in to comment.