-
Notifications
You must be signed in to change notification settings - Fork 591
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(source): support JSON schema addtionalProperties (map) #17110
Merged
xxchan
merged 4 commits into
main
from
06-05-feat_source_support_json_schema_addtionalproperties_map_
Jun 13, 2024
Merged
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,2 +1,2 @@ | ||
{"$schema":"https://json-schema.org/draft/2020-12/schema","$id":"https://example.com/product.schema.json","title":"Product","description":"A product from Acme's catalog","type":"object","properties":{"productId":{"description":"The unique identifier for a product","type":"integer"},"productName":{"description":"Name of the product","type":"string"},"price":{"description":"The price of the product","type":"number","exclusiveMinimum":0},"tags":{"description":"Tags for the product","type":"array","items":{"type":"string"},"minItems":1,"uniqueItems":true},"dimensions":{"type":"object","properties":{"length":{"type":"number"},"width":{"type":"number"},"height":{"type":"number"}},"required":["length","width","height"]}},"required":["productId","productName","price"]} | ||
{"productId":1,"productName":"An ice sculpture","price":12.5,"tags":["cold","ice"],"dimensions":{"length":7,"width":12,"height":9.5}} | ||
{"$schema":"https://json-schema.org/draft/2020-12/schema","$id":"https://example.com/product.schema.json","title":"Product","description":"A product from Acme's catalog","type":"object","properties":{"productId":{"description":"The unique identifier for a product","type":"integer"},"productName":{"description":"Name of the product","type":"string"},"price":{"description":"The price of the product","type":"number","exclusiveMinimum":0},"tags":{"description":"Tags for the product","type":"array","items":{"type":"string"},"minItems":1,"uniqueItems":true},"dimensions":{"type":"object","properties":{"length":{"type":"number"},"width":{"type":"number"},"height":{"type":"number"}},"required":["length","width","height"]},"map":{"type":"object","additionalProperties":{"type":"string"}},"notMap":{"type":"object","additionalProperties":{"type":"string"},"properties":{"a":{"type":"string"}}}},"required":["productId","productName","price"]} | ||
{"productId":1,"productName":"An ice sculpture","price":12.5,"tags":["cold","ice"],"dimensions":{"length":7,"width":12,"height":9.5},"map":{"foo":"bar"},"notMap":{"a":"b","ignored":"c"}} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -26,6 +26,7 @@ use std::collections::HashMap; | |
use anyhow::Context as _; | ||
use apache_avro::Schema; | ||
use jst::{convert_avro, Context}; | ||
use risingwave_connector_codec::decoder::avro::MapHandling; | ||
use risingwave_pb::plan_common::ColumnDesc; | ||
|
||
use super::util::{bytes_from_url, get_kafka_topic}; | ||
|
@@ -80,7 +81,7 @@ impl JsonAccessBuilder { | |
} | ||
} | ||
|
||
pub async fn schema_to_columns( | ||
pub async fn fetch_json_schema_and_map_to_columns( | ||
schema_location: &str, | ||
schema_registry_auth: Option<SchemaRegistryAuth>, | ||
props: &HashMap<String, String>, | ||
|
@@ -98,11 +99,21 @@ pub async fn schema_to_columns( | |
let bytes = bytes_from_url(url, None).await?; | ||
serde_json::from_slice(&bytes)? | ||
}; | ||
let context = Context::default(); | ||
let avro_schema = convert_avro(&json_schema, context).to_string(); | ||
json_schema_to_columns(&json_schema) | ||
} | ||
|
||
/// FIXME: when the JSON schema is invalid, it will panic. | ||
/// | ||
/// ## Notes on type conversion | ||
/// Map will be used when an object doesn't have `properties` but has `additionalProperties`. | ||
/// When an object has `properties` and `additionalProperties`, the latter will be ignored. | ||
/// <https://github.com/mozilla/jsonschema-transpiler/blob/fb715c7147ebd52427e0aea09b2bba2d539850b1/src/jsonschema.rs#L228-L280> | ||
/// | ||
/// TODO: examine other stuff like `oneOf`, `patternProperties`, etc. | ||
fn json_schema_to_columns(json_schema: &serde_json::Value) -> ConnectorResult<Vec<ColumnDesc>> { | ||
let avro_schema = convert_avro(json_schema, Context::default()).to_string(); | ||
let schema = Schema::parse_str(&avro_schema).context("failed to parse avro schema")?; | ||
// TODO: do we need to support map type here? | ||
avro_schema_to_column_descs(&schema, None).map_err(Into::into) | ||
avro_schema_to_column_descs(&schema, Some(MapHandling::Jsonb)).map_err(Into::into) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
} | ||
|
||
#[cfg(test)] | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This looks like a bug: when using external schema,
struct
's fields are not shown indescribe <table>
#17128