Re: Questions about Stream SQL plan

2016-06-14 Thread Cody Innowhere
Thanks Fabian, this clears my confusion. On Tue, Jun 14, 2016 at 3:21 PM, Fabian Hueske wrote: > Hi Cody, > > the monotone (or quasi-monotone) attribute required for grouping a stream > in Calcite is a generalization of the timestamp/watermark concept in Flink. > The timestamps in Flink are quas

Re: Questions about Stream SQL plan

2016-06-14 Thread Fabian Hueske
Hi Cody, the monotone (or quasi-monotone) attribute required for grouping a stream in Calcite is a generalization of the timestamp/watermark concept in Flink. The timestamps in Flink are quasi-monotone, i.e., they are increasing but might be slightly out of order. This out-of-orderness is controll

Questions about Stream SQL plan

2016-06-13 Thread Cody Innowhere
Hi guys, I went through Stream SQL doc on calcite website and have a little question about grouping. calcite's grouping requires that a table column must be monotonic or quasi-monotonic while in real world cases we don't necessarily have such fields in streams, unless we use a virtual field, say, t

Re: About Stream SQL

2016-02-23 Thread Julian Hyde
Sorry I misunderstood. As for ways to tell the system that it can make progress, the more the merrier. There's not a "best" mechanism. It depends on the business problem. A good engine should support several, including fully-ordered columns, punctuation, and slack, and let users chose on a per-stre

Re: About Stream SQL

2016-02-23 Thread Milinda Pathirage
Hi Julian, I agree with you. Calcite should stay away from physical properties of stream as much as possible. I was just trying to clarify the confusion regarding the punctuations and watermarks. My last question was not related to Calcite, rather to Flink and other implementations. Sorry for the

Re: About Stream SQL

2016-02-23 Thread Julian Hyde
As the author of the streaming SQL specification, I don't care at all how the system deduces that it is able to make progress. Just as the authors of the SQL standard don't care whether a vendor chooses to store records sorted and/or compressed. All the streaming SQL validator/optimizer needs to k

Re: About Stream SQL

2016-02-23 Thread Milinda Pathirage
Thank you Julian for the document. [1] is also a good read on punctuation. What I understood from reading [1] and MillWheel paper is that a low-watermark (or row-time bound) is a property maintained by operators and operators derive low-watermark by processing punctuations. One other thing mentio

Re: About Stream SQL

2016-02-22 Thread Julian Hyde
I’ve updated the Streaming reference guide as Fabian requested: http://calcite.apache.org/docs/stream.html Julian > On Feb 19, 2016, at 3:11 PM, Julian Hyde wrote: > > I gave a talk about streaming SQL at a Samza meetup. A lot of it is about the >

Re: About Stream SQL

2016-02-19 Thread Julian Hyde
I gave a talk about streaming SQL at a Samza meetup. A lot of it is about the semantics of streaming SQL, and I cover some ground that I don’t cover in the streams page[1]. The news item[2] gets you to both slides and video. In other news, I notice[3] that Spark 2.1 will contain “continuous SQL

Re: About Stream SQL

2016-02-17 Thread Milinda Pathirage
Hi Fabian, We did some work on stream joins [1]. I tested stream-to-relation joins with Samza. But not stream-to-stream joins. But never updated the streaming documentation. I'll send a pull request with some documentation on joins. Thanks Milinda [1] https://issues.apache.org/jira/browse/CALCIT

Re: About Stream SQL

2016-02-17 Thread Fabian Hueske
Hi, I agree, the Streaming page is a very good starting point for this discussion. As suggested by Julian, I created CALCITE-1090 to update the page such that it reflects the current state of the discussion (adding HOP and TUMBLE functions, punctuations). I can also help with that, e.g., by contri

Re: About Stream SQL

2016-02-14 Thread Julian Hyde
Fabian, Apologies for the late reply. I would rather that the specification for streaming SQL was not too prescriptive for how late events were handled. Approaches 1, 2 and 3 are all viable, and engines can differentiate themselves by the strength of their support for this. But for the SQL to

Re: About Stream SQL

2016-02-14 Thread Julian Hyde
件人: Fabian Hueske [mailto:fhue...@gmail.com] > 发送时间: 2016年2月6日 17:29 > 收件人: dev@calcite.apache.org > 主题: Re: About Stream SQL > > Excellent! I missed the punctuations in the todo list. > > What kind of strategies do you have in mind to handle events that arrive too > la

答复: About Stream SQL

2016-02-13 Thread Wanglan (Lan)
Hueske [mailto:fhue...@gmail.com] 发送时间: 2016年2月6日 17:29 收件人: dev@calcite.apache.org 主题: Re: About Stream SQL Excellent! I missed the punctuations in the todo list. What kind of strategies do you have in mind to handle events that arrive too late? I see 1. dropping of late events 2. computing an

答复: About Stream SQL

2016-02-13 Thread Wanglan (Lan)
Hueske [mailto:fhue...@gmail.com] 发送时间: 2016年2月6日 17:29 收件人: dev@calcite.apache.org 主题: Re: About Stream SQL Excellent! I missed the punctuations in the todo list. What kind of strategies do you have in mind to handle events that arrive too late? I see 1. dropping of late events 2. computing an

答复: About Stream SQL

2016-02-13 Thread Wanglan (Lan)
Hueske [mailto:fhue...@gmail.com] 发送时间: 2016年2月6日 17:29 收件人: dev@calcite.apache.org 主题: Re: About Stream SQL Excellent! I missed the punctuations in the todo list. What kind of strategies do you have in mind to handle events that arrive too late? I see 1. dropping of late events 2. computing an

Re: About Stream SQL

2016-02-06 Thread Fabian Hueske
Excellent! I missed the punctuations in the todo list. What kind of strategies do you have in mind to handle events that arrive too late? I see 1. dropping of late events 2. computing an updated window result for each late arriving element (implies that the window state is stored for a certain per

Re: About Stream SQL

2016-02-05 Thread Julian Hyde
Let me rephrase: The *majority* of the literature, of which I cited just one example, calls them punctuation, and a couple of recent papers out of Mountain View doesn't change that. There are some fine distinctions between punctuation, heartbeats, watermarks and rowtime bounds, mostly in terms of

Re: About Stream SQL

2016-02-05 Thread Ted Dunning
On Fri, Feb 5, 2016 at 5:10 PM, Julian Hyde wrote: > Yes, watermarks, absolutely. The "to do" list has "punctuation", which > is the same thing. (Actually, I prefer to call it "rowtime bound" > because it is feels more like a dynamic constraint than a piece of > data, but the literature[1] calls

Re: About Stream SQL

2016-02-05 Thread Julian Hyde
Yes, watermarks, absolutely. The "to do" list has "punctuation", which is the same thing. (Actually, I prefer to call it "rowtime bound" because it is feels more like a dynamic constraint than a piece of data, but the literature[1] calls them punctuation.) If a stream has punctuation enabled then

Re: About Stream SQL

2016-02-05 Thread Fabian Hueske
Hi, first of all, thanks for starting this discussion. As Stephan said before, the Flink community is working towards support for SQL on streams. IMO, it would be very nice if the different efforts for SQL on streams could converge towards a core set of semantics and syntax. I read the proposal o

Re: About Stream SQL

2016-02-05 Thread Julian Hyde
Stephan, I agree that we have a long way to go to get to a standard, but I think that means that we should start as soon as possible. By which I mean, rather than going ahead and creating Flink-specific extensions, let's have discussions about SQL extensions in a broad forum, pulling in members of

Re: About Stream SQL

2016-02-04 Thread Stephan Ewen
Hi! True, the Flink community is looking into stream SQL, and is currently building on top of Calcite. This is all going well, but we probably need some custom syntax around windowing. For Stream SQL Windowing, what I have seen so far in Calcite (correct me if I am wrong there), is pretty much a

Re: About Stream SQL

2016-02-04 Thread Julian Hyde
I totally agree with you. (Sorry for the delayed response; this week has been very busy.) There is a tendency of vendors (and projects) to think that their technology is unique, and superior to everyone else’s, and want to showcase it in their dialect of SQL. That is natural, and it’s OK, since

About Stream SQL

2016-01-29 Thread Wanglan (Lan)
Hi to all, I am from Huawei and am focusing on data stream processing. Recently I noticed that both in Storm community and Flink community there are endeavors to user Calcite as SQL parser to enable Storm/Flink to support SQL. They both want to supplemented or clarify Streaming SQL of calcite, e

About Stream SQL

2016-01-29 Thread Wanglan (Lan)
Hi to all, I am from Huawei and am focusing on data stream processing. Recently I noticed that both in Storm community and Flink community there are endeavors to user Calcite as SQL parser to enable Storm/Flink to support SQL. They both want to supplemented or clarify Streaming SQL of calcite, e