Hi Yuxia, Martijin,

Thanks for your feedback on FLIP-237!
My understanding is that FLIP-237 better focuses on thrift
encoding/decoding in Datastream/Table API/ Pyflink.
To address feedbacks, I made follow changes to FLIP-237
<https://cwiki.apache.org/confluence/display/FLINK/FLIP-237%3A+Thrift+Format+Support>
 doc

   - remove table schema section inference as flink doesn't have built-in
   support yet
   - remove paritialser/deser given this fits better as a kafka table
   source optimization that apply to various of encoding formats
   - align implementation with protol-buf flink support to keep code
   consistency

Please give another pass and let me know if you have any questions.

Chen

On Mon, May 30, 2022 at 6:34 PM Chen Qin <qinnc...@gmail.com> wrote:

>
>
> On Mon, May 30, 2022 at 7:35 AM Martijn Visser <martijnvis...@apache.org>
> wrote:
>
>> Hi Chen,
>>
>> I think the best starting point would be to create a FLIP [1]. One of the
>> important topics from my point of view is to make sure that such changes
>> are not only available for SQL users, but are also being considered for
>> Table API, DataStream and/or Python. There might be reasons why not to do
>> that, but then those considerations should also be captured in the FLIP.
>>
>> > thanks for piointer, working on Flip-237, stay tune
>
>> Another thing that would be interesting is how Thrift translates into
>> Flink
>> connectors & Flink formats. Or is your Thrift implementation only a
>> connector?
>>
> > it's flink-format for most part, hope it can help with pyflink not sure.
>
>>
>> Best regards,
>>
>> Martijn
>>
>> [1]
>>
>> https://cwiki.apache.org/confluence/display/FLINK/Flink+Improvement+Proposals
>>
>> Op zo 29 mei 2022 om 19:06 schreef Chen Qin <qinnc...@gmail.com>:
>>
>> > Hi there,
>> >
>> > We would like to discuss and potentially upstream our thrift support
>> > patches to flink.
>> >
>> > For some context, we have been internally patched flink-1.11.2 to
>> support
>> > FlinkSQL jobs read/write to thrift encoded kafka source/sink. Over the
>> > course of last 12 months, those patches supports a few features not
>> > available in open source master, including
>> >
>> >    - allow user defined inference thrift stub class name in table DDL,
>> >    Thrift binary <-> Row
>> >    - dynamic overwrite schema type information loaded from HiveCatalog
>> >    (Table only)
>> >    - forward compatible when kafka topic encode with new schema (adding
>> new
>> >    field)
>> >    - backward compatible when job with new schema handles input or state
>> >    with old schema
>> >
>> > With more FlinkSQL jobs in production, we expect maintenance of
>> divergent
>> > feature sets to increase in the next 6-12 months. Specifically
>> challenges
>> > around
>> >
>> >    - lack of systematic way to support inference based table/view ddl
>> >    (parity with hiveql serde
>> >    <
>> >
>> https://cwiki.apache.org/confluence/display/hive/serde#:~:text=SerDe%20Overview,-SerDe%20is%20short&text=Hive%20uses%20the%20SerDe%20interface,HDFS%20in%20any%20custom%20format
>> > .>
>> >    )
>> >    - lack of robust mapping from thrift field to row field
>> >    - dynamic update set of table with same inference class when
>> performing
>> >    schema change (e.g adding new field)
>> >    - minor lack of handle UNSET case, use NULL
>> >
>> > Please kindly provide pointers around the challenges section.
>> >
>> > Thanks,
>> > Chen, Pinterest.
>> >
>>
>

Reply via email to