IMO, it is better to have one feature that is reasonably well developed
instead of two half-baked features. That's why I proposed to advance the
Table API a bit further before starting the next big thing. I played around
with the Table API recently and I think it definitely needs a bit more
contributor attention and more features to be actually usable. Also since
all features of the SQL interface need to be included in the Table API
(given we follow the SQL on Table approach) it makes sense IMO to push the
Table API a bit further before going for the next thing.

2015-05-27 16:06 GMT+02:00 Stephan Ewen <se...@apache.org>:

> I see no reason why a SQL interface cannot be "bootstrapped" concurrently.
> It would initially not support many operations,
> but would act as a good source to test and drive functionality from the
> Table API.
>
>
> @Ted:
>
> I would like to learn a bit more about the stack and internal abstractions
> of Drill. It may make sense to
> reuse some of the query execution operators from Drill. I especially like
> the "learning schema on the fly" part of drill.
>
> Flink DataSets and Streams have a schema, but it may in several cases be a
> "schema lower bound", like the greatest common superclass.
> Those cases may benefit big time from Drill's ability to refine schema on
> the fly.
>
> That may be useful also in the Table API, making it again available to
> LINQ-like programs, and SQL scripts.
>
> On Wed, May 27, 2015 at 3:49 PM, Robert Metzger <rmetz...@apache.org>
> wrote:
>
> > I didn't know that paper...  Thanks for sharing.
> >
> > I've worked on a SQL layer for Stratosphere some time ago, using Apache
> > Calcite (called Optiq back then). I think the project provides a lot of
> > very good tooling for creating a SQL layer. So if we decide to go for SQL
> > on Flink, I would suggest to use Calcite.
> > I can also help you a bit with Calcite to get started with it.
> >
> > I agree with Fabian that it would probably make more sense for now to
> > enhance the Table API.
> > I think the biggest limitation right now is that it only supports POJOs.
> > We should also support Tuples (I know thats difficult to do), data from
> > HCatalog (that includes parquet & orc), JSON, ...
> > Then, I would add filter and projection pushdown into the table API.
> >
> >
> >
> > On Tue, May 26, 2015 at 10:03 PM, Ted Dunning <ted.dunn...@gmail.com>
> > wrote:
> >
> > > It would also be relatively simple (I think) to retarget drill to Flink
> > if
> > > Flink doesn't provide enough typing meta-data to do traditional SQL.
> > >
> > >
> > >
> > > On Tue, May 26, 2015 at 12:52 PM, Fabian Hueske <fhue...@gmail.com>
> > wrote:
> > >
> > > > Hi,
> > > >
> > > > Flink's Table API is pretty close to what SQL provides. IMO, the best
> > > > approach would be to leverage that and build a SQL parser (maybe
> > together
> > > > with a logical optimizer) on top of the Table API. Parser (and
> > optimizer)
> > > > could be built using Apache Calcite which is providing exactly this.
> > > >
> > > > Since the Table API is still a fairly new component and not very
> > feature
> > > > rich, it might make sense to extend and strengthen it before putting
> > > > something major on top.
> > > >
> > > > Cheers, Fabian
> > > >
> > > > 2015-05-26 21:38 GMT+02:00 Timo Walther <twal...@apache.org>:
> > > >
> > > > > Hey everyone,
> > > > >
> > > > > I would be interested in having a complete SQL API in Flink. How is
> > the
> > > > > status there? Is someone already working on it? If not, I would
> like
> > to
> > > > > work on it. I found
> http://ijcsi.org/papers/IJCSI-12-1-1-169-174.pdf
> > > but
> > > > > I couldn't find anything on the mailing list or Jira. Otherwise I
> > would
> > > > > open an issue and start a discussion about it there.
> > > > >
> > > > > Regards,
> > > > > Timo
> > > > >
> > > >
> > >
> >
>

Reply via email to