forked from cockroachdb/cockroach
-
Notifications
You must be signed in to change notification settings - Fork 0
/
create_logical_replication_stmt.go
495 lines (444 loc) · 15.3 KB
/
create_logical_replication_stmt.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
// Copyright 2024 The Cockroach Authors.
//
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.
package logical
import (
"context"
"fmt"
"strings"
"github.com/cockroachdb/cockroach/pkg/ccl/crosscluster"
"github.com/cockroachdb/cockroach/pkg/ccl/crosscluster/streamclient"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/repstream/streampb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/exprutil"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/asof"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/issuelink"
)
func init() {
sql.AddPlanHook("create logical replication stream", createLogicalReplicationStreamPlanHook, createLogicalReplicationStreamTypeCheck)
}
var streamCreationHeader = colinfo.ResultColumns{
{Name: "job_id", Typ: types.Int},
}
func createLogicalReplicationStreamPlanHook(
ctx context.Context, untypedStmt tree.Statement, p sql.PlanHookState,
) (sql.PlanHookRowFn, colinfo.ResultColumns, []sql.PlanNode, bool, error) {
stmt, ok := untypedStmt.(*tree.CreateLogicalReplicationStream)
if !ok {
return nil, nil, nil, false, nil
}
exprEval := p.ExprEvaluator("LOGICAL REPLICATION STREAM")
from, err := exprEval.String(ctx, stmt.PGURL)
if err != nil {
return nil, nil, nil, false, err
}
fn := func(ctx context.Context, _ []sql.PlanNode, resultsCh chan<- tree.Datums) (err error) {
defer func() {
if err == nil {
telemetry.Count("logical_replication_stream.started")
}
}()
ctx, span := tracing.ChildSpan(ctx, stmt.StatementTag())
defer span.Finish()
if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.V24_2) {
return pgerror.New(pgcode.FeatureNotSupported,
"replication job not supported before V24.2")
}
if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings,
"CREATE LOGICAL REPLICATION",
); err != nil {
return err
}
// TODO(dt): the global priv is a big hammer; should we be checking just on
// table(s) or database being replicated from and into?
if err := p.CheckPrivilege(
ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.REPLICATION,
); err != nil {
return err
}
if stmt.From.Database != "" {
return errors.UnimplementedErrorf(issuelink.IssueLink{}, "logical replication streams on databases are unsupported")
}
if len(stmt.From.Tables) != len(stmt.Into.Tables) {
return pgerror.New(pgcode.InvalidParameterValue, "the same number of source and destination tables must be specified")
}
options, err := evalLogicalReplicationOptions(ctx, stmt.Options, exprEval, p)
if err != nil {
return err
}
hasUDF := len(options.userFunctions) > 0 || options.defaultFunction != nil && options.defaultFunction.FunctionId != 0
mode := jobspb.LogicalReplicationDetails_Immediate
if m, ok := options.GetMode(); ok {
switch m {
case "immediate":
if hasUDF {
return pgerror.Newf(pgcode.InvalidParameterValue, "MODE = 'immediate' cannot be used with user-defined functions")
}
case "validated":
mode = jobspb.LogicalReplicationDetails_Validated
default:
return pgerror.Newf(pgcode.InvalidParameterValue, "unknown mode %q", m)
}
} else if hasUDF {
// UDFs imply applying changes via SQL, which implies validation.
mode = jobspb.LogicalReplicationDetails_Validated
}
var (
targetsDescription string
srcTableNames = make([]string, len(stmt.From.Tables))
repPairs = make([]jobspb.LogicalReplicationDetails_ReplicationPair, len(stmt.Into.Tables))
srcTableDescs = make([]*descpb.TableDescriptor, len(stmt.Into.Tables))
dstTableDescs = make([]*tabledesc.Mutable, len(stmt.Into.Tables))
)
for i := range stmt.From.Tables {
dstObjName, err := stmt.Into.Tables[i].ToUnresolvedObjectName(tree.NoAnnotation)
if err != nil {
return err
}
dstTableName := dstObjName.ToTableName()
prefix, td, err := resolver.ResolveMutableExistingTableObject(ctx, p, &dstTableName, true, tree.ResolveRequireTableDesc)
if err != nil {
return err
}
repPairs[i].DstDescriptorID = int32(td.GetID())
dstTableDescs[i] = td
// TODO(dt): remove when we support this via KV metadata.
var foundTSCol bool
for _, col := range td.GetColumns() {
if col.Name == originTimestampColumnName {
foundTSCol = true
if col.Type.Family() != types.DecimalFamily {
return errors.Newf(
"%s column must be type DECIMAL for use by logical replication", originTimestampColumnName,
)
}
break
}
}
if !foundTSCol {
return errors.WithHintf(errors.Newf(
"tables written to by logical replication currently require a %q DECIMAL column",
originTimestampColumnName,
), "try 'ALTER TABLE %s ADD COLUMN %s DECIMAL NOT VISIBLE DEFAULT NULL ON UPDATE NULL'",
dstObjName.String(), originTimestampColumnName,
)
}
tbNameWithSchema := tree.MakeTableNameWithSchema(
tree.Name(prefix.Database.GetName()),
tree.Name(prefix.Schema.GetName()),
tree.Name(td.GetName()),
)
srcTableNames[i] = stmt.From.Tables[i].String()
if i == 0 {
targetsDescription = tbNameWithSchema.FQString()
} else {
targetsDescription += ", " + tbNameWithSchema.FQString()
}
if mode != jobspb.LogicalReplicationDetails_Validated {
fks := td.OutboundForeignKeys()
for _, fk := range append(fks[:len(fks):len(fks)], td.InboundForeignKeys()...) {
// TODO(dt): move the constraint to un-validated for them.
if fk.IsConstraintValidated() {
return pgerror.Newf(pgcode.InvalidParameterValue, "only 'NOT VALID' foreign keys are only supported with MODE = 'validated'")
}
}
}
}
streamAddress := crosscluster.StreamAddress(from)
streamURL, err := streamAddress.URL()
if err != nil {
return err
}
streamAddress = crosscluster.StreamAddress(streamURL.String())
cleanedURI, err := cloud.SanitizeExternalStorageURI(from, nil)
if err != nil {
return err
}
client, err := streamclient.NewStreamClient(ctx, streamAddress, p.ExecCfg().InternalDB, streamclient.WithLogical())
if err != nil {
return err
}
defer func() {
_ = client.Close(ctx)
}()
if err := client.Dial(ctx); err != nil {
return err
}
spec, err := client.CreateForTables(ctx, &streampb.ReplicationProducerRequest{
TableNames: srcTableNames,
})
if err != nil {
return err
}
for i, name := range srcTableNames {
td := spec.TableDescriptors[name]
srcTableDescs[i] = &td
repPairs[i].SrcDescriptorID = int32(td.ID)
}
replicationStartTime := spec.ReplicationStartTime
progress := jobspb.LogicalReplicationProgress{}
if cursor, ok := options.GetCursor(); ok {
replicationStartTime = cursor
progress.ReplicatedTime = cursor
}
if uf, ok := options.GetUserFunctions(); ok {
for i, name := range srcTableNames {
repPairs[i].DstFunctionID = uf[name]
}
}
// Default conflict resolution if not set will be LWW
defaultConflictResolution := jobspb.LogicalReplicationDetails_DefaultConflictResolution{
ConflictResolutionType: jobspb.LogicalReplicationDetails_DefaultConflictResolution_LWW,
}
if cr, ok := options.GetDefaultFunction(); ok {
defaultConflictResolution = *cr
}
if buildutil.CrdbTestBuild {
if len(srcTableDescs) != len(dstTableDescs) {
panic("srcTableDescs and dstTableDescs should have the same length")
}
}
for i := range srcTableDescs {
err := tabledesc.CheckLogicalReplicationCompatibility(srcTableDescs[i], dstTableDescs[i].TableDesc(), options.SkipSchemaCheck())
if err != nil {
return err
}
}
jr := jobs.Record{
JobID: p.ExecCfg().JobRegistry.MakeJobID(),
Description: fmt.Sprintf("LOGICAL REPLICATION STREAM into %s from %s", targetsDescription, cleanedURI),
Username: p.User(),
Details: jobspb.LogicalReplicationDetails{
StreamID: uint64(spec.StreamID),
SourceClusterID: spec.SourceClusterID,
ReplicationStartTime: replicationStartTime,
SourceClusterConnStr: string(streamAddress),
ReplicationPairs: repPairs,
TableNames: srcTableNames,
DefaultConflictResolution: defaultConflictResolution,
IgnoreCDCIgnoredTTLDeletes: options.IgnoreCDCIgnoredTTLDeletes(),
Mode: mode,
MetricsLabel: options.metricsLabel,
},
Progress: progress,
}
if _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn(ctx, jr, jr.JobID, p.InternalSQLTxn()); err != nil {
return err
}
// Add the LDR job ID to the destination table descriptors.
b := p.InternalSQLTxn().KV().NewBatch()
for _, td := range dstTableDescs {
td.LDRJobIDs = append(td.LDRJobIDs, jr.JobID)
if err := p.InternalSQLTxn().Descriptors().WriteDescToBatch(ctx, true /* kvTrace */, td, b); err != nil {
return err
}
}
if err := p.InternalSQLTxn().KV().Run(ctx, b); err != nil {
return err
}
resultsCh <- tree.Datums{tree.NewDInt(tree.DInt(jr.JobID))}
return nil
}
return fn, streamCreationHeader, nil, false, nil
}
func createLogicalReplicationStreamTypeCheck(
ctx context.Context, untypedStmt tree.Statement, p sql.PlanHookState,
) (matched bool, header colinfo.ResultColumns, _ error) {
stmt, ok := untypedStmt.(*tree.CreateLogicalReplicationStream)
if !ok {
return false, nil, nil
}
toTypeCheck := []exprutil.ToTypeCheck{
exprutil.Strings{stmt.PGURL},
exprutil.Strings{
stmt.Options.Cursor,
stmt.Options.DefaultFunction,
stmt.Options.Mode,
stmt.Options.MetricsLabel,
},
exprutil.Bools{
stmt.Options.IgnoreCDCIgnoredTTLDeletes,
stmt.Options.SkipSchemaCheck,
},
}
if err := exprutil.TypeCheck(ctx, "LOGICAL REPLICATION STREAM", p.SemaCtx(),
toTypeCheck...,
); err != nil {
return false, nil, err
}
return true, streamCreationHeader, nil
}
type resolvedLogicalReplicationOptions struct {
cursor hlc.Timestamp
mode string
defaultFunction *jobspb.LogicalReplicationDetails_DefaultConflictResolution
// Mapping of table name to function descriptor
userFunctions map[string]int32
ignoreCDCIgnoredTTLDeletes bool
skipSchemaCheck bool
metricsLabel string
}
func evalLogicalReplicationOptions(
ctx context.Context,
options tree.LogicalReplicationOptions,
eval exprutil.Evaluator,
p sql.PlanHookState,
) (*resolvedLogicalReplicationOptions, error) {
r := &resolvedLogicalReplicationOptions{}
if options.Mode != nil {
mode, err := eval.String(ctx, options.Mode)
if err != nil {
return nil, err
}
r.mode = mode
}
if options.MetricsLabel != nil {
metricsLabel, err := eval.String(ctx, options.MetricsLabel)
if err != nil {
return nil, err
}
r.metricsLabel = metricsLabel
}
if options.Cursor != nil {
cursor, err := eval.String(ctx, options.Cursor)
if err != nil {
return nil, err
}
asOfClause := tree.AsOfClause{Expr: tree.NewStrVal(cursor)}
asOf, err := asof.Eval(ctx, asOfClause, p.SemaCtx(), &p.ExtendedEvalContext().Context)
if err != nil {
return nil, err
}
r.cursor = asOf.Timestamp
}
if options.DefaultFunction != nil {
defaultResolution := &jobspb.LogicalReplicationDetails_DefaultConflictResolution{}
defaultFnc, err := eval.String(ctx, options.DefaultFunction)
if err != nil {
return nil, err
}
switch strings.ToLower(defaultFnc) {
case "lww":
defaultResolution.ConflictResolutionType = jobspb.LogicalReplicationDetails_DefaultConflictResolution_LWW
case "dlq":
defaultResolution.ConflictResolutionType = jobspb.LogicalReplicationDetails_DefaultConflictResolution_DLQ
// This case will assume that a function name was passed in
// and we will try to resolve it.
default:
urn, err := parser.ParseFunctionName(defaultFnc)
if err != nil {
return nil, err
}
un := urn.ToUnresolvedName()
descID, err := lookupFunctionID(ctx, p, *un)
if err != nil {
return nil, err
}
defaultResolution.ConflictResolutionType = jobspb.LogicalReplicationDetails_DefaultConflictResolution_UDF
defaultResolution.FunctionId = descID
}
r.defaultFunction = defaultResolution
}
if options.UserFunctions != nil {
r.userFunctions = make(map[string]int32)
for tb, fnc := range options.UserFunctions {
objName, err := tb.ToUnresolvedObjectName(tree.NoAnnotation)
if err != nil {
return nil, err
}
un := fnc.ToUnresolvedObjectName().ToUnresolvedName()
descID, err := lookupFunctionID(ctx, p, *un)
if err != nil {
return nil, err
}
r.userFunctions[objName.String()] = descID
}
}
if options.IgnoreCDCIgnoredTTLDeletes == tree.DBoolTrue {
r.ignoreCDCIgnoredTTLDeletes = true
}
if options.SkipSchemaCheck == tree.DBoolTrue {
r.skipSchemaCheck = true
}
return r, nil
}
func lookupFunctionID(
ctx context.Context, p sql.PlanHookState, u tree.UnresolvedName,
) (int32, error) {
rf, err := p.ResolveFunction(ctx, tree.MakeUnresolvedFunctionName(&u), &p.SessionData().SearchPath)
if err != nil {
return 0, err
}
if len(rf.Overloads) > 1 {
return 0, errors.Newf("function %q has more than 1 overload", u.String())
}
fnOID := rf.Overloads[0].Oid
descID := typedesc.UserDefinedTypeOIDToID(fnOID)
if descID == 0 {
return 0, errors.Newf("function %q is not a user defined type", u.String())
}
return int32(descID), nil
}
func (r *resolvedLogicalReplicationOptions) GetCursor() (hlc.Timestamp, bool) {
if r == nil || r.cursor.IsEmpty() {
return hlc.Timestamp{}, false
}
return r.cursor, true
}
func (r *resolvedLogicalReplicationOptions) GetMode() (string, bool) {
if r == nil || r.mode == "" {
return "", false
}
return r.mode, true
}
func (r *resolvedLogicalReplicationOptions) GetDefaultFunction() (
*jobspb.LogicalReplicationDetails_DefaultConflictResolution,
bool,
) {
if r == nil || r.defaultFunction == nil {
return &jobspb.LogicalReplicationDetails_DefaultConflictResolution{}, false
}
return r.defaultFunction, true
}
func (r *resolvedLogicalReplicationOptions) GetUserFunctions() (map[string]int32, bool) {
if r == nil || r.userFunctions == nil {
return map[string]int32{}, false
}
return r.userFunctions, true
}
func (r *resolvedLogicalReplicationOptions) IgnoreCDCIgnoredTTLDeletes() bool {
if r == nil {
return false
}
return r.ignoreCDCIgnoredTTLDeletes
}
func (r *resolvedLogicalReplicationOptions) SkipSchemaCheck() bool {
if r == nil {
return false
}
return r.skipSchemaCheck
}