Re: TaskManager job lifecycle hooks

2017-12-11 Thread Ben Sidhom
Ah, I see. Yes, that should definitely be good enough (at least for a first
pass).

Do you know how most users tend to deploy and use Flink in practice?
Job-scoped clusters seem preferable for many reasons, but it's possible
that that's only really practical when using YARN/Kubernetes/Mesos. Would
we be be cutting out a large population by only "supporting" single-job
clusters?

On Sun, Dec 10, 2017 at 3:42 PM, Aljoscha Krettek 
wrote:

> Hi Ben,
>
> I think that's a good question but I also think that Erons answer is
> sufficient for an initial implementation. We suggest more and more to use a
> single "cluster" per job (either a single per-job YARN cluster or in
> Kubernetes or in Mesos), thus we don't really have to solve the problem of
> efficiently supporting multiple jobs per JobManager. Regarding Job lifetime
> vs. Task lifetime, the open() and close() methods that Eron mentioned are,
> for practical purposes, invoked at the beginning/end of a Job. For example,
> when a streaming pipeline starts the tasks corresponding to operators will
> be launched and they will stay around until the job is finished. I think
> this is different from a system such as Google Dataflow where you will see
> many more small tasks in the life of a streaming job and so it shouldn't be
> that big of a problem for Flink.
>
> What do you think?
>
> Best,
> Aljoscha
>
> > On 8. Dec 2017, at 00:40, Eron Wright  wrote:
> >
> > Could you speak to whether the lifecycle provided by RichFunction
> > (open/close) would fit the requirement?
> >
> > https://ci.apache.org/projects/flink/flink-docs-
> release-1.3/api/java/org/apache/flink/api/common/
> functions/RichFunction.html#open-org.apache.flink.
> configuration.Configuration-
> >
> > On Thu, Dec 7, 2017 at 1:57 PM, Ben Sidhom 
> > wrote:
> >
> >> Hey,
> >>
> >> I'm working on the Apache Beam  portability
> >> story
> >> and trying to figure out how we can get the Flink runner to support
> >> the new portability
> >> API .
> >>
> >> In order to get the runner to work with portable SDKs, we need to be
> able
> >> to spin up and manage user containers from the TaskManagers themselves.
> All
> >> communication with user code (effectively user-defined functions)
> happens
> >> over RPC endpoints between the container and the Flink worker threads.
> >> Unfortunately, we cannot assume that the container images themselves are
> >> small or that they are cheap to start up. For this reason, we cannot
> >> reasonably start and stop these external services once per task (e.g.,
> by
> >> wrapping service lifetimes within mapPartions). In order to support
> >> multiple jobs per JVM (either due to multiple task slots per manager or
> >> multiple jobs submitted to a cluster serially) , we need to know when to
> >> clean up resources associated with a particular job.
> >>
> >> Is there a way to do this in user code? Ideally, this would be something
> >> like a set of per-job startup and shutdown hooks that execute on each
> >> TaskManager that a particular job runs on. If this does not currently
> >> exist, how reasonable would it be to introduce client-facing APIs that
> >> would allow it? Is there a better approach for this lifecycle management
> >> that better fits into the Flink execution model?
> >>
> >> Thanks
> >> --
> >> -Ben
> >>
>
>


-- 
-Ben


[jira] [Created] (FLINK-8241) Remove ResultPartitionWriter related PrepareForTest annotations

2017-12-11 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-8241:
---

 Summary: Remove ResultPartitionWriter related PrepareForTest 
annotations
 Key: FLINK-8241
 URL: https://issues.apache.org/jira/browse/FLINK-8241
 Project: Flink
  Issue Type: Improvement
  Components: Network, Tests
Affects Versions: 1.5.0
Reporter: Chesnay Schepler
Assignee: Chesnay Schepler
 Fix For: 1.5.0


With the latest refactorings around the {{ResultPartitionWriter}} we no longer 
have to use {{@PrepareForTest}} annotations for it.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (FLINK-8240) Create unified interfaces to configure and instatiate TableSources

2017-12-11 Thread Timo Walther (JIRA)
Timo Walther created FLINK-8240:
---

 Summary: Create unified interfaces to configure and instatiate 
TableSources
 Key: FLINK-8240
 URL: https://issues.apache.org/jira/browse/FLINK-8240
 Project: Flink
  Issue Type: New Feature
  Components: Table API & SQL
Reporter: Timo Walther
Assignee: Timo Walther


At the moment every table source has different ways for configuration and 
instantiation. Some table source are tailored to a specific encoding (e.g., 
{{KafkaAvroTableSource}}, {{KafkaJsonTableSource}}) or only support one 
encoding for reading (e.g., {{CsvTableSource}}). Each of them might implement a 
builder or support table source converters for external catalogs.

The table sources should have a unified interface for discovery, defining 
common properties, and instantiation. The {{TableSourceConverters}} provide a 
similar functionality but use a external catalog. We might generialize this 
interface.

In general a table source declaration depends on the following parts:

- Source
  - Type (e.g. Kafka, Custom)
  - Properties (e.g. topic, connection info)
- Encoding
  - Type (e.g. Avro, JSON, CSV)
  - Schema (e.g. Avro class, JSON field names/types)
- Rowtime descriptor/Proctime
  - Watermark strategy and Watermark properties
  - Time attribute info
- Bucketization

This issue needs a design document before implementation. Any discussion is 
very welcome.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: [VOTE] Release 1.4.0, release candidate #3

2017-12-11 Thread Tzu-Li (Gordon) Tai
+1

- Staged Apache source & binary convenience releases looks good
- Built from source (macOS w/ Scala 2.11, hadoop-free, hadoop-2.8)
- Locally tested topic regex subscription for the Kafka consumer
- Quickstart projects looks good

Other things verified that are carried from previous RC votes
- Cluster tests on AWS with configuration detailed in [1], with special
focus on dynamic Kafka partition discovery
- Kinesis connector, Elasticsearch connector runs fine with cluster
execution + locally in IDE, without any dependency clashes

[1]
https://docs.google.com/document/d/1cOkycJwEKVjG_onnpl3bQNTq7uebh48zDtIJxceyU2E/edit#heading=h.sintcv4ccegd

On Mon, Dec 11, 2017 at 9:28 PM, Timo Walther  wrote:

> +1 (binding)
>
> - build the source locally
> - run various table programs
> - checked the resource consumption of table programs with retention
> enabled and disabled
> - built a quickstart project
> - tested the web ui submission (found https://issues.apache.org/jira
> /browse/FLINK-8187 but this is non-blocking)
>
>
> Am 12/11/17 um 2:16 PM schrieb Chesnay Schepler:
>
> +1 (binding)
>>
>> - checked contents of flink-dist for unshaded dependencies
>> - ran python examples (with/-out arguments) locally
>> - ran jobs on yarn on a cluster testing optional hadoop dependency
>> - verified that quickstarts work
>> - checked JM/TM logs for anything suspicious
>>
>> On 11.12.2017 11:29, Fabian Hueske wrote:
>>
>>> +1 (binding)
>>>
>>> - Checked hashes & signatures
>>> - Checked no binaries in source release
>>> - Checked Flink version in Quickstart pom files
>>>
>>> Cheers, Fabian
>>>
>>> 2017-12-11 11:26 GMT+01:00 Stefan Richter :
>>>
>>> +1 (non-binding)

 - did extensive cluster tests on Google Cloud with special focus on
 checkpointing and recovery and Kafka 0.11 end-to-end exactly-once +
 at-least-once.
 - build from source.

 Am 11.12.2017 um 09:53 schrieb Piotr Nowojski  >:
>
> Hi,
>
> +1 (non-binding)
>
> I have:
> - verified Scala and Java sample projects are creating and working
>
 properly and that Quickstart docs are ok

> - verified that ChildFirstClassloader allows user to run his
> application
>
 with some custom akka version

> - tested Kafka 0.11 end to end exactly once
> - did some manual checks whether docs/distribution files are ok
>
> Piotrek
>
> On 8 Dec 2017, at 16:49, Stephan Ewen  wrote:
>>
>> @Eron Given that this is actually an undocumented "internal" feature
>> at
>> this point, I would not expect that it is used heavily beyond Pravega.
>>
>> Unless you feel strongly that this is a major issue, I would go ahead
>>
> with

> the release...
>>
>> On Fri, Dec 8, 2017 at 3:18 PM, Aljoscha Krettek > >
>> wrote:
>>
>> Thanks for the update! I would also say it's not a blocker but we
>>>
>> should

> make sure that we don't break this after 1.4, then.
>>>
>>> On 7. Dec 2017, at 22:37, Eron Wright  wrote:

 Just discovered:  the removal of Flink's Future (FLINK-7252) causes
 a
 breaking change in connectors that use
 `org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook`,

>>> because

> `Future` is a type on one of the methods.

 To my knowledge, this affects only the Pravega connector.  Curious
 to

>>> know
>>>
 whether any other connectors are affected.  I don't think we (Dell

>>> EMC)

> consider it a blocker but it will mean that the connector is Flink

>>> 1.4+.

> Eron


 On Thu, Dec 7, 2017 at 12:25 PM, Aljoscha Krettek <

>>> aljos...@apache.org>

> wrote:

 I just noticed that I did a copy-and-paste error and the last
>
 paragraph

> about voting period should be this:
>
> The vote will be open for at least 72 hours. It is adopted by
>
 majority

> approval, with at least 3 PMC affirmative votes.
>
> Best,
> Aljoscha
>
> On 7. Dec 2017, at 19:24, Bowen Li 
>> wrote:
>>
>> I agree that it shouldn't block the release. The doc website part
>> is
>>
> even
>>>
 better!
>>
>> On Thu, Dec 7, 2017 at 1:09 AM, Aljoscha Krettek <
>>
> aljos...@apache.org>

> wrote:
>>
>> Good catch, yes. This shouldn't block the release, though, since
>>>
>> the

> doc
>>>
 is always built form the latest state of a release branch, i.e. the
>>>
>> 1.4
>>>
 doc

[jira] [Created] (FLINK-8239) Extend StreamTaskTestHarness to support TwoInput head operators

2017-12-11 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-8239:
---

 Summary: Extend StreamTaskTestHarness to support TwoInput head 
operators
 Key: FLINK-8239
 URL: https://issues.apache.org/jira/browse/FLINK-8239
 Project: Flink
  Issue Type: Improvement
  Components: Streaming, Tests
Affects Versions: 1.5.0
Reporter: Chesnay Schepler
Assignee: Chesnay Schepler
 Fix For: 1.5.0






--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (FLINK-8238) StreamTaskTestHarness should only allow one setup call

2017-12-11 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-8238:
---

 Summary: StreamTaskTestHarness should only allow one setup call
 Key: FLINK-8238
 URL: https://issues.apache.org/jira/browse/FLINK-8238
 Project: Flink
  Issue Type: Improvement
  Components: Streaming, Tests
Affects Versions: 1.5.0
Reporter: Chesnay Schepler
 Fix For: 1.5.0


The {{StreamTaskTestHarness}} offers 2 methods for settting up an 
{{OperatorChain}}:
* setupOutputForSingletonOperatorChain
* setupOperatorChain

We should add a check to make sure that only one of these methods is called. 
Calling both methods leads to undefined behavior by the task and is a bit 
tricky to debug.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: [VOTE] Release 1.4.0, release candidate #3

2017-12-11 Thread Timo Walther

+1 (binding)

- build the source locally
- run various table programs
- checked the resource consumption of table programs with retention 
enabled and disabled

- built a quickstart project
- tested the web ui submission (found 
https://issues.apache.org/jira/browse/FLINK-8187 but this is non-blocking)



Am 12/11/17 um 2:16 PM schrieb Chesnay Schepler:

+1 (binding)

- checked contents of flink-dist for unshaded dependencies
- ran python examples (with/-out arguments) locally
- ran jobs on yarn on a cluster testing optional hadoop dependency
- verified that quickstarts work
- checked JM/TM logs for anything suspicious

On 11.12.2017 11:29, Fabian Hueske wrote:

+1 (binding)

- Checked hashes & signatures
- Checked no binaries in source release
- Checked Flink version in Quickstart pom files

Cheers, Fabian

2017-12-11 11:26 GMT+01:00 Stefan Richter :


+1 (non-binding)

- did extensive cluster tests on Google Cloud with special focus on
checkpointing and recovery and Kafka 0.11 end-to-end exactly-once +
at-least-once.
- build from source.

Am 11.12.2017 um 09:53 schrieb Piotr Nowojski 
:


Hi,

+1 (non-binding)

I have:
- verified Scala and Java sample projects are creating and working

properly and that Quickstart docs are ok
- verified that ChildFirstClassloader allows user to run his 
application

with some custom akka version

- tested Kafka 0.11 end to end exactly once
- did some manual checks whether docs/distribution files are ok

Piotrek


On 8 Dec 2017, at 16:49, Stephan Ewen  wrote:

@Eron Given that this is actually an undocumented "internal" 
feature at
this point, I would not expect that it is used heavily beyond 
Pravega.


Unless you feel strongly that this is a major issue, I would go ahead

with

the release...

On Fri, Dec 8, 2017 at 3:18 PM, Aljoscha Krettek 


wrote:


Thanks for the update! I would also say it's not a blocker but we

should

make sure that we don't break this after 1.4, then.


On 7. Dec 2017, at 22:37, Eron Wright  wrote:

Just discovered:  the removal of Flink's Future (FLINK-7252) 
causes a

breaking change in connectors that use
`org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook`,

because

`Future` is a type on one of the methods.

To my knowledge, this affects only the Pravega connector.  
Curious to

know

whether any other connectors are affected.  I don't think we (Dell

EMC)

consider it a blocker but it will mean that the connector is Flink

1.4+.

Eron


On Thu, Dec 7, 2017 at 12:25 PM, Aljoscha Krettek <

aljos...@apache.org>

wrote:


I just noticed that I did a copy-and-paste error and the last

paragraph

about voting period should be this:

The vote will be open for at least 72 hours. It is adopted by

majority

approval, with at least 3 PMC affirmative votes.

Best,
Aljoscha

On 7. Dec 2017, at 19:24, Bowen Li  
wrote:


I agree that it shouldn't block the release. The doc website 
part is

even

better!

On Thu, Dec 7, 2017 at 1:09 AM, Aljoscha Krettek <

aljos...@apache.org>

wrote:


Good catch, yes. This shouldn't block the release, though, since

the

doc
is always built form the latest state of a release branch, 
i.e. the

1.4

doc

on the website will update as soon as the doc on the release-1.4

branch

is

updated.


On 6. Dec 2017, at 20:47, Bowen Li 

wrote:

Hi Aljoscha,

I found Flink's State doc and javaDoc are very ambiguous on 
what

the

replacement of FoldingState is, which will confuse a lot of

users. We

need

to fix it in 1.4 release.

I have submitted a PR at https://github.com/apache/

flink/pull/5129

Thanks,
Bowen


On Wed, Dec 6, 2017 at 5:56 AM, Aljoscha Krettek <

aljos...@apache.org>

wrote:


Hi everyone,

Please review and vote on release candidate #3 for the version

1.4.0,

as

follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific

comments)


The complete staging area is available for your review, which

includes:

* JIRA release notes [1],
* the official Apache source release and binary convenience

releases

to

be

deployed to dist.apache.org[2], which are signed with the key

with

fingerprint F2A67A8047499BBB3908D17AA8F4FD97121D7293 [3],
* all artifacts to be deployed to the Maven Central Repository

[4],

* source code tag "release-1.4.0-rc1" [5],
* website pull request listing the new release [6].

Please have a careful look at the website PR because I changed

some

wording and we're now also releasing a binary without Hadoop

dependencies.

Please use this document for coordinating testing efforts: [7]

The only change between RC1 and this RC2 is that the source

release

package does not include the erroneously included binary Ruby

dependencies

of the documentation anymore. Because of this I would like to

propose

a
shorter voting time and close the vote around the time that 
RC1

would

[jira] [Created] (FLINK-8237) BucketingSink throws NPE when Writer.duplicate returns null

2017-12-11 Thread JIRA
Gábor Hermann created FLINK-8237:


 Summary: BucketingSink throws NPE when Writer.duplicate returns 
null
 Key: FLINK-8237
 URL: https://issues.apache.org/jira/browse/FLINK-8237
 Project: Flink
  Issue Type: Bug
Reporter: Gábor Hermann
Priority: Minor


Users need to look into Flink code to find the cause. We could catch that null 
before even running the job.

{code:java}
java.lang.NullPointerException
at 
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.openNewPartFile(BucketingSink.java:546)
at 
org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.invoke(BucketingSink.java:441)
at 
org.apache.flink.streaming.api.operators.StreamSink.processElement(StreamSink.java:41)
at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:528)
at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:503)
at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:483)
at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:891)
at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:869)
at 
org.apache.flink.streaming.api.operators.StreamFilter.processElement(StreamFilter.java:40)
at 
org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:206)
at 
org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:69)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:263)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702)
{code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: [VOTE] Release 1.4.0, release candidate #3

2017-12-11 Thread Chesnay Schepler

+1 (binding)

- checked contents of flink-dist for unshaded dependencies
- ran python examples (with/-out arguments) locally
- ran jobs on yarn on a cluster testing optional hadoop dependency
- verified that quickstarts work
- checked JM/TM logs for anything suspicious

On 11.12.2017 11:29, Fabian Hueske wrote:

+1 (binding)

- Checked hashes & signatures
- Checked no binaries in source release
- Checked Flink version in Quickstart pom files

Cheers, Fabian

2017-12-11 11:26 GMT+01:00 Stefan Richter :


+1 (non-binding)

- did extensive cluster tests on Google Cloud with special focus on
checkpointing and recovery and Kafka 0.11 end-to-end exactly-once +
at-least-once.
- build from source.


Am 11.12.2017 um 09:53 schrieb Piotr Nowojski :

Hi,

+1 (non-binding)

I have:
- verified Scala and Java sample projects are creating and working

properly and that Quickstart docs are ok

- verified that ChildFirstClassloader allows user to run his application

with some custom akka version

- tested Kafka 0.11 end to end exactly once
- did some manual checks whether docs/distribution files are ok

Piotrek


On 8 Dec 2017, at 16:49, Stephan Ewen  wrote:

@Eron Given that this is actually an undocumented "internal" feature at
this point, I would not expect that it is used heavily beyond Pravega.

Unless you feel strongly that this is a major issue, I would go ahead

with

the release...

On Fri, Dec 8, 2017 at 3:18 PM, Aljoscha Krettek 
wrote:


Thanks for the update! I would also say it's not a blocker but we

should

make sure that we don't break this after 1.4, then.


On 7. Dec 2017, at 22:37, Eron Wright  wrote:

Just discovered:  the removal of Flink's Future (FLINK-7252) causes a
breaking change in connectors that use
`org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook`,

because

`Future` is a type on one of the methods.

To my knowledge, this affects only the Pravega connector.  Curious to

know

whether any other connectors are affected.  I don't think we (Dell

EMC)

consider it a blocker but it will mean that the connector is Flink

1.4+.

Eron


On Thu, Dec 7, 2017 at 12:25 PM, Aljoscha Krettek <

aljos...@apache.org>

wrote:


I just noticed that I did a copy-and-paste error and the last

paragraph

about voting period should be this:

The vote will be open for at least 72 hours. It is adopted by

majority

approval, with at least 3 PMC affirmative votes.

Best,
Aljoscha


On 7. Dec 2017, at 19:24, Bowen Li  wrote:

I agree that it shouldn't block the release. The doc website part is

even

better!

On Thu, Dec 7, 2017 at 1:09 AM, Aljoscha Krettek <

aljos...@apache.org>

wrote:


Good catch, yes. This shouldn't block the release, though, since

the

doc

is always built form the latest state of a release branch, i.e. the

1.4

doc

on the website will update as soon as the doc on the release-1.4

branch

is

updated.


On 6. Dec 2017, at 20:47, Bowen Li 

wrote:

Hi Aljoscha,

I found Flink's State doc and javaDoc are very ambiguous on what

the

replacement of FoldingState is, which will confuse a lot of

users. We

need

to fix it in 1.4 release.

I have submitted a PR at https://github.com/apache/

flink/pull/5129

Thanks,
Bowen


On Wed, Dec 6, 2017 at 5:56 AM, Aljoscha Krettek <

aljos...@apache.org>

wrote:


Hi everyone,

Please review and vote on release candidate #3 for the version

1.4.0,

as

follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific

comments)


The complete staging area is available for your review, which

includes:

* JIRA release notes [1],
* the official Apache source release and binary convenience

releases

to

be

deployed to dist.apache.org[2], which are signed with the key

with

fingerprint F2A67A8047499BBB3908D17AA8F4FD97121D7293 [3],
* all artifacts to be deployed to the Maven Central Repository

[4],

* source code tag "release-1.4.0-rc1" [5],
* website pull request listing the new release [6].

Please have a careful look at the website PR because I changed

some

wording and we're now also releasing a binary without Hadoop

dependencies.

Please use this document for coordinating testing efforts: [7]

The only change between RC1 and this RC2 is that the source

release

package does not include the erroneously included binary Ruby

dependencies

of the documentation anymore. Because of this I would like to

propose

a

shorter voting time and close the vote around the time that RC1

would

have

closed. This would mean closing by end of Wednesday. Please let

me

know

if

you disagree with this. The vote is adopted by majority approval,

with

at

least 3 PMC affirmative votes.

Thanks,
Your friendly Release Manager

[1] https://issues.apache.org/jira/secure/ReleaseNote.jspa?
projectId=12315522=12340533
[2] 

[jira] [Created] (FLINK-8236) Allow to set the parallelism of table queries

2017-12-11 Thread Timo Walther (JIRA)
Timo Walther created FLINK-8236:
---

 Summary: Allow to set the parallelism of table queries
 Key: FLINK-8236
 URL: https://issues.apache.org/jira/browse/FLINK-8236
 Project: Flink
  Issue Type: Improvement
  Components: Table API & SQL
Reporter: Timo Walther


Right now the parallelism of a table program is determined by the parallelism 
of the stream/batch environment. E.g., by default, tumbling window operators 
use the default parallelism of the environment. Simple project and select 
operations have the same parallelism as the inputs they are applied on.

While we cannot change forwarding operations because this would change the 
results when using retractions, it should be possible to change the parallelism 
for operators after shuffling operations.

It should be possible to specify the default parallelism of a table program in 
the {{TableConfig}} and/or {{QueryConfig}}. The configuration per query has 
higher precedence that the configuration per table environment.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (FLINK-8235) Cannot run spotbugs for single module

2017-12-11 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-8235:
---

 Summary: Cannot run spotbugs for single module
 Key: FLINK-8235
 URL: https://issues.apache.org/jira/browse/FLINK-8235
 Project: Flink
  Issue Type: Bug
  Components: Build System
Affects Versions: 1.4.0, 1.5.0
Reporter: Chesnay Schepler
Assignee: Chesnay Schepler
 Fix For: 1.5.0, 1.4.1


When running the spotbugs plugin ({{-Dspotbugs}}) in a sub-module of Flink the 
build will fail because it cannot find the exclusion file.

{code}
[ERROR] Could not find resource 'tools/maven/spotbugs-exclude.xml'. -> [Help 1]
{code}

The problem is that the configured relative path is resolved against the 
sub-module directory, and not the parent one.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: [VOTE] Release 1.4.0, release candidate #3

2017-12-11 Thread Fabian Hueske
+1 (binding)

- Checked hashes & signatures
- Checked no binaries in source release
- Checked Flink version in Quickstart pom files

Cheers, Fabian

2017-12-11 11:26 GMT+01:00 Stefan Richter :

> +1 (non-binding)
>
> - did extensive cluster tests on Google Cloud with special focus on
> checkpointing and recovery and Kafka 0.11 end-to-end exactly-once +
> at-least-once.
> - build from source.
>
> > Am 11.12.2017 um 09:53 schrieb Piotr Nowojski :
> >
> > Hi,
> >
> > +1 (non-binding)
> >
> > I have:
> > - verified Scala and Java sample projects are creating and working
> properly and that Quickstart docs are ok
> > - verified that ChildFirstClassloader allows user to run his application
> with some custom akka version
> > - tested Kafka 0.11 end to end exactly once
> > - did some manual checks whether docs/distribution files are ok
> >
> > Piotrek
> >
> >> On 8 Dec 2017, at 16:49, Stephan Ewen  wrote:
> >>
> >> @Eron Given that this is actually an undocumented "internal" feature at
> >> this point, I would not expect that it is used heavily beyond Pravega.
> >>
> >> Unless you feel strongly that this is a major issue, I would go ahead
> with
> >> the release...
> >>
> >> On Fri, Dec 8, 2017 at 3:18 PM, Aljoscha Krettek 
> >> wrote:
> >>
> >>> Thanks for the update! I would also say it's not a blocker but we
> should
> >>> make sure that we don't break this after 1.4, then.
> >>>
>  On 7. Dec 2017, at 22:37, Eron Wright  wrote:
> 
>  Just discovered:  the removal of Flink's Future (FLINK-7252) causes a
>  breaking change in connectors that use
>  `org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook`,
> because
>  `Future` is a type on one of the methods.
> 
>  To my knowledge, this affects only the Pravega connector.  Curious to
> >>> know
>  whether any other connectors are affected.  I don't think we (Dell
> EMC)
>  consider it a blocker but it will mean that the connector is Flink
> 1.4+.
> 
>  Eron
> 
> 
>  On Thu, Dec 7, 2017 at 12:25 PM, Aljoscha Krettek <
> aljos...@apache.org>
>  wrote:
> 
> > I just noticed that I did a copy-and-paste error and the last
> paragraph
> > about voting period should be this:
> >
> > The vote will be open for at least 72 hours. It is adopted by
> majority
> > approval, with at least 3 PMC affirmative votes.
> >
> > Best,
> > Aljoscha
> >
> >> On 7. Dec 2017, at 19:24, Bowen Li  wrote:
> >>
> >> I agree that it shouldn't block the release. The doc website part is
> >>> even
> >> better!
> >>
> >> On Thu, Dec 7, 2017 at 1:09 AM, Aljoscha Krettek <
> aljos...@apache.org>
> >> wrote:
> >>
> >>> Good catch, yes. This shouldn't block the release, though, since
> the
> >>> doc
> >>> is always built form the latest state of a release branch, i.e. the
> >>> 1.4
> > doc
> >>> on the website will update as soon as the doc on the release-1.4
> >>> branch
> > is
> >>> updated.
> >>>
>  On 6. Dec 2017, at 20:47, Bowen Li 
> wrote:
> 
>  Hi Aljoscha,
> 
>  I found Flink's State doc and javaDoc are very ambiguous on what
> the
>  replacement of FoldingState is, which will confuse a lot of
> users. We
> >>> need
>  to fix it in 1.4 release.
> 
>  I have submitted a PR at https://github.com/apache/
> flink/pull/5129
> 
>  Thanks,
>  Bowen
> 
> 
>  On Wed, Dec 6, 2017 at 5:56 AM, Aljoscha Krettek <
> >>> aljos...@apache.org>
>  wrote:
> 
> > Hi everyone,
> >
> > Please review and vote on release candidate #3 for the version
> >>> 1.4.0,
> > as
> > follows:
> > [ ] +1, Approve the release
> > [ ] -1, Do not approve the release (please provide specific
> >>> comments)
> >
> >
> > The complete staging area is available for your review, which
> > includes:
> > * JIRA release notes [1],
> > * the official Apache source release and binary convenience
> releases
> > to
> >>> be
> > deployed to dist.apache.org[2], which are signed with the key
> with
> > fingerprint F2A67A8047499BBB3908D17AA8F4FD97121D7293 [3],
> > * all artifacts to be deployed to the Maven Central Repository
> [4],
> > * source code tag "release-1.4.0-rc1" [5],
> > * website pull request listing the new release [6].
> >
> > Please have a careful look at the website PR because I changed
> some
> > wording and we're now also releasing a binary without Hadoop
> >>> dependencies.
> >
> > Please use this document for coordinating testing efforts: [7]
> >
> > The 

Re: [VOTE] Release 1.4.0, release candidate #3

2017-12-11 Thread Stefan Richter
+1 (non-binding)

- did extensive cluster tests on Google Cloud with special focus on 
checkpointing and recovery and Kafka 0.11 end-to-end exactly-once + 
at-least-once.
- build from source.

> Am 11.12.2017 um 09:53 schrieb Piotr Nowojski :
> 
> Hi,
> 
> +1 (non-binding)
> 
> I have:
> - verified Scala and Java sample projects are creating and working properly 
> and that Quickstart docs are ok
> - verified that ChildFirstClassloader allows user to run his application with 
> some custom akka version
> - tested Kafka 0.11 end to end exactly once
> - did some manual checks whether docs/distribution files are ok
> 
> Piotrek
> 
>> On 8 Dec 2017, at 16:49, Stephan Ewen  wrote:
>> 
>> @Eron Given that this is actually an undocumented "internal" feature at
>> this point, I would not expect that it is used heavily beyond Pravega.
>> 
>> Unless you feel strongly that this is a major issue, I would go ahead with
>> the release...
>> 
>> On Fri, Dec 8, 2017 at 3:18 PM, Aljoscha Krettek 
>> wrote:
>> 
>>> Thanks for the update! I would also say it's not a blocker but we should
>>> make sure that we don't break this after 1.4, then.
>>> 
 On 7. Dec 2017, at 22:37, Eron Wright  wrote:
 
 Just discovered:  the removal of Flink's Future (FLINK-7252) causes a
 breaking change in connectors that use
 `org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook`, because
 `Future` is a type on one of the methods.
 
 To my knowledge, this affects only the Pravega connector.  Curious to
>>> know
 whether any other connectors are affected.  I don't think we (Dell EMC)
 consider it a blocker but it will mean that the connector is Flink 1.4+.
 
 Eron
 
 
 On Thu, Dec 7, 2017 at 12:25 PM, Aljoscha Krettek 
 wrote:
 
> I just noticed that I did a copy-and-paste error and the last paragraph
> about voting period should be this:
> 
> The vote will be open for at least 72 hours. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
> 
> Best,
> Aljoscha
> 
>> On 7. Dec 2017, at 19:24, Bowen Li  wrote:
>> 
>> I agree that it shouldn't block the release. The doc website part is
>>> even
>> better!
>> 
>> On Thu, Dec 7, 2017 at 1:09 AM, Aljoscha Krettek 
>> wrote:
>> 
>>> Good catch, yes. This shouldn't block the release, though, since the
>>> doc
>>> is always built form the latest state of a release branch, i.e. the
>>> 1.4
> doc
>>> on the website will update as soon as the doc on the release-1.4
>>> branch
> is
>>> updated.
>>> 
 On 6. Dec 2017, at 20:47, Bowen Li  wrote:
 
 Hi Aljoscha,
 
 I found Flink's State doc and javaDoc are very ambiguous on what the
 replacement of FoldingState is, which will confuse a lot of users. We
>>> need
 to fix it in 1.4 release.
 
 I have submitted a PR at https://github.com/apache/flink/pull/5129
 
 Thanks,
 Bowen
 
 
 On Wed, Dec 6, 2017 at 5:56 AM, Aljoscha Krettek <
>>> aljos...@apache.org>
 wrote:
 
> Hi everyone,
> 
> Please review and vote on release candidate #3 for the version
>>> 1.4.0,
> as
> follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific
>>> comments)
> 
> 
> The complete staging area is available for your review, which
> includes:
> * JIRA release notes [1],
> * the official Apache source release and binary convenience releases
> to
>>> be
> deployed to dist.apache.org[2], which are signed with the key with
> fingerprint F2A67A8047499BBB3908D17AA8F4FD97121D7293 [3],
> * all artifacts to be deployed to the Maven Central Repository [4],
> * source code tag "release-1.4.0-rc1" [5],
> * website pull request listing the new release [6].
> 
> Please have a careful look at the website PR because I changed some
> wording and we're now also releasing a binary without Hadoop
>>> dependencies.
> 
> Please use this document for coordinating testing efforts: [7]
> 
> The only change between RC1 and this RC2 is that the source release
> package does not include the erroneously included binary Ruby
>>> dependencies
> of the documentation anymore. Because of this I would like to
>>> propose
> a
> shorter voting time and close the vote around the time that RC1
>>> would
>>> have
> closed. This would mean closing by end of Wednesday. Please let me
> know
>>> if
> you disagree with this. The 

Re: [VOTE] Release 1.4.0, release candidate #3

2017-12-11 Thread Piotr Nowojski
Hi,

+1 (non-binding)

I have:
- verified Scala and Java sample projects are creating and working properly and 
that Quickstart docs are ok
- verified that ChildFirstClassloader allows user to run his application with 
some custom akka version
- tested Kafka 0.11 end to end exactly once
- did some manual checks whether docs/distribution files are ok

Piotrek

> On 8 Dec 2017, at 16:49, Stephan Ewen  wrote:
> 
> @Eron Given that this is actually an undocumented "internal" feature at
> this point, I would not expect that it is used heavily beyond Pravega.
> 
> Unless you feel strongly that this is a major issue, I would go ahead with
> the release...
> 
> On Fri, Dec 8, 2017 at 3:18 PM, Aljoscha Krettek 
> wrote:
> 
>> Thanks for the update! I would also say it's not a blocker but we should
>> make sure that we don't break this after 1.4, then.
>> 
>>> On 7. Dec 2017, at 22:37, Eron Wright  wrote:
>>> 
>>> Just discovered:  the removal of Flink's Future (FLINK-7252) causes a
>>> breaking change in connectors that use
>>> `org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook`, because
>>> `Future` is a type on one of the methods.
>>> 
>>> To my knowledge, this affects only the Pravega connector.  Curious to
>> know
>>> whether any other connectors are affected.  I don't think we (Dell EMC)
>>> consider it a blocker but it will mean that the connector is Flink 1.4+.
>>> 
>>> Eron
>>> 
>>> 
>>> On Thu, Dec 7, 2017 at 12:25 PM, Aljoscha Krettek 
>>> wrote:
>>> 
 I just noticed that I did a copy-and-paste error and the last paragraph
 about voting period should be this:
 
 The vote will be open for at least 72 hours. It is adopted by majority
 approval, with at least 3 PMC affirmative votes.
 
 Best,
 Aljoscha
 
> On 7. Dec 2017, at 19:24, Bowen Li  wrote:
> 
> I agree that it shouldn't block the release. The doc website part is
>> even
> better!
> 
> On Thu, Dec 7, 2017 at 1:09 AM, Aljoscha Krettek 
> wrote:
> 
>> Good catch, yes. This shouldn't block the release, though, since the
>> doc
>> is always built form the latest state of a release branch, i.e. the
>> 1.4
 doc
>> on the website will update as soon as the doc on the release-1.4
>> branch
 is
>> updated.
>> 
>>> On 6. Dec 2017, at 20:47, Bowen Li  wrote:
>>> 
>>> Hi Aljoscha,
>>> 
>>> I found Flink's State doc and javaDoc are very ambiguous on what the
>>> replacement of FoldingState is, which will confuse a lot of users. We
>> need
>>> to fix it in 1.4 release.
>>> 
>>> I have submitted a PR at https://github.com/apache/flink/pull/5129
>>> 
>>> Thanks,
>>> Bowen
>>> 
>>> 
>>> On Wed, Dec 6, 2017 at 5:56 AM, Aljoscha Krettek <
>> aljos...@apache.org>
>>> wrote:
>>> 
 Hi everyone,
 
 Please review and vote on release candidate #3 for the version
>> 1.4.0,
 as
 follows:
 [ ] +1, Approve the release
 [ ] -1, Do not approve the release (please provide specific
>> comments)
 
 
 The complete staging area is available for your review, which
 includes:
 * JIRA release notes [1],
 * the official Apache source release and binary convenience releases
 to
>> be
 deployed to dist.apache.org[2], which are signed with the key with
 fingerprint F2A67A8047499BBB3908D17AA8F4FD97121D7293 [3],
 * all artifacts to be deployed to the Maven Central Repository [4],
 * source code tag "release-1.4.0-rc1" [5],
 * website pull request listing the new release [6].
 
 Please have a careful look at the website PR because I changed some
 wording and we're now also releasing a binary without Hadoop
>> dependencies.
 
 Please use this document for coordinating testing efforts: [7]
 
 The only change between RC1 and this RC2 is that the source release
 package does not include the erroneously included binary Ruby
>> dependencies
 of the documentation anymore. Because of this I would like to
>> propose
 a
 shorter voting time and close the vote around the time that RC1
>> would
>> have
 closed. This would mean closing by end of Wednesday. Please let me
 know
>> if
 you disagree with this. The vote is adopted by majority approval,
>> with
>> at
 least 3 PMC affirmative votes.
 
 Thanks,
 Your friendly Release Manager
 
 [1] https://issues.apache.org/jira/secure/ReleaseNote.jspa?
 projectId=12315522=12340533
 [2] http://people.apache.org/~aljoscha/flink-1.4.0-rc3/
 [3] https://dist.apache.org/repos/dist/release/flink/KEYS
 

[jira] [Created] (FLINK-8233) Retrieve ExecutionResult by REST polling

2017-12-11 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-8233:


 Summary: Retrieve ExecutionResult by REST polling
 Key: FLINK-8233
 URL: https://issues.apache.org/jira/browse/FLINK-8233
 Project: Flink
  Issue Type: Sub-task
  Components: REST
Affects Versions: 1.5.0
Reporter: Till Rohrmann
 Fix For: 1.5.0


Retrieve the {{ExecutionResult}} from a finished Flink job via the 
{{RestClusterClient}}.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (FLINK-8232) Create RestHandler for long running operations

2017-12-11 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-8232:


 Summary: Create RestHandler for long running operations
 Key: FLINK-8232
 URL: https://issues.apache.org/jira/browse/FLINK-8232
 Project: Flink
  Issue Type: Sub-task
  Components: REST
Affects Versions: 1.5.0
Reporter: Till Rohrmann
 Fix For: 1.5.0


We need a rest handler which can trigger long running asynchronous operations 
and creates a resource to check the operation's state. This resource should be 
returned in the location header and be completed once the long running 
operation has finished.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (FLINK-8231) Extend AbstractRestHandler to return response headers

2017-12-11 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-8231:


 Summary: Extend AbstractRestHandler to return response headers
 Key: FLINK-8231
 URL: https://issues.apache.org/jira/browse/FLINK-8231
 Project: Flink
  Issue Type: Sub-task
  Components: REST
Affects Versions: 1.5.0
Reporter: Till Rohrmann
 Fix For: 1.5.0


The {{AbstractRestHandler}} should be able to return a set of response headers 
in case that it creates a new resource (POST call) and wants to set the 
redirection header, for example.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)