Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: encode JSON in the same way as debezium #8865

Merged
merged 8 commits into from
Apr 3, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

22 changes: 22 additions & 0 deletions src/common/src/types/interval.rs
Original file line number Diff line number Diff line change
Expand Up @@ -843,6 +843,28 @@ impl ToText for crate::types::Interval {
}
}

impl Interval {
pub fn as_iso_8601(&self) -> String {
// ISO pattern - PnYnMnDTnHnMnS
let years = self.months / 12;
let months = self.months % 12;
let days = self.days;
let secs_fract = (self.usecs % USECS_PER_SEC).abs();
let total_secs = (self.usecs / USECS_PER_SEC).abs();
let hours = total_secs / 3600;
let minutes = (total_secs / 60) % 60;
let seconds = total_secs % 60;
let mut buf = [0u8; 7];
let fract_str = if secs_fract != 0 {
write!(buf.as_mut_slice(), ".{:06}", secs_fract).unwrap();
std::str::from_utf8(&buf).unwrap().trim_end_matches('0')
} else {
""
};
format!("P{years}Y{months}M{days}DT{hours}H{minutes}M{seconds}{fract_str}S")
}
}

impl Display for Interval {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
let years = self.months / 12;
Expand Down
1 change: 1 addition & 0 deletions src/connector/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] }
futures-async-stream = "0.2"
globset = "0.4.8"
google-cloud-pubsub = "0.7.0"
hex = "0.4"
http = "0.2"
http-serde = "1.1.0"
hyper = "0.14"
Expand Down
33 changes: 31 additions & 2 deletions src/connector/src/sink/kafka.rs
Original file line number Diff line number Diff line change
Expand Up @@ -411,11 +411,40 @@ pub fn chunk_to_json(chunk: StreamChunk, schema: &Schema) -> Result<Vec<String>>

fn fields_to_json(fields: &[Field]) -> Value {
let mut res = Vec::new();

fields.iter().for_each(|field| {
// mapping from 'https://debezium.io/documentation/reference/2.1/connectors/postgresql.html#postgresql-data-types'
let r#type = match field.data_type() {
risingwave_common::types::DataType::Boolean => "boolean",
risingwave_common::types::DataType::Int16 => "int16",
risingwave_common::types::DataType::Int32 => "int32",
risingwave_common::types::DataType::Int64 => "int64",
risingwave_common::types::DataType::Float32 => "float32",
risingwave_common::types::DataType::Float64 => "float64",
// currently, we only support handling decimal as string.
// https://debezium.io/documentation/reference/2.1/connectors/postgresql.html#postgresql-decimal-types
risingwave_common::types::DataType::Decimal => "string",

risingwave_common::types::DataType::Varchar => "string",

risingwave_common::types::DataType::Date => "int32",
risingwave_common::types::DataType::Time => "int64",
risingwave_common::types::DataType::Timestamp => "int64",
risingwave_common::types::DataType::Timestamptz => "string",
risingwave_common::types::DataType::Interval => "string",

risingwave_common::types::DataType::Bytea => "bytes",
risingwave_common::types::DataType::Jsonb => "string",
risingwave_common::types::DataType::Serial => "int32",
// since the original debezium pg support HSTORE via encoded as json string by default,
// we do the same here
risingwave_common::types::DataType::Struct(_) => "string",
risingwave_common::types::DataType::List { .. } => "string",
};
res.push(json!({
"field": field.name,
"optional": true,
"type": field.type_name,
"type": r#type,
}))
});

Expand Down Expand Up @@ -654,7 +683,7 @@ mod test {

let json_chunk = chunk_to_json(chunk, &schema).unwrap();
let schema_json = schema_to_json(&schema);
assert_eq!(schema_json.to_string(), "{\"fields\":[{\"field\":\"before\",\"fields\":[{\"field\":\"v1\",\"optional\":true,\"type\":\"\"},{\"field\":\"v2\",\"optional\":true,\"type\":\"\"},{\"field\":\"v3\",\"optional\":true,\"type\":\"\"}],\"optional\":true,\"type\":\"struct\"},{\"field\":\"after\",\"fields\":[{\"field\":\"v1\",\"optional\":true,\"type\":\"\"},{\"field\":\"v2\",\"optional\":true,\"type\":\"\"},{\"field\":\"v3\",\"optional\":true,\"type\":\"\"}],\"optional\":true,\"type\":\"struct\"}],\"optional\":false,\"type\":\"struct\"}");
assert_eq!(schema_json.to_string(), "{\"fields\":[{\"field\":\"before\",\"fields\":[{\"field\":\"v1\",\"optional\":true,\"type\":\"int32\"},{\"field\":\"v2\",\"optional\":true,\"type\":\"float32\"},{\"field\":\"v3\",\"optional\":true,\"type\":\"string\"}],\"optional\":true,\"type\":\"struct\"},{\"field\":\"after\",\"fields\":[{\"field\":\"v1\",\"optional\":true,\"type\":\"int32\"},{\"field\":\"v2\",\"optional\":true,\"type\":\"float32\"},{\"field\":\"v3\",\"optional\":true,\"type\":\"string\"}],\"optional\":true,\"type\":\"struct\"}],\"optional\":false,\"type\":\"struct\"}");
assert_eq!(
json_chunk[0].as_str(),
"{\"v1\":0,\"v2\":0.0,\"v3\":{\"v4\":0,\"v5\":0.0}}"
Expand Down
140 changes: 129 additions & 11 deletions src/connector/src/sink/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ use std::collections::HashMap;

use anyhow::anyhow;
use async_trait::async_trait;
use chrono::{Datelike, Timelike};
use enum_as_inner::EnumAsInner;
use risingwave_common::array::{ArrayError, ArrayResult, RowRef, StreamChunk};
use risingwave_common::catalog::{Field, Schema};
Expand Down Expand Up @@ -306,19 +307,27 @@ fn datum_to_json_object(field: &Field, datum: DatumRef<'_>) -> ArrayResult<Value
json!(v)
}
(DataType::Decimal, ScalarRefImpl::Decimal(v)) => {
// fixme
json!(v.to_text())
}
(
dt @ DataType::Date
| dt @ DataType::Time
| dt @ DataType::Timestamp
| dt @ DataType::Timestamptz
| dt @ DataType::Interval
| dt @ DataType::Bytea,
scalar,
) => {
json!(scalar.to_text_with_type(&dt))
(DataType::Timestamptz, ScalarRefImpl::Timestamp(v)) => {
json!(v.0.and_local_timezone(chrono::Utc).unwrap().to_rfc3339())
}
(DataType::Time, ScalarRefImpl::Time(v)) => {
// todo: just ignore the nanos part to avoid leap second complex
json!(v.0.num_seconds_from_midnight() as i64 * 1000)
}
(DataType::Date, ScalarRefImpl::Date(v)) => {
json!(v.0.num_days_from_ce())
}
(DataType::Timestamp, ScalarRefImpl::Timestamp(v)) => {
json!(v.0.timestamp_millis())
}
(DataType::Bytea, ScalarRefImpl::Bytea(v)) => {
json!(hex::encode(v))
}
// P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S
(DataType::Interval, ScalarRefImpl::Interval(v)) => {
json!(v.as_iso_8601())
}
(DataType::List { datatype }, ScalarRefImpl::List(list_ref)) => {
let mut vec = Vec::with_capacity(list_ref.values_ref().len());
Expand Down Expand Up @@ -351,3 +360,112 @@ fn datum_to_json_object(field: &Field, datum: DatumRef<'_>) -> ArrayResult<Value

Ok(value)
}

#[cfg(test)]
mod tests {

use risingwave_common::types::{Interval, ScalarImpl, Time, Timestamp};

use super::*;
#[test]
fn test_to_json_basic_type() {
let mock_field = Field {
data_type: DataType::Boolean,
name: Default::default(),
sub_fields: Default::default(),
type_name: Default::default(),
};
let boolean_value = datum_to_json_object(
&Field {
data_type: DataType::Boolean,
..mock_field.clone()
},
Some(ScalarImpl::Bool(false).as_scalar_ref_impl()),
)
.unwrap();
assert_eq!(boolean_value, json!(false));

let int16_value = datum_to_json_object(
&Field {
data_type: DataType::Int16,
..mock_field.clone()
},
Some(ScalarImpl::Int16(16).as_scalar_ref_impl()),
)
.unwrap();
assert_eq!(int16_value, json!(16));

let int64_value = datum_to_json_object(
&Field {
data_type: DataType::Int64,
..mock_field.clone()
},
Some(ScalarImpl::Int64(std::i64::MAX).as_scalar_ref_impl()),
)
.unwrap();
assert_eq!(
serde_json::to_string(&int64_value).unwrap(),
std::i64::MAX.to_string()
);

// https://github.com/debezium/debezium/blob/main/debezium-core/src/main/java/io/debezium/time/ZonedTimestamp.java
let tstz_str = "2018-01-26T18:30:09.453Z";
let tstz_value = datum_to_json_object(
&Field {
data_type: DataType::Timestamptz,
..mock_field.clone()
},
Some(
ScalarImpl::Timestamp(
chrono::DateTime::parse_from_rfc3339(tstz_str)
.unwrap()
.naive_utc()
.into(),
)
.as_scalar_ref_impl(),
),
)
.unwrap();
chrono::DateTime::parse_from_rfc3339(tstz_value.as_str().unwrap_or_default()).unwrap();

let ts_value = datum_to_json_object(
&Field {
data_type: DataType::Timestamp,
..mock_field.clone()
},
Some(
ScalarImpl::Timestamp(Timestamp::from_timestamp_uncheck(1000, 0))
.as_scalar_ref_impl(),
),
)
.unwrap();
assert_eq!(ts_value, json!(1000 * 1000));

// Represents the number of microseconds past midnigh, io.debezium.time.Time
let time_value = datum_to_json_object(
&Field {
data_type: DataType::Time,
..mock_field.clone()
},
Some(
ScalarImpl::Time(Time::from_num_seconds_from_midnight_uncheck(1000, 0))
.as_scalar_ref_impl(),
),
)
.unwrap();
assert_eq!(time_value, json!(1000 * 1000));

let interval_value = datum_to_json_object(
&Field {
data_type: DataType::Interval,
..mock_field
},
Some(
ScalarImpl::Interval(Interval::from_month_day_usec(13, 2, 1000000))
.as_scalar_ref_impl(),
),
)
.unwrap();
assert_eq!(interval_value, json!("P1Y1M2DT0H0M1S"));
}
}