Hi Chen, Everything on connector externalization is documented at https://cwiki.apache.org/confluence/display/FLINK/Externalized+Connector+development, including the links to the relevant discussions on that topic in the community.
Thanks, Martijn On Mon, Nov 28, 2022 at 11:59 AM Chen Qin <qinnc...@gmail.com> wrote: > Hi Martijn, > > I feel our proposal “shading libthrift in hive connector” seems pinching a > new problem “how to externalization connectors”. I assume there might be > some discussion in community already. If so please kindly pass some > contexts. > > I would incline take back shading proposal at this point. If user choose to > use flink hive connector and thrift format, they should be responsible to > keep libthrift version in sync. > > Chen > > > > On Mon, Nov 28, 2022 at 00:27 Martijn Visser <martijnvis...@apache.org> > wrote: > > > Hi Chen, > > > > While I agree that Hive Metastore is a crucial component for a lot of > > companies, this isn't the case for all companies. Right now it sounds > like > > Flink has to take on tech debt because users of Flink are running on > older > > versions of the Hive Metastore. I don't think that's a good idea at all. > > Like I said, we want to externalize the Hive connector so there's no root > > level config then available anymore. How would it then work? > > > > Best regards, > > > > Martijn > > > > On Sun, Nov 27, 2022 at 4:02 AM Chen Qin <qinnc...@gmail.com> wrote: > > > > > Hi Martjin, > > > > > > "shading Thrift libraries from the Hive connector" > > > Hivemetastore is foundational software running in many companies used > by > > > Spark/Flink... etc. Upgrading the hive metastore touches many pieces of > > > data engineering. If the user updates flink job jar dependency to the > > > latest 0.17, it would not guarantee both HMS and jar would work > properly. > > > and yes, 0.5-p6 is unfortunate internal tech debt we would work on > > outside > > > of this FLIP work. > > > > > > "KafkaSource and KafkaSink" > > > sounds good, this part seems outdated. > > > > > > "explain how a Thrift schema can be compiled/used in a SQL" > > > I see, our approach requires extra schema gen and jar load compared to > > > proto-buf implementation. Our internal implementation contains a > schema > > > inference patch that got moved out of this FLIP document. I agree it > > might > > > be worth removing compile requirement for ease of use. > > > > > > Chen > > > > > > > > > On Wed, Nov 23, 2022 at 6:42 AM Martijn Visser < > martijnvis...@apache.org > > > > > > wrote: > > > > > > > Hi Chen, > > > > > > > > I'm a bit skeptical of shading Thrift libraries from the Hive > > connector, > > > > especially with the plans to externalize connectors (including Hive). > > > Have > > > > we considered getting the versions in sync to avoid the need of any > > > > shading? > > > > > > > > > > The FLIP also shows a version of Thrift (0.5.0-p6) that I don't see > in > > > > Maven central, but the latest version there is 0.17.0. We should > > support > > > > the latest version. Do you know when Thrift expects to reach a major > > > > version? I'm not too fond of not having any major > version/compatibility > > > > guarantees. > > > > > > > > The FLIP mentions FlinkKafkaConsumer and FlinkKafkaProducer; these > are > > > > deprecated and should not be implemented, only KafkaSource and > > KafkaSink. > > > > > > > > Can you explain how a Thrift schema can be compiled/used in a SQL > > > > application, like also is done for Protobuf? > > > > > > > > > > > > > > https://nightlies.apache.org/flink/flink-docs-stable/docs/connectors/table/formats/protobuf/ > > > > > > > > Best regards, > > > > > > > > Martijn > > > > > > > > On Tue, Nov 22, 2022 at 6:44 PM Chen Qin <qinnc...@gmail.com> wrote: > > > > > > > > > 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. > > > > >>> > > > > > >>> > > > > >> > > > > > > > > > >