[ 
https://issues.apache.org/jira/browse/HADOOP-18839?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17757302#comment-17757302
 ] 

Steve Loughran commented on HADOOP-18839:
-----------------------------------------

also seen with a different third party store.

hypothesis: s3a retry policy retries on unrecoverable ssl handshake problems. 
these don't surface against AWS S3 (except maybe with proxy problems) which is 
why its rare. but it should be tractable.

{code}
22/05/25 10:53:53 DEBUG conn.ClientConnectionManagerFactory:
java.lang.reflect.InvocationTargetException
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
com.amazonaws.http.conn.ClientConnectionManagerFactory$Handler.invoke(ClientConnectionManagerFactory.java:76)
        at com.amazonaws.http.conn.$Proxy9.connect(Unknown Source)
        at 
com.amazonaws.thirdparty.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:393)
        at 
com.amazonaws.thirdparty.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236)
        at 
com.amazonaws.thirdparty.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186)
        at 
com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185)
        at 
com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83)
        at 
com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56)
        at 
com.amazonaws.http.apache.client.impl.SdkHttpClient.execute(SdkHttpClient.java:72)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1343)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1154)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:811)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:779)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:753)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:713)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:695)
        at 
com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:559)
        at 
com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:539)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5453)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5400)
        at 
com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1372)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getObjectMetadata$6(S3AFileSystem.java:2053)
        at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:412)
        at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:375)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:2043)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:2019)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3260)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3172)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:3040)
        at org.apache.hadoop.fs.Globber.getFileStatus(Globber.java:115)
        at org.apache.hadoop.fs.Globber.doGlob(Globber.java:349)
        at org.apache.hadoop.fs.Globber.glob(Globber.java:202)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:4297)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:4277)
        at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:353)
        at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:250)
        at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:233)
        at 
org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:104)
        at org.apache.hadoop.fs.shell.Command.run(Command.java:177)
        at org.apache.hadoop.fs.FsShell.run(FsShell.java:328)
        at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
        at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
        at org.apache.hadoop.fs.FsShell.main(FsShell.java:391)
Caused by: javax.net.ssl.SSLHandshakeException: 
sun.security.validator.ValidatorException: PKIX path building failed: 
sun.security.provider.certpath.SunCertPathBuilderException: unable to find 
valid certification path to requested target
        at sun.security.ssl.Alerts.getSSLException(Alerts.java:192)
        at sun.security.ssl.SSLSocketImpl.fatal(SSLSocketImpl.java:1946)
        at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:316)
        at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:310)
        at 
sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1639)
        at 
sun.security.ssl.ClientHandshaker.processMessage(ClientHandshaker.java:223)
        at sun.security.ssl.Handshaker.processLoop(Handshaker.java:1037)
        at sun.security.ssl.Handshaker.process_record(Handshaker.java:965)
        at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:1064)
        at 
sun.security.ssl.SSLSocketImpl.performInitialHandshake(SSLSocketImpl.java:1367)
        at 
sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1395)
        at 
sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1379)
        at 
com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:436)
        at 
com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.connectSocket(SSLConnectionSocketFactory.java:384)
        at 
com.amazonaws.thirdparty.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142)
        at 
com.amazonaws.thirdparty.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376)
        ... 47 more
Caused by: sun.security.validator.ValidatorException: PKIX path building 
failed: sun.security.provider.certpath.SunCertPathBuilderException: unable to 
find valid certification path to requested target
        at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:397)
        at 
sun.security.validator.PKIXValidator.engineValidate(PKIXValidator.java:302)
        at sun.security.validator.Validator.validate(Validator.java:262)
        at 
sun.security.ssl.X509TrustManagerImpl.validate(X509TrustManagerImpl.java:330)
        at 
sun.security.ssl.X509TrustManagerImpl.checkTrusted(X509TrustManagerImpl.java:237)
        at 
sun.security.ssl.X509TrustManagerImpl.checkServerTrusted(X509TrustManagerImpl.java:132)
        at 
sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1621)
        ... 58 more
Caused by: sun.security.provider.certpath.SunCertPathBuilderException: unable 
to find valid certification path to requested target
        at 
sun.security.provider.certpath.SunCertPathBuilder.build(SunCertPathBuilder.java:141)
        at 
sun.security.provider.certpath.SunCertPathBuilder.engineBuild(SunCertPathBuilder.java:126)
        at java.security.cert.CertPathBuilder.build(CertPathBuilder.java:280)
        at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:392)
        ... 64 more

{code}


> s3a client SSLException is raised after very long timeout "Unsupported or 
> unrecognized SSL message"
> ---------------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-18839
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18839
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.3.4
>            Reporter: Maxim Martynov
>            Priority: Minor
>         Attachments: host.log, ssl.log
>
>
> I've tried to connect from PySpark to Minio running in docker.
> Installing PySpark and starting Minio:
> {code:bash}
> pip install pyspark==3.4.1
> docker run --rm -d --hostname minio --name minio -p 9000:9000 -p 9001:9001 -e 
> MINIO_ACCESS_KEY=access -e MINIO_SECRET_KEY=Eevoh2wo0ui6ech0wu8oy
> 3feiR3eicha -e MINIO_ROOT_USER=admin -e 
> MINIO_ROOT_PASSWORD=iepaegaigi3ofa9TaephieSo1iecaesh bitnami/minio:latest
> docker exec minio mc mb test-bucket
> {code}
> Then create Spark session:
> {code:python}
> from pyspark.sql import SparkSession
> spark = SparkSession.builder\
>           .config("spark.jars.packages", 
> "org.apache.hadoop:hadoop-aws:3.3.4")\
>           .config("spark.hadoop.fs.s3a.endpoint", "localhost:9000")\
>           .config("spark.hadoop.fs.s3a.access.key", "access")\
>           .config("spark.hadoop.fs.s3a.secret.key", 
> "Eevoh2wo0ui6ech0wu8oy3feiR3eicha")\
>           .config("spark.hadoop.fs.s3a.aws.credentials.provider", 
> "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider")\
>           .getOrCreate()
> spark.sparkContext.setLogLevel("debug")
> {code}
> And try to access some object in a bucket:
> {code:python}
> import time
> begin = time.perf_counter()
> spark.read.format("csv").load("s3a://test-bucket/fake")
> end = time.perf_counter()
> py4j.protocol.Py4JJavaError: An error occurred while calling o40.load.
> : org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a://test-bucket/fake: com.amazonaws.SdkClientException: Unable to execute 
> HTTP request: Unsupported or unrecognized SSL message: Unable to execute HTTP 
> request: Unsupported or unrecognized SSL message
> ...
> {code}
> [^ssl.log]
> {code:python}
> >>> print((end-begin)/60)
> 14.72387898775002
> {code}
> I was waiting almost *15 minutes* to get the exception from Spark. The reason 
> was I tried to connect to endpoint with 
> {{{}fs.s3a.connection.ssl.enabled=true{}}}, but Minio is configured to listen 
> for HTTP protocol only.
> Is there any way to immediately raise exception if SSL connection cannot be 
> established?
> If I try to pass wrong endpoint, like {{{}localhos:9000{}}}, I'll get 
> exception like this in just 5 seconds:
> {code:java}
> : org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a://test-bucket/fake: com.amazonaws.SdkClientException: Unable to execute 
> HTTP request: test-bucket.localhos: Unable to execute HTTP request: 
> test-bucket.localhos
> ...
> {code}
> [^host.log]
> {code:python}
> >>> print((end-begin)/60)
> 0.09500707178334172
> >>> end-begin
> 5.700424307000503
> {code}
> I know about options like {{fs.s3a.attempts.maximum}} and 
> {{{}fs.s3a.retry.limit{}}}, setting them to 1 will cause raising exception 
> just immediately. But this does not look right.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to