Hi Xuyang, Timo and Sergey,

Thanks for the feedback! I'm kicking off the vote now.

Kind regards,
Gustavo

On Tue, 24 Mar 2026 at 10:59, Xuyang <[email protected]> wrote:

> Hi, Gustavo.
> Looks good to update the Internal API independently. +1.
>
>
>
>
>
> --
>
>     Best!
>     Xuyang
>
>
>
> At 2026-03-23 20:50:02, "Gustavo de Morais" <[email protected]>
> wrote:
> >Hi Sergey,
> >
> >Thanks for the message! I took a look at our support for UTF_16 and I
> don't
> >think any additional UTF_16 support is necessary for the scope of this
> >FLIP. For UTF-16 users should use Decode instead of Cast. Flink stores all
> >strings as UTF-8 internally, so DECODE(bytes, 'UTF-16') already handles
> >that case - it converts to a Java String and re-encodes as UTF-8 before
> >storage. Since the internal representation is always UTF-8, the validation
> >problem is fundamentally a UTF-8 concern and the functions are correctly
> >named. Adding validation functions support for all the character sets we
> >support for DECODE (‘US-ASCII’, ‘ISO-8859-1’, ‘UTF-8’, ‘UTF-16BE’,
> >‘UTF-16LE’, ‘UTF-16) might be something useful for some less frequent use
> >cases but I'd say this is out-of-scope for this FLIP.
> >
> >Kind regards,
> >Gustavo
> >
> >
> >On Fri, 20 Mar 2026 at 20:24, Sergey Nuyanzin <[email protected]>
> wrote:
> >
> >> Hi Gustavo
> >> thanks for the proposal
> >>
> >> I noticed that you are proposing usage of UTF8 in names (default cast
> >> to string is also using utf8)
> >> however I wonder if it makes sense to introduce similar utf16 similar
> >> functions as Flink supports this as well?
> >>
> >> On Fri, Mar 20, 2026 at 8:09 PM Gustavo de Morais
> >> <[email protected]> wrote:
> >> >
> >> > Hi Xuyang and Timo,
> >> >
> >> > Thanks for the positive feedback! Regarding your suggestions, Xuyang:
> >> >
> >> > 1. Yes, good point - we should add the fromUtf8Bytes(byte[], int, int)
> >> > overload as well.
> >> > 2. This is also relevant. If we want to do validation during ingestion
> >> > time, this might have performance implications. Since these are
> @Internal
> >> > APIs, they can be changed independently from the FLIP afterwards if it
> >> > makes sense. What's your opinion?
> >> >
> >> > Kind regards,
> >> > Gustavo
> >> >
> >> > On Fri, 20 Mar 2026 at 08:44, Xuyang <[email protected]> wrote:
> >> >
> >> > > Hi, Gustavo.
> >> > > Great catch! Thanks for driving this FLIP. Overall LGTM. I just have
> >> two
> >> > > minor points I'd like to confirm with you.
> >> > > 1. Should we also add the overload function `fromUtf8Bytes(byte[],
> int,
> >> > > int)` in StringData?
> >> > > 2. Callers like `ColumnarRowData#getString` and
> >> > > `ColumnarArrayData#getString`  call `StringData.fromBytes` directly.
> >> Should
> >> > > these call sites be migrated in a follow-up, or intentionally left
> >> as-is?
> >> > >
> >> > >
> >> > >
> >> > >
> >> > >
> >> > > --
> >> > >
> >> > >     Best!
> >> > >     Xuyang
> >> > >
> >> > >
> >> > >
> >> > > At 2026-03-19 22:37:28, "Timo Walther" <[email protected]> wrote:
> >> > > >Hi Gustavo,
> >> > > >
> >> > > >thank you for this excellent design document. And thanks for
> >> discovering
> >> > > >this data loss and driving the investigation. We should definitely
> fix
> >> > > >this shortcoming. Also looking at other vendors, it is definitly a
> >> cause
> >> > > >for false assumptions that lead to hard-to-debug inconsistencies.
> >> > > >
> >> > > >+1 for this proposal.
> >> > > >
> >> > > >Cheers,
> >> > > >Timo
> >> > > >
> >> > > >
> >> > > >On 19.03.26 15:23, Gustavo de Morais wrote:
> >> > > >> Hi everyone,
> >> > > >>
> >> > > >> Currently, CAST(bytes AS STRING) silently replaces any invalid
> UTF-8
> >> > > byte
> >> > > >> with U+FFFD (?). The substitution is irreversible and produces no
> >> > > warning -
> >> > > >> the pipeline keeps running while data is permanently corrupted
> >> > > >> downstream. This also means that a CAST from BYTES → STRING →
> BYTES
> >> is
> >> > > not
> >> > > >> idempotent, which prevents the engine from applying certain
> >> > > optimizations.
> >> > > >> For example, for preserving upsert keys after such CASTs.
> >> > > >>
> >> > > >> I'd like to start a discussion around defining and improving the
> >> default
> >> > > >> behavior. I've written a short FLIP [1] proposing new utility
> >> functions
> >> > > to
> >> > > >> handle this explicitly - similar to what other engines like Spark
> >> > > already
> >> > > >> do - and changing the default behavior to throw an error instead
> of
> >> > > >> silently corrupting data, while giving users clear options to
> deal
> >> with
> >> > > >> invalid bytes.
> >> > > >>
> >> > > >> Looking forward to your feedback and thoughts.
> >> > > >>
> >> > > >> Kind regards,
> >> > > >> Gustavo
> >> > > >>
> >> > > >> [1]
> >> > > >>
> >> > >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-568%3A+Strict+BYTES-to-STRING+CAST+with+UTF-8+Validation+Utilities
> >> > > >>
> >> > >
> >>
> >>
> >>
> >> --
> >> Best regards,
> >> Sergey
> >>
>

Reply via email to