Hi Xuefu,

Thanks for putting together the overview. I would like to add some more on
top of Timo's comments.
1,2. I agree with Timo that a proper catalog support should also address
the metadata compatibility issues. I was actually wondering if you are
referring to something like utilizing table stats for plan optimization?
4. If the key is to have users integrate Hive UDF without code changes to
Flink UDF, it shouldn't be a problem as Timo mentioned. Is your concern
mostly on the support of Hive UDFs that should be implemented in
Flink-table natively?
7,8. Correct me if I am wrong, but I feel like some of the related
components might have already been discussed in the longer term road map of
FLIP-24 [1]?
9. per Jorn's comment to stay clear from a tight dependency on Hive and
treat it as one "connector" system. Should we also consider treating
JDBC/ODBC driver as part of the component from the connector system instead
of having Flink to provide them?

Thanks,
Rong

[1]. https://cwiki.apache.org/confluence/display/FLINK/FLIP-24+-+SQL+Client

On Thu, Oct 11, 2018 at 12:46 AM Timo Walther <twal...@apache.org> wrote:

> Hi Xuefu,
>
> thanks for your proposal, it is a nice summary. Here are my thoughts to
> your list:
>
> 1. I think this is also on our current mid-term roadmap. Flink lacks a
> poper catalog support for a very long time. Before we can connect
> catalogs we need to define how to map all the information from a catalog
> to Flink's representation. This is why the work on the unified connector
> API [1] is going on for quite some time as it is the first approach to
> discuss and represent the pure characteristics of connectors.
> 2. It would be helpful to figure out what is missing in [1] to to ensure
> this point. I guess we will need a new design document just for a proper
> Hive catalog integration.
> 3. This is already work in progress. ORC has been merged, Parquet is on
> its way [1].
> 4. This should be easy. There was a PR in past that I reviewed but was
> not maintained anymore.
> 5. The type system of Flink SQL is very flexible. Only UNION type is
> missing.
> 6. A Flink SQL DDL is on the roadmap soon once we are done with [1].
> Support for Hive syntax also needs cooperation with Apache Calcite.
> 7-11. Long-term goals.
>
> I would also propose to start with a smaller scope where also current
> Flink SQL users can profit: 1, 2, 5, 3. This would allow to grow the
> Flink SQL ecosystem. After that we can aim to be fully compatible
> including syntax and UDFs (4, 6 etc.). Once the core is ready, we can
> work on the tooling (7, 8, 9) and performance (10, 11).
>
> @Jörn: Yes, we should not have a tight dependency on Hive. It should be
> treated as one "connector" system out of many.
>
> Thanks,
> Timo
>
> [1]
>
> https://docs.google.com/document/d/1Yaxp1UJUFW-peGLt8EIidwKIZEWrrA-pznWLuvaH39Y/edit?ts=5bb62df4#
> [2] https://github.com/apache/flink/pull/6483
>
> Am 11.10.18 um 07:54 schrieb Jörn Franke:
> > Would it maybe make sense to provide Flink as an engine on Hive
> („flink-on-Hive“)? Eg to address 4,5,6,8,9,10. this could be more loosely
> coupled than integrating hive in all possible flink core modules and thus
> introducing a very tight dependency to Hive in the core.
> > 1,2,3 could be achieved via a connector based on the Flink Table API.
> > Just as a proposal to start this Endeavour as independent projects (hive
> engine, connector) to avoid too tight coupling with Flink. Maybe in a more
> distant future if the Hive integration is heavily demanded one could then
> integrate it more tightly if needed.
> >
> > What is meant by 11?
> >> Am 11.10.2018 um 05:01 schrieb Zhang, Xuefu <xuef...@alibaba-inc.com>:
> >>
> >> Hi Fabian/Vno,
> >>
> >> Thank you very much for your encouragement inquiry. Sorry that I didn't
> see Fabian's email until I read Vino's response just now. (Somehow Fabian's
> went to the spam folder.)
> >>
> >> My proposal contains long-term and short-terms goals. Nevertheless, the
> effort will focus on the following areas, including Fabian's list:
> >>
> >> 1. Hive metastore connectivity - This covers both read/write access,
> which means Flink can make full use of Hive's metastore as its catalog (at
> least for the batch but can extend for streaming as well).
> >> 2. Metadata compatibility - Objects (databases, tables, partitions,
> etc) created by Hive can be understood by Flink and the reverse direction
> is true also.
> >> 3. Data compatibility - Similar to #2, data produced by Hive can be
> consumed by Flink and vise versa.
> >> 4. Support Hive UDFs - For all Hive's native udfs, Flink either
> provides its own implementation or make Hive's implementation work in
> Flink. Further, for user created UDFs in Hive, Flink SQL should provide a
> mechanism allowing user to import them into Flink without any code change
> required.
> >> 5. Data types -  Flink SQL should support all data types that are
> available in Hive.
> >> 6. SQL Language - Flink SQL should support SQL standard (such as
> SQL2003) with extension to support Hive's syntax and language features,
> around DDL, DML, and SELECT queries.
> >> 7.  SQL CLI - this is currently developing in Flink but more effort is
> needed.
> >> 8. Server - provide a server that's compatible with Hive's HiverServer2
> in thrift APIs, such that HiveServer2 users can reuse their existing client
> (such as beeline) but connect to Flink's thrift server instead.
> >> 9. JDBC/ODBC drivers - Flink may provide its own JDBC/ODBC drivers for
> other application to use to connect to its thrift server
> >> 10. Support other user's customizations in Hive, such as Hive Serdes,
> storage handlers, etc.
> >> 11. Better task failure tolerance and task scheduling at Flink runtime.
> >>
> >> As you can see, achieving all those requires significant effort and
> across all layers in Flink. However, a short-term goal could  include only
> core areas (such as 1, 2, 4, 5, 6, 7) or start  at a smaller scope (such as
> #3, #6).
> >>
> >> Please share your further thoughts. If we generally agree that this is
> the right direction, I could come up with a formal proposal quickly and
> then we can follow up with broader discussions.
> >>
> >> Thanks,
> >> Xuefu
> >>
> >>
> >>
> >> ------------------------------------------------------------------
> >> Sender:vino yang <yanghua1...@gmail.com>
> >> Sent at:2018 Oct 11 (Thu) 09:45
> >> Recipient:Fabian Hueske <fhue...@gmail.com>
> >> Cc:dev <dev@flink.apache.org>; Xuefu <xuef...@alibaba-inc.com>; user <
> u...@flink.apache.org>
> >> Subject:Re: [DISCUSS] Integrate Flink SQL well with Hive ecosystem
> >>
> >> Hi Xuefu,
> >>
> >> Appreciate this proposal, and like Fabian, it would look better if you
> can give more details of the plan.
> >>
> >> Thanks, vino.
> >>
> >> Fabian Hueske <fhue...@gmail.com> 于2018年10月10日周三 下午5:27写道:
> >> Hi Xuefu,
> >>
> >> Welcome to the Flink community and thanks for starting this discussion!
> Better Hive integration would be really great!
> >> Can you go into details of what you are proposing? I can think of a
> couple ways to improve Flink in that regard:
> >>
> >> * Support for Hive UDFs
> >> * Support for Hive metadata catalog
> >> * Support for HiveQL syntax
> >> * ???
> >>
> >> Best, Fabian
> >>
> >> Am Di., 9. Okt. 2018 um 19:22 Uhr schrieb Zhang, Xuefu <
> xuef...@alibaba-inc.com>:
> >> Hi all,
> >>
> >> Along with the community's effort, inside Alibaba we have explored
> Flink's potential as an execution engine not just for stream processing but
> also for batch processing. We are encouraged by our findings and have
> initiated our effort to make Flink's SQL capabilities full-fledged. When
> comparing what's available in Flink to the offerings from competitive data
> processing engines, we identified a major gap in Flink: a well integration
> with Hive ecosystem. This is crucial to the success of Flink SQL and batch
> due to the well-established data ecosystem around Hive. Therefore, we have
> done some initial work along this direction but there are still a lot of
> effort needed.
> >>
> >> We have two strategies in mind. The first one is to make Flink SQL
> full-fledged and well-integrated with Hive ecosystem. This is a similar
> approach to what Spark SQL adopted. The second strategy is to make Hive
> itself work with Flink, similar to the proposal in [1]. Each approach bears
> its pros and cons, but they don’t need to be mutually exclusive with each
> targeting at different users and use cases. We believe that both will
> promote a much greater adoption of Flink beyond stream processing.
> >>
> >> We have been focused on the first approach and would like to showcase
> Flink's batch and SQL capabilities with Flink SQL. However, we have also
> planned to start strategy #2 as the follow-up effort.
> >>
> >> I'm completely new to Flink(, with a short bio [2] below), though many
> of my colleagues here at Alibaba are long-time contributors. Nevertheless,
> I'd like to share our thoughts and invite your early feedback. At the same
> time, I am working on a detailed proposal on Flink SQL's integration with
> Hive ecosystem, which will be also shared when ready.
> >>
> >> While the ideas are simple, each approach will demand significant
> effort, more than what we can afford. Thus, the input and contributions
> from the communities are greatly welcome and appreciated.
> >>
> >> Regards,
> >>
> >>
> >> Xuefu
> >>
> >> References:
> >>
> >> [1] https://issues.apache.org/jira/browse/HIVE-10712
> >> [2] Xuefu Zhang is a long-time open source veteran, worked or working
> on many projects under Apache Foundation, of which he is also an honored
> member. About 10 years ago he worked in the Hadoop team at Yahoo where the
> projects just got started. Later he worked at Cloudera, initiating and
> leading the development of Hive on Spark project in the communities and
> across many organizations. Prior to joining Alibaba, he worked at Uber
> where he promoted Hive on Spark to all Uber's SQL on Hadoop workload and
> significantly improved Uber's cluster efficiency.
> >>
> >>
>
>

Reply via email to