-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
encoder.go
440 lines (394 loc) · 13.3 KB
/
encoder.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
// Copyright 2018 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
package changefeedccl
import (
"bytes"
"context"
"encoding/binary"
gojson "encoding/json"
"io/ioutil"
"net/http"
"net/url"
"path/filepath"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/json"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/pkg/errors"
)
const (
confluentSchemaContentType = `application/vnd.schemaregistry.v1+json`
confluentSubjectSuffixKey = `-key`
confluentSubjectSuffixValue = `-value`
confluentAvroWireFormatMagic = byte(0)
)
// encodeRow holds all the pieces necessary to encode a row change into a key or
// value.
type encodeRow struct {
// datums is the new value of a changed table row.
datums sqlbase.EncDatumRow
// updated is the mvcc timestamp corresponding to the latest update in
// `datums`.
updated hlc.Timestamp
// deleted is true if row is a deletion. In this case, only the primary
// key columns are guaranteed to be set in `datums`.
deleted bool
// tableDesc is a TableDescriptor for the table containing `datums`.
// It's valid for interpreting the row at `updated`.
tableDesc *sqlbase.TableDescriptor
}
// Encoder turns a row into a serialized changefeed key, value, or resolved
// timestamp. It represents one of the `format=` changefeed options.
type Encoder interface {
// EncodeKey encodes the primary key of the given row. The columns of the
// datums are expected to match 1:1 with the `Columns` field of the
// `TableDescriptor`, but only the primary key fields will be used. The
// returned bytes are only valid until the next call to Encode*.
EncodeKey(encodeRow) ([]byte, error)
// EncodeValue encodes the primary key of the given row. The columns of the
// datums are expected to match 1:1 with the `Columns` field of the
// `TableDescriptor`. The returned bytes are only valid until the next call
// to Encode*.
EncodeValue(encodeRow) ([]byte, error)
// EncodeResolvedTimestamp encodes a resolved timestamp payload for the
// given topic name. The returned bytes are only valid until the next call
// to Encode*.
EncodeResolvedTimestamp(string, hlc.Timestamp) ([]byte, error)
}
func getEncoder(opts map[string]string) (Encoder, error) {
switch formatType(opts[optFormat]) {
case ``, optFormatJSON:
return makeJSONEncoder(opts)
case optFormatAvro:
return newConfluentAvroEncoder(opts)
default:
return nil, errors.Errorf(`unknown %s: %s`, optFormat, opts[optFormat])
}
}
// jsonEncoder encodes changefeed entries as JSON. Keys are the primary key
// columns in a JSON array. Values are a JSON object mapping every column name
// to its value. Updated timestamps in rows and resolved timestamp payloads are
// stored in a sub-object under the `__crdb__` key in the top-level JSON object.
type jsonEncoder struct {
updatedField, wrapped, keyOnly, keyInValue bool
alloc sqlbase.DatumAlloc
buf bytes.Buffer
}
var _ Encoder = &jsonEncoder{}
func makeJSONEncoder(opts map[string]string) (*jsonEncoder, error) {
e := &jsonEncoder{
keyOnly: envelopeType(opts[optEnvelope]) == optEnvelopeKeyOnly,
wrapped: envelopeType(opts[optEnvelope]) == optEnvelopeWrapped,
}
_, e.updatedField = opts[optUpdatedTimestamps]
_, e.keyInValue = opts[optKeyInValue]
if e.keyInValue && !e.wrapped {
return nil, errors.Errorf(`%s is only usable with %s=%s`,
optKeyInValue, optEnvelope, optEnvelopeWrapped)
}
return e, nil
}
// EncodeKey implements the Encoder interface.
func (e *jsonEncoder) EncodeKey(row encodeRow) ([]byte, error) {
jsonEntries, err := e.encodeKeyRaw(row)
if err != nil {
return nil, err
}
j, err := json.MakeJSON(jsonEntries)
if err != nil {
return nil, err
}
e.buf.Reset()
j.Format(&e.buf)
return e.buf.Bytes(), nil
}
func (e *jsonEncoder) encodeKeyRaw(row encodeRow) ([]interface{}, error) {
colIdxByID := row.tableDesc.ColumnIdxMap()
jsonEntries := make([]interface{}, len(row.tableDesc.PrimaryIndex.ColumnIDs))
for i, colID := range row.tableDesc.PrimaryIndex.ColumnIDs {
idx, ok := colIdxByID[colID]
if !ok {
return nil, errors.Errorf(`unknown column id: %d`, colID)
}
datum, col := row.datums[idx], &row.tableDesc.Columns[idx]
if err := datum.EnsureDecoded(&col.Type, &e.alloc); err != nil {
return nil, err
}
var err error
jsonEntries[i], err = tree.AsJSON(datum.Datum)
if err != nil {
return nil, err
}
}
return jsonEntries, nil
}
// EncodeValue implements the Encoder interface.
func (e *jsonEncoder) EncodeValue(row encodeRow) ([]byte, error) {
if e.keyOnly || (!e.wrapped && row.deleted) {
return nil, nil
}
var after map[string]interface{}
if !row.deleted {
columns := row.tableDesc.Columns
after = make(map[string]interface{}, len(columns))
for i := range columns {
col := &columns[i]
datum := row.datums[i]
if err := datum.EnsureDecoded(&col.Type, &e.alloc); err != nil {
return nil, err
}
var err error
after[col.Name], err = tree.AsJSON(datum.Datum)
if err != nil {
return nil, err
}
}
}
var jsonEntries map[string]interface{}
if e.wrapped {
if after != nil {
jsonEntries = map[string]interface{}{`after`: after}
} else {
jsonEntries = map[string]interface{}{`after`: nil}
}
if e.keyInValue {
keyEntries, err := e.encodeKeyRaw(row)
if err != nil {
return nil, err
}
jsonEntries[`key`] = keyEntries
}
} else {
jsonEntries = after
}
if e.updatedField {
var meta map[string]interface{}
if e.wrapped {
meta = jsonEntries
} else {
meta = make(map[string]interface{}, 1)
jsonEntries[jsonMetaSentinel] = meta
}
meta[`updated`] = row.updated.AsOfSystemTime()
}
j, err := json.MakeJSON(jsonEntries)
if err != nil {
return nil, err
}
e.buf.Reset()
j.Format(&e.buf)
return e.buf.Bytes(), nil
}
// EncodeResolvedTimestamp implements the Encoder interface.
func (e *jsonEncoder) EncodeResolvedTimestamp(_ string, resolved hlc.Timestamp) ([]byte, error) {
meta := map[string]interface{}{
`resolved`: tree.TimestampToDecimal(resolved).Decimal.String(),
}
var jsonEntries interface{}
if e.wrapped {
jsonEntries = meta
} else {
jsonEntries = map[string]interface{}{
jsonMetaSentinel: meta,
}
}
return gojson.Marshal(jsonEntries)
}
// confluentAvroEncoder encodes changefeed entries as Avro's binary or textual
// JSON format. Keys are the primary key columns in a record. Values are all
// columns in a record.
type confluentAvroEncoder struct {
registryURL string
updatedField, keyOnly bool
keyCache map[tableIDAndVersion]confluentRegisteredKeySchema
valueCache map[tableIDAndVersion]confluentRegisteredEnvelopeSchema
resolvedCache map[string]confluentRegisteredEnvelopeSchema
}
type tableIDAndVersion uint64
func makeTableIDAndVersion(id sqlbase.ID, version sqlbase.DescriptorVersion) tableIDAndVersion {
return tableIDAndVersion(id)<<32 + tableIDAndVersion(version)
}
type confluentRegisteredKeySchema struct {
schema *avroDataRecord
registryID int32
}
type confluentRegisteredEnvelopeSchema struct {
schema *avroEnvelopeRecord
registryID int32
}
var _ Encoder = &confluentAvroEncoder{}
func newConfluentAvroEncoder(opts map[string]string) (*confluentAvroEncoder, error) {
e := &confluentAvroEncoder{registryURL: opts[optConfluentSchemaRegistry]}
switch opts[optEnvelope] {
case string(optEnvelopeKeyOnly):
e.keyOnly = true
case string(optEnvelopeWrapped):
default:
return nil, errors.Errorf(`%s=%s is not supported with %s=%s`,
optEnvelope, opts[optEnvelope], optFormat, optFormatAvro)
}
_, e.updatedField = opts[optUpdatedTimestamps]
if _, ok := opts[optKeyInValue]; ok {
return nil, errors.Errorf(`%s is not supported with %s=%s`,
optKeyInValue, optFormat, optFormatAvro)
}
if len(e.registryURL) == 0 {
return nil, errors.Errorf(`WITH option %s is required for %s=%s`,
optConfluentSchemaRegistry, optFormat, optFormatAvro)
}
e.keyCache = make(map[tableIDAndVersion]confluentRegisteredKeySchema)
e.valueCache = make(map[tableIDAndVersion]confluentRegisteredEnvelopeSchema)
e.resolvedCache = make(map[string]confluentRegisteredEnvelopeSchema)
return e, nil
}
// EncodeKey implements the Encoder interface.
func (e *confluentAvroEncoder) EncodeKey(row encodeRow) ([]byte, error) {
cacheKey := makeTableIDAndVersion(row.tableDesc.ID, row.tableDesc.Version)
registered, ok := e.keyCache[cacheKey]
if !ok {
var err error
registered.schema, err = indexToAvroSchema(row.tableDesc, &row.tableDesc.PrimaryIndex)
if err != nil {
return nil, err
}
// NB: This uses the kafka name escaper because it has to match the name
// of the kafka topic.
subject := SQLNameToKafkaName(row.tableDesc.Name) + confluentSubjectSuffixKey
registered.registryID, err = e.register(®istered.schema.avroRecord, subject)
if err != nil {
return nil, err
}
// TODO(dan): Bound the size of this cache.
e.keyCache[cacheKey] = registered
}
// https://docs.confluent.io/current/schema-registry/docs/serializer-formatter.html#wire-format
header := []byte{
confluentAvroWireFormatMagic,
0, 0, 0, 0, // Placeholder for the ID.
}
binary.BigEndian.PutUint32(header[1:5], uint32(registered.registryID))
return registered.schema.BinaryFromRow(header, row.datums)
}
// EncodeValue implements the Encoder interface.
func (e *confluentAvroEncoder) EncodeValue(row encodeRow) ([]byte, error) {
if e.keyOnly {
return nil, nil
}
cacheKey := makeTableIDAndVersion(row.tableDesc.ID, row.tableDesc.Version)
registered, ok := e.valueCache[cacheKey]
if !ok {
afterDataSchema, err := tableToAvroSchema(row.tableDesc)
if err != nil {
return nil, err
}
opts := avroEnvelopeOpts{afterField: true, updatedField: e.updatedField}
registered.schema, err = envelopeToAvroSchema(row.tableDesc.Name, opts, afterDataSchema)
if err != nil {
return nil, err
}
// NB: This uses the kafka name escaper because it has to match the name
// of the kafka topic.
subject := SQLNameToKafkaName(row.tableDesc.Name) + confluentSubjectSuffixValue
registered.registryID, err = e.register(®istered.schema.avroRecord, subject)
if err != nil {
return nil, err
}
// TODO(dan): Bound the size of this cache.
e.valueCache[cacheKey] = registered
}
var meta avroMetadata
if registered.schema.opts.updatedField {
meta = map[string]interface{}{
`updated`: row.updated,
}
}
var datums sqlbase.EncDatumRow
if !row.deleted {
datums = row.datums
}
// https://docs.confluent.io/current/schema-registry/docs/serializer-formatter.html#wire-format
header := []byte{
confluentAvroWireFormatMagic,
0, 0, 0, 0, // Placeholder for the ID.
}
binary.BigEndian.PutUint32(header[1:5], uint32(registered.registryID))
return registered.schema.BinaryFromRow(header, meta, datums)
}
// EncodeResolvedTimestamp implements the Encoder interface.
func (e *confluentAvroEncoder) EncodeResolvedTimestamp(
topic string, resolved hlc.Timestamp,
) ([]byte, error) {
registered, ok := e.resolvedCache[topic]
if !ok {
opts := avroEnvelopeOpts{resolvedField: true}
var err error
registered.schema, err = envelopeToAvroSchema(topic, opts, nil /* after */)
if err != nil {
return nil, err
}
// NB: This uses the kafka name escaper because it has to match the name
// of the kafka topic.
subject := SQLNameToKafkaName(topic) + confluentSubjectSuffixValue
registered.registryID, err = e.register(®istered.schema.avroRecord, subject)
if err != nil {
return nil, err
}
// TODO(dan): Bound the size of this cache.
e.resolvedCache[topic] = registered
}
var meta avroMetadata
if registered.schema.opts.resolvedField {
meta = map[string]interface{}{
`resolved`: resolved,
}
}
// https://docs.confluent.io/current/schema-registry/docs/serializer-formatter.html#wire-format
header := []byte{
confluentAvroWireFormatMagic,
0, 0, 0, 0, // Placeholder for the ID.
}
binary.BigEndian.PutUint32(header[1:5], uint32(registered.registryID))
return registered.schema.BinaryFromRow(header, meta, nil /* row */)
}
func (e *confluentAvroEncoder) register(schema *avroRecord, subject string) (int32, error) {
type confluentSchemaVersionRequest struct {
Schema string `json:"schema"`
}
type confluentSchemaVersionResponse struct {
ID int32 `json:"id"`
}
url, err := url.Parse(e.registryURL)
if err != nil {
return 0, err
}
url.Path = filepath.Join(url.EscapedPath(), `subjects`, subject, `versions`)
schemaStr := schema.codec.Schema()
if log.V(1) {
log.Infof(context.TODO(), "registering avro schema %s %s", url, schemaStr)
}
req := confluentSchemaVersionRequest{Schema: schemaStr}
var buf bytes.Buffer
if err := gojson.NewEncoder(&buf).Encode(req); err != nil {
return 0, err
}
resp, err := http.Post(url.String(), confluentSchemaContentType, &buf)
if err != nil {
return 0, err
}
defer resp.Body.Close()
if resp.StatusCode < 200 || resp.StatusCode >= 300 {
body, _ := ioutil.ReadAll(resp.Body)
return 0, errors.Errorf(`registering schema to %s %s: %s`, url.String(), resp.Status, body)
}
var res confluentSchemaVersionResponse
if err := gojson.NewDecoder(resp.Body).Decode(&res); err != nil {
return 0, err
}
return res.ID, nil
}