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

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

does this exception ever recover? if not, it looks like something to add to 
org.apache.hadoop.fs.s3a.S3ARetryPolicy 

A PR there will be very welcome -especially as you are the one who can verify 
it is being handled properly

{code}
3/08/03 11:25:11 DEBUG SSLConnectionSocketFactory: Enabled cipher 
suites:[TLS_ECDHE_ECDSA_WITH_AES_256_CBC_SHA384, 
TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA384, TLS_RSA_WITH_AES_256_CBC_SHA256, 
TLS_ECDH_ECDSA_WITH_AES_256_CBC_SHA384, TLS_ECDH_RSA_WITH_AES_256_CBC_SHA384, 
TLS_DHE_RSA_WITH_AES_256_CBC_SHA256, TLS_DHE_DSS_WITH_AES_256_CBC_SHA256, 
TLS_ECDHE_ECDSA_WITH_AES_256_CBC_SHA, TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA, 
TLS_RSA_WITH_AES_256_CBC_SHA, TLS_ECDH_ECDSA_WITH_AES_256_CBC_SHA, 
TLS_ECDH_RSA_WITH_AES_256_CBC_SHA, TLS_DHE_RSA_WITH_AES_256_CBC_SHA, 
TLS_DHE_DSS_WITH_AES_256_CBC_SHA, TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256, 
TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256, TLS_RSA_WITH_AES_128_CBC_SHA256, 
TLS_ECDH_ECDSA_WITH_AES_128_CBC_SHA256, TLS_ECDH_RSA_WITH_AES_128_CBC_SHA256, 
TLS_DHE_RSA_WITH_AES_128_CBC_SHA256, TLS_DHE_DSS_WITH_AES_128_CBC_SHA256, 
TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA, TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA, 
TLS_RSA_WITH_AES_128_CBC_SHA, TLS_ECDH_ECDSA_WITH_AES_128_CBC_SHA, 
TLS_ECDH_RSA_WITH_AES_128_CBC_SHA, TLS_DHE_RSA_WITH_AES_128_CBC_SHA, 
TLS_DHE_DSS_WITH_AES_128_CBC_SHA, TLS_EMPTY_RENEGOTIATION_INFO_SCSV]
23/08/03 11:25:11 DEBUG SSLConnectionSocketFactory: Starting handshake
23/08/03 11:25:11 DEBUG 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.$Proxy32.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:1346)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1157)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:814)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:781)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:755)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:715)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:697)
        at 
com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:561)
        at 
com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:541)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5456)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5403)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5397)
        at 
com.amazonaws.services.s3.AmazonS3Client.listObjectsV2(AmazonS3Client.java:971)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$listObjects$11(S3AFileSystem.java:2595)
        at 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:499)
        at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:414)
        at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:377)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.listObjects(S3AFileSystem.java:2586)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3832)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3688)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$isDirectory$35(S3AFileSystem.java:4724)
        at 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:499)
        at 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:444)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2337)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2356)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4722)
        at 
org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
        at 
org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:366)
        at 
org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:229)
        at 
org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:211)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:211)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:186)
        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 py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:374)
        at py4j.Gateway.invoke(Gateway.java:282)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at 
py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
        at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
        at java.lang.Thread.run(Thread.java:748)
Caused by: javax.net.ssl.SSLException: Unsupported or unrecognized SSL message
        at 
sun.security.ssl.SSLSocketInputRecord.handleUnknownRecord(SSLSocketInputRecord.java:448)
        at 
sun.security.ssl.SSLSocketInputRecord.decode(SSLSocketInputRecord.java:184)
        at sun.security.ssl.SSLTransport.decode(SSLTransport.java:109)
        at sun.security.ssl.SSLSocketImpl.decode(SSLSocketImpl.java:1383)
        at 
sun.security.ssl.SSLSocketImpl.readHandshakeRecord(SSLSocketImpl.java:1291)
        at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:435)
        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)
        ... 58 more

{code}

settings-wise,  you can start off with a very small initial retry interval, so 
even after retries it doesn't take so long to give up. 

{code}
fs.s3a.retry.interval = 50 ms
fs.s3a.retry.limit = 4

{code}

fs.s3a.attempts.maximum is what is passed down to the AWS SDK; if that is 
retrying on the exception then it is adding more delays. But this is the only 
retry  handler during block copy in rename, so it does have a role.



> SSLException is raised after very long timeout
> ----------------------------------------------
>
>                 Key: HADOOP-18839
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18839
>             Project: Hadoop Common
>          Issue Type: Bug
>          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: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to