-
Notifications
You must be signed in to change notification settings - Fork 570
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: impl key encode
clause for sink
#16377
Conversation
|
GitGuardian id | GitGuardian status | Secret | Commit | Filename | |
---|---|---|---|---|---|
9425213 | Triggered | Generic Password | 3ba944b | e2e_test/source/cdc/cdc.validate.postgres.slt | View secret |
🛠 Guidelines to remediate hardcoded secrets
- Understand the implications of revoking this secret by investigating where it is used in your code.
- Replace and store your secret safely. Learn here the best practices.
- Revoke and rotate this secret.
- If possible, rewrite git history. Rewriting git history is not a trivial act. You might completely break other contributing developers' workflow and you risk accidentally deleting legitimate data.
To avoid such incidents in the future consider
- following these best practices for managing and storing secrets including API keys and other credentials
- install secret detection on pre-commit to catch secret before it leaves your machine and ease remediation.
🦉 GitGuardian detects secrets in your source code to help developers and security teams secure the modern development process. You are seeing this because you or someone else with access to this repository has authorized GitGuardian to scan your pull request.
wait for #16575 merge first. |
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.
Overall LGTM
UpsertTemplate(UpsertFormatter<TemplateEncoder, TemplateEncoder>), | ||
UpsertTextTemplate(UpsertFormatter<TextEncoder, TemplateEncoder>), |
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.
So here we expand the cartesian product of
{Append-only, Upsert} x {Text (key) or not} x {Json, Proto, Template}
Is it possible to separate them?
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.
Possibly. The caller of formatter does not need to know whether it is AvroEncoder
or JsonEncoder
, but it still need to know whether the output of an encoder SerTo
a (utf8) String
or not (Vec<u8>
). So it may be possible to reduce into {Append-only, Upsert} x {String, Vec<u8>} x {String, Vec<u8>}
(or possibly {Append-only, Upsert} x {String, Vec<u8>} x {Vec<u8>}
if all MQs allow payload to be non-utf8 bytes). But the refactor wont fit into this PR. I will give it a try after this. #16628
| DataType::Int16 | ||
| DataType::Int32 | ||
| DataType::Int64 => {} | ||
_ => { |
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.
Please explicitly exhaust all data types here. It helps to you to think about all the data types, and also reminds the other people to maintain this code when new data type is introduced.
And I think many data types are missing, such as Decimal
, Int256
, Timestamp
, etc
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.
Refer to #16377 (comment) for a list of all data types, and why we want to be conservative on some data types initially.
(F::AppendOnly, E::Json, Some(E::Text)) => { | ||
Impl::AppendOnlyTextJson(build(p).await?) | ||
} | ||
(F::AppendOnly, E::Json, None | Some(_)) => Impl::AppendOnlyJson(build(p).await?), |
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.
Some(_)
shall not be ignored silently, although it probably already checked in frontend.
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.
After a second thought, I only keep (F::AppendOnly, E::Json, None)
for append-only json. Other key encode config is invalid and should be rejected (although it is done in the frontend).
pub fn new(schema: Schema, col_indices: Option<Vec<usize>>) -> Self { | ||
Self { |
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.
Prefer to restrict the data type here:
- allowed: bool, int16, int32, int64, varchar, serial, int256
- denied
- float32, float64
1e6::float4
can be either1e+06
or1000000
- interval, struct, list, bytea
- PostgreSQL
ToText
unfriendly to downstream
- PostgreSQL
- jsonb
- float32, float64
- maybe
- decimal
1.0
and1.00
are equal asdecimal
but unequal as text
- date
-0001-01-01
is same as0002-01-01 BC
- time, timestamptz
12:34:56.020
is same as12:34:56.02
(i.e. pad to 3/6/9 or not)
- timestamp
2024-01-01 12:34:56
is neither ISO 8601 nor RFC 3339 but a mixture of both.
- decimal
if let Some(format_desc) = &format_desc | ||
&& let Some(key_encode) = &format_desc.key_encode |
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.
Encoder-specific check in EncoderBuild::build()
and ::new()
. This file can be untouched.
The differences between new
and build
:
New
is supposed to be the minimal parameter required by the encoder itself. So it shall accept a singlecol_index: usize
.Build
is supposed to adapt the common input to the ones required bynew
. So it checks whetherpk_indices
contains exactly one column.
primary_key = 'id') | ||
format plain encode json ( | ||
force_append_only='true' | ||
) key encode text ; |
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.
Ingest it back with include key as
and compare it with expected value?
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.
already included in the test.
Signed-off-by: tabVersion <[email protected]>
Signed-off-by: tabVersion <[email protected]>
Signed-off-by: tabVersion <[email protected]>
Signed-off-by: tabVersion <[email protected]>
Signed-off-by: tabVersion <[email protected]>
Signed-off-by: tabVersion <[email protected]>
let datum = row.datum_at(col_index); | ||
result = datum.to_text_with_type(&self.schema.fields[col_index].data_type); |
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.
Second point of #16377 (comment) unresolved:
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.
Oh I thought it is to_text_with_type
's default behavior. Let me fix.
Signed-off-by: tabVersion <[email protected]>
Canal, | ||
// Keyword::UPSERT | ||
Upsert, | ||
// Keyword::PLAIN |
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.
Is it intended? Or just a formatter issue?
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.
It seems a RustRover's bug, it has its own formatter instead of using the rust toolchain one.
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.
I remember there is an option like "use reustfmt instead of the built-in formatter". Btw, don't forget to fix this
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.
Fixed already
Signed-off-by: tabVersion <[email protected]>
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.
Rest LGTM
let data_type = &self.schema.fields[col_index].data_type; | ||
if data_type == &DataType::Boolean { | ||
result = datum | ||
.unwrap_or(ScalarRefImpl::Bool(false)) |
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.
Good point ... I do not feel outputting false
is right when the input is null. I just checked ToText for Datum
and it is outputting NULL
. Maybe keep consistent here.
As for PostgreSQL, there is no text or binary format for null. It is always indicated by a side channel.
- The
typoutput
procedure for eachpg_type
always returnnull::varchar
. - In
psql
it is empty string by default but customizable via\pset null
. - The
typoutput
of an array element (rather than the whole array) outputsNULL
. - The
typoutput
of a record field (rather than the whole record) outputs empty string.
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.
fixed. And thanks for explaining the pg behavior. Then why we make an exception for the boolean type? Directly using the ToText
trait meets the behavior above and is consistent with other types.
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.
why we make an exception for the boolean type
PostgreSQL ToText |
PostgreSQL v::varchar |
Popular in other programming languages | TextEncoder |
|
---|---|---|---|---|
bool | t /f |
true /false |
true /false |
follow v::varchar |
list | {1,2,3} |
{1,2,3} |
[1,2,3] |
disallowed |
int | 49 |
49 |
49 |
49 (rather than 0x31 ) |
So actually we only allowed unambiguous data types 😂 Or we could say PostgreSQL also made an exception for bool in v::varchar
which we follow, while it is unfortunate there is no such trait ...
Signed-off-by: tabVersion <[email protected]>
Signed-off-by: tabVersion <[email protected]>
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.
LGTM!
Canal, | ||
// Keyword::UPSERT | ||
Upsert, | ||
// Keyword::PLAIN |
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.
I remember there is an option like "use reustfmt instead of the built-in formatter". Btw, don't forget to fix this
I hereby agree to the terms of the RisingWave Labs, Inc. Contributor License Agreement.
What's changed and what's your intention?
as title, introducing
key encode
clause in sink statement.the whole statement looks like:
Checklist
./risedev check
(or alias,./risedev c
)Documentation
Release note
If this PR includes changes that directly affect users or other significant modifications relevant to the community, kindly draft a release note to provide a concise summary of these changes. Please prioritize highlighting the impact these changes will have on users.
adding the
key encode
at the end of the sink statement.The key encode can only be
TEXT
now, and the primary key specified in SQL should be one and only one of the types (varchar
,bool
,small int
,int
,big int
)