I don't think this is a configuration problem, but a bug in Flink. But
we'll have to dig a little deeper to be sure.
Besides the actual SSL problem, what concerns me is that we didn't fail
earlier. If a bug in the SSL setup prevents
the up- or download of jars then we should fail earlier. Looping in Nico
who may have some input.
On 04.10.2017 22:58, Aniket Deshpande wrote:
Hi Chesnay,
Thanks for the reply. After your suggestion, I found out that setting
/blob.service.ssl.enabled: false/ solved the issue and now all the
pipelines run as expected.
So, the issue is kinda narrowed down to blob service ssl now.
I also checked the jobmanager logs when blob ssl is enabled and I see
the following error:
/2017-10-03 23:28:50.459 [BLOB connection for /<jm_ip>:46932] ERROR
org.apache.flink.runtime.blob.BlobServerConnection - Error while
executing BLOB connection.
javax.net.ssl.SSLHandshakeException: Received fatal alert:
certificate_unknown
at sun.security.ssl.Alerts.getSSLException(Alerts.java:192)
at sun.security.ssl.Alerts.getSSLException(Alerts.java:154)
at
sun.security.ssl.SSLSocketImpl.recvAlert(SSLSocketImpl.java:2023)
at
sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:1125)
at
sun.security.ssl.SSLSocketImpl.performInitialHandshake(SSLSocketImpl.java:1375)
at
sun.security.ssl.SSLSocketImpl.readDataRecord(SSLSocketImpl.java:928)
at sun.security.ssl.AppInputStream.read(AppInputStream.java:105)
at sun.security.ssl.AppInputStream.read(AppInputStream.java:71)
at
org.apache.flink.runtime.blob.BlobServerConnection.run(BlobServerConnection.java:119)
/
So, is there some additional steps that I have to follow for enabling
SSL for blob service?
On Wed, Oct 4, 2017 at 4:09 PM, Eron Wright <eronwri...@gmail.com
<mailto:eronwri...@gmail.com>> wrote:
By following Chesney's recommendation we will hopefully uncover an
SSL error that is being masked. Another thing to try is to disable
hostname verification (it is enabled by default) to see whether
the certificate is being rejected.
On Wed, Oct 4, 2017 at 5:15 AM, Chesnay Schepler
<ches...@apache.org <mailto:ches...@apache.org>> wrote:
something that would also help us narrow down the problematic
area is to enable SSL for one component at a time and see
which one causesd the job to fail.
On 04.10.2017 14:11, Chesnay Schepler wrote:
The configuration looks reasonable. Just to be sure, are the
paths accessible by all nodes?
As a first step, could you set the logging level to DEBUG (by
modifying the 'conf/log4j.properties' file), resubmit the job
(after a cluster restart) and check the Job- and TaskManager
logs for any exception?
On 04.10.2017 03:15, Aniket Deshpande wrote:
Background: We have a setup of Flink 1.3.1 along with a
secure MAPR cluster (Flink is running on mapr client nodes).
We run this flink cluster via flink-jobmanager.sh
<http://flink-jobmanager.sh> foreground and
flink-taskmanager.sh <http://flink-taskmanager.sh>
foreground command via Marathon. In order for us to make
this work, we had to add -Djavax.net
<http://-Djavax.net>.ssl.trustStore="$JAVA_HOME/jre/lib/security/cacerts" in
flink-console.sh <http://flink-console.sh> as extra JVM arg
(otherwise, flink was taking MAPR's ssl_truststore as
default truststore and then we were facing issues for any
3rd party jars like aws_sdk etc.). This entire setup was
working fine as it is and we could submit our jars and the
pipelines ran without any problem
Problem: We started experimenting with enabling ssl for all
communication for Flink. For this, we followed
https://ci.apache.org/projects/flink/flink-docs-release-1.3/setup/security-ssl.html
<https://ci.apache.org/projects/flink/flink-docs-release-1.3/setup/security-ssl.html>
for
generating CA and keystore. I added the following properties
to flink-conf.yaml:
security.ssl.enabled: true
security.ssl.keystore: /opt/flink/certs/node1.keystore
security.ssl.keystore-password: <password>
security.ssl.key-password: <password>
security.ssl.truststore: /opt/flink/certs/ca.truststore
security.ssl.truststore-password: <password>
jobmanager.web.ssl.enabled: true
taskmanager.data.ssl.enabled: true
blob.service.ssl.enabled: true
akka.ssl.enabled: true
We then spin up a cluster and tried submitting the same job
which was working before. We get the following erros:
org.apache.flink.streaming.runtime.tasks.StreamTaskException:
Cannot load user class:
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09
ClassLoader info: URL ClassLoader:
Class not resolvable through given classloader.
at
org.apache.flink.streaming.api.graph.StreamConfig.getStreamOperator(StreamConfig.java:229)
at
org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(OperatorChain.java:95)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:230)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702)
at java.lang.Thread.run(Thread.java:748)
This error disappears when we remove the ssl config
properties i.e run flink cluster without ssl enabled.
So, did we miss any steps for enabling ssl?
P.S.: We tried removing the extra JVm arg mentioned above,
but still get the same error.
--
Aniket
--
Yours Sincerely,
Aniket S Deshpande.