+1

- verified checksum and gpg files
- verified LICENSE and NOTICE: NOTICE didn't change from 1.5, LICENSE had one 
unnecessary part removed

Side comment: I'm not sure whether the "Verify that the LICENSE and NOTICE file 
is correct for the binary and source releases" part is valid anymore because we 
only have one LICENSE and NOTICE file. also "The LICENSE and NOTICE files in 
flink-dist/src/main/flink-bin refer to the binary distribution and mention all 
of Flink's Maven dependencies as well" can be dropped because we don't have 
them anymore.

I came to the same conclusion on dependencies. I used tools/list_deps.py and 
diff'ed the output for 1.5 and 1.6, that's probably what Chesnay also did ... 
:-)

> On 8. Aug 2018, at 14:43, Chesnay Schepler <ches...@apache.org> wrote:
> 
> +1
> 
> - verified source release contains no binaries
> - verified correct versions in source release
> - verified compilation, tests and E2E-tests pass (on travis)
> - verified checksum and gpg files
> 
> New dependencies (excluding dependencies where we simply depend on a 
> different version now):
>    Apache licensed:
>        io.confluent:common-utils:jar:3.3.1
>        io.confluent:kafka-schema-registry-client:jar:3.3.1
>        io.prometheus:simpleclient_pushgateway:jar:0.3.0
>        various Apache Nifi dependencies
>        various Apache Parquet dependencies
>        various ElasticSearch dependencies
>    CDDL:
>        javax.ws.rs:javax.ws.rs-api:jar:2.1
>    Bouncycastle (MIT-like):
>        org.bouncycastle:bcpkix-jdk15on:jar:1.59
>        org.bouncycastle:bcprov-jdk15on:jar:1.59
>    MIT:
>        org.projectlombok:lombok:jar:1.16.20
> 
> On 08.08.2018 13:28, Till Rohrmann wrote:
>> Thanks for reporting these problems Chesnay. The usage string in
>> `standalone-job.sh` is out dated and should be updated. The same applies to
>> the typo.
>> 
>> When calling `standalone-job.sh start --job-classname foobar.Job` please
>> make sure that the user code jar is contained in the classpath (e.g.
>> putting the jar in the lib directory). Documenting this behaviour is part
>> of the pending issue FLINK-10001.
>> 
>> We should fix all of these issues. They are, however, no release blockers.
>> 
>> Cheers,
>> Till
>> 
>> On Wed, Aug 8, 2018 at 11:31 AM Chesnay Schepler <ches...@apache.org> wrote:
>> 
>>> I found some issues with the standalone-job.sh script.
>>> 
>>> I ran "./bin/standalone-job.sh start" as described by the usage string.
>>> 
>>>     2018-08-08 09:22:34,385 ERROR
>>>     org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>     Could not parse command line arguments [--configDir,
>>>     /home/zento/svn/flink-1.6.0/flink-1.6.0/conf].
>>>     org.apache.flink.runtime.entrypoint.FlinkParseException: Failed to
>>>     parse the command line arguments.
>>>              at
>>> 
>>> org.apache.flink.runtime.entrypoint.parser.CommandLineParser.parse(CommandLineParser.java:52)
>>>              at
>>> 
>>> org.apache.flink.container.entrypoint.StandaloneJobClusterEntryPoint.main(StandaloneJobClusterEntryPoint.java:143)
>>>     Caused by: org.apache.commons.cli.MissingOptionException: Missing
>>>     required option: j
>>>              at
>>> 
>>> org.apache.commons.cli.DefaultParser.checkRequiredOptions(DefaultParser.java:199)
>>>              at
>>>     org.apache.commons.cli.DefaultParser.parse(DefaultParser.java:130)
>>>              at
>>>     org.apache.commons.cli.DefaultParser.parse(DefaultParser.java:81)
>>>              at
>>> 
>>> org.apache.flink.runtime.entrypoint.parser.CommandLineParser.parse(CommandLineParser.java:50)
>>>             ... 1 more
>>> 
>>> The script should fail earlier if no jar is provided, with a better
>>> error message.
>>> It is also undocumented, and the usage instructions don't appear correct.
>>> 
>>> Passing a jar with the -j option leads to a ClassNotFoundException:
>>> "./bin/standalone-job.sh start -j examples/streaming/WordCount.jar"
>>> 
>>>     2018-08-08 09:26:30,562 ERROR
>>>     org.apache.flink.runtime.entrypoint.ClusterEntrypoint         -
>>>     Cluster initialization failed.
>>>     java.lang.reflect.UndeclaredThrowableException
>>>              at
>>> 
>>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1854)
>>>              at
>>> 
>>> org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
>>>              at
>>> 
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:189)
>>>              at
>>> 
>>> org.apache.flink.container.entrypoint.StandaloneJobClusterEntryPoint.main(StandaloneJobClusterEntryPoint.java:158)
>>>     Caused by: org.apache.flink.util.FlinkException: Could not load the
>>>     provied entrypoint class.
>>>              at
>>> 
>>> org.apache.flink.container.entrypoint.StandaloneJobClusterEntryPoint.createPackagedProgram(StandaloneJobClusterEntryPoint.java:92)
>>>              at
>>> 
>>> org.apache.flink.container.entrypoint.StandaloneJobClusterEntryPoint.retrieveJobGraph(StandaloneJobClusterEntryPoint.java:75)
>>>              at
>>> 
>>> org.apache.flink.runtime.entrypoint.JobClusterEntrypoint.createDispatcher(JobClusterEntrypoint.java:107)
>>>              at
>>> 
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startClusterComponents(ClusterEntrypoint.java:353)
>>>              at
>>> 
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:232)
>>>              at
>>> 
>>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$0(ClusterEntrypoint.java:190)
>>>              at java.security.AccessController.doPrivileged(Native Method)
>>>              at javax.security.auth.Subject.doAs(Subject.java:422)
>>>              at
>>> 
>>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1836)
>>>              ... 3 more
>>>     Caused by: java.lang.ClassNotFoundException:
>>>     examples/streaming/WordCount.jar
>>>              at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
>>>              at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>>>              at
>>>     sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:338)
>>>              at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>>>              at
>>> 
>>> org.apache.flink.container.entrypoint.StandaloneJobClusterEntryPoint.createPackagedProgram(StandaloneJobClusterEntryPoint.java:89)
>>>              ... 11 more
>>> 
>>> So this seems to not work at all, but maybe I'm using it wrong?
>>> 
>>> (There's also typo in "Could not load the provied entrypoint class")
>>> 
>>> On 08.08.2018 10:33, Piotr Nowojski wrote:
>>>> +1 from my side
>>>> 
>>>> I’ve spent some time playing around with various examples (batching,
>>> streaming and SQL) on EMR 6 nodes cluster with yarn deployment, with
>>> different configuration options (number of task
>>> managers/memory/Flip6/credit base flow control/metrics) and everything
>>> looks now fine (after fixing
>>> https://issues.apache.org/jira/browse/FLINK-9969 <
>>> https://issues.apache.org/jira/browse/FLINK-9969> ).
>>>> Piotrek
>>>> 
>>>>> On 7 Aug 2018, at 17:17, Till Rohrmann <trohrm...@apache.org> wrote:
>>>>> 
>>>>> Hi everyone,
>>>>> Please review and vote on the release candidate #4 for the version
>>> 1.6.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 1F302569A96CFFD5 [3],
>>>>> * all artifacts to be deployed to the Maven Central Repository [4],
>>>>> * source code tag "release-1.6.0-rc4" [5],
>>>>> * website pull request listing the new release and adding announcement
>>> blog
>>>>> post [6].
>>>>> 
>>>>> Please use this document for coordinating testing efforts: [7]
>>>>> 
>>>>> The vote will be shortened since we only adde a minor fix on top of the
>>> RC
>>>>> 3. It will close on Wednesday 6:30pm CET. It 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&version=12342760
>>>>> [2] https://dist.apache.org/repos/dist/dev/flink/flink-1.6.0/
>>>>> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
>>>>> [4]
>>> https://repository.apache.org/content/repositories/orgapacheflink-1178
>>>>> [5] https://github.com/apache/flink/tree/release-1.6.0-rc4
>>>>> [6] https://github.com/apache/flink-web/pull/117
>>>>> [7]
>>>>> 
>>> https://docs.google.com/document/d/1upBFZQ7tbaSkYvDiLqfUFXKg8Xxs-lVheEfb66e4jpo/edit?usp=sharing
>>>>> Pro-tip: you can create a settings.xml file with these contents:
>>>>> 
>>>>> <settings>
>>>>> <activeProfiles>
>>>>>   <activeProfile>flink-1.6.0</activeProfile>
>>>>> </activeProfiles>
>>>>> <profiles>
>>>>>   <profile>
>>>>>     <id>flink-1.6.0</id>
>>>>>     <repositories>
>>>>>       <repository>
>>>>>         <id>flink-1.6.0</id>
>>>>>         <url>
>>>>> 
>>>>> https://repository.apache.org/content/repositories/orgapacheflink-1178/
>>>>>         </url>
>>>>>       </repository>
>>>>>       <repository>
>>>>>         <id>archetype</id>
>>>>>         <url>
>>>>> 
>>>>> https://repository.apache.org/content/repositories/orgapacheflink-1178/
>>>>>         </url>
>>>>>       </repository>
>>>>>     </repositories>
>>>>>   </profile>
>>>>> </profiles>
>>>>> </settings>
>>>>> 
>>>>> And reference that in you maven commands via --settings
>>>>> path/to/settings.xml. This is useful for creating a quickstart based on
>>> the
>>>>> staged release and for building against the staged jars.
>>> 
> 

Reply via email to