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
496
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

use std::collections::BTreeMap;
use std::fmt;
use std::sync::LazyLock;

use anyhow::Ok;
use byteorder::{NetworkEndian, WriteBytesExt};
use chrono::Timelike;
use itertools::Itertools;
use mz_avro::types::{DecimalValue, ToAvro, Value};
use mz_avro::Schema;
use mz_ore::cast::CastFrom;
use mz_repr::adt::jsonb::JsonbRef;
use mz_repr::adt::numeric::{self, NUMERIC_AGG_MAX_PRECISION, NUMERIC_DATUM_MAX_PRECISION};
use mz_repr::{ColumnName, ColumnType, Datum, GlobalId, RelationDesc, Row, ScalarType};
use serde_json::json;

use crate::encode::{column_names_and_types, Encode, TypedDatum};
use crate::envelopes::{self, DBZ_ROW_TYPE_ID, ENVELOPE_CUSTOM_NAMES};
use crate::json::{build_row_schema_json, SchemaOptions};

// TODO(rkhaitan): this schema intentionally omits the data_collections field
// that is typically present in Debezium transaction metadata topics. See
// https://debezium.io/documentation/reference/connectors/postgresql.html#postgresql-transaction-metadata
// for more information. We chose to omit this field because it is redundant
// for sinks where each consistency topic corresponds to exactly one sink.
// We will need to add it in order to be able to reingest sinked topics.
static DEBEZIUM_TRANSACTION_SCHEMA: LazyLock<Schema> = LazyLock::new(|| {
    Schema::parse(&json!({
        "type": "record",
        "name": "envelope",
        "fields": [
            {
                "name": "id",
                "type": "string"
            },
            {
                "name": "status",
                "type": "string"
            },
            {
                "name": "event_count",
                "type": [
                  "null",
                  "long"
                ]
            },
            {
                "name": "data_collections",
                "type": [
                    "null",
                    {
                        "type": "array",
                        "items": {
                            "type": "record",
                            "name": "data_collection",
                            "fields": [
                                {
                                    "name": "data_collection",
                                    "type": "string"
                                },
                                {
                                    "name": "event_count",
                                    "type": "long"
                                },
                            ]
                        }
                    }
                ],
                "default": null,
            },
        ]
    }))
    .expect("valid schema constructed")
});

fn encode_avro_header(buf: &mut Vec<u8>, schema_id: i32) {
    // The first byte is a magic byte (0) that indicates the Confluent
    // serialization format version, and the next four bytes are a
    // 32-bit schema ID.
    //
    // https://docs.confluent.io/platform/current/schema-registry/fundamentals/serdes-develop/index.html#wire-format
    buf.write_u8(0).expect("writing to vec cannot fail");
    buf.write_i32::<NetworkEndian>(schema_id)
        .expect("writing to vec cannot fail");
}

fn encode_message_unchecked(
    schema_id: i32,
    row: Row,
    schema: &Schema,
    columns: &[(ColumnName, ColumnType)],
) -> Vec<u8> {
    let mut buf = vec![];
    encode_avro_header(&mut buf, schema_id);
    let value = encode_datums_as_avro(row.iter(), columns);
    mz_avro::encode_unchecked(&value, schema, &mut buf);
    buf
}

#[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Clone)]
pub enum DocTarget {
    Type(GlobalId),
    Field {
        object_id: GlobalId,
        column_name: ColumnName,
    },
}

impl DocTarget {
    fn id(&self) -> GlobalId {
        match self {
            DocTarget::Type(object_id) => *object_id,
            DocTarget::Field { object_id, .. } => *object_id,
        }
    }
}

/// Generates an Avro schema
pub struct AvroSchemaGenerator {
    columns: Vec<(ColumnName, ColumnType)>,
    schema: Schema,
}

impl fmt::Debug for AvroSchemaGenerator {
    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
        f.debug_struct("SchemaGenerator")
            .field("writer_schema", &self.schema())
            .finish()
    }
}

impl AvroSchemaGenerator {
    pub fn new(
        desc: RelationDesc,
        debezium: bool,
        mut doc_options: BTreeMap<DocTarget, String>,
        avro_fullname: &str,
        set_null_defaults: bool,
        sink_from: Option<GlobalId>,
        use_custom_envelope_names: bool,
    ) -> Result<Self, anyhow::Error> {
        let mut columns = column_names_and_types(desc);
        if debezium {
            columns = envelopes::dbz_envelope(columns);
            // With DEBEZIUM envelope the message is wrapped into "before" and "after"
            // with `DBZ_ROW_TYPE_ID` instead of `sink_from`.
            // Replacing comments for the columns and type in `sink_from` to `DBZ_ROW_TYPE_ID`.
            if let Some(sink_from_id) = sink_from {
                let mut new_column_docs = BTreeMap::new();
                doc_options.iter().for_each(|(k, v)| {
                    if k.id() == sink_from_id {
                        match k {
                            DocTarget::Field { column_name, .. } => {
                                new_column_docs.insert(
                                    DocTarget::Field {
                                        object_id: DBZ_ROW_TYPE_ID,
                                        column_name: column_name.clone(),
                                    },
                                    v.clone(),
                                );
                            }
                            DocTarget::Type(_) => {
                                new_column_docs.insert(DocTarget::Type(DBZ_ROW_TYPE_ID), v.clone());
                            }
                        }
                    }
                });
                doc_options.append(&mut new_column_docs);
                doc_options.retain(|k, _v| k.id() != sink_from_id);
            }
        }
        let custom_names = if use_custom_envelope_names {
            &ENVELOPE_CUSTOM_NAMES
        } else {
            &BTreeMap::new()
        };
        let row_schema = build_row_schema_json(
            &columns,
            avro_fullname,
            custom_names,
            sink_from,
            &SchemaOptions {
                set_null_defaults,
                doc_comments: doc_options,
            },
        )?;
        let schema = Schema::parse(&row_schema).expect("valid schema constructed");
        Ok(AvroSchemaGenerator { columns, schema })
    }

    pub fn schema(&self) -> &Schema {
        &self.schema
    }

    pub fn columns(&self) -> &[(ColumnName, ColumnType)] {
        &self.columns
    }
}

/// Manages encoding of Avro-encoded bytes.
pub struct AvroEncoder {
    columns: Vec<(ColumnName, ColumnType)>,
    schema: Schema,
    schema_id: i32,
}

impl fmt::Debug for AvroEncoder {
    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
        f.debug_struct("AvroEncoder")
            .field("writer_schema", &self.schema)
            .finish()
    }
}

impl AvroEncoder {
    pub fn new(desc: RelationDesc, debezium: bool, schema: &str, schema_id: i32) -> Self {
        let mut columns = column_names_and_types(desc);
        if debezium {
            columns = envelopes::dbz_envelope(columns);
        };
        AvroEncoder {
            columns,
            schema: Schema::parse(&serde_json::from_str(schema).expect("valid schema json"))
                .expect("valid schema"),
            schema_id,
        }
    }
}

impl Encode for AvroEncoder {
    fn encode_unchecked(&self, row: Row) -> Vec<u8> {
        encode_message_unchecked(self.schema_id, row, &self.schema, &self.columns)
    }

    fn hash(&self, buf: &[u8]) -> u64 {
        // Compute a stable hash by ignoring the avro header which might contain a
        // non-deterministic schema id.
        let (_schema_id, payload) = crate::confluent::extract_avro_header(buf).unwrap();
        seahash::hash(payload)
    }
}

/// Encodes a sequence of `Datum` as Avro (key and value), using supplied column names and types.
pub fn encode_datums_as_avro<'a, I>(datums: I, names_types: &[(ColumnName, ColumnType)]) -> Value
where
    I: IntoIterator<Item = Datum<'a>>,
{
    let value_fields: Vec<(String, Value)> = names_types
        .iter()
        .zip_eq(datums)
        .map(|((name, typ), datum)| {
            let name = name.as_str().to_owned();
            (name, TypedDatum::new(datum, typ).avro())
        })
        .collect();
    let v = Value::Record(value_fields);
    v
}

impl<'a> mz_avro::types::ToAvro for TypedDatum<'a> {
    fn avro(self) -> Value {
        let TypedDatum { datum, typ } = self;
        if typ.nullable && datum.is_null() {
            Value::Union {
                index: 0,
                inner: Box::new(Value::Null),
                n_variants: 2,
                null_variant: Some(0),
            }
        } else {
            let mut val = match &typ.scalar_type {
                ScalarType::AclItem => Value::String(datum.unwrap_acl_item().to_string()),
                ScalarType::Bool => Value::Boolean(datum.unwrap_bool()),
                ScalarType::PgLegacyChar => {
                    Value::Fixed(1, datum.unwrap_uint8().to_le_bytes().into())
                }
                ScalarType::Int16 => Value::Int(i32::from(datum.unwrap_int16())),
                ScalarType::Int32 => Value::Int(datum.unwrap_int32()),
                ScalarType::Int64 => Value::Long(datum.unwrap_int64()),
                ScalarType::UInt16 => Value::Fixed(2, datum.unwrap_uint16().to_be_bytes().into()),
                ScalarType::UInt32 => Value::Fixed(4, datum.unwrap_uint32().to_be_bytes().into()),
                ScalarType::UInt64 => Value::Fixed(8, datum.unwrap_uint64().to_be_bytes().into()),
                ScalarType::Oid
                | ScalarType::RegClass
                | ScalarType::RegProc
                | ScalarType::RegType => {
                    Value::Fixed(4, datum.unwrap_uint32().to_be_bytes().into())
                }
                ScalarType::Float32 => Value::Float(datum.unwrap_float32()),
                ScalarType::Float64 => Value::Double(datum.unwrap_float64()),
                ScalarType::Numeric { max_scale } => {
                    let mut d = datum.unwrap_numeric().0;
                    let (unscaled, precision, scale) = match max_scale {
                        Some(max_scale) => {
                            // Values must be rescaled to resaturate trailing zeroes
                            numeric::rescale(&mut d, max_scale.into_u8()).unwrap();
                            (
                                numeric::numeric_to_twos_complement_be(d).to_vec(),
                                NUMERIC_DATUM_MAX_PRECISION,
                                max_scale.into_u8(),
                            )
                        }
                        // Decimals without specified scale must nonetheless be
                        // expressed as a fixed scale, so we write everything as
                        // a 78-digit number with a scale of 39, which
                        // definitively expresses all valid numeric values.
                        None => (
                            numeric::numeric_to_twos_complement_wide(d).to_vec(),
                            NUMERIC_AGG_MAX_PRECISION,
                            NUMERIC_DATUM_MAX_PRECISION,
                        ),
                    };
                    Value::Decimal(DecimalValue {
                        unscaled,
                        precision: usize::cast_from(precision),
                        scale: usize::cast_from(scale),
                    })
                }
                ScalarType::Date => Value::Date(datum.unwrap_date().unix_epoch_days()),
                ScalarType::Time => Value::Long({
                    let time = datum.unwrap_time();
                    i64::from(time.num_seconds_from_midnight()) * 1_000_000
                        + i64::from(time.nanosecond()) / 1_000
                }),
                ScalarType::Timestamp { .. } => {
                    Value::Timestamp(datum.unwrap_timestamp().to_naive())
                }
                ScalarType::TimestampTz { .. } => {
                    Value::Timestamp(datum.unwrap_timestamptz().to_naive())
                }
                // SQL intervals and Avro durations differ quite a lot (signed
                // vs unsigned, different int sizes), so SQL intervals are their
                // own bespoke type.
                ScalarType::Interval => Value::Fixed(16, {
                    let iv = datum.unwrap_interval();
                    let mut buf = Vec::with_capacity(16);
                    buf.extend(iv.months.to_le_bytes());
                    buf.extend(iv.days.to_le_bytes());
                    buf.extend(iv.micros.to_le_bytes());
                    debug_assert_eq!(buf.len(), 16);
                    buf
                }),
                ScalarType::Bytes => Value::Bytes(Vec::from(datum.unwrap_bytes())),
                ScalarType::String | ScalarType::VarChar { .. } | ScalarType::PgLegacyName => {
                    Value::String(datum.unwrap_str().to_owned())
                }
                ScalarType::Char { length } => {
                    let s = mz_repr::adt::char::format_str_pad(datum.unwrap_str(), *length);
                    Value::String(s)
                }
                ScalarType::Jsonb => Value::Json(JsonbRef::from_datum(datum).to_serde_json()),
                ScalarType::Uuid => Value::Uuid(datum.unwrap_uuid()),
                ty @ (ScalarType::Array(..) | ScalarType::Int2Vector | ScalarType::List { .. }) => {
                    let list = match ty {
                        ScalarType::Array(_) | ScalarType::Int2Vector => {
                            datum.unwrap_array().elements()
                        }
                        ScalarType::List { .. } => datum.unwrap_list(),
                        _ => unreachable!(),
                    };

                    let values = list
                        .into_iter()
                        .map(|datum| {
                            TypedDatum::new(
                                datum,
                                &ColumnType {
                                    nullable: true,
                                    scalar_type: ty.unwrap_collection_element_type().clone(),
                                },
                            )
                            .avro()
                        })
                        .collect();
                    Value::Array(values)
                }
                ScalarType::Map { value_type, .. } => {
                    let map = datum.unwrap_map();
                    let elements = map
                        .into_iter()
                        .map(|(key, datum)| {
                            let value = TypedDatum::new(
                                datum,
                                &ColumnType {
                                    nullable: true,
                                    scalar_type: (**value_type).clone(),
                                },
                            )
                            .avro();
                            (key.to_string(), value)
                        })
                        .collect();
                    Value::Map(elements)
                }
                ScalarType::Record { fields, .. } => {
                    let list = datum.unwrap_list();
                    let fields = fields
                        .iter()
                        .zip(&list)
                        .map(|((name, typ), datum)| {
                            let name = name.to_string();
                            let datum = TypedDatum::new(datum, typ);
                            let value = datum.avro();
                            (name, value)
                        })
                        .collect();
                    Value::Record(fields)
                }
                ScalarType::MzTimestamp => Value::String(datum.unwrap_mz_timestamp().to_string()),
                ScalarType::Range { .. } => Value::String(datum.unwrap_range().to_string()),
                ScalarType::MzAclItem => Value::String(datum.unwrap_mz_acl_item().to_string()),
            };
            if typ.nullable {
                val = Value::Union {
                    index: 1,
                    inner: Box::new(val),
                    n_variants: 2,
                    null_variant: Some(0),
                };
            }
            val
        }
    }
}

pub fn get_debezium_transaction_schema() -> &'static Schema {
    &DEBEZIUM_TRANSACTION_SCHEMA
}

pub fn encode_debezium_transaction_unchecked(
    schema_id: i32,
    collection: &str,
    id: &str,
    status: &str,
    message_count: Option<i64>,
) -> Vec<u8> {
    let mut buf = Vec::new();
    encode_avro_header(&mut buf, schema_id);

    let transaction_id = Value::String(id.to_owned());
    let status = Value::String(status.to_owned());
    let event_count = match message_count {
        None => Value::Union {
            index: 0,
            inner: Box::new(Value::Null),
            n_variants: 2,
            null_variant: Some(0),
        },
        Some(count) => Value::Union {
            index: 1,
            inner: Box::new(Value::Long(count)),
            n_variants: 2,
            null_variant: Some(0),
        },
    };

    let data_collections = if let Some(message_count) = message_count {
        let collection = Value::Record(vec![
            ("data_collection".into(), Value::String(collection.into())),
            ("event_count".into(), Value::Long(message_count)),
        ]);
        Value::Union {
            index: 1,
            inner: Box::new(Value::Array(vec![collection])),
            n_variants: 2,
            null_variant: Some(0),
        }
    } else {
        Value::Union {
            index: 0,
            inner: Box::new(Value::Null),
            n_variants: 2,
            null_variant: Some(0),
        }
    };

    let record_contents = vec![
        ("id".into(), transaction_id),
        ("status".into(), status),
        ("event_count".into(), event_count),
        ("data_collections".into(), data_collections),
    ];
    let avro = Value::Record(record_contents);
    debug_assert!(avro.validate(DEBEZIUM_TRANSACTION_SCHEMA.top_node()));
    mz_avro::encode_unchecked(&avro, &DEBEZIUM_TRANSACTION_SCHEMA, &mut buf);
    buf
}