-
Notifications
You must be signed in to change notification settings - Fork 15
/
transactor.go
394 lines (327 loc) · 11.2 KB
/
transactor.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
package connector
import (
"context"
"encoding/json"
"fmt"
"strings"
"text/template"
"cloud.google.com/go/bigquery"
"github.com/estuary/connectors/go/dbt"
m "github.com/estuary/connectors/go/protocols/materialize"
boilerplate "github.com/estuary/connectors/materialize-boilerplate"
sql "github.com/estuary/connectors/materialize-sql"
pf "github.com/estuary/flow/go/protocols/flow"
pm "github.com/estuary/flow/go/protocols/materialize"
log "github.com/sirupsen/logrus"
"go.gazette.dev/core/consumer/protocol"
"google.golang.org/api/iterator"
)
type transactor struct {
fence *sql.Fence
cfg *config
client *client
bucketPath string
bucket string
bindings []*binding
be *boilerplate.BindingEvents
}
func newTransactor(
ctx context.Context,
ep *sql.Endpoint,
fence sql.Fence,
bindings []sql.Table,
open pm.Request_Open,
is *boilerplate.InfoSchema,
be *boilerplate.BindingEvents,
) (_ m.Transactor, _ *boilerplate.MaterializeOptions, err error) {
cfg := ep.Config.(*config)
client, err := cfg.client(ctx)
if err != nil {
return nil, nil, err
}
t := &transactor{
cfg: cfg,
fence: &fence,
client: client,
bucketPath: cfg.BucketPath,
bucket: cfg.Bucket,
be: be,
}
for _, binding := range bindings {
// The name of the table itself is always the last element of the path.
table := binding.TableShape.Path[len(binding.TableShape.Path)-1]
// The dataset for the table is always the second to last element of the path. Dataset names
// can only contain letters, numbers, and underscores, so translateFlowIdentifier is not
// needed when using this part of the path directly.
dataset := binding.TableShape.Path[len(binding.TableShape.Path)-2]
// Lookup metadata for the table to build the schema for the external file that will be used
// for loading data. Schema definitions from the actual table columns are queried instead of
// directly using the dialect's output for JSON schema type to provide some degree in
// flexibility in changing the dialect and having it still work for existing tables. As long
// as the JSON encoding of the values is the same they may be used for columns that would
// have been created differently due to evolution of the dialect's column types.
meta, err := client.bigqueryClient.DatasetInProject(cfg.ProjectID, dataset).Table(translateFlowIdentifier(table)).Metadata(ctx)
if err != nil {
return nil, nil, fmt.Errorf("getting table metadata: %w", err)
}
log.WithFields(log.Fields{
"table": table,
"collection": binding.Source.String(),
"schemaJson": meta.Schema,
}).Debug("bigquery schema for table")
fieldSchemas := make(map[string]*bigquery.FieldSchema)
for _, f := range meta.Schema {
fieldSchemas[f.Name] = f
}
if err = t.addBinding(binding, fieldSchemas); err != nil {
return nil, nil, fmt.Errorf("addBinding of %s: %w", binding.Path, err)
}
}
opts := &boilerplate.MaterializeOptions{
ExtendedLogging: true,
AckSchedule: &boilerplate.AckScheduleOption{
Config: cfg.Schedule,
Jitter: []byte(cfg.ProjectID + cfg.Dataset),
},
}
return t, opts, nil
}
func (t *transactor) addBinding(target sql.Table, fieldSchemas map[string]*bigquery.FieldSchema) error {
loadSchema, err := schemaForCols(target.KeyPtrs(), fieldSchemas)
if err != nil {
return err
}
storeSchema, err := schemaForCols(target.Columns(), fieldSchemas)
if err != nil {
return err
}
b := &binding{
target: target,
loadFile: newStagedFile(t.client.cloudStorageClient, t.bucket, t.bucketPath, loadSchema),
storeFile: newStagedFile(t.client.cloudStorageClient, t.bucket, t.bucketPath, storeSchema),
}
for _, m := range []struct {
sql *string
tpl *template.Template
}{
{&b.tempTableName, tplTempTableName},
{&b.loadQuerySQL, tplLoadQuery},
{&b.storeInsertSQL, tplStoreInsert},
{&b.storeUpdateSQL, tplStoreUpdate},
} {
var err error
if *m.sql, err = sql.RenderTableTemplate(target, m.tpl); err != nil {
return err
}
}
t.bindings = append(t.bindings, b)
return nil
}
func schemaForCols(cols []*sql.Column, fieldSchemas map[string]*bigquery.FieldSchema) ([]*bigquery.FieldSchema, error) {
s := make([]*bigquery.FieldSchema, 0, len(cols))
for idx, col := range cols {
schema, ok := fieldSchemas[translateFlowIdentifier(col.Field)]
if !ok {
return nil, fmt.Errorf("could not find metadata for field '%s'", col.Field)
}
// Use a placeholder value instead of the actual field name for the external table schema.
// This allows for materialized tables to use "Flexible column names", which is not yet
// supported by external tables. A similar placeholder is used in the generated SQL queries
// to match this.
schema.Name = fmt.Sprintf("c%d", idx)
s = append(s, schema)
}
return s, nil
}
func (t *transactor) UnmarshalState(state json.RawMessage) error { return nil }
func (t *transactor) Acknowledge(ctx context.Context) (*pf.ConnectorState, error) { return nil, nil }
func (t *transactor) Load(it *m.LoadIterator, loaded func(int, json.RawMessage) error) error {
var ctx = it.Context()
for it.Next() {
var b = t.bindings[it.Binding]
b.loadFile.start()
converted, err := b.target.ConvertKey(it.Key)
if err != nil {
return fmt.Errorf("converting load key: %w", err)
}
if err = b.loadFile.encodeRow(ctx, converted); err != nil {
return fmt.Errorf("writing normalized key to keyfile: %w", err)
}
}
if it.Err() != nil {
return it.Err()
}
// Build the queries of all documents across all bindings that were requested.
var subqueries []string
// This is the map of external table references we will populate.
var edcTableDefs = make(map[string]bigquery.ExternalData)
for idx, b := range t.bindings {
if !b.loadFile.started {
// No loads for this binding.
continue
}
subqueries = append(subqueries, b.loadQuerySQL)
delete, err := b.loadFile.flush()
if err != nil {
return fmt.Errorf("flushing load file for binding[%d]: %w", idx, err)
}
defer delete(ctx)
edcTableDefs[b.tempTableName] = b.loadFile.edc()
}
if len(subqueries) == 0 {
return nil // Nothing to load.
}
// Build the query across all tables.
queryStr := strings.Join(subqueries, "\nUNION ALL\n") + ";"
query := t.client.newQuery(queryStr)
query.TableDefinitions = edcTableDefs // Tell bigquery where to get the external references in gcs.
ll := log.WithField("query", queryStr)
t.be.StartedEvaluatingLoads()
job, err := t.client.runQuery(ctx, query)
if err != nil {
ll.WithError(err).Error("client runQuery failed")
return fmt.Errorf("load query: %w", err)
}
bqit, err := job.Read(ctx)
if err != nil {
ll.WithError(err).Error("job read failed")
return fmt.Errorf("load job read: %w", err)
}
t.be.FinishedEvaluatingLoads()
for {
var bd bindingDocument
if err = bqit.Next(&bd); err == iterator.Done {
break
} else if err != nil {
ll.WithError(err).Error("query results iterator failed")
return fmt.Errorf("load row read: %w", err)
}
if err = loaded(bd.Binding, bd.Document); err != nil {
return fmt.Errorf("load row loaded: %w", err)
}
}
return nil
}
func (t *transactor) Store(it *m.StoreIterator) (m.StartCommitFunc, error) {
var ctx = it.Context()
cleanupFiles := []func(context.Context){}
var lastBinding = -1
for it.Next() {
if lastBinding == -1 {
lastBinding = it.Binding
}
if lastBinding != it.Binding {
// Flush the staged file(s) for the binding now that it's stores are
// fully processed.
var b = t.bindings[lastBinding]
// There may be no staged file if the binding has received nothing
// but hard deletion requests for keys that aren't in the
// destination table.
if b.storeFile.started {
cleanupFn, err := b.storeFile.flush()
if err != nil {
return nil, fmt.Errorf("flushing staged files for collection %q: %w", b.target.Source.String(), err)
}
cleanupFiles = append(cleanupFiles, cleanupFn)
}
lastBinding = it.Binding
}
var b = t.bindings[it.Binding]
var flowDocument = it.RawJSON
if t.cfg.HardDelete && it.Delete {
if it.Exists {
flowDocument = json.RawMessage(`"delete"`)
} else {
// Ignore items which do not exist and are already deleted
continue
}
}
b.storeFile.start()
b.hasData = true
if converted, err := b.target.ConvertAll(it.Key, it.Values, flowDocument); err != nil {
return nil, fmt.Errorf("converting store parameters: %w", err)
} else if err = b.storeFile.encodeRow(ctx, converted); err != nil {
return nil, fmt.Errorf("encoding Store to scratch file: %w", err)
}
}
if it.Err() != nil {
return nil, it.Err()
}
// Flush the final binding.
if lastBinding != -1 {
var b = t.bindings[lastBinding]
cleanupFn, err := b.storeFile.flush()
if err != nil {
return nil, fmt.Errorf("final binding flushing staged files for collection %q: %w", b.target.Source.String(), err)
}
cleanupFiles = append(cleanupFiles, cleanupFn)
}
return func(ctx context.Context, runtimeCheckpoint *protocol.Checkpoint) (*pf.ConnectorState, m.OpFuture) {
var err error
if t.fence.Checkpoint, err = runtimeCheckpoint.Marshal(); err != nil {
return nil, m.FinishedOperation(fmt.Errorf("marshalling checkpoint: %w", err))
}
return nil, pf.RunAsyncOperation(func() error { return t.commit(ctx, cleanupFiles) })
}, nil
}
func (t *transactor) commit(ctx context.Context, cleanupFiles []func(context.Context)) error {
defer func() {
for _, f := range cleanupFiles {
f(ctx)
}
}()
// Build the slice of transactions required for a commit.
var subqueries []string
subqueries = append(subqueries, `
BEGIN
BEGIN TRANSACTION;
`)
// First we must validate the fence has not been modified.
var fenceUpdate strings.Builder
if err := tplUpdateFence.Execute(&fenceUpdate, t.fence); err != nil {
return fmt.Errorf("evaluating fence template: %w", err)
}
subqueries = append(subqueries, fenceUpdate.String())
// This is the map of external table references we will populate. Loop through the bindings and
// append the SQL for that table.
var edcTableDefs = make(map[string]bigquery.ExternalData)
for _, b := range t.bindings {
if !b.hasData {
// No stores for this binding.
continue
}
edcTableDefs[b.tempTableName] = b.storeFile.edc()
if b.target.DeltaUpdates {
subqueries = append(subqueries, b.storeInsertSQL)
} else {
subqueries = append(subqueries, b.storeUpdateSQL)
}
// Reset for the next round.
b.hasData = false
}
// Complete the transaction and return the appropriate error.
subqueries = append(subqueries, `
COMMIT TRANSACTION;
END;
`)
// Build the bigquery query of the combined subqueries.
queryString := strings.Join(subqueries, "\n")
query := t.client.newQuery(queryString)
query.TableDefinitions = edcTableDefs // Tell the query where to get the external references in gcs.
// This returns a single row with the error status of the query.
if _, err := t.client.runQuery(ctx, query); err != nil {
log.WithField("query", queryString).Error("query failed")
return fmt.Errorf("commit query: %w", err)
}
if t.cfg.DBTJobTrigger.Enabled() {
log.Info("store: dbt job trigger")
if err := dbt.JobTrigger(t.cfg.DBTJobTrigger); err != nil {
return fmt.Errorf("triggering dbt job: %w", err)
}
}
return nil
}
func (t *transactor) Destroy() {
_ = t.client.bigqueryClient.Close()
_ = t.client.cloudStorageClient.Close()
}