Hi!
That does not sound right, I agree. Can you tell us a bit more?
- What version of Flink are you using?
- I assume the NIO loop is executed by a Netty thread. Can you tell us
whether it is from a "io.netty.*" thread, or a "org.jboss.netty.*" thread?
The former is from Flink's data network thr
Andra Lungu created FLINK-1976:
--
Summary: Add ForwardedFields* hints for the optimizer
Key: FLINK-1976
URL: https://issues.apache.org/jira/browse/FLINK-1976
Project: Flink
Issue Type: Wish
Andra Lungu created FLINK-1975:
--
Summary: Graph getUndirected improvement
Key: FLINK-1975
URL: https://issues.apache.org/jira/browse/FLINK-1975
Project: Flink
Issue Type: Improvement
>From Customers AS c1, Customers AS c2
this is two dataset how can use filter function implemented this
On Tuesday, May 5, 2015 9:30 AM, Matthias J. Sax
wrote:
If the sub-query is not correlated, you can (always) transform it into a
simple join query.
However, the query you show is
If the sub-query is not correlated, you can (always) transform it into a
simple join query.
However, the query you show is no valid SQL... Did you mean:
SELECT C_CUSTKEY,C_NAME
FROM Customers
WHERE C_MKTSEGMENT =
(SELECT C_MKTSEGMENT FROM Customers WHERE C_ADDRESS="MG9kdTD2WBHm")
In this case,
how can implemented subquery in flinkexample
SELECT C_CUSTKEY,C_NAME
FROM Customers where C_MKTSEGMENT=(select C_CUSTKEY,C_MKTSEGMENT from Customers
where C_ADDRESS="MG9kdTD2WBHm")
Hi everyone,
Everytime when I am running jvisualvm on one of the machines in our cluster
during a Flink job, I see that NioEventLoop.select() is taking 50% to 70% CPU
self-time. I wonder how severe this is. It might be busy-waiting time that
cannot be filled otherwise, but I wanted to ask you i
Johannes created FLINK-1974:
---
Summary: JobExecutionResult NetRuntime - document the type or the
result
Key: FLINK-1974
URL: https://issues.apache.org/jira/browse/FLINK-1974
Project: Flink
Issue Ty
So I gather I should go forward with this? If no-one objects I will
open a Jira and work on this.
On Tue, May 5, 2015 at 4:14 PM, Aljoscha Krettek wrote:
> Yes, because the handling of punctuations depends on the operator: A
> MapOperator can just forward them while a windowed join or reduce can
Yes, because the handling of punctuations depends on the operator: A
MapOperator can just forward them while a windowed join or reduce can
only forward them after emitting the correct windows or results.
On Tue, May 5, 2015 at 3:58 PM, Paris Carbone wrote:
> By watermark handling I meant making p
By watermark handling I meant making punctuations explicit and
forwarding/modifying them at the operator level. I think this is clear so far.
> On 05 May 2015, at 15:41, Aljoscha Krettek wrote:
>
> There is no watermark handling yet. :D
>
> But this would enable me to do this.
>
> On Tue, May
Does the operator have to know about barriers actually?
My first intuition would be that the operator reacts to a barrier the same
way as to a punctuation/watermark.
The outside driver handles the barriers as follows
1) Punctuate operator
2) Draw operator state snapshot
3) send output barriers
I agree with Gyula on this one. Barriers should better not be exposed to the
operator. They are system events for state management. Apart from that,
watermark handling seems to be on a right track, I like it so far.
> On 05 May 2015, at 15:26, Aljoscha Krettek wrote:
>
> I don't know, I just p
There is no watermark handling yet. :D
But this would enable me to do this.
On Tue, May 5, 2015 at 3:39 PM, Paris Carbone wrote:
> I agree with Gyula on this one. Barriers should better not be exposed to the
> operator. They are system events for state management. Apart from that,
> watermark
oops, meant "iterators" :-)
On Tue, May 5, 2015 at 3:04 PM, Aljoscha Krettek
wrote:
> What do you mean by "losing iterations"?
>
> For the pros and cons:
>
> Cons: I can't think of any, since most of the operators are chainable
> already and already behave like a collector.
>
> Pros:
> - Unifie
Yes, we dont need that method there. Snapshots are handled as a call to the
streamtask from the input reader.
On Tuesday, May 5, 2015, Aljoscha Krettek wrote:
> I don't know, I just put that there because other people are working
> on the checkpointing/barrier thing. So there would need to be so
I don't know, I just put that there because other people are working
on the checkpointing/barrier thing. So there would need to be some
functionality there at some point.
Or maybe it is not required there and can be handled in the
StreamTask. Others might know this better than I do right now.
On
What would the processBarrier method do?
On Tuesday, May 5, 2015, Aljoscha Krettek wrote:
> I'm using the term punctuation and watermark interchangeably here
> because for practical purposes they do the same thing. I'm not sure
> what you meant with your comment about those.
>
> For the Operator
I'm using the term punctuation and watermark interchangeably here
because for practical purposes they do the same thing. I'm not sure
what you meant with your comment about those.
For the Operator interface I'm thinking about something like this:
abstract class OneInputStreamOperator {
publi
I think this a good idea in general. I would try to minimize the methods we
include and make the ones that we keep very concrete. For instance i would
not have the receive barrier method as that is handled on a totally
different level already. And instead of punctuation I would directly add a
metho
What do you mean by "losing iterations"?
For the pros and cons:
Cons: I can't think of any, since most of the operators are chainable
already and already behave like a collector.
Pros:
- Unified model for operators, chainable operators don't have to
worry about input iterators and the collect i
Can you give us a rough idea of the pros and cons? Do we lose some
functionality by getting rid of iterations?
Kostas
On Tue, May 5, 2015 at 1:37 PM, Aljoscha Krettek
wrote:
> Hi Folks,
> while working on introducing source-assigned timestamps into streaming
> (https://issues.apache.org/jira/br
Hi Folks,
while working on introducing source-assigned timestamps into streaming
(https://issues.apache.org/jira/browse/FLINK-1967) I thought about how
the punctuations (low watermarks) can be pushed through the system.
The problem is, that operators can have two ways of getting input: 1.
They read
23 matches
Mail list logo