[jira] [Created] (FLINK-2869) Apply JMH on IOManagerPerformanceBenchmark class.

2015-10-19 Thread GaoLun (JIRA)
GaoLun created FLINK-2869:
-

 Summary: Apply JMH on IOManagerPerformanceBenchmark class.
 Key: FLINK-2869
 URL: https://issues.apache.org/jira/browse/FLINK-2869
 Project: Flink
  Issue Type: Sub-task
  Components: Tests
Reporter: GaoLun
Assignee: GaoLun
Priority: Minor


JMH is a Java harness for building, running, and analysing 
nano/micro/milli/macro benchmarks.Use JMH to replace the old micro benchmarks 
method in order to get much more accurate results.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: A proposal about skew data handling in Flink

2015-10-19 Thread Fabian Hueske
Hi,

First of all, thanks a lot for this extensive proposal! It contains a lot
of good observations and techniques how to address data skew.

I have a few remarks:

1) The terms Input and Output Contract were introduced in the first
scientific publications and are not used anymore. Input Contract are what
we call operators or transformations today, the concept of output contract
is completely gone.
In the current code, we have operators like Map, Reduce, and Join that
describe how data needs to be organized (by key, etc.) and UDFs that
process the data.

2) I would categorize skew as follows:

- UDF Call Complexity Skew: The input cardinalities of UDF calls differ
(only applicable to group-based operators such as GroupReduce and CoGroup)
or the computational complexity of UDF calls depends on the data and varies
a lot. UDF calls are the smallest parallelizable unit. It is not possible
to change that without changing the semantics. Combiners can help to reduce
the effect of skew for group-based operators.

- Input Partition Skew: The cardinality of parallel partitions varies. This
is handled by Flink as follows:
- Lazy split assignment for data sources
- Operators that do not require special partitioning (Map, Filter,
Cross, etc.) just consume the output partitions of the preceding operator.
Rebalance() can be used to enforce round-robin partitioning to equalize
size of all partitions.
- Operators that require key-based partitioning use hash partitioning.
Range partitioning can help address significant data skew.

- UDF Call Skew: The number of UDF calls per parallel partition varies.
This can be an issue for n-m joins which essentially result in Cartesian
products.
- UDF Call Skew is most relevant for Joins
- UDF Call Skew for Map, Reduce, CoGroup, Cross can be controlled by
controlling Input Partition Skew

3) I agree that we should not try to detect and automatically fix data skew
(at the moment) but give users tools to manually manage skew.

4) I would focus on addressing the Input Partition Skew problem. UDF Call
Complexity Skew cannot be addressed because it would change the semantics
of operators. UDF Call Skew is only affecting joins and much harder to
solve.

5) I wonder how much the practical gain is to address the
Non-Range-Equally-Splittable case compared to the added code complexity. In
general, tackling skew is a very good idea, but solving corner cases with
quite complex methods might make future features more complicated to add.
Hence, I would propose to focus on the common and "easy" cases first.

I would address Input Partition Skew first and ignore the
Non-Range-Equally-Splittable case for now. We can do this in two steps:

1) Add the "simple" range partitioner as in your pull request for unary
operators (explicit range partitioning, total order, groupBy). Once the
sampling happens online, this is a very good addition to Flink.
2) Add the "simple" range partitioner also for binary operators (join,
coGroup). This will be a bit more tricky, because we need to do a
coordinated decision for both inputs.
3) Expose range partitioning for GroupBy, Join, CoGroup to the API, maybe
through optimizer hints.

Since we want to have this transparently handled by the API and engine, we
need to add a lot of these features into the optimizer, or
JobGraphGenerator to be more precisely.

Does that make sense to you?

Cheers, Fabian

2015-10-16 17:13 GMT+02:00 Fabian Hueske :

> Hi,
>
> thanks for starting a discussion about data skew! I agree, it's a
> important issue that can cause a lot of problems.
> I'll have a look at your proposal and add comments soon.
>
> Thanks, Fabian
>
> 2015-10-15 12:24 GMT+02:00 Li, Chengxiang :
>
>> Dear all,
>> In many real world use case, data are nature to be skewed. For example,
>> in social network, famous people get much more "follow" than others, a hot
>> tweet would be transferred millions of times. and the purchased records of
>> normal product can never compared to hot products. While at the same time,
>> Flink runtime assume that all tasks consume same size resources, this's not
>> always true. Skew data handling try to make skewed data fit into Flink's
>> runtime.
>> I write a proposal about skew data handling in Flink, you can read it at
>> https://docs.google.com/document/d/1ma060BUlhXDqeFmviEO7Io4CXLKgrAXIfeDYldvZsKI/edit?usp=sharing
>> .
>> Any comments and feedback are welcome, you can comment on the google doc,
>> or reply this email thread directly.
>>
>> Thanks
>> Chengxiang
>>
>
>


Re: Powered by Flink

2015-10-19 Thread Fabian Hueske
Thanks for starting this Kostas.

I think the list is quite hidden in the wiki. Should we link from
flink.apache.org to that page?

Cheers, Fabian

2015-10-19 14:50 GMT+02:00 Kostas Tzoumas :

> Hi everyone,
>
> I started a "Powered by Flink" wiki page, listing some of the
> organizations that are using Flink:
>
> https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink
>
> If you would like to be added to the list, just send me a short email with
> your organization's name and a description and I will add you to the wiki
> page.
>
> Best,
> Kostas
>


Re: Powered by Flink

2015-10-19 Thread Márton Balassi
Thanks for starting and big +1 for making it more prominent.

On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske  wrote:

> Thanks for starting this Kostas.
>
> I think the list is quite hidden in the wiki. Should we link from
> flink.apache.org to that page?
>
> Cheers, Fabian
>
> 2015-10-19 14:50 GMT+02:00 Kostas Tzoumas :
>
>> Hi everyone,
>>
>> I started a "Powered by Flink" wiki page, listing some of the
>> organizations that are using Flink:
>>
>> https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink
>>
>> If you would like to be added to the list, just send me a short email
>> with your organization's name and a description and I will add you to the
>> wiki page.
>>
>> Best,
>> Kostas
>>
>
>


Powered by Flink

2015-10-19 Thread Kostas Tzoumas
Hi everyone,

I started a "Powered by Flink" wiki page, listing some of the organizations
that are using Flink:

https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink

If you would like to be added to the list, just send me a short email with
your organization's name and a description and I will add you to the wiki
page.

Best,
Kostas


Re: Powered by Flink

2015-10-19 Thread Fabian Hueske
Sounds good +1

2015-10-19 14:57 GMT+02:00 Márton Balassi :

> Thanks for starting and big +1 for making it more prominent.
>
> On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske  wrote:
>
>> Thanks for starting this Kostas.
>>
>> I think the list is quite hidden in the wiki. Should we link from
>> flink.apache.org to that page?
>>
>> Cheers, Fabian
>>
>> 2015-10-19 14:50 GMT+02:00 Kostas Tzoumas :
>>
>>> Hi everyone,
>>>
>>> I started a "Powered by Flink" wiki page, listing some of the
>>> organizations that are using Flink:
>>>
>>> https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink
>>>
>>> If you would like to be added to the list, just send me a short email
>>> with your organization's name and a description and I will add you to the
>>> wiki page.
>>>
>>> Best,
>>> Kostas
>>>
>>
>>
>


Re: Powered by Flink

2015-10-19 Thread Kostas Tzoumas
yes, definitely. How about a link under the Community drop-down that points
to the wiki page?

On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske  wrote:

> Thanks for starting this Kostas.
>
> I think the list is quite hidden in the wiki. Should we link from
> flink.apache.org to that page?
>
> Cheers, Fabian
>
> 2015-10-19 14:50 GMT+02:00 Kostas Tzoumas :
>
>> Hi everyone,
>>
>> I started a "Powered by Flink" wiki page, listing some of the
>> organizations that are using Flink:
>>
>> https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink
>>
>> If you would like to be added to the list, just send me a short email
>> with your organization's name and a description and I will add you to the
>> wiki page.
>>
>> Best,
>> Kostas
>>
>
>


Kafka source stuck while canceling

2015-10-19 Thread Gyula Fóra
Hey guys,

Has anyone ever got something similar working with the kafka sources?

11:52:48,838 WARN  org.apache.flink.runtime.taskmanager.Task
  - Task 'Source: Kafka[***] (3/4)' did not react to cancelling signal,
but is stuck in method:
 org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:208)
org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
java.lang.Thread.run(Thread.java:745)

The failure was caused by a different operator in the pipeline, but the job
could never be fully cancelled and restarted due to this error.

Any idea is appreciated :)

Cheers,
Gyula


Re: Powered by Flink

2015-10-19 Thread Suneel Marthi
+1 to this.

On Mon, Oct 19, 2015 at 3:00 PM, Fabian Hueske  wrote:

> Sounds good +1
>
> 2015-10-19 14:57 GMT+02:00 Márton Balassi :
>
> > Thanks for starting and big +1 for making it more prominent.
> >
> > On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske 
> wrote:
> >
> >> Thanks for starting this Kostas.
> >>
> >> I think the list is quite hidden in the wiki. Should we link from
> >> flink.apache.org to that page?
> >>
> >> Cheers, Fabian
> >>
> >> 2015-10-19 14:50 GMT+02:00 Kostas Tzoumas :
> >>
> >>> Hi everyone,
> >>>
> >>> I started a "Powered by Flink" wiki page, listing some of the
> >>> organizations that are using Flink:
> >>>
> >>> https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink
> >>>
> >>> If you would like to be added to the list, just send me a short email
> >>> with your organization's name and a description and I will add you to
> the
> >>> wiki page.
> >>>
> >>> Best,
> >>> Kostas
> >>>
> >>
> >>
> >
>


Re: Powered by Flink

2015-10-19 Thread Maximilian Michels
+1 Let's collect in the Wiki for now. At some point in time, we might
want to have a dedicated page on the Flink homepage.

On Mon, Oct 19, 2015 at 3:31 PM, Timo Walther  wrote:
> Ah ok, sorry. I think linking to the wiki is also ok.
>
>
> On 19.10.2015 15:18, Fabian Hueske wrote:
>>
>> @Timo: The proposal was to keep the list in the wiki (can be easily
>> extended) but link from the main website to the wiki page.
>>
>> 2015-10-19 15:16 GMT+02:00 Timo Walther :
>>
>>> +1 for adding it to the website instead of wiki.
>>> "Who is using Flink?" is always a question difficult to answer to
>>> interested users.
>>>
>>>
>>> On 19.10.2015 15:08, Suneel Marthi wrote:
>>>
>>> +1 to this.
>>>
>>> On Mon, Oct 19, 2015 at 3:00 PM, Fabian Hueske  wrote:
>>>
 Sounds good +1

 2015-10-19 14:57 GMT+02:00 Márton Balassi < 
 balassi.mar...@gmail.com>:

> Thanks for starting and big +1 for making it more prominent.
>
> On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske < 

 fhue...@gmail.com> wrote:
>>
>> Thanks for starting this Kostas.
>>
>> I think the list is quite hidden in the wiki. Should we link from
>> flink.apache.org to that page?
>>
>> Cheers, Fabian
>>
>> 2015-10-19 14:50 GMT+02:00 Kostas Tzoumas < 

 ktzou...@apache.org>:
>>>
>>> Hi everyone,
>>>
>>> I started a "Powered by Flink" wiki page, listing some of the
>>> organizations that are using Flink:
>>>
>>> https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink
>>>
>>> If you would like to be added to the list, just send me a short email
>>> with your organization's name and a description and I will add you to

 the
>>>
>>> wiki page.
>>>
>>> Best,
>>> Kostas
>>>
>>
>>>
>>>
>


Re: Powered by Flink

2015-10-19 Thread Timo Walther

+1 for adding it to the website instead of wiki.
"Who is using Flink?" is always a question difficult to answer to 
interested users.


On 19.10.2015 15:08, Suneel Marthi wrote:

+1 to this.

On Mon, Oct 19, 2015 at 3:00 PM, Fabian Hueske > wrote:


Sounds good +1

2015-10-19 14:57 GMT+02:00 Márton Balassi
>:

> Thanks for starting and big +1 for making it more prominent.
>
> On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske
> wrote:
>
>> Thanks for starting this Kostas.
>>
>> I think the list is quite hidden in the wiki. Should we link from
>> flink.apache.org  to that page?
>>
>> Cheers, Fabian
>>
>> 2015-10-19 14:50 GMT+02:00 Kostas Tzoumas >:
>>
>>> Hi everyone,
>>>
>>> I started a "Powered by Flink" wiki page, listing some of the
>>> organizations that are using Flink:
>>>
>>> https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink
>>>
>>> If you would like to be added to the list, just send me a
short email
>>> with your organization's name and a description and I will add
you to the
>>> wiki page.
>>>
>>> Best,
>>> Kostas
>>>
>>
>>
>






Re: Powered by Flink

2015-10-19 Thread Timo Walther

Ah ok, sorry. I think linking to the wiki is also ok.

On 19.10.2015 15:18, Fabian Hueske wrote:

@Timo: The proposal was to keep the list in the wiki (can be easily
extended) but link from the main website to the wiki page.

2015-10-19 15:16 GMT+02:00 Timo Walther :


+1 for adding it to the website instead of wiki.
"Who is using Flink?" is always a question difficult to answer to
interested users.


On 19.10.2015 15:08, Suneel Marthi wrote:

+1 to this.

On Mon, Oct 19, 2015 at 3:00 PM, Fabian Hueske  wrote:


Sounds good +1

2015-10-19 14:57 GMT+02:00 Márton Balassi < 
balassi.mar...@gmail.com>:


Thanks for starting and big +1 for making it more prominent.

On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske < 

fhue...@gmail.com> wrote:

Thanks for starting this Kostas.

I think the list is quite hidden in the wiki. Should we link from
flink.apache.org to that page?

Cheers, Fabian

2015-10-19 14:50 GMT+02:00 Kostas Tzoumas < 

ktzou...@apache.org>:

Hi everyone,

I started a "Powered by Flink" wiki page, listing some of the
organizations that are using Flink:

https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink

If you would like to be added to the list, just send me a short email
with your organization's name and a description and I will add you to

the

wiki page.

Best,
Kostas










Re: Powered by Flink

2015-10-19 Thread Fabian Hueske
@Timo: The proposal was to keep the list in the wiki (can be easily
extended) but link from the main website to the wiki page.

2015-10-19 15:16 GMT+02:00 Timo Walther :

> +1 for adding it to the website instead of wiki.
> "Who is using Flink?" is always a question difficult to answer to
> interested users.
>
>
> On 19.10.2015 15:08, Suneel Marthi wrote:
>
> +1 to this.
>
> On Mon, Oct 19, 2015 at 3:00 PM, Fabian Hueske  wrote:
>
>> Sounds good +1
>>
>> 2015-10-19 14:57 GMT+02:00 Márton Balassi < 
>> balassi.mar...@gmail.com>:
>>
>> > Thanks for starting and big +1 for making it more prominent.
>> >
>> > On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske < 
>> fhue...@gmail.com> wrote:
>> >
>> >> Thanks for starting this Kostas.
>> >>
>> >> I think the list is quite hidden in the wiki. Should we link from
>> >> flink.apache.org to that page?
>> >>
>> >> Cheers, Fabian
>> >>
>> >> 2015-10-19 14:50 GMT+02:00 Kostas Tzoumas < 
>> ktzou...@apache.org>:
>> >>
>> >>> Hi everyone,
>> >>>
>> >>> I started a "Powered by Flink" wiki page, listing some of the
>> >>> organizations that are using Flink:
>> >>>
>> >>> https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink
>> >>>
>> >>> If you would like to be added to the list, just send me a short email
>> >>> with your organization's name and a description and I will add you to
>> the
>> >>> wiki page.
>> >>>
>> >>> Best,
>> >>> Kostas
>> >>>
>> >>
>> >>
>> >
>>
>
>
>


Re: Powered by Flink

2015-10-19 Thread Matthias J. Sax
+1

On 10/19/2015 04:05 PM, Maximilian Michels wrote:
> +1 Let's collect in the Wiki for now. At some point in time, we might
> want to have a dedicated page on the Flink homepage.
> 
> On Mon, Oct 19, 2015 at 3:31 PM, Timo Walther  wrote:
>> Ah ok, sorry. I think linking to the wiki is also ok.
>>
>>
>> On 19.10.2015 15:18, Fabian Hueske wrote:
>>>
>>> @Timo: The proposal was to keep the list in the wiki (can be easily
>>> extended) but link from the main website to the wiki page.
>>>
>>> 2015-10-19 15:16 GMT+02:00 Timo Walther :
>>>
 +1 for adding it to the website instead of wiki.
 "Who is using Flink?" is always a question difficult to answer to
 interested users.


 On 19.10.2015 15:08, Suneel Marthi wrote:

 +1 to this.

 On Mon, Oct 19, 2015 at 3:00 PM, Fabian Hueske  wrote:

> Sounds good +1
>
> 2015-10-19 14:57 GMT+02:00 Márton Balassi < 
> balassi.mar...@gmail.com>:
>
>> Thanks for starting and big +1 for making it more prominent.
>>
>> On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske < 
>
> fhue...@gmail.com> wrote:
>>>
>>> Thanks for starting this Kostas.
>>>
>>> I think the list is quite hidden in the wiki. Should we link from
>>> flink.apache.org to that page?
>>>
>>> Cheers, Fabian
>>>
>>> 2015-10-19 14:50 GMT+02:00 Kostas Tzoumas < 
>
> ktzou...@apache.org>:

 Hi everyone,

 I started a "Powered by Flink" wiki page, listing some of the
 organizations that are using Flink:

 https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink

 If you would like to be added to the list, just send me a short email
 with your organization's name and a description and I will add you to
>
> the

 wiki page.

 Best,
 Kostas

>>>


>>



signature.asc
Description: OpenPGP digital signature


[jira] [Created] (FLINK-2870) Add support for accumulating/discarding for Event-Time Windows

2015-10-19 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-2870:
---

 Summary: Add support for accumulating/discarding for Event-Time 
Windows
 Key: FLINK-2870
 URL: https://issues.apache.org/jira/browse/FLINK-2870
 Project: Flink
  Issue Type: Sub-task
  Components: Streaming
Reporter: Aljoscha Krettek
Assignee: Aljoscha Krettek


This would allow to specify whether windows should be discarded after the 
trigger fires or kept in the operator if late elements arrive.

When keeping elements, the user would also have to specify an allowed lateness 
time after which the window contents are discarded without emitting any further 
window evaluation result.

If elements arrive after the allowed lateness they would trigger the window 
immediately with only the one single element.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (FLINK-2872) Update the documentation for Scala part to add readFileOfPrimitives

2015-10-19 Thread Henry Saputra (JIRA)
Henry Saputra created FLINK-2872:


 Summary: Update the documentation for Scala part to add 
readFileOfPrimitives
 Key: FLINK-2872
 URL: https://issues.apache.org/jira/browse/FLINK-2872
 Project: Flink
  Issue Type: Bug
  Components: Documentation
Reporter: Henry Saputra
Assignee: Henry Saputra
Priority: Minor


Currently the Scala part of the ExecutionEnvironment missing 
readFileOfPrimitives to create Dataset from file for primitive types.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


RE: A proposal about skew data handling in Flink

2015-10-19 Thread Li, Chengxiang
Thanks a lot for the comments, Fabian. I agree with you on the plan mostly, 
just add some more thoughts about  Non-Range-Equally-Splittable case here.
1. Let's assume a case which 10% data is skewed on certain key, in this case, 
as long as the parallelism is larger than 10, it would fit into 
Non-Range-Equally-Splittable case. So it should not be very corner case of skew 
issue.
2. In proposal, the solution of Non-Range-Equally-Splittable case is based on 2 
new RangePartitioner and little optimizer logic, which has been touched already 
in the plan #1, #2. It does not require to change anything about the operator 
semantics, so if we have a good partitioner abstraction, I think it does not 
add much complexity for Flink to handle this kind of issue. 
It should not block anything, after finished the simple case, we would have 
more knowledge about the implementation details, then we can look back at this 
issue, and decide whether it's deserved to be resolved at the cost.

Thanks
Chengxiang 
-Original Message-
From: Fabian Hueske [mailto:fhue...@gmail.com] 
Sent: Monday, October 19, 2015 7:15 PM
To: dev@flink.apache.org
Subject: Re: A proposal about skew data handling in Flink

Hi,

First of all, thanks a lot for this extensive proposal! It contains a lot of 
good observations and techniques how to address data skew.

I have a few remarks:

1) The terms Input and Output Contract were introduced in the first scientific 
publications and are not used anymore. Input Contract are what we call 
operators or transformations today, the concept of output contract is 
completely gone.
In the current code, we have operators like Map, Reduce, and Join that describe 
how data needs to be organized (by key, etc.) and UDFs that process the data.

2) I would categorize skew as follows:

- UDF Call Complexity Skew: The input cardinalities of UDF calls differ (only 
applicable to group-based operators such as GroupReduce and CoGroup) or the 
computational complexity of UDF calls depends on the data and varies a lot. UDF 
calls are the smallest parallelizable unit. It is not possible to change that 
without changing the semantics. Combiners can help to reduce the effect of skew 
for group-based operators.

- Input Partition Skew: The cardinality of parallel partitions varies. This is 
handled by Flink as follows:
- Lazy split assignment for data sources
- Operators that do not require special partitioning (Map, Filter, Cross, 
etc.) just consume the output partitions of the preceding operator.
Rebalance() can be used to enforce round-robin partitioning to equalize size of 
all partitions.
- Operators that require key-based partitioning use hash partitioning.
Range partitioning can help address significant data skew.

- UDF Call Skew: The number of UDF calls per parallel partition varies.
This can be an issue for n-m joins which essentially result in Cartesian 
products.
- UDF Call Skew is most relevant for Joins
- UDF Call Skew for Map, Reduce, CoGroup, Cross can be controlled by 
controlling Input Partition Skew

3) I agree that we should not try to detect and automatically fix data skew (at 
the moment) but give users tools to manually manage skew.

4) I would focus on addressing the Input Partition Skew problem. UDF Call 
Complexity Skew cannot be addressed because it would change the semantics of 
operators. UDF Call Skew is only affecting joins and much harder to solve.

5) I wonder how much the practical gain is to address the 
Non-Range-Equally-Splittable case compared to the added code complexity. In 
general, tackling skew is a very good idea, but solving corner cases with quite 
complex methods might make future features more complicated to add.
Hence, I would propose to focus on the common and "easy" cases first.

I would address Input Partition Skew first and ignore the 
Non-Range-Equally-Splittable case for now. We can do this in two steps:

1) Add the "simple" range partitioner as in your pull request for unary 
operators (explicit range partitioning, total order, groupBy). Once the 
sampling happens online, this is a very good addition to Flink.
2) Add the "simple" range partitioner also for binary operators (join, 
coGroup). This will be a bit more tricky, because we need to do a coordinated 
decision for both inputs.
3) Expose range partitioning for GroupBy, Join, CoGroup to the API, maybe 
through optimizer hints.

Since we want to have this transparently handled by the API and engine, we need 
to add a lot of these features into the optimizer, or JobGraphGenerator to be 
more precisely.

Does that make sense to you?

Cheers, Fabian

2015-10-16 17:13 GMT+02:00 Fabian Hueske :

> Hi,
>
> thanks for starting a discussion about data skew! I agree, it's a 
> important issue that can cause a lot of problems.
> I'll have a look at your proposal and add comments soon.
>
> Thanks, Fabian
>
> 2015-10-15 12:24 GMT+02:00 Li, Chengxiang :
>
>>