[jira] [Commented] (SPARK-47008) Spark to support S3 Express One Zone Storage

2024-05-13 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-47008?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17846014#comment-17846014
 ] 

Steve Loughran commented on SPARK-47008:


yes, that looks like it. real PITA this feature, though apparently its there to 
let you know that you have outstanding uploads to purge -no lifecycle rules, 
see.

FWIW you can explictly create the real situation with a touch command under a 
__magic path:

{code}
hadoop fs -touch s3a://stevel--usw1-az2--x-s3/cli/__magic/__base/d/file.txt
{code}

this creates an incomplete upload under /cli//d/file.txt

> Spark to support S3 Express One Zone Storage
> 
>
> Key: SPARK-47008
> URL: https://issues.apache.org/jira/browse/SPARK-47008
> Project: Spark
>  Issue Type: Sub-task
>  Components: Spark Core
>Affects Versions: 4.0.0
>Reporter: Steve Loughran
>Priority: Major
>
> Hadoop 3.4.0 adds support for AWS S3 Express One Zone Storage.
> Most of this is transparent. However, one aspect which can surface as an 
> issue is that these stores report prefixes in a listing when there are 
> pending uploads, *even when there are no files underneath*
> This leads to a situation where a listStatus of a path returns a list of file 
> status entries which appears to contain one or more directories -but a 
> listStatus on that path raises a FileNotFoundException: there is nothing 
> there.
> HADOOP-18996 handles this in all of hadoop code, including FileInputFormat, 
> A filesystem can now be probed for inconsistent directoriy listings through 
> {{fs.hasPathCapability(path, "fs.capability.directory.listing.inconsistent")}}
> If true, then treewalking code SHOULD NOT report a failure if, when walking 
> into a subdirectory, a list/getFileStatus on that directory raises a 
> FileNotFoundException.
> Although most of this is handled in the hadoop code, but there some places 
> where treewalking is done inside spark These need to be identified and make 
> resilient to failure on the recurse down the tree
> * SparkHadoopUtil list methods , 
> * especially listLeafStatuses used by OrcFileOperator
> org.apache.spark.util.Utils#fetchHcfsFile
> {{org.apache.hadoop.fs.FileUtil.maybeIgnoreMissingDirectory()}} can assist 
> here, or the logic can be replicated. Using the hadoop implementation would 
> be better from a maintenance perspective



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-48123) Provide a constant table schema for querying structured logs

2024-05-07 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-48123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17844245#comment-17844245
 ] 

Steve Loughran commented on SPARK-48123:


this doesn't handle nested stack traces. I seem to have my comments here 
ignored. let me repeat

*  deep nested are common, especially those coming from networks, where have to 
translate things like aws sdk errors into meaningful and well known exceptions.
* these consist of a chain of exceptions, each with their own message and stack 
trace
* any log format which fails to anticipate or support these is inadequate to 
diagnose a large portion of the stack traces a failing app will generate
* thus destroying its utility value

has a decision been made to ignore my requirements?

> Provide a constant table schema for querying structured logs
> 
>
> Key: SPARK-48123
> URL: https://issues.apache.org/jira/browse/SPARK-48123
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Affects Versions: 4.0.0
>Reporter: Gengliang Wang
>Assignee: Gengliang Wang
>Priority: Major
>  Labels: pull-request-available
> Fix For: 4.0.0
>
>
> Providing a table schema LOG_SCHEMA, so that users can load structured logs 
> with the following:
> ```
> spark.read.schema(LOG_SCHEMA).json(logPath)
> ```



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2024-03-21 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17829521#comment-17829521
 ] 

Steve Loughran commented on SPARK-38330:


[~jpanda] a bit late but your problem is the WONTFIX issue HADOOP-17017. your 
dotted bucket really doesn't match the AWS certificates. either switch to path 
access as documented in s3a docs or (better) use a bucket without dots in its 
name.

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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 
> 

[jira] [Commented] (SPARK-41392) spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin

2024-03-01 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17822572#comment-17822572
 ] 

Steve Loughran commented on SPARK-41392:


expect an official release this week; this pr will ensure it works

> spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin
> ---
>
> Key: SPARK-41392
> URL: https://issues.apache.org/jira/browse/SPARK-41392
> Project: Spark
>  Issue Type: Sub-task
>  Components: Build
>Affects Versions: 4.0.0
>Reporter: Steve Loughran
>Assignee: Yang Jie
>Priority: Major
>  Labels: pull-request-available
> Fix For: 4.0.0
>
>
> on hadoop trunk (but not the 3.3.x line), spark builds fail with a CNFE
> {code}
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> {code}
> full stack
> {code}
> [ERROR] Failed to execute goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile 
> (scala-test-compile-first) on project spark-sql_2.12: Execution 
> scala-test-compile-first of goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile failed: A required 
> class was missing while executing 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> [ERROR] -
> [ERROR] realm =plugin>net.alchim31.maven:scala-maven-plugin:4.7.2
> [ERROR] strategy = org.codehaus.plexus.classworlds.strategy.SelfFirstStrategy
> [ERROR] urls[0] = 
> file:/Users/stevel/.m2/repository/net/alchim31/maven/scala-maven-plugin/4.7.2/scala-maven-plugin-4.7.2.jar
> [ERROR] urls[1] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/shared/maven-dependency-tree/3.2.0/maven-dependency-tree-3.2.0.jar
> [ERROR] urls[2] = 
> file:/Users/stevel/.m2/repository/org/eclipse/aether/aether-util/1.0.0.v20140518/aether-util-1.0.0.v20140518.jar
> [ERROR] urls[3] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/reporting/maven-reporting-api/3.1.1/maven-reporting-api-3.1.1.jar
> [ERROR] urls[4] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-sink-api/1.11.1/doxia-sink-api-1.11.1.jar
> [ERROR] urls[5] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-logging-api/1.11.1/doxia-logging-api-1.11.1.jar
> [ERROR] urls[6] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/maven-archiver/3.6.0/maven-archiver-3.6.0.jar
> [ERROR] urls[7] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-io/3.4.0/plexus-io-3.4.0.jar
> [ERROR] urls[8] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-interpolation/1.26/plexus-interpolation-1.26.jar
> [ERROR] urls[9] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-exec/1.3/commons-exec-1.3.jar
> [ERROR] urls[10] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-utils/3.4.2/plexus-utils-3.4.2.jar
> [ERROR] urls[11] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-archiver/4.5.0/plexus-archiver-4.5.0.jar
> [ERROR] urls[12] = 
> file:/Users/stevel/.m2/repository/commons-io/commons-io/2.11.0/commons-io-2.11.0.jar
> [ERROR] urls[13] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-compress/1.21/commons-compress-1.21.jar
> [ERROR] urls[14] = 
> file:/Users/stevel/.m2/repository/org/iq80/snappy/snappy/0.4/snappy-0.4.jar
> [ERROR] urls[15] = 
> file:/Users/stevel/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar
> [ERROR] urls[16] = 
> file:/Users/stevel/.m2/repository/com/github/luben/zstd-jni/1.5.2-4/zstd-jni-1.5.2-4.jar
> [ERROR] urls[17] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc_2.13/1.7.1/zinc_2.13-1.7.1.jar
> [ERROR] urls[18] = 
> file:/Users/stevel/.m2/repository/org/scala-lang/scala-library/2.13.8/scala-library-2.13.8.jar
> [ERROR] urls[19] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-core_2.13/1.7.1/zinc-core_2.13-1.7.1.jar
> [ERROR] urls[20] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-apiinfo_2.13/1.7.1/zinc-apiinfo_2.13-1.7.1.jar
> [ERROR] urls[21] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-bridge_2.13/1.7.1/compiler-bridge_2.13-1.7.1.jar
> [ERROR] urls[22] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-classpath_2.13/1.7.1/zinc-classpath_2.13-1.7.1.jar
> [ERROR] urls[23] = 
> file:/Users/stevel/.m2/repository/org/scala-lang/scala-compiler/2.13.8/scala-compiler-2.13.8.jar
> [ERROR] urls[24] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-interface/1.7.1/compiler-interface-1.7.1.jar
> [ERROR] urls[25] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/util-interface/1.7.0/util-interface-1.7.0.jar
> [ERROR] urls[26] = 
> 

[jira] [Updated] (SPARK-41392) spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin

2024-02-28 Thread Steve Loughran (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-41392?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran updated SPARK-41392:
---
Priority: Major  (was: Minor)

> spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin
> ---
>
> Key: SPARK-41392
> URL: https://issues.apache.org/jira/browse/SPARK-41392
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 4.0.0
>Reporter: Steve Loughran
>Priority: Major
>
> on hadoop trunk (but not the 3.3.x line), spark builds fail with a CNFE
> {code}
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> {code}
> full stack
> {code}
> [ERROR] Failed to execute goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile 
> (scala-test-compile-first) on project spark-sql_2.12: Execution 
> scala-test-compile-first of goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile failed: A required 
> class was missing while executing 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> [ERROR] -
> [ERROR] realm =plugin>net.alchim31.maven:scala-maven-plugin:4.7.2
> [ERROR] strategy = org.codehaus.plexus.classworlds.strategy.SelfFirstStrategy
> [ERROR] urls[0] = 
> file:/Users/stevel/.m2/repository/net/alchim31/maven/scala-maven-plugin/4.7.2/scala-maven-plugin-4.7.2.jar
> [ERROR] urls[1] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/shared/maven-dependency-tree/3.2.0/maven-dependency-tree-3.2.0.jar
> [ERROR] urls[2] = 
> file:/Users/stevel/.m2/repository/org/eclipse/aether/aether-util/1.0.0.v20140518/aether-util-1.0.0.v20140518.jar
> [ERROR] urls[3] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/reporting/maven-reporting-api/3.1.1/maven-reporting-api-3.1.1.jar
> [ERROR] urls[4] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-sink-api/1.11.1/doxia-sink-api-1.11.1.jar
> [ERROR] urls[5] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-logging-api/1.11.1/doxia-logging-api-1.11.1.jar
> [ERROR] urls[6] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/maven-archiver/3.6.0/maven-archiver-3.6.0.jar
> [ERROR] urls[7] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-io/3.4.0/plexus-io-3.4.0.jar
> [ERROR] urls[8] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-interpolation/1.26/plexus-interpolation-1.26.jar
> [ERROR] urls[9] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-exec/1.3/commons-exec-1.3.jar
> [ERROR] urls[10] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-utils/3.4.2/plexus-utils-3.4.2.jar
> [ERROR] urls[11] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-archiver/4.5.0/plexus-archiver-4.5.0.jar
> [ERROR] urls[12] = 
> file:/Users/stevel/.m2/repository/commons-io/commons-io/2.11.0/commons-io-2.11.0.jar
> [ERROR] urls[13] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-compress/1.21/commons-compress-1.21.jar
> [ERROR] urls[14] = 
> file:/Users/stevel/.m2/repository/org/iq80/snappy/snappy/0.4/snappy-0.4.jar
> [ERROR] urls[15] = 
> file:/Users/stevel/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar
> [ERROR] urls[16] = 
> file:/Users/stevel/.m2/repository/com/github/luben/zstd-jni/1.5.2-4/zstd-jni-1.5.2-4.jar
> [ERROR] urls[17] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc_2.13/1.7.1/zinc_2.13-1.7.1.jar
> [ERROR] urls[18] = 
> file:/Users/stevel/.m2/repository/org/scala-lang/scala-library/2.13.8/scala-library-2.13.8.jar
> [ERROR] urls[19] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-core_2.13/1.7.1/zinc-core_2.13-1.7.1.jar
> [ERROR] urls[20] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-apiinfo_2.13/1.7.1/zinc-apiinfo_2.13-1.7.1.jar
> [ERROR] urls[21] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-bridge_2.13/1.7.1/compiler-bridge_2.13-1.7.1.jar
> [ERROR] urls[22] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-classpath_2.13/1.7.1/zinc-classpath_2.13-1.7.1.jar
> [ERROR] urls[23] = 
> file:/Users/stevel/.m2/repository/org/scala-lang/scala-compiler/2.13.8/scala-compiler-2.13.8.jar
> [ERROR] urls[24] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-interface/1.7.1/compiler-interface-1.7.1.jar
> [ERROR] urls[25] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/util-interface/1.7.0/util-interface-1.7.0.jar
> [ERROR] urls[26] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-persist-core-assembly/1.7.1/zinc-persist-core-assembly-1.7.1.jar
> [ERROR] urls[27] = 
> 

[jira] [Commented] (SPARK-41392) spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin

2024-02-28 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17821694#comment-17821694
 ] 

Steve Loughran commented on SPARK-41392:


Hadoop 3.4.0 RC2 exhibits this; spark needs its patches in

> spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin
> ---
>
> Key: SPARK-41392
> URL: https://issues.apache.org/jira/browse/SPARK-41392
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 4.0.0
>Reporter: Steve Loughran
>Priority: Minor
>
> on hadoop trunk (but not the 3.3.x line), spark builds fail with a CNFE
> {code}
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> {code}
> full stack
> {code}
> [ERROR] Failed to execute goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile 
> (scala-test-compile-first) on project spark-sql_2.12: Execution 
> scala-test-compile-first of goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile failed: A required 
> class was missing while executing 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> [ERROR] -
> [ERROR] realm =plugin>net.alchim31.maven:scala-maven-plugin:4.7.2
> [ERROR] strategy = org.codehaus.plexus.classworlds.strategy.SelfFirstStrategy
> [ERROR] urls[0] = 
> file:/Users/stevel/.m2/repository/net/alchim31/maven/scala-maven-plugin/4.7.2/scala-maven-plugin-4.7.2.jar
> [ERROR] urls[1] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/shared/maven-dependency-tree/3.2.0/maven-dependency-tree-3.2.0.jar
> [ERROR] urls[2] = 
> file:/Users/stevel/.m2/repository/org/eclipse/aether/aether-util/1.0.0.v20140518/aether-util-1.0.0.v20140518.jar
> [ERROR] urls[3] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/reporting/maven-reporting-api/3.1.1/maven-reporting-api-3.1.1.jar
> [ERROR] urls[4] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-sink-api/1.11.1/doxia-sink-api-1.11.1.jar
> [ERROR] urls[5] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-logging-api/1.11.1/doxia-logging-api-1.11.1.jar
> [ERROR] urls[6] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/maven-archiver/3.6.0/maven-archiver-3.6.0.jar
> [ERROR] urls[7] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-io/3.4.0/plexus-io-3.4.0.jar
> [ERROR] urls[8] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-interpolation/1.26/plexus-interpolation-1.26.jar
> [ERROR] urls[9] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-exec/1.3/commons-exec-1.3.jar
> [ERROR] urls[10] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-utils/3.4.2/plexus-utils-3.4.2.jar
> [ERROR] urls[11] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-archiver/4.5.0/plexus-archiver-4.5.0.jar
> [ERROR] urls[12] = 
> file:/Users/stevel/.m2/repository/commons-io/commons-io/2.11.0/commons-io-2.11.0.jar
> [ERROR] urls[13] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-compress/1.21/commons-compress-1.21.jar
> [ERROR] urls[14] = 
> file:/Users/stevel/.m2/repository/org/iq80/snappy/snappy/0.4/snappy-0.4.jar
> [ERROR] urls[15] = 
> file:/Users/stevel/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar
> [ERROR] urls[16] = 
> file:/Users/stevel/.m2/repository/com/github/luben/zstd-jni/1.5.2-4/zstd-jni-1.5.2-4.jar
> [ERROR] urls[17] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc_2.13/1.7.1/zinc_2.13-1.7.1.jar
> [ERROR] urls[18] = 
> file:/Users/stevel/.m2/repository/org/scala-lang/scala-library/2.13.8/scala-library-2.13.8.jar
> [ERROR] urls[19] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-core_2.13/1.7.1/zinc-core_2.13-1.7.1.jar
> [ERROR] urls[20] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-apiinfo_2.13/1.7.1/zinc-apiinfo_2.13-1.7.1.jar
> [ERROR] urls[21] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-bridge_2.13/1.7.1/compiler-bridge_2.13-1.7.1.jar
> [ERROR] urls[22] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-classpath_2.13/1.7.1/zinc-classpath_2.13-1.7.1.jar
> [ERROR] urls[23] = 
> file:/Users/stevel/.m2/repository/org/scala-lang/scala-compiler/2.13.8/scala-compiler-2.13.8.jar
> [ERROR] urls[24] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-interface/1.7.1/compiler-interface-1.7.1.jar
> [ERROR] urls[25] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/util-interface/1.7.0/util-interface-1.7.0.jar
> [ERROR] urls[26] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-persist-core-assembly/1.7.1/zinc-persist-core-assembly-1.7.1.jar
> [ERROR] urls[27] = 
> 

[jira] [Created] (SPARK-47008) Spark to support S3 Express One Zone Storage

2024-02-08 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-47008:
--

 Summary: Spark to support S3 Express One Zone Storage
 Key: SPARK-47008
 URL: https://issues.apache.org/jira/browse/SPARK-47008
 Project: Spark
  Issue Type: Sub-task
  Components: Spark Core
Affects Versions: 3.5.1
Reporter: Steve Loughran


Hadoop 3.4.0 adds support for AWS S3 Express One Zone Storage.

Most of this is transparent. However, one aspect which can surface as an issue 
is that these stores report prefixes in a listing when there are pending 
uploads, *even when there are no files underneath*

This leads to a situation where a listStatus of a path returns a list of file 
status entries which appears to contain one or more directories -but a 
listStatus on that path raises a FileNotFoundException: there is nothing there.

HADOOP-18996 handles this in all of hadoop code, including FileInputFormat, 

A filesystem can now be probed for inconsistent directoriy listings through 
{{fs.hasPathCapability(path, "fs.capability.directory.listing.inconsistent")}}

If true, then treewalking code SHOULD NOT report a failure if, when walking 
into a subdirectory, a list/getFileStatus on that directory raises a 
FileNotFoundException.

Although most of this is handled in the hadoop code, but there some places 
where treewalking is done inside spark These need to be identified and make 
resilient to failure on the recurse down the tree

* SparkHadoopUtil list methods , 
* especially listLeafStatuses used by OrcFileOperator
org.apache.spark.util.Utils#fetchHcfsFile

{{org.apache.hadoop.fs.FileUtil.maybeIgnoreMissingDirectory()}} can assist 
here, or the logic can be replicated. Using the hadoop implementation would be 
better from a maintenance perspective




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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-45404) Support AWS_ENDPOINT_URL env variable

2024-01-22 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-45404?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17809536#comment-17809536
 ] 

Steve Loughran commented on SPARK-45404:


Just saw this while working on SPARK-35878. 

If you are copying endpoints then you may also want to think about picking up 
the region from AWS_REGION too.

The full list of env vars which I have collected by looking in the AWS SDKs is 
up at 
https://github.com/steveloughran/cloudstore/blob/main/src/main/java/org/apache/hadoop/fs/store/diag/S3ADiagnosticsInfo.java#L379
I do not know what they all mean or do, only that if I get a support call I 
want to know if anyone has been setting them.



> Support AWS_ENDPOINT_URL env variable
> -
>
> Key: SPARK-45404
> URL: https://issues.apache.org/jira/browse/SPARK-45404
> Project: Spark
>  Issue Type: Sub-task
>  Components: Spark Core
>Affects Versions: 4.0.0
>Reporter: Dongjoon Hyun
>Assignee: Dongjoon Hyun
>Priority: Major
>  Labels: pull-request-available
> Fix For: 4.0.0
>
>




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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Updated] (SPARK-46793) Revert S3A endpoint fixup logic of SPARK-35878

2024-01-22 Thread Steve Loughran (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-46793?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran updated SPARK-46793:
---
Summary: Revert S3A endpoint fixup logic of SPARK-35878  (was: Revert 
region fixup logic of SPARK-35878)

> Revert S3A endpoint fixup logic of SPARK-35878
> --
>
> Key: SPARK-46793
> URL: https://issues.apache.org/jira/browse/SPARK-46793
> Project: Spark
>  Issue Type: Sub-task
>  Components: Spark Core
>Affects Versions: 3.5.0, 3.4.3
>Reporter: Steve Loughran
>Priority: Major
>
> The v2 SDK does its region resolution "differently", and the changes of 
> SPARK-35878 actually create problems.
> That PR went in to fix  a regression in Hadoop 3.3.1 which has been fixed 
> since 3.3.2; removing it is not going to cause problems on anyone not using 
> the 3.3.1 release, which is 3 years old and replaced by multiple follow on 
> 3.3.x releases



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-46793) Revert region fixup logic of SPARK-35878

2024-01-22 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-46793:
--

 Summary: Revert region fixup logic of SPARK-35878
 Key: SPARK-46793
 URL: https://issues.apache.org/jira/browse/SPARK-46793
 Project: Spark
  Issue Type: Sub-task
  Components: Spark Core
Affects Versions: 3.5.0, 3.4.3
Reporter: Steve Loughran


The v2 SDK does its region resolution "differently", and the changes of 
SPARK-35878 actually create problems.

That PR went in to fix  a regression in Hadoop 3.3.1 which has been fixed since 
3.3.2; removing it is not going to cause problems on anyone not using the 3.3.1 
release, which is 3 years old and replaced by multiple follow on 3.3.x releases



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-46247) Invalid bucket file error when reading from bucketed table created with PathOutputCommitProtocol

2024-01-18 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-46247?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17808227#comment-17808227
 ] 

Steve Loughran commented on SPARK-46247:


why is the file invalid? any more stack trace?

# try using s3a:// as the prefix all the way through
# is there really a "." at the end of the filenames.

The directory committer was netflix's design for incremental update of an 
existing table, where a partition could be deleted before new data was 
committed.

unless you want to do this, use the magic or (second best) staging committer


> Invalid bucket file error when reading from bucketed table created with 
> PathOutputCommitProtocol
> 
>
> Key: SPARK-46247
> URL: https://issues.apache.org/jira/browse/SPARK-46247
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.5.0
>Reporter: Никита Соколов
>Priority: Major
>
> I am trying to create an external partioned bucketed table using this code:
> {code:java}
> spark.read.parquet("s3://faucct/input")
>   .repartition(128, col("product_id"))
>   .write.partitionBy("features_date").bucketBy(128, "product_id")
>   .option("path", "s3://faucct/tmp/output")
>   .option("compression", "uncompressed")
>   .saveAsTable("tmp.output"){code}
> At first it took more time than expected because it had to rename a lot of 
> files in the end, which requires copying in S3. But I have used the 
> configuration from the documentation – 
> [https://spark.apache.org/docs/3.0.0-preview/cloud-integration.html#committing-work-into-cloud-storage-safely-and-fast]:
> {code:java}
> spark.hadoop.fs.s3a.committer.name directory
> spark.sql.sources.commitProtocolClass 
> org.apache.spark.internal.io.cloud.PathOutputCommitProtocol
> spark.sql.parquet.output.committer.class 
> org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter {code}
> It is properly partitioned: every partition_date has exactly 128 files named 
> like 
> [part-00117-43293810-d0e9-4eee-9be8-e9e50a3e10fd_00117-5eb66a54-2fbb-4775-8f3b-3040b2966a71.c000.parquet|https://s3.console.aws.amazon.com/s3/object/joom-analytics-recom?region=eu-central-1=recom/dataset/best/best-to-cart-rt/user-product-v4/to_cart-faucct/fnw/ipw/msv2/2023-09-15/14d/tmp_3/features_date%3D2023-09-01/part-00117-43293810-d0e9-4eee-9be8-e9e50a3e10fd_00117-5eb66a54-2fbb-4775-8f3b-3040b2966a71.c000.parquet].
> Then I am trying to join this table with another one, for example like this:
> {code:java}
> spark.table("tmp.output").repartition(128, $"product_id")
>   .join(spark.table("tmp.output").repartition(128, $"product_id"), 
> Seq("product_id")).count(){code}
> Because of the configuration I get the following errors:
> {code:java}
> org.apache.spark.SparkException: [INVALID_BUCKET_FILE] Invalid bucket file: 
> s3://faucct/tmp/output/features_date=2023-09-01/part-0-43293810-d0e9-4eee-9be8-e9e50a3e10fd_0-5eb66a54-2fbb-4775-8f3b-3040b2966a71.c000.parquet.
>   at 
> org.apache.spark.sql.errors.QueryExecutionErrors$.invalidBucketFile(QueryExecutionErrors.scala:2731)
>   at 
> org.apache.spark.sql.execution.FileSourceScanExec.$anonfun$createBucketedReadRDD$5(DataSourceScanExec.scala:636)
>  {code}



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-44124) Upgrade AWS SDK to v2

2023-10-25 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-44124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17779460#comment-17779460
 ] 

Steve Loughran commented on SPARK-44124:


good document

# I think you could consider cutting the kinesis module entirely. its lack of 
maintenance and bug reports indicates nobody uses it.
# we don't have a timetable for a 3.4.x release. end of year for the beta phase 
has been discussed -but I'm not going to manage that release. There's a lot of 
other changes there, and we are still trying to stabilize that v2 code in 
HADOOP-18886 with some fairly major regressions surfacing HADOOP-18945.

one strateigy
# If you cut kinesis then its only the k8s tests which use the sdk...and you 
don't need to explicitly redistribute either sdk jar.
# remove the exclusion of the v1 aws-sdk from hadoop-cloud module imports and 
then all hadoop releases built on v1 get that jar included in the distro
# and if anyone builds with v2 then they'll get the exact v2 release that the 
s3a connector has been built with. this matters as the v2 sdk is a fairly major 
migration and any bugrep against the s3a connector which doesn't use the sdk 
version we released against is going to left to the submitter to troubleshoot.


> Upgrade AWS SDK to v2
> -
>
> Key: SPARK-44124
> URL: https://issues.apache.org/jira/browse/SPARK-44124
> Project: Spark
>  Issue Type: Sub-task
>  Components: Build
>Affects Versions: 4.0.0
>Reporter: Dongjoon Hyun
>Priority: Major
>
> Here is a design doc:
> https://docs.google.com/document/d/1nGWbGTqxuFBG2ftfYYXxzrkipINILfWCOwse36yg7Ig/edit?usp=sharing



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38958) Override S3 Client in Spark Write/Read calls

2023-09-11 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38958?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17763725#comment-17763725
 ] 

Steve Loughran commented on SPARK-38958:


[~hershalb] hadoop trunk is now on v2 sdk, but we are still stabilising client 
binding. 



> Override S3 Client in Spark Write/Read calls
> 
>
> Key: SPARK-38958
> URL: https://issues.apache.org/jira/browse/SPARK-38958
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Affects Versions: 3.2.1
>Reporter: Hershal
>Priority: Major
>
> Hello,
> I have been working to use spark to read and write data to S3. Unfortunately, 
> there are a few S3 headers that I need to add to my spark read/write calls. 
> After much looking, I have not found a way to replace the S3 client that 
> spark uses to make the read/write calls. I also have not found a 
> configuration that allows me to pass in S3 headers. Here is an example of 
> some common S3 request headers 
> ([https://docs.aws.amazon.com/AmazonS3/latest/API/RESTCommonRequestHeaders.html).]
>  Does there already exist functionality to add S3 headers to spark read/write 
> calls or pass in a custom client that would pass these headers on every 
> read/write request? Appreciate the help and feedback
>  
> Thanks,



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-44884) Spark doesn't create SUCCESS file in Spark 3.3.0+ when partitionOverwriteMode is dynamic

2023-08-25 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-44884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17759063#comment-17759063
 ] 

Steve Loughran commented on SPARK-44884:


so using insert overwrite. yes, what happens there is that the entire job is 
written to a temporary subdir, and on job complete the partition directories 
are updated, but not any files on the root path created by job committer itself

> Spark doesn't create SUCCESS file in Spark 3.3.0+ when partitionOverwriteMode 
> is dynamic
> 
>
> Key: SPARK-44884
> URL: https://issues.apache.org/jira/browse/SPARK-44884
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.3.0
>Reporter: Dipayan Dev
>Priority: Critical
> Attachments: image-2023-08-20-18-46-53-342.png, 
> image-2023-08-25-13-01-42-137.png
>
>
> The issue is not happening in Spark 2.x (I am using 2.4.0), but only in 3.3.0 
> (tested with 3.4.1 as well)
> Code to reproduce the issue
>  
> {code:java}
> scala> spark.conf.set("spark.sql.sources.partitionOverwriteMode", "dynamic") 
> scala> val DF = Seq(("test1", 123)).toDF("name", "num")
> scala> DF.write.option("path", 
> "gs://test_bucket/table").mode("overwrite").partitionBy("num").format("orc").saveAsTable("test_schema.test_tb1")
>  {code}
>  
> The above code succeeds and creates external Hive table, but {*}there is no 
> SUCCESS file generated{*}.
> Adding the content of the bucket after table creation
> !image-2023-08-25-13-01-42-137.png|width=500,height=130!
>  The same code when running with spark 2.4.0 (with or without external path), 
> generates the SUCCESS file.
> {code:java}
> scala> 
> DF.write.mode(SaveMode.Overwrite).partitionBy("num").format("orc").saveAsTable("test_schema.test_tb1"){code}
> !image-2023-08-20-18-46-53-342.png|width=465,height=166!
>  



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-44884) Spark doesn't create SUCCESS file when external path is passed

2023-08-23 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-44884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17758167#comment-17758167
 ] 

Steve Loughran commented on SPARK-44884:


i'm not trying to replicate it; i have too many other things to do. in open 
source, sadly, everyone gets to fend for themselves, and i'm not actually a 
spark developer. i'd suggest looking at what changed in .saveAsTable to see 
what possible changes may be to blame...

> Spark doesn't create SUCCESS file when external path is passed
> --
>
> Key: SPARK-44884
> URL: https://issues.apache.org/jira/browse/SPARK-44884
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.3.0
>Reporter: Dipayan Dev
>Priority: Critical
> Attachments: image-2023-08-20-18-08-38-531.png, 
> image-2023-08-20-18-46-53-342.png
>
>
> The issue is not happening in Spark 2.x (I am using 2.4.0), but only in 3.3.0
> Code to reproduce the issue.
>  
> {code:java}
> scala> spark.conf.set("spark.sql.orc.char.enabled", true)
> scala> val DF = Seq(("test1", 123)).toDF("name", "num")
> scala> DF.write.option("path", 
> "gs://test_dd123/").mode(SaveMode.Overwrite).partitionBy("num").format("orc").saveAsTable("test_schema.table_name")
> 23/08/20 12:31:43 WARN SessionState: METASTORE_FILTER_HOOK will be ignored, 
> since hive.security.authorization.manager is set to instance of 
> HiveAuthorizerFactory.   {code}
> The above code succeeds and creates the External Hive table, but {*}there is 
> no SUCCESS file generated{*}. The same code when running spark 2.4.0, 
> generating a SUCCESS file.
> Adding the content of the bucket after table creation
>  
> !image-2023-08-20-18-08-38-531.png|width=453,height=162!
>  
> But when I don’t pass the external path as following, the SUCCESS file is 
> generated
> {code:java}
> scala> 
> DF.write.mode(SaveMode.Overwrite).partitionBy("num").format("orc").saveAsTable("us_wm_supply_chain_rcv_pre_prod.test_tb1")
>  {code}
> !image-2023-08-20-18-46-53-342.png|width=465,height=166!
>  



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-44884) Spark doesn't create SUCCESS file when external path is passed

2023-08-22 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-44884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17757547#comment-17757547
 ] 

Steve Loughran commented on SPARK-44884:


[~dipayandev] i don't think think anyone has disabled the option; doesn't 
surface in my test setup (manifest and s3a committers).

Afraid you are going to have to debug it yourself, as it is your env which has 
the problem.

does everything work if you use .saveAs() rather than .saveAsTable()?

> Spark doesn't create SUCCESS file when external path is passed
> --
>
> Key: SPARK-44884
> URL: https://issues.apache.org/jira/browse/SPARK-44884
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.3.0
>Reporter: Dipayan Dev
>Priority: Critical
> Attachments: image-2023-08-20-18-08-38-531.png, 
> image-2023-08-20-18-46-53-342.png
>
>
> The issue is not happening in Spark 2.x (I am using 2.4.0), but only in 3.3.0
> Code to reproduce the issue.
>  
> {code:java}
> scala> spark.conf.set("spark.sql.orc.char.enabled", true)
> scala> val DF = Seq(("test1", 123)).toDF("name", "num")
> scala> DF.write.option("path", 
> "gs://test_dd123/").mode(SaveMode.Overwrite).partitionBy("num").format("orc").saveAsTable("test_schema.table_name")
> 23/08/20 12:31:43 WARN SessionState: METASTORE_FILTER_HOOK will be ignored, 
> since hive.security.authorization.manager is set to instance of 
> HiveAuthorizerFactory.   {code}
> The above code succeeds and creates the External Hive table, but {*}there is 
> no SUCCESS file generated{*}. The same code when running spark 2.4.0, 
> generating a SUCCESS file.
> Adding the content of the bucket after table creation
>  
> !image-2023-08-20-18-08-38-531.png|width=453,height=162!
>  
> But when I don’t pass the external path as following, the SUCCESS file is 
> generated
> {code:java}
> scala> 
> DF.write.mode(SaveMode.Overwrite).partitionBy("num").format("orc").saveAsTable("us_wm_supply_chain_rcv_pre_prod.test_tb1")
>  {code}
> !image-2023-08-20-18-46-53-342.png|width=465,height=166!
>  



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38958) Override S3 Client in Spark Write/Read calls

2023-08-22 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38958?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17757364#comment-17757364
 ] 

Steve Loughran commented on SPARK-38958:


[~hershalb] we are about to merge the v2 sdk feature set; it'd be good for you 
to see if your changes work there.

as for static headers, I could imagine something like we added in HADOOP-17833 
for adding headers to created files.

# Define a well know prefix, e.g {{fs.s3a.request.headers.))
# every key which matches fs.s3a.request.headers.* becomes a header; the value 
the header value.

the alternative is as done for custom signers, a list of key=value separated by 
commas.

> Override S3 Client in Spark Write/Read calls
> 
>
> Key: SPARK-38958
> URL: https://issues.apache.org/jira/browse/SPARK-38958
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Affects Versions: 3.2.1
>Reporter: Hershal
>Priority: Major
>
> Hello,
> I have been working to use spark to read and write data to S3. Unfortunately, 
> there are a few S3 headers that I need to add to my spark read/write calls. 
> After much looking, I have not found a way to replace the S3 client that 
> spark uses to make the read/write calls. I also have not found a 
> configuration that allows me to pass in S3 headers. Here is an example of 
> some common S3 request headers 
> ([https://docs.aws.amazon.com/AmazonS3/latest/API/RESTCommonRequestHeaders.html).]
>  Does there already exist functionality to add S3 headers to spark read/write 
> calls or pass in a custom client that would pass these headers on every 
> read/write request? Appreciate the help and feedback
>  
> Thanks,



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-44884) Spark doesn't create SUCCESS file when external path is passed

2023-08-21 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-44884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17757036#comment-17757036
 ] 

Steve Loughran commented on SPARK-44884:


this is created in the committer; for hadoop-mapreduce ones  
"mapreduce.fileoutputcommitter.marksuccessfuljobs"; is the flag to enable this; 
if it is not being created then it'll be down to how saveAsTable commits work

> Spark doesn't create SUCCESS file when external path is passed
> --
>
> Key: SPARK-44884
> URL: https://issues.apache.org/jira/browse/SPARK-44884
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.3.0
>Reporter: Dipayan Dev
>Priority: Critical
> Attachments: image-2023-08-20-18-08-38-531.png, 
> image-2023-08-20-18-46-53-342.png
>
>
> The issue is not happening in Spark 2.x (I am using 2.4.0), but only in 3.3.0
> Code to reproduce the issue.
>  
> {code:java}
> scala> spark.conf.set("spark.sql.orc.char.enabled", true)
> scala> val DF = Seq(("test1", 123)).toDF("name", "num")
> scala> DF.write.option("path", 
> "gs://test_dd123/").mode(SaveMode.Overwrite).partitionBy("num").format("orc").saveAsTable("test_schema.table_name")
> 23/08/20 12:31:43 WARN SessionState: METASTORE_FILTER_HOOK will be ignored, 
> since hive.security.authorization.manager is set to instance of 
> HiveAuthorizerFactory.   {code}
> The above code succeeds and creates the External Hive table, but {*}there is 
> no SUCCESS file generated{*}. The same code when running spark 2.4.0, 
> generating a SUCCESS file.
> Adding the content of the bucket after table creation
>  
> !image-2023-08-20-18-08-38-531.png|width=453,height=162!
>  
> But when I don’t pass the external path as following, the SUCCESS file is 
> generated
> {code:java}
> scala> 
> DF.write.mode(SaveMode.Overwrite).partitionBy("num").format("orc").saveAsTable("us_wm_supply_chain_rcv_pre_prod.test_tb1")
>  {code}
> !image-2023-08-20-18-46-53-342.png|width=465,height=166!
>  



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Resolved] (SPARK-44883) Spark insertInto with location GCS bucket root causes NPE

2023-08-21 Thread Steve Loughran (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-44883?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran resolved SPARK-44883.

Resolution: Duplicate

> Spark insertInto with location GCS bucket root causes NPE
> -
>
> Key: SPARK-44883
> URL: https://issues.apache.org/jira/browse/SPARK-44883
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.3.0
>Reporter: Dipayan Dev
>Priority: Minor
>
> In our Organisation, we are using GCS bucket root location to point to our 
> Hive table. Dataproc's latest 2.1 uses *Spark* *3.3.0* and this needs to be 
> fixed.
> Spark Scala code to reproduce this issue
> {noformat}
> val DF = Seq(("test1", 123)).toDF("name", "num")
> DF.write.option("path", 
> "gs://test_dd123/").mode(SaveMode.Overwrite).partitionBy("num").format("orc").saveAsTable("schema_name.table_name")
> val DF1 = Seq(("test2", 125)).toDF("name", "num")
> DF.write.mode(SaveMode.Overwrite).format("orc").insertInto("schema_name.table_name")
> java.lang.NullPointerException
>   at org.apache.hadoop.fs.Path.(Path.java:141)
>   at org.apache.hadoop.fs.Path.(Path.java:120)
>   at org.apache.hadoop.fs.Path.suffix(Path.java:441)
>   at 
> org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.$anonfun$getCustomPartitionLocations$1(InsertIntoHadoopFsRelationCommand.scala:254)
>  {noformat}
> Looks like the issue is coming from Hadoop Path. 
> {noformat}
> scala> import org.apache.hadoop.fs.Path
> import org.apache.hadoop.fs.Path
> scala> val path: Path = new Path("gs://test_dd123/")
> path: org.apache.hadoop.fs.Path = gs://test_dd123/
> scala> path.suffix("/num=123")
> java.lang.NullPointerException
>   at org.apache.hadoop.fs.Path.(Path.java:150)
>   at org.apache.hadoop.fs.Path.(Path.java:129)
>   at org.apache.hadoop.fs.Path.suffix(Path.java:450){noformat}
> Path.suffix throughs NPE when writing into GS buckets root. 
>  



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-44124) Upgrade AWS SDK to v2

2023-08-15 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-44124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17754698#comment-17754698
 ] 

Steve Loughran commented on SPARK-44124:


+will need to make sure any classloaders set up to pass down com.amazonaws to 
children (e.g. hive classloader) now pass down software.amazon

> Upgrade AWS SDK to v2
> -
>
> Key: SPARK-44124
> URL: https://issues.apache.org/jira/browse/SPARK-44124
> Project: Spark
>  Issue Type: Sub-task
>  Components: Build
>Affects Versions: 4.0.0
>Reporter: Dongjoon Hyun
>Priority: Major
>




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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-44116) Utilize Hadoop vectorized APIs

2023-07-31 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-44116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17749266#comment-17749266
 ] 

Steve Loughran commented on SPARK-44116:


If this gets into the libraries, you don't need explicit support in spark 
unless you really want to do your own.

what could be good is replacing FileSystem.open() with the openFile() builder, 
passing in your read policy and any file status/file length you have. saves 
HEAD requests and tunes GET/prefetching based on expected use.

> Utilize Hadoop vectorized APIs
> --
>
> Key: SPARK-44116
> URL: https://issues.apache.org/jira/browse/SPARK-44116
> Project: Spark
>  Issue Type: Sub-task
>  Components: Spark Core
>Affects Versions: 4.0.0
>Reporter: Dongjoon Hyun
>Priority: Major
>
> Apache Hadoop 3.3.5+ supports vectorized APIs.



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-44124) Upgrade AWS SDK to v2

2023-07-31 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-44124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17749264#comment-17749264
 ] 

Steve Loughran commented on SPARK-44124:


we are soon to move hadoop trunk up to SDK v2, stabilise it with a goal to 
backport ASAP: HADOOP-18818 any help with testing this would be wonderful.

for spark
* usual classpath issues (bundle.jar + aws-crt.jar)
* spark-kinesis will need to move to the sdk or spark continues to ship v1 with 
that module (i.e. they can coexist)
* any testing people can do
apart from classpath pain, unless people are actually extending bits of the s3a 
client by plugging in classes (credentials, signing, delegation,...) the move 
seems painless

> Upgrade AWS SDK to v2
> -
>
> Key: SPARK-44124
> URL: https://issues.apache.org/jira/browse/SPARK-44124
> Project: Spark
>  Issue Type: Sub-task
>  Components: Build
>Affects Versions: 4.0.0
>Reporter: Dongjoon Hyun
>Priority: Major
>




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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-44042) SPIP: PySpark Test Framework

2023-06-21 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-44042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17735646#comment-17735646
 ] 

Steve Loughran commented on SPARK-44042:


* you can create an independent git repo for this (ASF self service) and so 
have a different release timetable and get it out to older versions. 
* This does require testing across versions, but separate modules can do that. 
see https://github.com/apache/hadoop-api-shim for an example. Base module has 
tests, separate mvn module to run those tests against a given version
* testing test runners is fun
* anything which can be done to provide meaningful reports, stack traces, *and 
set a good example to users*  is critical. The design goal should be (for all 
test frameworks, IMO), "does the output report on its own provide enough 
information to diagnose and fix the failure" -the way junit  baseline "assert 
failure at line 308" doesn't

> SPIP: PySpark Test Framework
> 
>
> Key: SPARK-44042
> URL: https://issues.apache.org/jira/browse/SPARK-44042
> Project: Spark
>  Issue Type: New Feature
>  Components: PySpark
>Affects Versions: 3.5.0
>Reporter: Amanda Liu
>Priority: Major
>
> Currently, there's no official PySpark test framework, but only various 
> open-source repos and blog posts. Many of these open-source resources are 
> very popular, which demonstrates user-demand for PySpark testing 
> capabilities. 
> [spark-testing-base|https://github.com/holdenk/spark-testing-base] has 1.4k 
> stars, and [chispa|https://github.com/MrPowers/chispa] has 532k 
> downloads/month. However, it can be confusing for users to piece together 
> disparate resources to write their own PySpark tests (see [The Elephant in 
> the Room: How to Write PySpark 
> Tests|https://towardsdatascience.com/the-elephant-in-the-room-how-to-write-pyspark-unit-tests-a5073acabc34]).
>  We can streamline and simplify the testing process by incorporating test 
> features, such as a PySpark Test Base class (which allows tests to share 
> Spark sessions) and test util functions (for example, asserting dataframe and 
> schema equality). Please see the full SPIP document attached: 
> [https://docs.google.com/document/d/1OkyBn3JbEHkkQgSQ45Lq82esXjr9rm2Vj7Ih_4zycRc/edit#heading=h.f5f0u2riv07v].



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-41599) Memory leak in FileSystem.CACHE when submitting apps to secure cluster using InProcessLauncher

2023-06-16 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17733454#comment-17733454
 ] 

Steve Loughran commented on SPARK-41599:


correct. remember, all the source of hadoop is there for you to open in your 
IDE -it's the only way to be sure

> Memory leak in FileSystem.CACHE when submitting apps to secure cluster using 
> InProcessLauncher
> --
>
> Key: SPARK-41599
> URL: https://issues.apache.org/jira/browse/SPARK-41599
> Project: Spark
>  Issue Type: Bug
>  Components: Deploy, YARN
>Affects Versions: 3.1.2
>Reporter: Maciej Smolenski
>Priority: Major
> Attachments: InProcLaunchFsIssue.scala, 
> SPARK-41599-fixes-to-limit-FileSystem-CACHE-size-when-using-InProcessLauncher.diff
>
>
> When submitting spark application in kerberos environment the credentials of 
> 'current user' (UserGroupInformation.getCurrentUser()) are being modified.
> Filesystem.CACHE entries contain 'current user' (with user credentials) as a 
> key.
> Submitting many spark applications using InProcessLauncher cause that 
> FileSystem.CACHE becomes bigger and bigger.
> Finally process exits because of OutOfMemory error.
> Code for reproduction attached.
>  
> Output from running 'jmap -histo' on reproduction jvm shows that the number 
> of FileSystem$Cache$Key increases in time:
> time: #instances class
> 1671533274: 2 org.apache.hadoop.fs.FileSystem$Cache$Key
> 167155: 11 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533395: 21 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533455: 30 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533515: 39 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533576: 48 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533636: 57 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533696: 66 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533757: 75 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533817: 84 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533877: 93 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533937: 102 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533998: 111 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534058: 120 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534118: 135 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534178: 140 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534239: 150 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534299: 159 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534359: 168 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534419: 177 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534480: 186 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534540: 195 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534600: 204 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534661: 213 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534721: 222 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534781: 231 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534841: 240 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534902: 249 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534962: 257 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535022: 264 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535083: 273 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535143: 282 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535203: 291 org.apache.hadoop.fs.FileSystem$Cache$Key



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-43170) The spark sql like statement is pushed down to parquet for execution, but the data cannot be queried

2023-04-28 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-43170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17717611#comment-17717611
 ] 

Steve Loughran commented on SPARK-43170:


FWIW, using  S3 URLs  's3://x/dwm_user_app_action_sum_all' means it's an 
AWS EMR deployment, with their private fork of spark, etc. you might want to 
raise a support case there

> The spark sql like statement is pushed down to parquet for execution, but the 
> data cannot be queried
> 
>
> Key: SPARK-43170
> URL: https://issues.apache.org/jira/browse/SPARK-43170
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.2
>Reporter: todd
>Priority: Major
> Attachments: image-2023-04-18-10-59-30-199.png, 
> image-2023-04-19-10-59-44-118.png, screenshot-1.png
>
>
> --DDL
> CREATE TABLE `ecom_dwm`.`dwm_user_app_action_sum_all` (
>   `gaid` STRING COMMENT '',
>   `beyla_id` STRING COMMENT '',
>   `dt` STRING,
>   `hour` STRING,
>   `appid` STRING COMMENT '包名')
> USING parquet
> PARTITIONED BY (dt, hour, appid)
> LOCATION 's3://x/dwm_user_app_action_sum_all'
> – partitions  info
> show partitions ecom_dwm.dwm_user_app_action_sum_all PARTITION 
> (dt='20230412');
>  
> dt=20230412/hour=23/appid=blibli.mobile.commerce
> dt=20230412/hour=23/appid=cn.shopee.app
> dt=20230412/hour=23/appid=cn.shopee.br
> dt=20230412/hour=23/appid=cn.shopee.id
> dt=20230412/hour=23/appid=cn.shopee.my
> dt=20230412/hour=23/appid=cn.shopee.ph
>  
> — query
> select DISTINCT(appid) from ecom_dwm.dwm_user_app_action_sum_all
> where dt='20230412' and appid like '%shopee%'
>  
> --result
>  nodata 
>  
> — other
> I use spark3.0.1 version and trino query engine to query the data。
>  
>  
> The physical execution node formed by spark 3.2
> (3) Scan parquet ecom_dwm.dwm_user_app_action_sum_all Output [3]: [dt#63, 
> hour#64, appid#65|#63, hour#64, appid#65] Batched: true Location: 
> InMemoryFileIndex []
> PartitionFilters: [isnotnull(dt#63), isnotnull(appid#65), (dt#63 = 20230412), 
> Contains(appid#65, shopee)|#63), isnotnull(appid#65), (dt#63 = 20230412), 
> Contains(appid#65, shopee)] ReadSchema: struct<>
>  
>  
> !image-2023-04-18-10-59-30-199.png!
>  
>  – sql plan detail
> {code:java}
> == Physical Plan ==
> CollectLimit (9)
> +- InMemoryTableScan (1)
>   +- InMemoryRelation (2)
> +- * HashAggregate (8)
>+- Exchange (7)
>   +- * HashAggregate (6)
>  +- * Project (5)
> +- * ColumnarToRow (4)
>+- Scan parquet 
> ecom_dwm.dwm_user_app_action_sum_all (3)
> (1) InMemoryTableScan
> Output [1]: [appid#65]
> Arguments: [appid#65]
> (2) InMemoryRelation
> Arguments: [appid#65], 
> CachedRDDBuilder(org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer@ab5af13,StorageLevel(disk,
>  memory, deserialized, 1 replicas),*(2) HashAggregate(keys=[appid#65], 
> functions=[], output=[appid#65])
> +- Exchange hashpartitioning(appid#65, 200), ENSURE_REQUIREMENTS, [plan_id=24]
>+- *(1) HashAggregate(keys=[appid#65], functions=[], output=[appid#65])
>   +- *(1) Project [appid#65]
>  +- *(1) ColumnarToRow
> +- FileScan parquet 
> ecom_dwm.dwm_user_app_action_sum_all[dt#63,hour#64,appid#65] Batched: true, 
> DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(0 paths)[], 
> PartitionFilters: [isnotnull(dt#63), isnotnull(appid#65), (dt#63 = 20230412), 
> StartsWith(appid#65, com)], PushedFilters: [], ReadSchema: struct<>
> ,None)
> (3) Scan parquet ecom_dwm.dwm_user_app_action_sum_all
> Output [3]: [dt#63, hour#64, appid#65]
> Batched: true
> Location: InMemoryFileIndex []
> PartitionFilters: [isnotnull(dt#63), isnotnull(appid#65), (dt#63 = 20230412), 
> StartsWith(appid#65, com)]
> ReadSchema: struct<>
> (4) ColumnarToRow [codegen id : 1]
> Input [3]: [dt#63, hour#64, appid#65]
> (5) Project [codegen id : 1]
> Output [1]: [appid#65]
> Input [3]: [dt#63, hour#64, appid#65]
> (6) HashAggregate [codegen id : 1]
> Input [1]: [appid#65]
> Keys [1]: [appid#65]
> Functions: []
> Aggregate Attributes: []
> Results [1]: [appid#65]
> (7) Exchange
> Input [1]: [appid#65]
> Arguments: hashpartitioning(appid#65, 200), ENSURE_REQUIREMENTS, [plan_id=24]
> (8) HashAggregate [codegen id : 2]
> Input [1]: [appid#65]
> Keys [1]: [appid#65]
> Functions: []
> Aggregate Attributes: []
> Results [1]: [appid#65]
> (9) CollectLimit
> Input [1]: [appid#65]
> Arguments: 1 {code}



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-40034) PathOutputCommitters to work with dynamic partition overwrite

2023-03-01 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-40034?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17694969#comment-17694969
 ] 

Steve Loughran commented on SPARK-40034:


thanks for the update. I will get that new pr done ready for a release with the 
3.3.5 binaries.

(Fwiw insert overwrite does work for parquet data where the hack of binding 
through a subclass is used, as that then works through the base protocol; its 
using ORC where things play up

> PathOutputCommitters to work with dynamic partition overwrite
> -
>
> Key: SPARK-40034
> URL: https://issues.apache.org/jira/browse/SPARK-40034
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core, SQL
>Affects Versions: 3.5.0
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>Priority: Minor
> Fix For: 3.5.0
>
>
> sibling of MAPREDUCE-7403: allow PathOutputCommitter implementation to 
> declare that they support the semantics required by spark dynamic 
> partitioning:
> * rename to work as expected
> * working dir to be on same fs as final dir
> They will do this through implementing StreamCapabilities and adding a new 
> probe, "mapreduce.job.committer.dynamic.partitioning" ; the spark side 
> changes are to
> * postpone rejection of dynamic partition overwrite until the output 
> committer is created
> * allow it if the committer implements StreamCapabilities and returns true 
> for {{hasCapability("mapreduce.job.committer.dynamic.partitioning")))
> this isn't going to be supported by the s3a committers, they don't meet the 
> requirements. The manifest committer of MAPREDUCE-7341 running against abfs 
> and gcs does work. 



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-42537) Remove obsolete/superfluous imports in spark-hadoop-cloud module

2023-02-23 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-42537?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17692617#comment-17692617
 ] 

Steve Loughran commented on SPARK-42537:


FYI +[~dannycjones].
I'm getting build issues related to compiling spark against the hadoop 3.3.5 
RC1 because sparks jackson-cbor maven download is playing up, *even though it's 
not been needed for years*

> Remove obsolete/superfluous imports in spark-hadoop-cloud module
> 
>
> Key: SPARK-42537
> URL: https://issues.apache.org/jira/browse/SPARK-42537
> Project: Spark
>  Issue Type: Improvement
>  Components: Build
>Affects Versions: 3.4.0
>Reporter: Steve Loughran
>Priority: Minor
>
> The explicit imports into hadoop-cloud are obsolete
> * the hadoop-cloud-storage pom is a cut down export of the bindings to the 
> various cloud stores in their hadoop-* modules
> * it's been shipping since hadoop 2.10
> * its grown to include cos and allyun support
> * fairly well tested
> * actually cuts removed support (hadoop-openstack) when withdrawn. Hadoop 
> 3.3.5 has done this, leaving a stub jar there just to avoid breaking 
> downstream builds like spark's current setup.
> hadoop-cloud-storage *should* be all that's needed.
> I know that the spark hadoop-2 profile still references the (long unsupported 
> 2.7.x), but if you are using those releases then really you aren't going to 
> talk to cloud infra
> * no abfs connector
> * s3n connector which won't authenticate with any of the aws regions launched 
> in the past 5-8 years
> * gcs connector won't work (its java11+; hadoop 3.2.x is minimum for java11 
> clients)
> * none of the new chinese cloud services
> * s3a connector very outdated.
> * s3a connector using unshaded aws client which is unlikely to work with 
> versions of jackson, httpclient written in the last 5 years, has trouble on 
> java8 etc.
> Proposed
> * hadoop-2 profile to be the minimal hadoop-aws and hadoop-azure dependencies 
> in the code today. cutting to the empty set would be better, but a bit more 
> radical
> * hadoop-3 profile to pull in hadoop-cloud-storage (excluding aws sdk as 
> today), *and nothing else*
> This will simplify everyone's life as there are fewer dependencies to 
> reconcile. 
> see also SPARK-39969 proposing making the hadoop-aws versions of the 
> aws-sdk-bundle the normative one, as it is now newer than the spark-kinesis 
> import and more broadly tested



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-42537) Remove obsolete/superfluous imports in spark-hadoop-cloud module

2023-02-23 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-42537:
--

 Summary: Remove obsolete/superfluous imports in spark-hadoop-cloud 
module
 Key: SPARK-42537
 URL: https://issues.apache.org/jira/browse/SPARK-42537
 Project: Spark
  Issue Type: Improvement
  Components: Build
Affects Versions: 3.4.0
Reporter: Steve Loughran


The explicit imports into hadoop-cloud are obsolete

* the hadoop-cloud-storage pom is a cut down export of the bindings to the 
various cloud stores in their hadoop-* modules
* it's been shipping since hadoop 2.10
* its grown to include cos and allyun support
* fairly well tested
* actually cuts removed support (hadoop-openstack) when withdrawn. Hadoop 3.3.5 
has done this, leaving a stub jar there just to avoid breaking downstream 
builds like spark's current setup.

hadoop-cloud-storage *should* be all that's needed.

I know that the spark hadoop-2 profile still references the (long unsupported 
2.7.x), but if you are using those releases then really you aren't going to 
talk to cloud infra
* no abfs connector
* s3n connector which won't authenticate with any of the aws regions launched 
in the past 5-8 years
* gcs connector won't work (its java11+; hadoop 3.2.x is minimum for java11 
clients)
* none of the new chinese cloud services
* s3a connector very outdated.
* s3a connector using unshaded aws client which is unlikely to work with 
versions of jackson, httpclient written in the last 5 years, has trouble on 
java8 etc.

Proposed
* hadoop-2 profile to be the minimal hadoop-aws and hadoop-azure dependencies 
in the code today. cutting to the empty set would be better, but a bit more 
radical
* hadoop-3 profile to pull in hadoop-cloud-storage (excluding aws sdk as 
today), *and nothing else*

This will simplify everyone's life as there are fewer dependencies to 
reconcile. 

see also SPARK-39969 proposing making the hadoop-aws versions of the 
aws-sdk-bundle the normative one, as it is now newer than the spark-kinesis 
import and more broadly tested




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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-40034) PathOutputCommitters to work with dynamic partition overwrite

2023-01-19 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-40034?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17678814#comment-17678814
 ] 

Steve Loughran commented on SPARK-40034:


Note that these changes aren't sufficient. The hadoop 3.3.5 manifest committer 
can tell spark that it has the rename semantics needed, but the protocol is 
still broken and the downstream tests I wrote aren't sufficient. 

SPARK-41551 will have a fix

> PathOutputCommitters to work with dynamic partition overwrite
> -
>
> Key: SPARK-40034
> URL: https://issues.apache.org/jira/browse/SPARK-40034
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core, SQL
>Affects Versions: 3.4.0
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>Priority: Minor
> Fix For: 3.4.0
>
>
> sibling of MAPREDUCE-7403: allow PathOutputCommitter implementation to 
> declare that they support the semantics required by spark dynamic 
> partitioning:
> * rename to work as expected
> * working dir to be on same fs as final dir
> They will do this through implementing StreamCapabilities and adding a new 
> probe, "mapreduce.job.committer.dynamic.partitioning" ; the spark side 
> changes are to
> * postpone rejection of dynamic partition overwrite until the output 
> committer is created
> * allow it if the committer implements StreamCapabilities and returns true 
> for {{hasCapability("mapreduce.job.committer.dynamic.partitioning")))
> this isn't going to be supported by the s3a committers, they don't meet the 
> requirements. The manifest committer of MAPREDUCE-7341 running against abfs 
> and gcs does work. 



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-41599) Memory leak in FileSystem.CACHE when submitting apps to secure cluster using InProcessLauncher

2023-01-18 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17678189#comment-17678189
 ] 

Steve Loughran commented on SPARK-41599:


well, the challenge there becomes "not changing that UGI" or at least closing 
all fs instances for that ugi before that happens

> Memory leak in FileSystem.CACHE when submitting apps to secure cluster using 
> InProcessLauncher
> --
>
> Key: SPARK-41599
> URL: https://issues.apache.org/jira/browse/SPARK-41599
> Project: Spark
>  Issue Type: Bug
>  Components: Deploy, YARN
>Affects Versions: 3.1.2
>Reporter: Maciej Smolenski
>Priority: Major
> Attachments: InProcLaunchFsIssue.scala, 
> SPARK-41599-fixes-to-limit-FileSystem-CACHE-size-when-using-InProcessLauncher.diff
>
>
> When submitting spark application in kerberos environment the credentials of 
> 'current user' (UserGroupInformation.getCurrentUser()) are being modified.
> Filesystem.CACHE entries contain 'current user' (with user credentials) as a 
> key.
> Submitting many spark applications using InProcessLauncher cause that 
> FileSystem.CACHE becomes bigger and bigger.
> Finally process exits because of OutOfMemory error.
> Code for reproduction attached.
>  
> Output from running 'jmap -histo' on reproduction jvm shows that the number 
> of FileSystem$Cache$Key increases in time:
> time: #instances class
> 1671533274: 2 org.apache.hadoop.fs.FileSystem$Cache$Key
> 167155: 11 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533395: 21 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533455: 30 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533515: 39 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533576: 48 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533636: 57 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533696: 66 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533757: 75 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533817: 84 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533877: 93 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533937: 102 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533998: 111 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534058: 120 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534118: 135 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534178: 140 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534239: 150 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534299: 159 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534359: 168 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534419: 177 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534480: 186 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534540: 195 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534600: 204 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534661: 213 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534721: 222 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534781: 231 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534841: 240 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534902: 249 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534962: 257 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535022: 264 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535083: 273 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535143: 282 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535203: 291 org.apache.hadoop.fs.FileSystem$Cache$Key



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-41599) Memory leak in FileSystem.CACHE when submitting apps to secure cluster using InProcessLauncher

2022-12-23 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651672#comment-17651672
 ] 

Steve Loughran commented on SPARK-41599:


apps can call  FileSystem.closeAllForUGI() to remove all cached entries for a 
given user. It's how multi-user services (hive, tez) stay in control

> Memory leak in FileSystem.CACHE when submitting apps to secure cluster using 
> InProcessLauncher
> --
>
> Key: SPARK-41599
> URL: https://issues.apache.org/jira/browse/SPARK-41599
> Project: Spark
>  Issue Type: Bug
>  Components: Deploy, YARN
>Affects Versions: 3.1.2
>Reporter: Maciej Smolenski
>Priority: Major
> Attachments: InProcLaunchFsIssue.scala, 
> SPARK-41599-fixes-to-limit-FileSystem-CACHE-size-when-using-InProcessLauncher.diff
>
>
> When submitting spark application in kerberos environment the credentials of 
> 'current user' (UserGroupInformation.getCurrentUser()) are being modified.
> Filesystem.CACHE entries contain 'current user' (with user credentials) as a 
> key.
> Submitting many spark applications using InProcessLauncher cause that 
> FileSystem.CACHE becomes bigger and bigger.
> Finally process exits because of OutOfMemory error.
> Code for reproduction attached.
>  
> Output from running 'jmap -histo' on reproduction jvm shows that the number 
> of FileSystem$Cache$Key increases in time:
> time: #instances class
> 1671533274: 2 org.apache.hadoop.fs.FileSystem$Cache$Key
> 167155: 11 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533395: 21 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533455: 30 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533515: 39 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533576: 48 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533636: 57 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533696: 66 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533757: 75 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533817: 84 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533877: 93 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533937: 102 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533998: 111 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534058: 120 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534118: 135 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534178: 140 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534239: 150 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534299: 159 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534359: 168 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534419: 177 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534480: 186 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534540: 195 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534600: 204 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534661: 213 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534721: 222 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534781: 231 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534841: 240 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534902: 249 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534962: 257 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535022: 264 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535083: 273 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535143: 282 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535203: 291 org.apache.hadoop.fs.FileSystem$Cache$Key



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-41599) Memory leak in FileSystem.CACHE when submitting apps to secure cluster using InProcessLauncher

2022-12-23 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651614#comment-17651614
 ] 

Steve Loughran commented on SPARK-41599:


1. try explicitly disabling the cache for that fs schema
2. there's a new 3.3.5 rc0 out at  
https://dist.apache.org/repos/dist/dev/hadoop/hadoop-3.3.5-RC0/  ; spark picks 
up the jars if you build it with the right settings. eg


{code}
 build/sbt -Dhadoop.version=3.3.5 -Psnapshots-and-staging
{code}

that does fix the s3a instrumentation leakage, though if s3a fs instances are 
not being close()d they can still leak thread pools

> Memory leak in FileSystem.CACHE when submitting apps to secure cluster using 
> InProcessLauncher
> --
>
> Key: SPARK-41599
> URL: https://issues.apache.org/jira/browse/SPARK-41599
> Project: Spark
>  Issue Type: Bug
>  Components: Deploy, YARN
>Affects Versions: 3.1.2
>Reporter: Maciej Smolenski
>Priority: Major
> Attachments: InProcLaunchFsIssue.scala, 
> adding_fs_to_cache_traces_140722.txt, 
> adding_fs_to_cache_with_staging_dir_set_traces_142334.txt
>
>
> When submitting spark application in kerberos environment the credentials of 
> 'current user' (UserGroupInformation.getCurrentUser()) are being modified.
> Filesystem.CACHE entries contain 'current user' (with user credentials) as a 
> key.
> Submitting many spark applications using InProcessLauncher cause that 
> FileSystem.CACHE becomes bigger and bigger.
> Finally process exits because of OutOfMemory error.
> Code for reproduction attached.
>  
> Output from running 'jmap -histo' on reproduction jvm shows that the number 
> of FileSystem$Cache$Key increases in time:
> time: #instances class
> 1671533274: 2 org.apache.hadoop.fs.FileSystem$Cache$Key
> 167155: 11 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533395: 21 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533455: 30 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533515: 39 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533576: 48 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533636: 57 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533696: 66 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533757: 75 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533817: 84 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533877: 93 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533937: 102 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533998: 111 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534058: 120 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534118: 135 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534178: 140 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534239: 150 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534299: 159 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534359: 168 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534419: 177 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534480: 186 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534540: 195 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534600: 204 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534661: 213 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534721: 222 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534781: 231 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534841: 240 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534902: 249 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534962: 257 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535022: 264 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535083: 273 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535143: 282 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535203: 291 org.apache.hadoop.fs.FileSystem$Cache$Key



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-41551) Improve/complete PathOutputCommitProtocol support for dynamic partitioning

2022-12-22 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651384#comment-17651384
 ] 

Steve Loughran commented on SPARK-41551:


PR up. PathOutputCommitProtocol stops anyone trying to use a parent dir as the 
absolute path in dynamic update mode, as 
HadoopMapReduceCommitProtocol.commitJob() will blindly delete the entire dir 
tree at that point. I'm not convinced that feature is particularly safe.

> Improve/complete PathOutputCommitProtocol support for dynamic partitioning
> --
>
> Key: SPARK-41551
> URL: https://issues.apache.org/jira/browse/SPARK-41551
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.3.1
>Reporter: Steve Loughran
>Priority: Minor
>
> Followup to SPARK-40034 as 
> * that is incomplete as it doesn't record the partitions
> * as long at the job doesn't call `newTaskTempFileAbsPath()`, and slow 
> renames are ok, both s3a committers are actually OK to use.
> It's only that newTaskTempFileAbsPath operation which is unsupported in s3a 
> committers; the post-job dir rename is O(data) but file by file rename is 
> correct for a non-atomic job commit.
> # Cut PathOutputCommitProtocol.newTaskTempFile; to update super 
> partitionPaths (needs a setter). The superclass can't just say if (committer 
> instance of PathOutputCommitter as spark-core needs to compile with older 
> hadoop versions)
> # downgrade failure in setup to log (info?)
> # retain failure in the newTaskTempFileAbsPath call.
> Testing: yes



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-41599) Memory leak in FileSystem.CACHE when submitting apps to secure cluster using InProcessLauncher

2022-12-22 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651201#comment-17651201
 ] 

Steve Loughran commented on SPARK-41599:


either the fs is being created by ((FileSystem.newInstance()}} and the code 
isn't calling close() after, or caching is disabled with 
"fs.$SCHEME.impl.disable.cache" set to true.

There's also {{HADOOP-17313. FileSystem.get to support slow-to-instantiate FS 
clients}} which handles many threads calling get() on slow to create 
clients...but that only surfaced as an issue in large worker process

> Memory leak in FileSystem.CACHE when submitting apps to secure cluster using 
> InProcessLauncher
> --
>
> Key: SPARK-41599
> URL: https://issues.apache.org/jira/browse/SPARK-41599
> Project: Spark
>  Issue Type: Bug
>  Components: Deploy, YARN
>Affects Versions: 3.1.2
>Reporter: Maciej Smolenski
>Priority: Major
> Attachments: InProcLaunchFsIssue.scala
>
>
> When submitting spark application in kerberos environment the credentials of 
> 'current user' (UserGroupInformation.getCurrentUser()) are being modified.
> Filesystem.CACHE entries contain 'current user' (with user credentials) as a 
> key.
> Submitting many spark applications using InProcessLauncher cause that 
> FileSystem.CACHE becomes bigger and bigger.
> Finally process exits because of OutOfMemory error.
> Code for reproduction attached.
>  
> Output from running 'jmap -histo' on reproduction jvm shows that the number 
> of FileSystem$Cache$Key increases in time:
> time: #instances class
> 1671533274: 2 org.apache.hadoop.fs.FileSystem$Cache$Key
> 167155: 11 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533395: 21 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533455: 30 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533515: 39 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533576: 48 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533636: 57 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533696: 66 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533757: 75 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533817: 84 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533877: 93 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533937: 102 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671533998: 111 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534058: 120 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534118: 135 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534178: 140 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534239: 150 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534299: 159 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534359: 168 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534419: 177 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534480: 186 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534540: 195 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534600: 204 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534661: 213 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534721: 222 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534781: 231 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534841: 240 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534902: 249 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671534962: 257 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535022: 264 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535083: 273 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535143: 282 org.apache.hadoop.fs.FileSystem$Cache$Key
> 1671535203: 291 org.apache.hadoop.fs.FileSystem$Cache$Key



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-41551) Improve/complete PathOutputCommitProtocol support for dynamic partitioning

2022-12-20 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17649775#comment-17649775
 ] 

Steve Loughran commented on SPARK-41551:


So there's an interesting little "feature" of 
HadoopMapReduceCommitProtocol.newTaskTempFile() which is: 

If you call newTaskTempFile(tac, None, ext) when dynamicPartitionOverwrite is 
true, *and* spark-core was compiled with assertions -Xelide-below at a level 
which excludes assert(), then in job commit the entire directory tree is 
destroyed -both output and (implicitly) the .spark-staging dir. makes for a 
fairly messy job failure.

The good news: spark builds don't do that, and since spark-core/spark-sql 
itself doesn't seem to invoke newTaskTempFile(_, None, _) in dynamic partition 
mode, it's not a serious risk. Is it worth hardening?

> Improve/complete PathOutputCommitProtocol support for dynamic partitioning
> --
>
> Key: SPARK-41551
> URL: https://issues.apache.org/jira/browse/SPARK-41551
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.3.1
>Reporter: Steve Loughran
>Priority: Minor
>
> Followup to SPARK-40034 as 
> * that is incomplete as it doesn't record the partitions
> * as long at the job doesn't call `newTaskTempFileAbsPath()`, and slow 
> renames are ok, both s3a committers are actually OK to use.
> It's only that newTaskTempFileAbsPath operation which is unsupported in s3a 
> committers; the post-job dir rename is O(data) but file by file rename is 
> correct for a non-atomic job commit.
> # Cut PathOutputCommitProtocol.newTaskTempFile; to update super 
> partitionPaths (needs a setter). The superclass can't just say if (committer 
> instance of PathOutputCommitter as spark-core needs to compile with older 
> hadoop versions)
> # downgrade failure in setup to log (info?)
> # retain failure in the newTaskTempFileAbsPath call.
> Testing: yes



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-41551) Improve/complete PathOutputCommitProtocol support for dynamic partitioning

2022-12-16 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-41551:
--

 Summary: Improve/complete PathOutputCommitProtocol support for 
dynamic partitioning
 Key: SPARK-41551
 URL: https://issues.apache.org/jira/browse/SPARK-41551
 Project: Spark
  Issue Type: Improvement
  Components: SQL
Affects Versions: 3.3.1
Reporter: Steve Loughran


Followup to SPARK-40034 as 
* that is incomplete as it doesn't record the partitions
* as long at the job doesn't call `newTaskTempFileAbsPath()`, and slow renames 
are ok, both s3a committers are actually OK to use.

It's only that newTaskTempFileAbsPath operation which is unsupported in s3a 
committers; the post-job dir rename is O(data) but file by file rename is 
correct for a non-atomic job commit.

# Cut PathOutputCommitProtocol.newTaskTempFile; to update super partitionPaths 
(needs a setter). The superclass can't just say if (committer instance of 
PathOutputCommitter as spark-core needs to compile with older hadoop versions)
# downgrade failure in setup to log (info?)
# retain failure in the newTaskTempFileAbsPath call.

Testing: yes




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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-41392) spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin

2022-12-06 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17643774#comment-17643774
 ] 

Steve Loughran commented on SPARK-41392:


may relate to the bouncy castle 1.68 update of HADOOP-1756 -but this is also in 
the 3.3.5/3.3 branches and spark is happy there. so there must be more to it

> spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin
> ---
>
> Key: SPARK-41392
> URL: https://issues.apache.org/jira/browse/SPARK-41392
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 3.4.0
>Reporter: Steve Loughran
>Priority: Minor
>
> on hadoop trunk (but not the 3.3.x line), spark builds fail with a CNFE
> {code}
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> {code}
> full stack
> {code}
> [ERROR] Failed to execute goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile 
> (scala-test-compile-first) on project spark-sql_2.12: Execution 
> scala-test-compile-first of goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile failed: A required 
> class was missing while executing 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> [ERROR] -
> [ERROR] realm =plugin>net.alchim31.maven:scala-maven-plugin:4.7.2
> [ERROR] strategy = org.codehaus.plexus.classworlds.strategy.SelfFirstStrategy
> [ERROR] urls[0] = 
> file:/Users/stevel/.m2/repository/net/alchim31/maven/scala-maven-plugin/4.7.2/scala-maven-plugin-4.7.2.jar
> [ERROR] urls[1] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/shared/maven-dependency-tree/3.2.0/maven-dependency-tree-3.2.0.jar
> [ERROR] urls[2] = 
> file:/Users/stevel/.m2/repository/org/eclipse/aether/aether-util/1.0.0.v20140518/aether-util-1.0.0.v20140518.jar
> [ERROR] urls[3] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/reporting/maven-reporting-api/3.1.1/maven-reporting-api-3.1.1.jar
> [ERROR] urls[4] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-sink-api/1.11.1/doxia-sink-api-1.11.1.jar
> [ERROR] urls[5] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-logging-api/1.11.1/doxia-logging-api-1.11.1.jar
> [ERROR] urls[6] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/maven-archiver/3.6.0/maven-archiver-3.6.0.jar
> [ERROR] urls[7] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-io/3.4.0/plexus-io-3.4.0.jar
> [ERROR] urls[8] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-interpolation/1.26/plexus-interpolation-1.26.jar
> [ERROR] urls[9] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-exec/1.3/commons-exec-1.3.jar
> [ERROR] urls[10] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-utils/3.4.2/plexus-utils-3.4.2.jar
> [ERROR] urls[11] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-archiver/4.5.0/plexus-archiver-4.5.0.jar
> [ERROR] urls[12] = 
> file:/Users/stevel/.m2/repository/commons-io/commons-io/2.11.0/commons-io-2.11.0.jar
> [ERROR] urls[13] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-compress/1.21/commons-compress-1.21.jar
> [ERROR] urls[14] = 
> file:/Users/stevel/.m2/repository/org/iq80/snappy/snappy/0.4/snappy-0.4.jar
> [ERROR] urls[15] = 
> file:/Users/stevel/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar
> [ERROR] urls[16] = 
> file:/Users/stevel/.m2/repository/com/github/luben/zstd-jni/1.5.2-4/zstd-jni-1.5.2-4.jar
> [ERROR] urls[17] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc_2.13/1.7.1/zinc_2.13-1.7.1.jar
> [ERROR] urls[18] = 
> file:/Users/stevel/.m2/repository/org/scala-lang/scala-library/2.13.8/scala-library-2.13.8.jar
> [ERROR] urls[19] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-core_2.13/1.7.1/zinc-core_2.13-1.7.1.jar
> [ERROR] urls[20] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-apiinfo_2.13/1.7.1/zinc-apiinfo_2.13-1.7.1.jar
> [ERROR] urls[21] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-bridge_2.13/1.7.1/compiler-bridge_2.13-1.7.1.jar
> [ERROR] urls[22] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-classpath_2.13/1.7.1/zinc-classpath_2.13-1.7.1.jar
> [ERROR] urls[23] = 
> file:/Users/stevel/.m2/repository/org/scala-lang/scala-compiler/2.13.8/scala-compiler-2.13.8.jar
> [ERROR] urls[24] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-interface/1.7.1/compiler-interface-1.7.1.jar
> [ERROR] urls[25] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/util-interface/1.7.0/util-interface-1.7.0.jar
> [ERROR] urls[26] = 
> 

[jira] [Commented] (SPARK-41392) spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin

2022-12-05 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-41392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17643492#comment-17643492
 ] 

Steve Loughran commented on SPARK-41392:


MBP m1 with

{code}
 uname -a
Darwin stevel-MBP16 21.6.0 Darwin Kernel Version 21.6.0: Thu Sep 29 20:13:56 
PDT 2022; root:xnu-8020.240.7~1/RELEASE_ARM64_T6000 arm64

{code}

java 8

{code}
 java -version
openjdk version "1.8.0_322"
OpenJDK Runtime Environment (Zulu 8.60.0.21-CA-macos-aarch64) (build 
1.8.0_322-b06)
OpenJDK 64-Bit Server VM (Zulu 8.60.0.21-CA-macos-aarch64) (build 25.322-b06, 
mixed mode)

{code}
 build/mvn invokes homebrew maven which I run at -T 1 as sometimes the build 
hangs (maven bug, presumably)

{code}
build/mvn -v
Using `mvn` from path: /opt/homebrew/bin/mvn
Apache Maven 3.8.6 (84538c9988a25aec085021c365c560670ad80f63)
Maven home: /opt/homebrew/Cellar/maven/3.8.6/libexec
Java version: 1.8.0_322, vendor: Azul Systems, Inc., runtime: 
/Library/Java/JavaVirtualMachines/zulu-8.jdk/Contents/Home/jre
Default locale: en_GB, platform encoding: UTF-8
OS name: "mac os x", version: "12.6.1", arch: "aarch64", family: "mac"
{code}

this setup works with older hadoop releases (inc the forthcoming 3.3.5), 
somehow the plugin can't cope with the trunk release


> spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin
> ---
>
> Key: SPARK-41392
> URL: https://issues.apache.org/jira/browse/SPARK-41392
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 3.4.0
>Reporter: Steve Loughran
>Priority: Minor
>
> on hadoop trunk (but not the 3.3.x line), spark builds fail with a CNFE
> {code}
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> {code}
> full stack
> {code}
> [ERROR] Failed to execute goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile 
> (scala-test-compile-first) on project spark-sql_2.12: Execution 
> scala-test-compile-first of goal 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile failed: A required 
> class was missing while executing 
> net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
> org/bouncycastle/jce/provider/BouncyCastleProvider
> [ERROR] -
> [ERROR] realm =plugin>net.alchim31.maven:scala-maven-plugin:4.7.2
> [ERROR] strategy = org.codehaus.plexus.classworlds.strategy.SelfFirstStrategy
> [ERROR] urls[0] = 
> file:/Users/stevel/.m2/repository/net/alchim31/maven/scala-maven-plugin/4.7.2/scala-maven-plugin-4.7.2.jar
> [ERROR] urls[1] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/shared/maven-dependency-tree/3.2.0/maven-dependency-tree-3.2.0.jar
> [ERROR] urls[2] = 
> file:/Users/stevel/.m2/repository/org/eclipse/aether/aether-util/1.0.0.v20140518/aether-util-1.0.0.v20140518.jar
> [ERROR] urls[3] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/reporting/maven-reporting-api/3.1.1/maven-reporting-api-3.1.1.jar
> [ERROR] urls[4] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-sink-api/1.11.1/doxia-sink-api-1.11.1.jar
> [ERROR] urls[5] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-logging-api/1.11.1/doxia-logging-api-1.11.1.jar
> [ERROR] urls[6] = 
> file:/Users/stevel/.m2/repository/org/apache/maven/maven-archiver/3.6.0/maven-archiver-3.6.0.jar
> [ERROR] urls[7] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-io/3.4.0/plexus-io-3.4.0.jar
> [ERROR] urls[8] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-interpolation/1.26/plexus-interpolation-1.26.jar
> [ERROR] urls[9] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-exec/1.3/commons-exec-1.3.jar
> [ERROR] urls[10] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-utils/3.4.2/plexus-utils-3.4.2.jar
> [ERROR] urls[11] = 
> file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-archiver/4.5.0/plexus-archiver-4.5.0.jar
> [ERROR] urls[12] = 
> file:/Users/stevel/.m2/repository/commons-io/commons-io/2.11.0/commons-io-2.11.0.jar
> [ERROR] urls[13] = 
> file:/Users/stevel/.m2/repository/org/apache/commons/commons-compress/1.21/commons-compress-1.21.jar
> [ERROR] urls[14] = 
> file:/Users/stevel/.m2/repository/org/iq80/snappy/snappy/0.4/snappy-0.4.jar
> [ERROR] urls[15] = 
> file:/Users/stevel/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar
> [ERROR] urls[16] = 
> file:/Users/stevel/.m2/repository/com/github/luben/zstd-jni/1.5.2-4/zstd-jni-1.5.2-4.jar
> [ERROR] urls[17] = 
> file:/Users/stevel/.m2/repository/org/scala-sbt/zinc_2.13/1.7.1/zinc_2.13-1.7.1.jar
> [ERROR] urls[18] = 
> file:/Users/stevel/.m2/repository/org/scala-lang/scala-library/2.13.8/scala-library-2.13.8.jar
> [ERROR] urls[19] = 
> 

[jira] [Created] (SPARK-41392) spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in scala-maven plugin

2022-12-05 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-41392:
--

 Summary: spark builds against hadoop trunk/3.4.0-SNAPSHOT fail in 
scala-maven plugin
 Key: SPARK-41392
 URL: https://issues.apache.org/jira/browse/SPARK-41392
 Project: Spark
  Issue Type: Bug
  Components: Build
Affects Versions: 3.4.0
Reporter: Steve Loughran


on hadoop trunk (but not the 3.3.x line), spark builds fail with a CNFE

{code}
net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
org/bouncycastle/jce/provider/BouncyCastleProvider

{code}

full stack

{code}
[ERROR] Failed to execute goal 
net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile 
(scala-test-compile-first) on project spark-sql_2.12: Execution 
scala-test-compile-first of goal 
net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile failed: A required 
class was missing while executing 
net.alchim31.maven:scala-maven-plugin:4.7.2:testCompile: 
org/bouncycastle/jce/provider/BouncyCastleProvider
[ERROR] -
[ERROR] realm =plugin>net.alchim31.maven:scala-maven-plugin:4.7.2
[ERROR] strategy = org.codehaus.plexus.classworlds.strategy.SelfFirstStrategy
[ERROR] urls[0] = 
file:/Users/stevel/.m2/repository/net/alchim31/maven/scala-maven-plugin/4.7.2/scala-maven-plugin-4.7.2.jar
[ERROR] urls[1] = 
file:/Users/stevel/.m2/repository/org/apache/maven/shared/maven-dependency-tree/3.2.0/maven-dependency-tree-3.2.0.jar
[ERROR] urls[2] = 
file:/Users/stevel/.m2/repository/org/eclipse/aether/aether-util/1.0.0.v20140518/aether-util-1.0.0.v20140518.jar
[ERROR] urls[3] = 
file:/Users/stevel/.m2/repository/org/apache/maven/reporting/maven-reporting-api/3.1.1/maven-reporting-api-3.1.1.jar
[ERROR] urls[4] = 
file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-sink-api/1.11.1/doxia-sink-api-1.11.1.jar
[ERROR] urls[5] = 
file:/Users/stevel/.m2/repository/org/apache/maven/doxia/doxia-logging-api/1.11.1/doxia-logging-api-1.11.1.jar
[ERROR] urls[6] = 
file:/Users/stevel/.m2/repository/org/apache/maven/maven-archiver/3.6.0/maven-archiver-3.6.0.jar
[ERROR] urls[7] = 
file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-io/3.4.0/plexus-io-3.4.0.jar
[ERROR] urls[8] = 
file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-interpolation/1.26/plexus-interpolation-1.26.jar
[ERROR] urls[9] = 
file:/Users/stevel/.m2/repository/org/apache/commons/commons-exec/1.3/commons-exec-1.3.jar
[ERROR] urls[10] = 
file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-utils/3.4.2/plexus-utils-3.4.2.jar
[ERROR] urls[11] = 
file:/Users/stevel/.m2/repository/org/codehaus/plexus/plexus-archiver/4.5.0/plexus-archiver-4.5.0.jar
[ERROR] urls[12] = 
file:/Users/stevel/.m2/repository/commons-io/commons-io/2.11.0/commons-io-2.11.0.jar
[ERROR] urls[13] = 
file:/Users/stevel/.m2/repository/org/apache/commons/commons-compress/1.21/commons-compress-1.21.jar
[ERROR] urls[14] = 
file:/Users/stevel/.m2/repository/org/iq80/snappy/snappy/0.4/snappy-0.4.jar
[ERROR] urls[15] = 
file:/Users/stevel/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar
[ERROR] urls[16] = 
file:/Users/stevel/.m2/repository/com/github/luben/zstd-jni/1.5.2-4/zstd-jni-1.5.2-4.jar
[ERROR] urls[17] = 
file:/Users/stevel/.m2/repository/org/scala-sbt/zinc_2.13/1.7.1/zinc_2.13-1.7.1.jar
[ERROR] urls[18] = 
file:/Users/stevel/.m2/repository/org/scala-lang/scala-library/2.13.8/scala-library-2.13.8.jar
[ERROR] urls[19] = 
file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-core_2.13/1.7.1/zinc-core_2.13-1.7.1.jar
[ERROR] urls[20] = 
file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-apiinfo_2.13/1.7.1/zinc-apiinfo_2.13-1.7.1.jar
[ERROR] urls[21] = 
file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-bridge_2.13/1.7.1/compiler-bridge_2.13-1.7.1.jar
[ERROR] urls[22] = 
file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-classpath_2.13/1.7.1/zinc-classpath_2.13-1.7.1.jar
[ERROR] urls[23] = 
file:/Users/stevel/.m2/repository/org/scala-lang/scala-compiler/2.13.8/scala-compiler-2.13.8.jar
[ERROR] urls[24] = 
file:/Users/stevel/.m2/repository/org/scala-sbt/compiler-interface/1.7.1/compiler-interface-1.7.1.jar
[ERROR] urls[25] = 
file:/Users/stevel/.m2/repository/org/scala-sbt/util-interface/1.7.0/util-interface-1.7.0.jar
[ERROR] urls[26] = 
file:/Users/stevel/.m2/repository/org/scala-sbt/zinc-persist-core-assembly/1.7.1/zinc-persist-core-assembly-1.7.1.jar
[ERROR] urls[27] = 
file:/Users/stevel/.m2/repository/org/scala-lang/modules/scala-parallel-collections_2.13/0.2.0/scala-parallel-collections_2.13-0.2.0.jar
[ERROR] urls[28] = 
file:/Users/stevel/.m2/repository/org/scala-sbt/io_2.13/1.7.0/io_2.13-1.7.0.jar
[ERROR] urls[29] = 
file:/Users/stevel/.m2/repository/com/swoval/file-tree-views/2.1.9/file-tree-views-2.1.9.jar
[ERROR] urls[30] = 
file:/Users/stevel/.m2/repository/net/java/dev/jna/jna/5.12.0/jna-5.12.0.jar
[ERROR] urls[31] = 

[jira] [Commented] (SPARK-38934) Provider TemporaryAWSCredentialsProvider has no credentials

2022-10-21 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17622378#comment-17622378
 ] 

Steve Loughran commented on SPARK-38934:


sounds like there is a race condition, which surfaces now startup probes for 
bucket existence are skipped for performance reasons.

set fs.s3a.bucket.probe to 2 to reinstate that probe

> Provider TemporaryAWSCredentialsProvider has no credentials
> ---
>
> Key: SPARK-38934
> URL: https://issues.apache.org/jira/browse/SPARK-38934
> Project: Spark
>  Issue Type: Bug
>  Components: Kubernetes, Spark Core
>Affects Versions: 3.2.1
>Reporter: Lily
>Priority: Major
>
>  
> We are using Jupyter Hub on K8s as a notebook based development environment 
> and Spark on K8s as a backend cluster of Jupyter Hub on K8s with Spark 3.2.1 
> and Hadoop 3.3.1.
> When we run a code like the one below in the Jupyter Hub on K8s,
>  
> {code:java}
> val perm = ... // get AWS temporary credential by AWS STS from AWS assumed 
> role
> // set AWS temporary credential
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.aws.credentials.provider", 
> "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider")
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", 
> perm.credential.accessKeyID)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", 
> perm.credential.secretAccessKey)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.session.token", 
> perm.credential.sessionToken)
> // execute simple Spark action
> spark.read.format("parquet").load("s3a:///*").show(1) {code}
>  
>  
> the first few executors left a warning like the one below in the first code 
> execution, but we were able to get the proper result thanks to Spark task 
> retry function. 
> {code:java}
> 22/04/18 09:13:50 WARN TaskSetManager: Lost task 2.0 in stage 0.0 (TID 2) 
> (10.197.5.15 executor 1): java.nio.file.AccessDeniedException: 
> s3a:///.parquet: 
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider 
> TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206)
>   at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
>   at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:2810)
>   at 
> org.apache.spark.util.HadoopFSUtils$.listLeafFiles(HadoopFSUtils.scala:225)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$6(HadoopFSUtils.scala:136)
>   at scala.collection.immutable.Stream.map(Stream.scala:418)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$4(HadoopFSUtils.scala:126)
>   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:131)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: 
> Provider TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130)
>   at 
> org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:842)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:792)
>   at 
> 

[jira] [Updated] (SPARK-29729) Upgrade ASM to 7.2

2022-10-17 Thread Steve Loughran (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-29729?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran updated SPARK-29729:
---
Description: this patch is required for spark to build with any version of 
bouncy castle jar with multi-version java .class files. see HADOOP-17563 for 
details. 

> Upgrade ASM to 7.2
> --
>
> Key: SPARK-29729
> URL: https://issues.apache.org/jira/browse/SPARK-29729
> Project: Spark
>  Issue Type: Improvement
>  Components: Build
>Affects Versions: 3.0.0
>Reporter: Dongjoon Hyun
>Assignee: Dongjoon Hyun
>Priority: Major
> Fix For: 3.0.0
>
>
> this patch is required for spark to build with any version of bouncy castle 
> jar with multi-version java .class files. see HADOOP-17563 for details. 



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-40640) SparkHadoopUtil to set origin of hadoop/hive config options

2022-10-03 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-40640:
--

 Summary: SparkHadoopUtil to set origin of hadoop/hive config 
options
 Key: SPARK-40640
 URL: https://issues.apache.org/jira/browse/SPARK-40640
 Project: Spark
  Issue Type: Improvement
  Components: Spark Core
Affects Versions: 3.3.2
Reporter: Steve Loughran


Hadoop's Configuration class allows the caller to declare the origin of an 
option in the {{set(key, value, source)}} method.

Having spark do this for all programmatically set values helps people trying to 
debug where values came from. This is particularly relevant when looking at AWS 
env var propagation.




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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-40567) SharedState to redact secrets when propagating them to HadoopConf

2022-09-26 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-40567:
--

 Summary: SharedState to redact secrets when propagating them to 
HadoopConf
 Key: SPARK-40567
 URL: https://issues.apache.org/jira/browse/SPARK-40567
 Project: Spark
  Issue Type: Improvement
  Components: SQL
Affects Versions: 3.3.0
Reporter: Steve Loughran



When SharedState propagates (key, value) pairs from initialConfigs to 
HadoopConf, it logs the values at debug.

If the config contained secrets (cloud credentials, etc) the log will contain 
them.

The org.apache.hadoop.conf.ConfigRedactor class will redact values of all keys 
matching a patten in "hadoop.security.sensitive-config-keys"; this is 
configured by default to be


{code}
  "secret$",
  "password$",
  "ssl.keystore.pass$",
  "fs.s3.*[Ss]ecret.?[Kk]ey",
  "fs.s3a.*.server-side-encryption.key",
  "fs.s3a.encryption.algorithm",
  "fs.s3a.encryption.key",
  "fs.azure\\.account.key.*",
  "credential$",
  "oauth.*secret",
  "oauth.*password",
  "oauth.*token",
"hadoop.security.sensitive-config-keys"
{code}

...And it may be extended in site configs/future hadoop releases

Spark should be using the redactor for log hygiene/security





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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-40286) Load Data from S3 deletes data source file

2022-09-01 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-40286?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17598837#comment-17598837
 ] 

Steve Loughran commented on SPARK-40286:


this is EMR. can you repliacate in an ASF spark release through the s3a 
connector and committers?

if you can replicate, especiallly  in spark standadone, turn spark and 
org.apache.hadoop.fs.s3a logging on to debug and see what it says

> Load Data from S3 deletes data source file
> --
>
> Key: SPARK-40286
> URL: https://issues.apache.org/jira/browse/SPARK-40286
> Project: Spark
>  Issue Type: Question
>  Components: Documentation
>Affects Versions: 3.2.1
>Reporter: Drew
>Priority: Major
>
> Hello, 
> I'm using spark to [load 
> data|https://spark.apache.org/docs/latest/sql-ref-syntax-dml-load.html] into 
> a hive table through Pyspark, and when I load data from a path in Amazon S3, 
> the original file is getting wiped from the Directory. The file is found, and 
> is populating the table with data. I also tried to add the `Local` clause but 
> that throws an error when looking for the file. When looking through the 
> documentation it doesn't explicitly state that this is the intended behavior.
> Thanks in advance!
> {code:java}
> spark.sql("CREATE TABLE src (key INT, value STRING) STORED AS textfile")
> spark.sql("LOAD DATA INPATH 's3://bucket/kv1.txt' OVERWRITE INTO TABLE 
> src"){code}



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-40287) Load Data using Spark by a single partition moves entire dataset under same location in S3

2022-09-01 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-40287?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17598835#comment-17598835
 ] 

Steve Loughran commented on SPARK-40287:


does this happen when
# you switch to an ASF spark build with the s3a connector
# and use an s3a committer safe to use with spark

this is clearly EMR (s3:// URLs), so they have to be the people to talk to if 
you can't replicate it in the apache code

> Load Data using Spark by a single partition moves entire dataset under same 
> location in S3
> --
>
> Key: SPARK-40287
> URL: https://issues.apache.org/jira/browse/SPARK-40287
> Project: Spark
>  Issue Type: Question
>  Components: Spark Core
>Affects Versions: 3.2.1
>Reporter: Drew
>Priority: Major
>
> Hello,
> I'm experiencing an issue in PySpark when creating a hive table and loading 
> in the data to the table. So I'm using an Amazon s3 bucket as a data location 
> and I'm creating a table as parquet and trying to load data into that table 
> by a single partition, and I'm seeing some weird behavior. When selecting the 
> data location in s3 of a parquet file to load into my table. All of the data 
> is moved into the specified location in my create table command including the 
> partitions I didn't specify in the load data command. For example:
> {code:java}
> # create a data frame in pyspark with partitions
> df = spark.createDataFrame([("a", 1, "x"), ("b", 2, "y"), ("c", 3, "y")], 
> ["c1", "c2", "p"])
> # save it to S3
> df.write.format("parquet").mode("overwrite").partitionBy("p").save("s3://bucket/data/")
> {code}
> In the current state S3 should have a new folder `data` with two folders 
> which contain a parquet file in each partition. 
>   
>  - s3://bucket/data/p=x/
>     - part-1.snappy.parquet
>  - s3://bucket/data/p=y/
>     - part-2.snappy.parquet
>     - part-3.snappy.parquet
>  
> {code:java}
> # create new table
> spark.sql("create table src (c1 string,c2 int) PARTITIONED BY (p string) 
> STORED AS parquet LOCATION 's3://bucket/new/'")
> # load the saved table data from s3 specifying single partition value x
> spark.sql("LOAD DATA INPATH 's3://bucket/data/'INTO TABLE src PARTITION 
> (p='x')")
> spark.sql("select * from src").show()
> # output: 
> # +---+---+---+
> # | c1| c2|  p|
> # +---+---+---+
> # +---+---+---+
> {code}
> After running the `load data` command, and looking at the table I'm left with 
> no data loaded in. When checking S3 the data source we saved earlier is moved 
> under `s3://bucket/new/` oddly enough it also brought over the other 
> partitions along with it directory structure listed below. 
> - s3://bucket/new/
>     - p=x/
>         - p=x/
>             - part-1.snappy.parquet
>         - p=y/
>             - part-2.snappy.parquet
>             - part-3.snappy.parquet
> Is this the intended behavior of loading the data in from a partitioned 
> parquet file? Is the previous file supposed to be moved/deleted from source 
> directory? 



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38934) Provider TemporaryAWSCredentialsProvider has no credentials

2022-08-26 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17585275#comment-17585275
 ] 

Steve Loughran commented on SPARK-38934:


[~graceee318] try explicitly setting the aws secrets as a per-bucket option, If 
the bucket is called  "my-bucket", set them in your config into these options

{code}

spark.hadoop.fs.s3a.bucket.my-bucket.access.key
spark.hadoop.fs.s3a.bucket.my-bucket.secret.key
spark.hadoop.fs.s3a.bucket.my-bucket.session.token

{code}

This will stop any env var propagation from interfering with the values for the 
bucket

> Provider TemporaryAWSCredentialsProvider has no credentials
> ---
>
> Key: SPARK-38934
> URL: https://issues.apache.org/jira/browse/SPARK-38934
> Project: Spark
>  Issue Type: Bug
>  Components: Kubernetes, Spark Core
>Affects Versions: 3.2.1
>Reporter: Lily
>Priority: Major
>
>  
> We are using Jupyter Hub on K8s as a notebook based development environment 
> and Spark on K8s as a backend cluster of Jupyter Hub on K8s with Spark 3.2.1 
> and Hadoop 3.3.1.
> When we run a code like the one below in the Jupyter Hub on K8s,
>  
> {code:java}
> val perm = ... // get AWS temporary credential by AWS STS from AWS assumed 
> role
> // set AWS temporary credential
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.aws.credentials.provider", 
> "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider")
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", 
> perm.credential.accessKeyID)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", 
> perm.credential.secretAccessKey)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.session.token", 
> perm.credential.sessionToken)
> // execute simple Spark action
> spark.read.format("parquet").load("s3a:///*").show(1) {code}
>  
>  
> the first few executors left a warning like the one below in the first code 
> execution, but we were able to get the proper result thanks to Spark task 
> retry function. 
> {code:java}
> 22/04/18 09:13:50 WARN TaskSetManager: Lost task 2.0 in stage 0.0 (TID 2) 
> (10.197.5.15 executor 1): java.nio.file.AccessDeniedException: 
> s3a:///.parquet: 
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider 
> TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206)
>   at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
>   at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:2810)
>   at 
> org.apache.spark.util.HadoopFSUtils$.listLeafFiles(HadoopFSUtils.scala:225)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$6(HadoopFSUtils.scala:136)
>   at scala.collection.immutable.Stream.map(Stream.scala:418)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$4(HadoopFSUtils.scala:126)
>   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:131)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: 
> Provider TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130)
>   at 
> org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266)
>   at 
> 

[jira] [Reopened] (SPARK-38934) Provider TemporaryAWSCredentialsProvider has no credentials

2022-08-26 Thread Steve Loughran (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-38934?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran reopened SPARK-38934:


> Provider TemporaryAWSCredentialsProvider has no credentials
> ---
>
> Key: SPARK-38934
> URL: https://issues.apache.org/jira/browse/SPARK-38934
> Project: Spark
>  Issue Type: Bug
>  Components: Kubernetes, Spark Core
>Affects Versions: 3.2.1
>Reporter: Lily
>Priority: Major
>
>  
> We are using Jupyter Hub on K8s as a notebook based development environment 
> and Spark on K8s as a backend cluster of Jupyter Hub on K8s with Spark 3.2.1 
> and Hadoop 3.3.1.
> When we run a code like the one below in the Jupyter Hub on K8s,
>  
> {code:java}
> val perm = ... // get AWS temporary credential by AWS STS from AWS assumed 
> role
> // set AWS temporary credential
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.aws.credentials.provider", 
> "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider")
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", 
> perm.credential.accessKeyID)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", 
> perm.credential.secretAccessKey)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.session.token", 
> perm.credential.sessionToken)
> // execute simple Spark action
> spark.read.format("parquet").load("s3a:///*").show(1) {code}
>  
>  
> the first few executors left a warning like the one below in the first code 
> execution, but we were able to get the proper result thanks to Spark task 
> retry function. 
> {code:java}
> 22/04/18 09:13:50 WARN TaskSetManager: Lost task 2.0 in stage 0.0 (TID 2) 
> (10.197.5.15 executor 1): java.nio.file.AccessDeniedException: 
> s3a:///.parquet: 
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider 
> TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206)
>   at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
>   at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:2810)
>   at 
> org.apache.spark.util.HadoopFSUtils$.listLeafFiles(HadoopFSUtils.scala:225)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$6(HadoopFSUtils.scala:136)
>   at scala.collection.immutable.Stream.map(Stream.scala:418)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$4(HadoopFSUtils.scala:126)
>   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:131)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: 
> Provider TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130)
>   at 
> org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:842)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:792)
>   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)
>   

[jira] [Commented] (SPARK-38934) Provider TemporaryAWSCredentialsProvider has no credentials

2022-08-26 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17585268#comment-17585268
 ] 

Steve Loughran commented on SPARK-38934:


staring at this some more, as there's enough occurrences of this with spark 
alone that maybe there is a problem, not with the s3a code, but how spark 
passed configurations around, including env var adoption.

> Provider TemporaryAWSCredentialsProvider has no credentials
> ---
>
> Key: SPARK-38934
> URL: https://issues.apache.org/jira/browse/SPARK-38934
> Project: Spark
>  Issue Type: Bug
>  Components: Kubernetes, Spark Core
>Affects Versions: 3.2.1
>Reporter: Lily
>Priority: Major
>
>  
> We are using Jupyter Hub on K8s as a notebook based development environment 
> and Spark on K8s as a backend cluster of Jupyter Hub on K8s with Spark 3.2.1 
> and Hadoop 3.3.1.
> When we run a code like the one below in the Jupyter Hub on K8s,
>  
> {code:java}
> val perm = ... // get AWS temporary credential by AWS STS from AWS assumed 
> role
> // set AWS temporary credential
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.aws.credentials.provider", 
> "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider")
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", 
> perm.credential.accessKeyID)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", 
> perm.credential.secretAccessKey)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.session.token", 
> perm.credential.sessionToken)
> // execute simple Spark action
> spark.read.format("parquet").load("s3a:///*").show(1) {code}
>  
>  
> the first few executors left a warning like the one below in the first code 
> execution, but we were able to get the proper result thanks to Spark task 
> retry function. 
> {code:java}
> 22/04/18 09:13:50 WARN TaskSetManager: Lost task 2.0 in stage 0.0 (TID 2) 
> (10.197.5.15 executor 1): java.nio.file.AccessDeniedException: 
> s3a:///.parquet: 
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider 
> TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206)
>   at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
>   at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:2810)
>   at 
> org.apache.spark.util.HadoopFSUtils$.listLeafFiles(HadoopFSUtils.scala:225)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$6(HadoopFSUtils.scala:136)
>   at scala.collection.immutable.Stream.map(Stream.scala:418)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$4(HadoopFSUtils.scala:126)
>   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:131)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: 
> Provider TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130)
>   at 
> org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:842)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:792)
>   at 
> 

[jira] [Commented] (SPARK-38954) Implement sharing of cloud credentials among driver and executors

2022-08-17 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38954?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17580711#comment-17580711
 ] 

Steve Loughran commented on SPARK-38954:


any plans to put the PR up? i'm curious about what you've done.

The hadoop s3a delegation tokens can be used to collect credentials and 
encryption secrets at spark launch, pass them to workers, though there's no 
mechanism to update tokens during the life of a session.

you might want to look at this code, and experiment with it.

if you are doing your own provider, do update credentials at least 30s before 
they expire, and add some sync blocks so that 30 threads don't all try and do 
it independently. 

> Implement sharing of cloud credentials among driver and executors
> -
>
> Key: SPARK-38954
> URL: https://issues.apache.org/jira/browse/SPARK-38954
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core
>Affects Versions: 3.4.0
>Reporter: Parth Chandra
>Priority: Major
>
> Currently Spark uses external implementations (e.g. hadoop-aws) to access 
> cloud services like S3. In order to access the actual service, these 
> implementations use credentials provider implementations that obtain 
> credentials to allow access to the cloud service.
> These credentials are typically session credentials, which means that they 
> expire after a fixed time. Sometimes, this expiry can be only an hour and for 
> a spark job that runs for many hours (or spark streaming job that runs 
> continuously), the credentials have to be renewed periodically.
> In many organizations, the process of getting credentials may multi-step. The 
> organization has an identity provider service that provides authentication 
> for the user, while the cloud service provider provides authorization for the 
> roles the user has access to. Once the user is authenticated and her role 
> verified, the credentials are generated for a new session.
> In a large setup with hundreds of Spark jobs and thousands of executors, each 
> executor is then spending a lot of time getting credentials and this may put 
> unnecessary load on the backend authentication services.
> The alleviate this, we can use Spark's architecture to obtain the credentials 
> once in the driver and push the credentials to the executors. In addition, 
> the driver can check the expiry of the credentials and push updated 
> credentials to the executors. This is relatively easy to do since the rpc 
> mechanism to implement this is already in place and is used similarly for 
> Kerberos delegation tokens.
>   



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38445) Are hadoop committers used in Structured Streaming?

2022-08-17 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38445?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17580710#comment-17580710
 ] 

Steve Loughran commented on SPARK-38445:


SPARK-40039 might address this

> Are hadoop committers used in Structured Streaming?
> ---
>
> Key: SPARK-38445
> URL: https://issues.apache.org/jira/browse/SPARK-38445
> Project: Spark
>  Issue Type: Question
>  Components: Spark Core
>Affects Versions: 3.2.1
>Reporter: Martin Andersson
>Priority: Major
>  Labels: structured-streaming
>
> At the company I work at we're using Spark Structured Streaming to sink 
> messages on kafka to HDFS. We're in the late stages of migrating this 
> component to instead sink messages to AWS S3, and in connection with that we 
> hit upon a couple of issues regarding hadoop committers.
> I've come to understand that the default "file" committer (documented 
> [here|https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/committers.html#Switching_to_an_S3A_Committer])
>  is unsafe to use in S3, which is why [this page in the spark 
> documentation|https://spark.apache.org/docs/3.2.1/cloud-integration.html] 
> recommends using the "directory" (i.e. staging) committer, and in later 
> versions of hadoop they also recommend to use the "magic" committer.
> However, it's not clear whether spark structured streaming even use 
> committers. There's no "_SUCCESS" file in destination (as compared to normal 
> spark jobs), and the documentation regarding committers used in streaming is 
> non-existent.
> Can anyone please shed some light on this?



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Comment Edited] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2022-08-17 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17580707#comment-17580707
 ] 

Steve Loughran edited comment on SPARK-38330 at 8/17/22 9:46 AM:
-

bq. Is there a way to work-around this issue while waiting for a version of 
Spark which uses hadoop 3.3.4 (Spark 3.4?)

remove all jars with cos in the title from your classpath

note, emr is unaffected by this. so are cloudera products, primarily because 
they never backported the cos module. this is why it didn't show up in those 
tests.


was (Author: ste...@apache.org):
bq. Is there a way to work-around this issue while waiting for a version of 
Spark which uses hadoop 3.3.4 (Spark 3.4?)

remove all jars with cos in the title from your classpath

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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 
> 

[jira] [Comment Edited] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2022-08-17 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17580707#comment-17580707
 ] 

Steve Loughran edited comment on SPARK-38330 at 8/17/22 9:45 AM:
-

bq. Is there a way to work-around this issue while waiting for a version of 
Spark which uses hadoop 3.3.4 (Spark 3.4?)

remove all jars with cos in the title from your classpath


was (Author: ste...@apache.org):
remove all jars with cos in the title from your classpath

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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 
> 

[jira] [Commented] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2022-08-17 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17580707#comment-17580707
 ] 

Steve Loughran commented on SPARK-38330:


remove all jars with cos in the title from your classpath

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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:1333)
>   at 
> 

[jira] [Commented] (SPARK-40039) Introducing a streaming checkpoint file manager based on Hadoop's Abortable interface

2022-08-11 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-40039?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17578334#comment-17578334
 ] 

Steve Loughran commented on SPARK-40039:


doesn't actualy use MPU; if you haven't uploaded any data yet it simply doesn't 
do that write in close();l

remember, all s3a/abfs input and output streams print their IOStatistics, if 
you don't believe me, print them!

> Introducing a streaming checkpoint file manager based on Hadoop's Abortable 
> interface
> -
>
> Key: SPARK-40039
> URL: https://issues.apache.org/jira/browse/SPARK-40039
> Project: Spark
>  Issue Type: Improvement
>  Components: Structured Streaming
>Affects Versions: 3.4.0
>Reporter: Attila Zsolt Piros
>Assignee: Attila Zsolt Piros
>Priority: Major
>
> Currently on S3 the checkpoint file manager (called 
> FileContextBasedCheckpointFileManager) is based on rename. So when a file is 
> opened for an atomic stream a temporary file used instead and when the stream 
> is committed the file is renamed.
> But on S3 a rename will be a file copy. So it has some serious performance 
> implication.
> But on Hadoop 3 there is new interface introduce called *Abortable* and 
> *S3AFileSystem* has this capability which is implemented by on top S3's 
> multipart upload. So when the file is committed a POST is sent 
> ([https://docs.aws.amazon.com/AmazonS3/latest/API/API_CompleteMultipartUpload.html])
>  and when aborted a DELETE will be send 
> ([https://docs.aws.amazon.com/AmazonS3/latest/API/API_AbortMultipartUpload.html])



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Updated] (SPARK-40034) PathOutputCommitters to work with dynamic partition overwrite

2022-08-10 Thread Steve Loughran (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-40034?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran updated SPARK-40034:
---
Summary: PathOutputCommitters to work with dynamic partition overwrite  
(was: PathOutputCommitters to work with dynamic partition overwrite -if they 
support it)

> PathOutputCommitters to work with dynamic partition overwrite
> -
>
> Key: SPARK-40034
> URL: https://issues.apache.org/jira/browse/SPARK-40034
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core, SQL
>Affects Versions: 3.4.0
>Reporter: Steve Loughran
>Priority: Minor
>
> sibling of MAPREDUCE-7403: allow PathOutputCommitter implementation to 
> declare that they support the semantics required by spark dynamic 
> partitioning:
> * rename to work as expected
> * working dir to be on same fs as final dir
> They will do this through implementing StreamCapabilities and adding a new 
> probe, "mapreduce.job.committer.dynamic.partitioning" ; the spark side 
> changes are to
> * postpone rejection of dynamic partition overwrite until the output 
> committer is created
> * allow it if the committer implements StreamCapabilities and returns true 
> for {{hasCapability("mapreduce.job.committer.dynamic.partitioning")))
> this isn't going to be supported by the s3a committers, they don't meet the 
> requirements. The manifest committer of MAPREDUCE-7341 running against abfs 
> and gcs does work. 



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-40034) PathOutputCommitters to work with dynamic partition overwrite -if they support it

2022-08-10 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-40034:
--

 Summary: PathOutputCommitters to work with dynamic partition 
overwrite -if they support it
 Key: SPARK-40034
 URL: https://issues.apache.org/jira/browse/SPARK-40034
 Project: Spark
  Issue Type: Improvement
  Components: Spark Core, SQL
Affects Versions: 3.4.0
Reporter: Steve Loughran


sibling of MAPREDUCE-7403: allow PathOutputCommitter implementation to declare 
that they support the semantics required by spark dynamic partitioning:

* rename to work as expected
* working dir to be on same fs as final dir

They will do this through implementing StreamCapabilities and adding a new 
probe, "mapreduce.job.committer.dynamic.partitioning" ; the spark side changes 
are to
* postpone rejection of dynamic partition overwrite until the output committer 
is created
* allow it if the committer implements StreamCapabilities and returns true for 
{{hasCapability("mapreduce.job.committer.dynamic.partitioning")))

this isn't going to be supported by the s3a committers, they don't meet the 
requirements. The manifest committer of MAPREDUCE-7341 running against abfs and 
gcs does work. 




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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-39969) Spark AWS SDK and kinesis dependencies lagging hadoop-aws and s3a

2022-08-09 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-39969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17577389#comment-17577389
 ] 

Steve Loughran commented on SPARK-39969:


there's an AWS SDK CVE which is fixed with HADOOP-18344; the s3a code and the 
spark kinesis module aren't vulnerable to it, but people will be complaining 
anyway

> Spark AWS SDK and kinesis dependencies lagging hadoop-aws and s3a
> -
>
> Key: SPARK-39969
> URL: https://issues.apache.org/jira/browse/SPARK-39969
> Project: Spark
>  Issue Type: Improvement
>  Components: Build
>Affects Versions: 3.4.0
>Reporter: Steve Loughran
>Priority: Minor
>
> The AWS SDK and matching kinesis versions are now a few iterations behind 
> what is shipping in hadoop 3.3.x. ( see HADOOP-18068 and HADOOP-18344)
> * this updates dependencies/bundling of jackson and httpclient
> * no problems upgrading other than some test regressions
> catching up would be good, as it means that recent s3a releases are not 
> qualified with the AWS SDK release spark is pulling in -and if there is any 
> problem. it'll be a spark team issue. 



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-39863) Upgrade Hadoop to 3.3.4

2022-08-03 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-39863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17574722#comment-17574722
 ] 

Steve Loughran commented on SPARK-39863:


probably should follow this with an upgrade of the aws sdk in spark SPARK-39969

> Upgrade Hadoop to 3.3.4
> ---
>
> Key: SPARK-39863
> URL: https://issues.apache.org/jira/browse/SPARK-39863
> Project: Spark
>  Issue Type: Improvement
>  Components: Build
>Affects Versions: 3.4.0
>Reporter: Chao Sun
>Priority: Major
>
> This JIRA tracks the progress of upgrading Hadoop dependency to 3.3.4



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-39969) Spark AWS SDK and kinesis dependencies lagging hadoop-aws and s3a

2022-08-03 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-39969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17574720#comment-17574720
 ] 

Steve Loughran commented on SPARK-39969:


note: although the latest release fixes the latest set of jackson CVEs in the 
aws shaded jar, the s3 client doesn't use the vulnerable libraries. don't know 
about the rest of the sdk.j it will at least stop code security analysis tools 
from complaining so much

> Spark AWS SDK and kinesis dependencies lagging hadoop-aws and s3a
> -
>
> Key: SPARK-39969
> URL: https://issues.apache.org/jira/browse/SPARK-39969
> Project: Spark
>  Issue Type: Improvement
>  Components: Build
>Affects Versions: 3.4.0
>Reporter: Steve Loughran
>Priority: Minor
>
> The AWS SDK and matching kinesis versions are now a few iterations behind 
> what is shipping in hadoop 3.3.x. ( see HADOOP-18068 and HADOOP-18344)
> * this updates dependencies/bundling of jackson and httpclient
> * no problems upgrading other than some test regressions
> catching up would be good, as it means that recent s3a releases are not 
> qualified with the AWS SDK release spark is pulling in -and if there is any 
> problem. it'll be a spark team issue. 



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-39969) Spark AWS SDK and kinesis dependencies lagging hadoop-aws and s3a

2022-08-03 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-39969:
--

 Summary: Spark AWS SDK and kinesis dependencies lagging hadoop-aws 
and s3a
 Key: SPARK-39969
 URL: https://issues.apache.org/jira/browse/SPARK-39969
 Project: Spark
  Issue Type: Improvement
  Components: Build
Affects Versions: 3.4.0
Reporter: Steve Loughran


The AWS SDK and matching kinesis versions are now a few iterations behind what 
is shipping in hadoop 3.3.x. ( see HADOOP-18068 and HADOOP-18344)

* this updates dependencies/bundling of jackson and httpclient
* no problems upgrading other than some test regressions

catching up would be good, as it means that recent s3a releases are not 
qualified with the AWS SDK release spark is pulling in -and if there is any 
problem. it'll be a spark team issue. 



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Updated] (SPARK-39969) Spark AWS SDK and kinesis dependencies lagging hadoop-aws and s3a

2022-08-03 Thread Steve Loughran (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-39969?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran updated SPARK-39969:
---
Priority: Minor  (was: Major)

> Spark AWS SDK and kinesis dependencies lagging hadoop-aws and s3a
> -
>
> Key: SPARK-39969
> URL: https://issues.apache.org/jira/browse/SPARK-39969
> Project: Spark
>  Issue Type: Improvement
>  Components: Build
>Affects Versions: 3.4.0
>Reporter: Steve Loughran
>Priority: Minor
>
> The AWS SDK and matching kinesis versions are now a few iterations behind 
> what is shipping in hadoop 3.3.x. ( see HADOOP-18068 and HADOOP-18344)
> * this updates dependencies/bundling of jackson and httpclient
> * no problems upgrading other than some test regressions
> catching up would be good, as it means that recent s3a releases are not 
> qualified with the AWS SDK release spark is pulling in -and if there is any 
> problem. it'll be a spark team issue. 



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38934) Provider TemporaryAWSCredentialsProvider has no credentials

2022-08-01 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17573653#comment-17573653
 ] 

Steve Loughran commented on SPARK-38934:


bq. our system set the provider as WebIdentityTokenCredentialsProvider as a 
default, I had to explicitly set as TemporaryAWSCredentialsProvider.

if TemporaryAWSCredentialsProvider had a bug then it would be a JIRA


> Provider TemporaryAWSCredentialsProvider has no credentials
> ---
>
> Key: SPARK-38934
> URL: https://issues.apache.org/jira/browse/SPARK-38934
> Project: Spark
>  Issue Type: Bug
>  Components: Kubernetes, Spark Core
>Affects Versions: 3.2.1
>Reporter: Lily
>Priority: Major
>
>  
> We are using Jupyter Hub on K8s as a notebook based development environment 
> and Spark on K8s as a backend cluster of Jupyter Hub on K8s with Spark 3.2.1 
> and Hadoop 3.3.1.
> When we run a code like the one below in the Jupyter Hub on K8s,
>  
> {code:java}
> val perm = ... // get AWS temporary credential by AWS STS from AWS assumed 
> role
> // set AWS temporary credential
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.aws.credentials.provider", 
> "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider")
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", 
> perm.credential.accessKeyID)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", 
> perm.credential.secretAccessKey)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.session.token", 
> perm.credential.sessionToken)
> // execute simple Spark action
> spark.read.format("parquet").load("s3a:///*").show(1) {code}
>  
>  
> the first few executors left a warning like the one below in the first code 
> execution, but we were able to get the proper result thanks to Spark task 
> retry function. 
> {code:java}
> 22/04/18 09:13:50 WARN TaskSetManager: Lost task 2.0 in stage 0.0 (TID 2) 
> (10.197.5.15 executor 1): java.nio.file.AccessDeniedException: 
> s3a:///.parquet: 
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider 
> TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206)
>   at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
>   at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:2810)
>   at 
> org.apache.spark.util.HadoopFSUtils$.listLeafFiles(HadoopFSUtils.scala:225)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$6(HadoopFSUtils.scala:136)
>   at scala.collection.immutable.Stream.map(Stream.scala:418)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$4(HadoopFSUtils.scala:126)
>   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:131)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: 
> Provider TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130)
>   at 
> org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:842)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:792)
>   at 
> 

[jira] [Commented] (SPARK-38934) Provider TemporaryAWSCredentialsProvider has no credentials

2022-08-01 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17573652#comment-17573652
 ] 

Steve Loughran commented on SPARK-38934:


because its your deployment setup, not anybody's code

> Provider TemporaryAWSCredentialsProvider has no credentials
> ---
>
> Key: SPARK-38934
> URL: https://issues.apache.org/jira/browse/SPARK-38934
> Project: Spark
>  Issue Type: Bug
>  Components: Kubernetes, Spark Core
>Affects Versions: 3.2.1
>Reporter: Lily
>Priority: Major
>
>  
> We are using Jupyter Hub on K8s as a notebook based development environment 
> and Spark on K8s as a backend cluster of Jupyter Hub on K8s with Spark 3.2.1 
> and Hadoop 3.3.1.
> When we run a code like the one below in the Jupyter Hub on K8s,
>  
> {code:java}
> val perm = ... // get AWS temporary credential by AWS STS from AWS assumed 
> role
> // set AWS temporary credential
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.aws.credentials.provider", 
> "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider")
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", 
> perm.credential.accessKeyID)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", 
> perm.credential.secretAccessKey)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.session.token", 
> perm.credential.sessionToken)
> // execute simple Spark action
> spark.read.format("parquet").load("s3a:///*").show(1) {code}
>  
>  
> the first few executors left a warning like the one below in the first code 
> execution, but we were able to get the proper result thanks to Spark task 
> retry function. 
> {code:java}
> 22/04/18 09:13:50 WARN TaskSetManager: Lost task 2.0 in stage 0.0 (TID 2) 
> (10.197.5.15 executor 1): java.nio.file.AccessDeniedException: 
> s3a:///.parquet: 
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider 
> TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206)
>   at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
>   at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:2810)
>   at 
> org.apache.spark.util.HadoopFSUtils$.listLeafFiles(HadoopFSUtils.scala:225)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$6(HadoopFSUtils.scala:136)
>   at scala.collection.immutable.Stream.map(Stream.scala:418)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$4(HadoopFSUtils.scala:126)
>   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:131)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: 
> Provider TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130)
>   at 
> org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:842)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:792)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:779)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:753)
>   at 
> 

[jira] [Commented] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2022-07-29 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17573141#comment-17573141
 ] 

Steve Loughran commented on SPARK-38330:


the hadoop 3.3.4 rC0 will fix this with that cut of the cos jar

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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:1333)
>   at 
> 

[jira] [Commented] (SPARK-38934) Provider TemporaryAWSCredentialsProvider has no credentials

2022-07-29 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17573138#comment-17573138
 ] 

Steve Loughran commented on SPARK-38934:


so that's a config problem? not a bug? closing

> Provider TemporaryAWSCredentialsProvider has no credentials
> ---
>
> Key: SPARK-38934
> URL: https://issues.apache.org/jira/browse/SPARK-38934
> Project: Spark
>  Issue Type: Bug
>  Components: Kubernetes, Spark Core
>Affects Versions: 3.2.1
>Reporter: Lily
>Priority: Major
>
>  
> We are using Jupyter Hub on K8s as a notebook based development environment 
> and Spark on K8s as a backend cluster of Jupyter Hub on K8s with Spark 3.2.1 
> and Hadoop 3.3.1.
> When we run a code like the one below in the Jupyter Hub on K8s,
>  
> {code:java}
> val perm = ... // get AWS temporary credential by AWS STS from AWS assumed 
> role
> // set AWS temporary credential
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.aws.credentials.provider", 
> "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider")
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", 
> perm.credential.accessKeyID)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", 
> perm.credential.secretAccessKey)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.session.token", 
> perm.credential.sessionToken)
> // execute simple Spark action
> spark.read.format("parquet").load("s3a:///*").show(1) {code}
>  
>  
> the first few executors left a warning like the one below in the first code 
> execution, but we were able to get the proper result thanks to Spark task 
> retry function. 
> {code:java}
> 22/04/18 09:13:50 WARN TaskSetManager: Lost task 2.0 in stage 0.0 (TID 2) 
> (10.197.5.15 executor 1): java.nio.file.AccessDeniedException: 
> s3a:///.parquet: 
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider 
> TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206)
>   at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
>   at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:2810)
>   at 
> org.apache.spark.util.HadoopFSUtils$.listLeafFiles(HadoopFSUtils.scala:225)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$6(HadoopFSUtils.scala:136)
>   at scala.collection.immutable.Stream.map(Stream.scala:418)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$4(HadoopFSUtils.scala:126)
>   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:131)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: 
> Provider TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130)
>   at 
> org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:842)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:792)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:779)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:753)
>   at 
> 

[jira] [Resolved] (SPARK-38934) Provider TemporaryAWSCredentialsProvider has no credentials

2022-07-29 Thread Steve Loughran (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-38934?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran resolved SPARK-38934.

Resolution: Invalid

> Provider TemporaryAWSCredentialsProvider has no credentials
> ---
>
> Key: SPARK-38934
> URL: https://issues.apache.org/jira/browse/SPARK-38934
> Project: Spark
>  Issue Type: Bug
>  Components: Kubernetes, Spark Core
>Affects Versions: 3.2.1
>Reporter: Lily
>Priority: Major
>
>  
> We are using Jupyter Hub on K8s as a notebook based development environment 
> and Spark on K8s as a backend cluster of Jupyter Hub on K8s with Spark 3.2.1 
> and Hadoop 3.3.1.
> When we run a code like the one below in the Jupyter Hub on K8s,
>  
> {code:java}
> val perm = ... // get AWS temporary credential by AWS STS from AWS assumed 
> role
> // set AWS temporary credential
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.aws.credentials.provider", 
> "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider")
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.access.key", 
> perm.credential.accessKeyID)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", 
> perm.credential.secretAccessKey)
> spark.sparkContext.hadoopConfiguration.set("fs.s3a.session.token", 
> perm.credential.sessionToken)
> // execute simple Spark action
> spark.read.format("parquet").load("s3a:///*").show(1) {code}
>  
>  
> the first few executors left a warning like the one below in the first code 
> execution, but we were able to get the proper result thanks to Spark task 
> retry function. 
> {code:java}
> 22/04/18 09:13:50 WARN TaskSetManager: Lost task 2.0 in stage 0.0 (TID 2) 
> (10.197.5.15 executor 1): java.nio.file.AccessDeniedException: 
> s3a:///.parquet: 
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider 
> TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206)
>   at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
>   at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:2810)
>   at 
> org.apache.spark.util.HadoopFSUtils$.listLeafFiles(HadoopFSUtils.scala:225)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$6(HadoopFSUtils.scala:136)
>   at scala.collection.immutable.Stream.map(Stream.scala:418)
>   at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$4(HadoopFSUtils.scala:126)
>   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:131)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: 
> Provider TemporaryAWSCredentialsProvider has no credentials
>   at 
> org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130)
>   at 
> org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:842)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:792)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:779)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:753)
>   at 
> 

[jira] [Commented] (SPARK-38958) Override S3 Client in Spark Write/Read calls

2022-07-29 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38958?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17573137#comment-17573137
 ] 

Steve Loughran commented on SPARK-38958:


#. api is public, but we have changed the api incompatibly twice. it's a 
builder/parameter object pattern to try and reduce this, but you will probably 
still need to be in sync with the build.
# branch-3.3 (you get to make the release yourself) lets you add whatever 
headers you want when creating a file
https://github.com/apache/hadoop/blob/trunk/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md#-s3a-specific-options


> Override S3 Client in Spark Write/Read calls
> 
>
> Key: SPARK-38958
> URL: https://issues.apache.org/jira/browse/SPARK-38958
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Affects Versions: 3.2.1
>Reporter: Hershal
>Priority: Major
>
> Hello,
> I have been working to use spark to read and write data to S3. Unfortunately, 
> there are a few S3 headers that I need to add to my spark read/write calls. 
> After much looking, I have not found a way to replace the S3 client that 
> spark uses to make the read/write calls. I also have not found a 
> configuration that allows me to pass in S3 headers. Here is an example of 
> some common S3 request headers 
> ([https://docs.aws.amazon.com/AmazonS3/latest/API/RESTCommonRequestHeaders.html).]
>  Does there already exist functionality to add S3 headers to spark read/write 
> calls or pass in a custom client that would pass these headers on every 
> read/write request? Appreciate the help and feedback
>  
> Thanks,



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-33088) Enhance ExecutorPlugin API to include methods for task start and end events

2022-07-25 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-33088?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17570936#comment-17570936
 ] 

Steve Loughran commented on SPARK-33088:


i;m playing with this and IOStatistics collection in hadoop 3.3.3+ 
(HADOOP-16830). 

Has anyone any examples of implementations of this i can look at?

i'm particularly curious as to how the driver-side plugin can get notified of 
task start/complete/fail, so can register accumulators, extract their values 
and publish them, which is what i want to do. Are people using a different 
plugin point there?

> Enhance ExecutorPlugin API to include methods for task start and end events
> ---
>
> Key: SPARK-33088
> URL: https://issues.apache.org/jira/browse/SPARK-33088
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Affects Versions: 3.1.0
>Reporter: Samuel Souza
>Assignee: Samuel Souza
>Priority: Major
> Fix For: 3.1.0
>
>
> On [SPARK-24918|https://issues.apache.org/jira/browse/SPARK-24918]'s 
> [SIPP|https://docs.google.com/document/d/1a20gHGMyRbCM8aicvq4LhWfQmoA5cbHBQtyqIA2hgtc/view#|https://docs.google.com/document/d/1a20gHGMyRbCM8aicvq4LhWfQmoA5cbHBQtyqIA2hgtc/edit#],
>  it was raised to potentially add methods to ExecutorPlugin interface on task 
> start and end:
> {quote}The basic interface can just be a marker trait, as that allows a 
> plugin to monitor general characteristics of the JVM (eg. monitor memory or 
> take thread dumps).   Optionally, we could include methods for task start and 
> end events.   This would allow more control on monitoring – eg., you could 
> start polling thread dumps only if there was a task from a particular stage 
> that had been taking too long. But anything task related is a bit trickier to 
> decide the right api. Should the task end event also get the failure reason? 
> Should those events get called in the same thread as the task runner, or in 
> another thread?
> {quote}
> The ask is to add exactly that. I've put up a draft PR [in our fork of 
> spark|https://github.com/palantir/spark/pull/713] and I'm happy to push it 
> upstream. Also happy to receive comments on what's the right interface to 
> expose - not opinionated on that front, tried to expose the simplest 
> interface for now.
> The main reason for this ask is to propagate tracing information from the 
> driver to the executors 
> ([SPARK-21962|https://issues.apache.org/jira/browse/SPARK-21962] has some 
> context). On 
> [HADOOP-15566|https://issues.apache.org/jira/browse/HADOOP-15566] I see we're 
> discussing how to add tracing to the Apache ecosystem, but my problem is 
> slightly different: I want to use this interface to propagate tracing 
> information to my framework of choice. If the Hadoop issue gets solved we'll 
> have a framework to communicate tracing information inside the Apache 
> ecosystem, but it's highly unlikely that all Spark users will use the same 
> common framework. Therefore we should still provide plugin interfaces where 
> the tracing information can be propagated appropriately.
> To give more color, in our case the tracing information is [stored in a 
> thread 
> local|https://github.com/palantir/tracing-java/blob/4.9.0/tracing/src/main/java/com/palantir/tracing/Tracer.java#L61],
>  therefore it needs to be set in the same thread which is executing the task. 
> [*]
> While our framework is specific, I imagine such an interface could be useful 
> in general. Happy to hear your thoughts about it.
> [*] Something I did not mention was how to propagate the tracing information 
> from the driver to the executors. For that I intend to use 1. the driver's 
> localProperties, which 2. will be eventually propagated to the executors' 
> TaskContext, which 3. I'll be able to access from the methods above.



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

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-29250) Upgrade to Hadoop 3.3.1

2022-06-13 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-29250?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17553744#comment-17553744
 ] 

Steve Loughran commented on SPARK-29250:


use whatever version the spark release was built with if you want least stress. 

> Upgrade to Hadoop 3.3.1
> ---
>
> Key: SPARK-29250
> URL: https://issues.apache.org/jira/browse/SPARK-29250
> Project: Spark
>  Issue Type: Improvement
>  Components: Build
>Affects Versions: 3.2.0
>Reporter: Dongjoon Hyun
>Assignee: Chao Sun
>Priority: Major
>  Labels: releasenotes
> Fix For: 3.2.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38954) Implement sharing of cloud credentials among driver and executors

2022-05-23 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38954?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17541002#comment-17541002
 ] 

Steve Loughran commented on SPARK-38954:


what is the strategy for having the workers get the secrets into the cloud 
connectors? 

> Implement sharing of cloud credentials among driver and executors
> -
>
> Key: SPARK-38954
> URL: https://issues.apache.org/jira/browse/SPARK-38954
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core
>Affects Versions: 3.4.0
>Reporter: Parth Chandra
>Priority: Major
>
> Currently Spark uses external implementations (e.g. hadoop-aws) to access 
> cloud services like S3. In order to access the actual service, these 
> implementations use credentials provider implementations that obtain 
> credentials to allow access to the cloud service.
> These credentials are typically session credentials, which means that they 
> expire after a fixed time. Sometimes, this expiry can be only an hour and for 
> a spark job that runs for many hours (or spark streaming job that runs 
> continuously), the credentials have to be renewed periodically.
> In many organizations, the process of getting credentials may multi-step. The 
> organization has an identity provider service that provides authentication 
> for the user, while the cloud service provider provides authorization for the 
> roles the user has access to. Once the user is authenticated and her role 
> verified, the credentials are generated for a new session.
> In a large setup with hundreds of Spark jobs and thousands of executors, each 
> executor is then spending a lot of time getting credentials and this may put 
> unnecessary load on the backend authentication services.
> The alleviate this, we can use Spark's architecture to obtain the credentials 
> once in the driver and push the credentials to the executors. In addition, 
> the driver can check the expiry of the credentials and push updated 
> credentials to the executors. This is relatively easy to do since the rpc 
> mechanism to implement this is already in place and is used similarly for 
> Kerberos delegation tokens.
>   



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2022-04-21 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17525719#comment-17525719
 ] 

Steve Loughran commented on SPARK-38330:


aws sdk does its own thing sometimes, from what we see. best not to look too 
close

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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:1333)
>   at 
> 

[jira] [Commented] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2022-04-18 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17523891#comment-17523891
 ] 

Steve Loughran commented on SPARK-38330:


FWIW I'm not 100% sure this is fixed, as we've had local impala test runs fail 
if hadoop-cos was on the classpath.

hadoop-3.3.3 will have the unshaded wildfly references though, so switching to 
openssl will be an option. please test it when I get the RC out this week

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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)
>   

[jira] [Commented] (SPARK-38445) Are hadoop committers used in Structured Streaming?

2022-04-05 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38445?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17517556#comment-17517556
 ] 

Steve Loughran commented on SPARK-38445:


not suppoorted unless you provide the PR for a new committer.

hadoop 3.3.1 added an abort() call on an output stream in order to make a 
zero-rename committer possible here...you would initiate a write to the final 
destination, but call abort() before close() if you needed to abort. as no 
output will appear if the process dies, failures won't be visible (billable of 
course, if you don't purge uploads)

> Are hadoop committers used in Structured Streaming?
> ---
>
> Key: SPARK-38445
> URL: https://issues.apache.org/jira/browse/SPARK-38445
> Project: Spark
>  Issue Type: Question
>  Components: Spark Core
>Affects Versions: 3.2.1
>Reporter: Martin Andersson
>Priority: Major
>  Labels: structured-streaming
>
> At the company I work at we're using Spark Structured Streaming to sink 
> messages on kafka to HDFS. We're in the late stages of migrating this 
> component to instead sink messages to AWS S3, and in connection with that we 
> hit upon a couple of issues regarding hadoop committers.
> I've come to understand that the default "file" committer (documented 
> [here|https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/committers.html#Switching_to_an_S3A_Committer])
>  is unsafe to use in S3, which is why [this page in the spark 
> documentation|https://spark.apache.org/docs/3.2.1/cloud-integration.html] 
> recommends using the "directory" (i.e. staging) committer, and in later 
> versions of hadoop they also recommend to use the "magic" committer.
> However, it's not clear whether spark structured streaming even use 
> committers. There's no "_SUCCESS" file in destination (as compared to normal 
> spark jobs), and the documentation regarding committers used in streaming is 
> non-existent.
> Can anyone please shed some light on this?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38652) K8S IT Test DepsTestsSuite blocks with PathIOException in hadoop-aws-3.3.2

2022-03-25 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38652?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17512458#comment-17512458
 ] 

Steve Loughran commented on SPARK-38652:


have you tried running the same suite against an aws s3 endpoint?

> K8S IT Test DepsTestsSuite blocks with PathIOException in hadoop-aws-3.3.2
> --
>
> Key: SPARK-38652
> URL: https://issues.apache.org/jira/browse/SPARK-38652
> Project: Spark
>  Issue Type: Bug
>  Components: Kubernetes, Tests
>Affects Versions: 3.3.0
>Reporter: qian
>Priority: Major
>
> DepsTestsSuite in k8s IT test is blocked with PathIOException in 
> hadoop-aws-3.3.2. Exception Message is as follow
> {code:java}
> Exception in thread "main" org.apache.spark.SparkException: Uploading file 
> /Users/hengzhen.sq/IdeaProjects/spark/dist/examples/jars/spark-examples_2.12-3.4.0-SNAPSHOT.jar
>  failed...
> at 
> org.apache.spark.deploy.k8s.KubernetesUtils$.uploadFileUri(KubernetesUtils.scala:332)
> 
> at 
> org.apache.spark.deploy.k8s.KubernetesUtils$.$anonfun$uploadAndTransformFileUris$1(KubernetesUtils.scala:277)
> 
> at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286) 
>
> at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)   
>  
> at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)  
>   
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
> at scala.collection.TraversableLike.map(TraversableLike.scala:286)
> at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
> at scala.collection.AbstractTraversable.map(Traversable.scala:108)
> at 
> org.apache.spark.deploy.k8s.KubernetesUtils$.uploadAndTransformFileUris(KubernetesUtils.scala:275)
> 
> at 
> org.apache.spark.deploy.k8s.features.BasicDriverFeatureStep.$anonfun$getAdditionalPodSystemProperties$1(BasicDriverFeatureStep.scala:187)
>
> at scala.collection.immutable.List.foreach(List.scala:431)
> at 
> org.apache.spark.deploy.k8s.features.BasicDriverFeatureStep.getAdditionalPodSystemProperties(BasicDriverFeatureStep.scala:178)
> 
> at 
> org.apache.spark.deploy.k8s.submit.KubernetesDriverBuilder.$anonfun$buildFromFeatures$5(KubernetesDriverBuilder.scala:86)
> at 
> scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
> 
> at 
> scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)   
>  
> at scala.collection.immutable.List.foldLeft(List.scala:91)
> at 
> org.apache.spark.deploy.k8s.submit.KubernetesDriverBuilder.buildFromFeatures(KubernetesDriverBuilder.scala:84)
> 
> at 
> org.apache.spark.deploy.k8s.submit.Client.run(KubernetesClientApplication.scala:104)
> 
> at 
> org.apache.spark.deploy.k8s.submit.KubernetesClientApplication.$anonfun$run$5(KubernetesClientApplication.scala:248)
> 
> at 
> org.apache.spark.deploy.k8s.submit.KubernetesClientApplication.$anonfun$run$5$adapted(KubernetesClientApplication.scala:242)
> at org.apache.spark.util.Utils$.tryWithResource(Utils.scala:2738)
> at 
> org.apache.spark.deploy.k8s.submit.KubernetesClientApplication.run(KubernetesClientApplication.scala:242)
> 
> at 
> org.apache.spark.deploy.k8s.submit.KubernetesClientApplication.start(KubernetesClientApplication.scala:214)
> 
> at 
> org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:958)
> 
> at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180) 
>
> at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
> at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
> at 
> org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1046)  
>   
> at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1055)
> at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)Caused by: 
> org.apache.spark.SparkException: Error uploading file 
> spark-examples_2.12-3.4.0-SNAPSHOT.jar
> at 
> org.apache.spark.deploy.k8s.KubernetesUtils$.uploadFileToHadoopCompatibleFS(KubernetesUtils.scala:355)
> 
> at 
> org.apache.spark.deploy.k8s.KubernetesUtils$.uploadFileUri(KubernetesUtils.scala:328)
> 
> ... 30 more
> Caused by: org.apache.hadoop.fs.PathIOException: `Cannot get relative path 
> for 
> URI:file:///Users/hengzhen.sq/IdeaProjects/spark/dist/examples/jars/spark-examples_2.12-3.4.0-SNAPSHOT.jar':
>  Input/output error
> at 
> org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation.getFinalPath(CopyFromLocalOperation.java:365)
> 
> at 
> 

[jira] [Commented] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2022-03-22 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17510830#comment-17510830
 ] 

Steve Loughran commented on SPARK-38330:


the hadoop fix is in, but it will take a while.
note that on hadoop 3.3.1+, if you can switch to the unshaded aws sdk, then you 
can change the http client version.

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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 
> 

[jira] [Commented] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2022-03-16 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17507557#comment-17507557
 ] 

Steve Loughran commented on SPARK-38330:


sorry about that. try enabling path style access and see if that helps

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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:1333)
>   at 
> 

[jira] [Commented] (SPARK-38330) Certificate doesn't match any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]

2022-03-10 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17504286#comment-17504286
 ] 

Steve Loughran commented on SPARK-38330:


this is a hadoop issue -create a Jira there and file as a causes.

 
 # the aws sdk bundled jar has its own httpclient, so upgrading that may fix it
 # and recent hadoop releases let you switch to openssl if it is on the system, 
so has it handling certs

> Certificate doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
> --
>
> Key: SPARK-38330
> URL: https://issues.apache.org/jira/browse/SPARK-38330
> Project: Spark
>  Issue Type: Bug
>  Components: EC2
>Affects Versions: 3.2.1
> Environment: Spark 3.2.1 built with `hadoop-cloud` flag.
> Direct access to s3 using default file committer.
> JDK8.
>  
>Reporter: André F.
>Priority: Major
>
> Trying to run any job after bumping our Spark version from 3.1.2 to 3.2.1, 
> lead us to the current exception while reading files on s3:
> {code:java}
> org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on 
> s3a:///.parquet: com.amazonaws.SdkClientException: Unable to 
> execute HTTP request: Certificate for  doesn't match 
> any of the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com]: 
> Unable to execute HTTP request: Certificate for  doesn't match any of 
> the subject alternative names: [*.s3.amazonaws.com, s3.amazonaws.com] at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) at 
> org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3351)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>  at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.isDirectory(S3AFileSystem.java:4277) 
> at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:54)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:370)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:274) 
> at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$3(DataFrameReader.scala:245)
>  at scala.Option.getOrElse(Option.scala:189) at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:245) at 
> org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:596) {code}
>  
> {code:java}
> Caused by: javax.net.ssl.SSLPeerUnverifiedException: Certificate for 
>  doesn't match any of the subject alternative names: 
> [*.s3.amazonaws.com, s3.amazonaws.com]
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.verifyHostname(SSLConnectionSocketFactory.java:507)
>   at 
> com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(SSLConnectionSocketFactory.java:437)
>   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)
>   at sun.reflect.GeneratedMethodAccessor36.invoke(Unknown Source)
>   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.$Proxy16.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 
> 

[jira] [Resolved] (SPARK-31911) Using S3A staging committer, pending uploads are committed more than once and listed incorrectly in _SUCCESS data

2022-03-10 Thread Steve Loughran (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-31911?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran resolved SPARK-31911.

Fix Version/s: 3.0.1
   2.4.7
   Resolution: Fixed

> Using S3A staging committer, pending uploads are committed more than once and 
> listed incorrectly in _SUCCESS data
> -
>
> Key: SPARK-31911
> URL: https://issues.apache.org/jira/browse/SPARK-31911
> Project: Spark
>  Issue Type: Bug
>  Components: Input/Output
>Affects Versions: 2.4.4
>Reporter: Brandon
>Priority: Major
> Fix For: 3.0.1, 2.4.7
>
>
> First of all thanks for the great work on the S3 committers. I was able set 
> up the directory staging committer in my environment following docs at 
> [https://github.com/apache/spark/blob/master/docs/cloud-integration.md#committing-work-into-cloud-storage-safely-and-fast]
>  and tested one of my Spark applications using it. The Spark version is 2.4.4 
> with Hadoop 3.2.1 and the cloud committer bindings. The application writes 
> multiple DataFrames to ORC/Parquet in S3, submitting them as write jobs to 
> Spark in parallel.
> I think I'm seeing a bug where the staging committer will complete pending 
> uploads more than once. The main symptom how I discovered this is that the 
> _SUCCESS data files under each table will contain overlapping file names that 
> belong to separate tables. From my reading of the code, that's because the 
> filenames in _SUCCESS reflect which multipart uploads were completed in the 
> commit for that particular table.
> An example:
> Concurrently, fire off DataFrame.write.orc("s3a://bucket/a") and 
> DataFrame.write.orc("s3a://bucket/b"). Suppose each table has one partition 
> so writes one partition file.
> When the two writes are done,
>  * /a/_SUCCESS contains two filenames: /a/part- and /b/part-.
>  * /b/_SUCCESS contains the same two filenames.
> Setting S3A logs to debug, I see the commitJob operation belonging to table a 
> includes completing the uploads of /a/part- and /b/part-. Then again, 
> commitJob for table b includes the same completions. I haven't had a problem 
> yet, but I wonder if having these extra requests would become an issue at 
> higher scale, where dozens of commits with hundreds of files may be happening 
> concurrently in the application.
> I believe this may be caused from the way the pendingSet files are stored in 
> the staging directory. They are stored under one directory named by the 
> jobID, in the Hadoop code. However, for all write jobs executed by the Spark 
> application, the jobID passed to Hadoop is the same - the application ID. 
> Maybe the staging commit algorithm was built on the assumption that each 
> instance of the algorithm would use a unique random jobID.
> [~ste...@apache.org] , [~rdblue] Having seen your names on most of this work 
> (thank you), I would be interested to know your thoughts on this. Also it's 
> my first time opening a bug here, so let me know if there's anything else I 
> can do to help report the issue.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-31911) Using S3A staging committer, pending uploads are committed more than once and listed incorrectly in _SUCCESS data

2022-03-10 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-31911?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17504273#comment-17504273
 ] 

Steve Loughran commented on SPARK-31911:


I'm going to close as fixed now; the spark changes will have done it.

> Using S3A staging committer, pending uploads are committed more than once and 
> listed incorrectly in _SUCCESS data
> -
>
> Key: SPARK-31911
> URL: https://issues.apache.org/jira/browse/SPARK-31911
> Project: Spark
>  Issue Type: Bug
>  Components: Input/Output
>Affects Versions: 2.4.4
>Reporter: Brandon
>Priority: Major
>
> First of all thanks for the great work on the S3 committers. I was able set 
> up the directory staging committer in my environment following docs at 
> [https://github.com/apache/spark/blob/master/docs/cloud-integration.md#committing-work-into-cloud-storage-safely-and-fast]
>  and tested one of my Spark applications using it. The Spark version is 2.4.4 
> with Hadoop 3.2.1 and the cloud committer bindings. The application writes 
> multiple DataFrames to ORC/Parquet in S3, submitting them as write jobs to 
> Spark in parallel.
> I think I'm seeing a bug where the staging committer will complete pending 
> uploads more than once. The main symptom how I discovered this is that the 
> _SUCCESS data files under each table will contain overlapping file names that 
> belong to separate tables. From my reading of the code, that's because the 
> filenames in _SUCCESS reflect which multipart uploads were completed in the 
> commit for that particular table.
> An example:
> Concurrently, fire off DataFrame.write.orc("s3a://bucket/a") and 
> DataFrame.write.orc("s3a://bucket/b"). Suppose each table has one partition 
> so writes one partition file.
> When the two writes are done,
>  * /a/_SUCCESS contains two filenames: /a/part- and /b/part-.
>  * /b/_SUCCESS contains the same two filenames.
> Setting S3A logs to debug, I see the commitJob operation belonging to table a 
> includes completing the uploads of /a/part- and /b/part-. Then again, 
> commitJob for table b includes the same completions. I haven't had a problem 
> yet, but I wonder if having these extra requests would become an issue at 
> higher scale, where dozens of commits with hundreds of files may be happening 
> concurrently in the application.
> I believe this may be caused from the way the pendingSet files are stored in 
> the staging directory. They are stored under one directory named by the 
> jobID, in the Hadoop code. However, for all write jobs executed by the Spark 
> application, the jobID passed to Hadoop is the same - the application ID. 
> Maybe the staging commit algorithm was built on the assumption that each 
> instance of the algorithm would use a unique random jobID.
> [~ste...@apache.org] , [~rdblue] Having seen your names on most of this work 
> (thank you), I would be interested to know your thoughts on this. Also it's 
> my first time opening a bug here, so let me know if there's anything else I 
> can do to help report the issue.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-38394) build of spark sql against hadoop-3.4.0-snapshot failing with bouncycastle classpath error

2022-03-02 Thread Steve Loughran (Jira)
Steve Loughran created SPARK-38394:
--

 Summary: build of spark sql against hadoop-3.4.0-snapshot failing 
with bouncycastle classpath error
 Key: SPARK-38394
 URL: https://issues.apache.org/jira/browse/SPARK-38394
 Project: Spark
  Issue Type: Bug
  Components: Build
Affects Versions: 3.3.0
Reporter: Steve Loughran


builidng spark master with {{-Dhadoop.version=3.4.0-SNAPSHOT}} and a local 
hadoop build breaks in the sbt compiler plugin


{code}
[ERROR] Failed to execute goal 
net.alchim31.maven:scala-maven-plugin:4.3.0:testCompile 
(scala-test-compile-first) on project spark-sql_2.12: Execution 
scala-test-compile-first of goal 
net.alchim31.maven:scala-maven-plugin:4.3.0:testCompile failed: A required 
class was missing while executing 
net.alchim31.maven:scala-maven-plugin:4.3.0:testCompile: 
org/bouncycastle/jce/provider/BouncyCastleProvider
[ERROR] -
[ERROR] realm =plugin>net.alchim31.maven:scala-maven-plugin:4.3.0

{code}

* this is the classpath of the sbt compiler
* hadoop hasn't been doing anything related to bouncy castle.

setting scala-maven-plugin to 3.4.0 makes this go away, i.e. reapplying 
SPARK-36547

the implication here is that the plugin version is going to have to be 
configured in different profiles.






--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38115) No spark conf to control the path of _temporary when writing to target filesystem

2022-02-22 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38115?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17496022#comment-17496022
 ] 

Steve Loughran commented on SPARK-38115:


bq. Is there any config as such to stop using FileOutputCommiter, because we 
didn't set any conf explicitly to use the committers.
https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/committers.html

bq. Just I am looking if I can use conf/options to manage temporary location as 
staging and have target path as primary

no, because the commit-by-rename mechanism is broken on s3; tuning temp dir 
location isn't going to fix that

> No spark conf to control the path of _temporary when writing to target 
> filesystem
> -
>
> Key: SPARK-38115
> URL: https://issues.apache.org/jira/browse/SPARK-38115
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core
>Affects Versions: 2.4.8, 3.2.1
>Reporter: kk
>Priority: Minor
>  Labels: spark, spark-conf, spark-sql, spark-submit
>
> No default spark conf or param to control the '_temporary' path when writing 
> to filesystem.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-38115) No spark conf to control the path of _temporary when writing to target filesystem

2022-02-15 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-38115?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17492810#comment-17492810
 ] 

Steve Loughran commented on SPARK-38115:


* stop using the classic FileOutputCommitter for your work, unless you like 
waiting a long time for your jobs to complete. along with a risk of corrupt 
data in the presence of worker failures.
* the choice of where temporary paths go is a function of the committer, not 
the spark codebase. the s3a staging committer uses the local fs. for example
* the magic committer does work under _temporary, but it doesn't write the 
final data there. it's "magic", after all. l

> No spark conf to control the path of _temporary when writing to target 
> filesystem
> -
>
> Key: SPARK-38115
> URL: https://issues.apache.org/jira/browse/SPARK-38115
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core
>Affects Versions: 2.4.8, 3.2.1
>Reporter: kk
>Priority: Minor
>  Labels: spark, spark-conf, spark-sql, spark-submit
>
> No default spark conf or param to control the '_temporary' path when writing 
> to filesystem.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Comment Edited] (SPARK-37814) Migrating from log4j 1 to log4j 2

2022-02-08 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-37814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17488804#comment-17488804
 ] 

Steve Loughran edited comment on SPARK-37814 at 2/8/22, 12:04 PM:
--

everyone is aware of the log4j 1.2 issues, but they are much less critical than 
log4j2 as they are (a) only local network vulnerabilities and (b) only through 
appender services which nobody ever deploys.

if you have your own downstream distribution, know that reload4j is a drop in 
replacement for log4j 1.2.17 without the vulnerabilities, making it a 
low-stress upgrade https://github.com/qos-ch/reload4j
hadoop is going to be adopting that on all its maintenance branches while the 
more traumatic logging upgrade is done on trunk 


was (Author: ste...@apache.org):
everyone is aware of the log4j issues, but they are much less critical than 
log4j2 as they are (a) only local network vulnerabilities and (b) only through 
appender services which nobody ever deploys.

if you have your own downstream distribution, know that reload4j is a drop in 
replacement for log4j 1.2.17 without the vulnerabilities, making it a 
low-stress upgrade https://github.com/qos-ch/reload4j
hadoop is going to be adopting that on all its maintenance branches while the 
more traumatic logging upgrade is done on trunk 

> Migrating from log4j 1 to log4j 2
> -
>
> Key: SPARK-37814
> URL: https://issues.apache.org/jira/browse/SPARK-37814
> Project: Spark
>  Issue Type: Umbrella
>  Components: Build
>Affects Versions: 3.3.0
>Reporter: L. C. Hsieh
>Assignee: L. C. Hsieh
>Priority: Major
>  Labels: releasenotes
> Fix For: 3.3.0
>
>
> This is umbrella ticket for all tasks related to migrating to log4j2.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-37814) Migrating from log4j 1 to log4j 2

2022-02-08 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-37814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17488804#comment-17488804
 ] 

Steve Loughran commented on SPARK-37814:


everyone is aware of the log4j issues, but they are much less critical than 
log4j2 as they are (a) only local network vulnerabilities and (b) only through 
appender services which nobody ever deploys.

if you have your own downstream distribution, know that reload4j is a drop in 
replacement for log4j 1.2.17 without the vulnerabilities, making it a 
low-stress upgrade https://github.com/qos-ch/reload4j
hadoop is going to be adopting that on all its maintenance branches while the 
more traumatic logging upgrade is done on trunk 

> Migrating from log4j 1 to log4j 2
> -
>
> Key: SPARK-37814
> URL: https://issues.apache.org/jira/browse/SPARK-37814
> Project: Spark
>  Issue Type: Umbrella
>  Components: Build
>Affects Versions: 3.3.0
>Reporter: L. C. Hsieh
>Assignee: L. C. Hsieh
>Priority: Major
>  Labels: releasenotes
> Fix For: 3.3.0
>
>
> This is umbrella ticket for all tasks related to migrating to log4j2.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-37771) Race condition in withHiveState and limited logic in IsolatedClientLoader result in ClassNotFoundException

2022-02-02 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-37771?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17485973#comment-17485973
 ] 

Steve Loughran commented on SPARK-37771:


[~ivan.sadikov] -any update here?

> Race condition in withHiveState and limited logic in IsolatedClientLoader 
> result in ClassNotFoundException
> --
>
> Key: SPARK-37771
> URL: https://issues.apache.org/jira/browse/SPARK-37771
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.0, 3.1.2, 3.2.0
>Reporter: Ivan Sadikov
>Priority: Major
>
> There is a race condition between creating a Hive client and loading classes 
> that do not appear in shared prefixes config. For example, we confirmed that 
> the code fails for the following configuration:
> {code:java}
> spark.sql.hive.metastore.version 0.13.0
> spark.sql.hive.metastore.jars maven
> spark.sql.hive.metastore.sharedPrefixes  com.amazonaws prefix>
> spark.hadoop.fs.s3a.impl org.apache.hadoop.fs.s3a.S3AFileSystem{code}
> And code: 
> {code:java}
> -- Prerequisite commands to set up the table
> -- drop table if exists ivan_test_2;
> -- create table ivan_test_2 (a int, part string) using csv location 
> 's3://bucket/hive-test' partitioned by (part);
> -- insert into ivan_test_2 values (1, 'a'); 
> -- Command that triggers failure
> ALTER TABLE ivan_test_2 ADD PARTITION (part='b') LOCATION 
> 's3://bucket/hive-test'{code}
>  
> Stacktrace (line numbers might differ):
> {code:java}
> 21/12/22 04:37:05 DEBUG IsolatedClientLoader: shared class: 
> org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
> 21/12/22 04:37:05 DEBUG IsolatedClientLoader: shared class: 
> org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider
> 21/12/22 04:37:05 DEBUG IsolatedClientLoader: hive class: 
> com.amazonaws.auth.EnvironmentVariableCredentialsProvider - null
> 21/12/22 04:37:05 ERROR S3AFileSystem: Failed to initialize S3AFileSystem for 
> path s3://bucket/hive-test
> java.io.IOException: From option fs.s3a.aws.credentials.provider 
> java.lang.ClassNotFoundException: Class 
> com.amazonaws.auth.EnvironmentVariableCredentialsProvider not found
>     at 
> org.apache.hadoop.fs.s3a.S3AUtils.loadAWSProviderClasses(S3AUtils.java:725)
>     at 
> org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:688)
>     at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:411)
>     at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3469)
>     at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174)
>     at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574)
>     at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521)
>     at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540)
>     at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365)
>     at org.apache.hadoop.hive.metastore.Warehouse.getFs(Warehouse.java:112)
>     at 
> org.apache.hadoop.hive.metastore.Warehouse.getDnsPath(Warehouse.java:144)
>     at 
> org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createLocationForAddedPartition(HiveMetaStore.java:1993)
>     at 
> org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.add_partitions_core(HiveMetaStore.java:1865)
>     at 
> org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.add_partitions_req(HiveMetaStore.java:1910)
>     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 
> org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:105)
>     at com.sun.proxy.$Proxy58.add_partitions_req(Unknown Source)
>     at 
> org.apache.hadoop.hive.metastore.HiveMetaStoreClient.add_partitions(HiveMetaStoreClient.java:457)
>     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 
> org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:89)
>     at com.sun.proxy.$Proxy59.add_partitions(Unknown Source)
>     at 
> org.apache.hadoop.hive.ql.metadata.Hive.createPartitions(Hive.java:1514)
>     at 
> org.apache.spark.sql.hive.client.Shim_v0_13.createPartitions(HiveShim.scala:773)
>     at 
> org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$createPartitions$1(HiveClientImpl.scala:683)
>     at 

[jira] [Commented] (SPARK-37771) Race condition in withHiveState and limited logic in IsolatedClientLoader result in ClassNotFoundException

2022-01-07 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-37771?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17470842#comment-17470842
 ] 

Steve Loughran commented on SPARK-37771:


probably related to HADOOP-17372, which makes sure the hive classloader isn't 
picked up for class lookups in the config

try with hadoop 3.3.1 binaries

> Race condition in withHiveState and limited logic in IsolatedClientLoader 
> result in ClassNotFoundException
> --
>
> Key: SPARK-37771
> URL: https://issues.apache.org/jira/browse/SPARK-37771
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.0, 3.1.2, 3.2.0
>Reporter: Ivan Sadikov
>Priority: Major
>
> There is a race condition between creating a Hive client and loading classes 
> that do not appear in shared prefixes config. For example, we confirmed that 
> the code fails for the following configuration:
> {code:java}
> spark.sql.hive.metastore.version 0.13.0
> spark.sql.hive.metastore.jars maven
> spark.sql.hive.metastore.sharedPrefixes  com.amazonaws prefix>
> spark.hadoop.fs.s3a.impl org.apache.hadoop.fs.s3a.S3AFileSystem{code}
> And code: 
> {code:java}
> -- Prerequisite commands to set up the table
> -- drop table if exists ivan_test_2;
> -- create table ivan_test_2 (a int, part string) using csv location 
> 's3://bucket/hive-test' partitioned by (part);
> -- insert into ivan_test_2 values (1, 'a'); 
> -- Command that triggers failure
> ALTER TABLE ivan_test_2 ADD PARTITION (part='b') LOCATION 
> 's3://bucket/hive-test'{code}
>  
> Stacktrace (line numbers might differ):
> {code:java}
> 21/12/22 04:37:05 DEBUG IsolatedClientLoader: shared class: 
> org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
> 21/12/22 04:37:05 DEBUG IsolatedClientLoader: shared class: 
> org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider
> 21/12/22 04:37:05 DEBUG IsolatedClientLoader: hive class: 
> com.amazonaws.auth.EnvironmentVariableCredentialsProvider - null
> 21/12/22 04:37:05 ERROR S3AFileSystem: Failed to initialize S3AFileSystem for 
> path s3://bucket/hive-test
> java.io.IOException: From option fs.s3a.aws.credentials.provider 
> java.lang.ClassNotFoundException: Class 
> com.amazonaws.auth.EnvironmentVariableCredentialsProvider not found
>     at 
> org.apache.hadoop.fs.s3a.S3AUtils.loadAWSProviderClasses(S3AUtils.java:725)
>     at 
> org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:688)
>     at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:411)
>     at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3469)
>     at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174)
>     at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574)
>     at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521)
>     at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540)
>     at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365)
>     at org.apache.hadoop.hive.metastore.Warehouse.getFs(Warehouse.java:112)
>     at 
> org.apache.hadoop.hive.metastore.Warehouse.getDnsPath(Warehouse.java:144)
>     at 
> org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createLocationForAddedPartition(HiveMetaStore.java:1993)
>     at 
> org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.add_partitions_core(HiveMetaStore.java:1865)
>     at 
> org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.add_partitions_req(HiveMetaStore.java:1910)
>     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 
> org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:105)
>     at com.sun.proxy.$Proxy58.add_partitions_req(Unknown Source)
>     at 
> org.apache.hadoop.hive.metastore.HiveMetaStoreClient.add_partitions(HiveMetaStoreClient.java:457)
>     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 
> org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:89)
>     at com.sun.proxy.$Proxy59.add_partitions(Unknown Source)
>     at 
> org.apache.hadoop.hive.ql.metadata.Hive.createPartitions(Hive.java:1514)
>     at 
> org.apache.spark.sql.hive.client.Shim_v0_13.createPartitions(HiveShim.scala:773)
>     at 
> 

[jira] [Commented] (SPARK-37814) Migrating from log4j 1 to log4j 2

2022-01-05 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-37814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17469178#comment-17469178
 ] 

Steve Loughran commented on SPARK-37814:


be good to link to all issues related to this, e.g test log problems, so other 
projects doing the same work know what to look for

> Migrating from log4j 1 to log4j 2
> -
>
> Key: SPARK-37814
> URL: https://issues.apache.org/jira/browse/SPARK-37814
> Project: Spark
>  Issue Type: Umbrella
>  Components: Build
>Affects Versions: 3.3.0
>Reporter: L. C. Hsieh
>Assignee: L. C. Hsieh
>Priority: Major
>  Labels: releasenotes
> Fix For: 3.3.0
>
>
> This is umbrella ticket for all tasks related to migrating to log4j2.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Comment Edited] (SPARK-6305) Add support for log4j 2.x to Spark

2021-12-30 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-6305?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17466876#comment-17466876
 ] 

Steve Loughran edited comment on SPARK-6305 at 12/30/21, 6:44 PM:
--

If anyone wants a version of a log4j 1.17 without the known (and never used) 
CVE, you could grab the Cloudera patched JAR. ASF projects are not allowed to 
release their own builds of other projects, so I'm afraid you are not allowed 
to include this in Apache releases.
https://mvnrepository.com/artifact/log4j/log4j/1.2.17-cloudera1
You don't need to move to log4j 2, and if you are, now is a good time to look 
at alternatives e.g. logback. I suspect this is what Hadoop will pick up in the 
new year, 


was (Author: ste...@apache.org):
If anyone wants a version of a log4j 1.17 without the known (and never used) 
CVE, you could grab the Cloudera patched JAR. ASF projects are not allowed to 
release their own builds of other projects, so I'm afraid you were not allowed 
her to include this in Apache releases.
https://mvnrepository.com/artifact/log4j/log4j/1.2.17-cloudera1
You don't need to move to log4j 2, and if you are, now is a good time to look 
at alternatives e.g. logback. I suspect this is what Hadoop will pick up in the 
new year, 

> Add support for log4j 2.x to Spark
> --
>
> Key: SPARK-6305
> URL: https://issues.apache.org/jira/browse/SPARK-6305
> Project: Spark
>  Issue Type: Improvement
>  Components: Build
>Reporter: Tal Sliwowicz
>Assignee: L. C. Hsieh
>Priority: Minor
> Fix For: 3.3.0
>
>
> log4j 2 requires replacing the slf4j binding and adding the log4j jars in the 
> classpath. Since there are shaded jars, it must be done during the build.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-37630) Security issue from Log4j 1.X exploit

2021-12-30 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-37630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17466918#comment-17466918
 ] 

Steve Loughran commented on SPARK-37630:


nobody does. 
you can find a patched jar at 
https://mvnrepository.com/artifact/log4j/log4j/1.2.17-cloudera1
this has been shipping in Cloudera artefacts for a long time and nobody has 
ever filed a bug report saying their logs have stopped working

> Security issue from Log4j 1.X exploit
> -
>
> Key: SPARK-37630
> URL: https://issues.apache.org/jira/browse/SPARK-37630
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 2.4.8, 3.2.0
>Reporter: Ismail H
>Priority: Major
>  Labels: security
>
> log4j is being used in version [1.2.17|#L122]]
>  
> This version has been deprecated and since [then have a known issue that 
> hasn't been adressed in 1.X 
> versions|https://www.cvedetails.com/cve/CVE-2019-17571/].
>  
> *Solution:*
>  * Upgrade log4j to version 2.15.0 which correct all known issues. [Last 
> known issues |https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2021-44228]



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-6305) Add support for log4j 2.x to Spark

2021-12-30 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-6305?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17466876#comment-17466876
 ] 

Steve Loughran commented on SPARK-6305:
---

If anyone wants a version of a log4j 1.17 without the known (and never used) 
CVE, you could grab the Cloudera patched JAR. ASF projects are not allowed to 
release their own builds of other projects, so I'm afraid you were not allowed 
her to include this in Apache releases.
https://mvnrepository.com/artifact/log4j/log4j/1.2.17-cloudera1
You don't need to move to log4j 2, and if you are, now is a good time to look 
at alternatives e.g. logback. I suspect this is what Hadoop will pick up in the 
new year, 

> Add support for log4j 2.x to Spark
> --
>
> Key: SPARK-6305
> URL: https://issues.apache.org/jira/browse/SPARK-6305
> Project: Spark
>  Issue Type: Improvement
>  Components: Build
>Reporter: Tal Sliwowicz
>Assignee: L. C. Hsieh
>Priority: Minor
> Fix For: 3.3.0
>
>
> log4j 2 requires replacing the slf4j binding and adding the log4j jars in the 
> classpath. Since there are shaded jars, it must be done during the build.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-23977) Add commit protocol binding to Hadoop 3.1 PathOutputCommitter mechanism

2021-11-02 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-23977?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17437259#comment-17437259
 ] 

Steve Loughran commented on SPARK-23977:


[~gumartinm] can I draw your attention to Apache Iceberg?

meanwhile
MAPREDUCE-7341 adds a high performance targeting abfs and gcs; all tree 
scanning is in task commit, which is atomic; job commit aggressively 
parallelised and optimized for stores whose listStatusIterator calls are 
incremental with prefetching: we can start processing at the first page of task 
manifests found in a listing well the second Page is still being retrieved. 
Also in there: rate limiting, IO Statistics Collection.

HADOOP-17833 I will pick up some of that work, including incremental loading 
and rate limiting. And if we can keep reads and writes below the S3 IOPS 
limits, we should be able to avoid situations where we have to start sleeping 
and re-trying.

HADOOP-17981 is my homework this week -emergency work to deal with a rare but 
current failure in abfs under heavy load.


> Add commit protocol binding to Hadoop 3.1 PathOutputCommitter mechanism
> ---
>
> Key: SPARK-23977
> URL: https://issues.apache.org/jira/browse/SPARK-23977
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.4.0
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>Priority: Minor
> Fix For: 3.0.0
>
>
> Hadoop 3.1 adds a mechanism for job-specific and store-specific committers 
> (MAPREDUCE-6823, MAPREDUCE-6956), and one key implementation, S3A committers, 
> HADOOP-13786
> These committers deliver high-performance output of MR and spark jobs to S3, 
> and offer the key semantics which Spark depends on: no visible output until 
> job commit, a failure of a task at an stage, including partway through task 
> commit, can be handled by executing and committing another task attempt. 
> In contrast, the FileOutputFormat commit algorithms on S3 have issues:
> * Awful performance because files are copied by rename
> * FileOutputFormat v1: weak task commit failure recovery semantics as the 
> (v1) expectation: "directory renames are atomic" doesn't hold.
> * S3 metadata eventual consistency can cause rename to miss files or fail 
> entirely (SPARK-15849)
> Note also that FileOutputFormat "v2" commit algorithm doesn't offer any of 
> the commit semantics w.r.t observability of or recovery from task commit 
> failure, on any filesystem.
> The S3A committers address these by way of uploading all data to the 
> destination through multipart uploads, uploads which are only completed in 
> job commit.
> The new {{PathOutputCommitter}} factory mechanism allows applications to work 
> with the S3A committers and any other, by adding a plugin mechanism into the 
> MRv2 FileOutputFormat class, where it job config and filesystem configuration 
> options can dynamically choose the output committer.
> Spark can use these with some binding classes to 
> # Add a subclass of {{HadoopMapReduceCommitProtocol}} which uses the MRv2 
> classes and {{PathOutputCommitterFactory}} to create the committers.
> # Add a {{BindingParquetOutputCommitter extends ParquetOutputCommitter}}
> to wire up Parquet output even when code requires the committer to be a 
> subclass of {{ParquetOutputCommitter}}
> This patch builds on SPARK-23807 for setting up the dependencies.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-36024) Switch the datasource example due to the depreciation of the dataset

2021-10-08 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-36024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17426166#comment-17426166
 ] 

Steve Loughran commented on SPARK-36024:


Amazon are being very nice here and keeping the landsat index file for us: )

even so, it'd be good to move onto something more structured anyway, just to 
discourage people using .csv.gz as a persistence format

> Switch the datasource example due to the depreciation of the dataset
> 
>
> Key: SPARK-36024
> URL: https://issues.apache.org/jira/browse/SPARK-36024
> Project: Spark
>  Issue Type: Documentation
>  Components: Documentation
>Affects Versions: 3.1.2
>Reporter: Leona Yoda
>Priority: Trivial
>
> The S3 bucket that used for an example in "Integration with Cloud 
> Infrastructures" document will be deleted on Jul 1, 2021 
> [https://registry.opendata.aws/landsat-8/ 
> |https://registry.opendata.aws/landsat-8/]
> The dataset will move to another bucket but it requires `--request-payer 
> requester` option so users have to pay S3 cost. 
> [https://registry.opendata.aws/usgs-landsat/]
>  
> So I think it's better to change the datasource like this.
> [https://github.com/yoda-mon/spark/commit/cdb24acdbb57a429e5bf1729502653b91a600022]
>  
> I chose [NYC Taxi data| 
> [https://registry.opendata.aws/nyc-tlc-trip-records-pds/|https://registry.opendata.aws/nyc-tlc-trip-records-pds/),]]
>  here for an example. 
>  Unlike landat data it's not compressed, but it is just an example and there 
> are several tutorials using Spark  (e.g. 
> [https://github.com/aws-samples/amazon-eks-apache-spark-etl-sample)]
>  
> Reed test result
> {code:java}
> scala> sc.textFile("s3a://nyc-tlc/misc/taxi 
> _zone_lookup.csv").take(10).foreach(println) 
> "LocationID","Borough","Zone","service_zone" 1,"EWR","Newark Airport","EWR" 
> 2,"Queens","Jamaica Bay","Boro Zone" 3,"Bronx","Allerton/Pelham 
> Gardens","Boro Zone" 4,"Manhattan","Alphabet City","Yellow Zone" 5,"Staten 
> Island","Arden Heights","Boro Zone" 6,"Staten Island","Arrochar/Fort 
> Wadsworth","Boro Zone" 7,"Queens","Astoria","Boro Zone" 8,"Queens","Astoria 
> Park","Boro Zone" 9,"Queens","Auburndale","Boro Zone"
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-36761) spark-examples_2.12-3.0.2.jar DFSReadWriteTest S3A Implementation

2021-10-08 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-36761?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17426163#comment-17426163
 ] 

Steve Loughran commented on SPARK-36761:


something in the code has got the default cluster FS and then 
({{FileSystem.get(Configuration)}} but is then invoking a method on it with a 
path beginning with the s3a schema 
{{FileSystem.exists(s3a://d14/import/rajtestsp33/dfs_read_write_test, 
expected)}}.'



> spark-examples_2.12-3.0.2.jar DFSReadWriteTest S3A Implementation
> -
>
> Key: SPARK-36761
> URL: https://issues.apache.org/jira/browse/SPARK-36761
> Project: Spark
>  Issue Type: Bug
>  Components: Examples
>Affects Versions: 3.0.2
>Reporter: Raj
>Priority: Major
>
> Dear Team,
> I am using Spark3 to test the s3a storage writing. Part of test i am invoking 
> DFSReadWriteTest from spark-examples_2.12-3.0.2.jar file. I am passing the 
> arguments as below
> spark-submit --verbose --driver-java-options 
> "-Dlog4j.configuration=file:/home/myid/log4j.properties" --conf 
> "spark.executor.extraJavaOptions='-Dlog4j.configuration=file:/home/myid/log4j.properties'"
>  --driver-class-path 
> "/usr/hdp/3.1.5.0-152/hadoop/ceph-rgw-sts-auth-6.jar,/opt/spark3/jars/hadoop-aws-3.1.1.3.1.5.0-152.jar"
>  --class org.apache.spark.examples.DFSReadWriteTest --deploy-mode client 
> --executor-memory 1G --num-executors 3 --conf 
> "spark.hadoop.fs.s3a.refreshTokenFile='/home/myid/keycloaktoken/tokenfile'" 
> /opt/spark3/examples/jars/spark-examples_2.12-3.0.2.jar 
> "/home/myid/sparkreadtest.txt" "s3a://d14/import/rajtestsp33"
> The Program fails with the message Wrong FS ( It seems the file system 
> comparison fails in checkpath)
> Exception in thread "main" java.lang.IllegalArgumentException: Wrong FS: 
> s3a://d14/import/rajtestsp33/dfs_read_write_test, expected: 
> hdfs://pphdpException in thread "main" java.lang.IllegalArgumentException: 
> Wrong FS: s3a://d14/import/rajtestsp33/dfs_read_write_test, expected: 
> hdfs://pphdp at 
> org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:730) at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getPathName(DistributedFileSystem.java:234)
>  at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1577)
>  at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1574)
>  at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>  at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1589)
>  at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1683) at 
> org.apache.spark.examples.DFSReadWriteTest$.main(DFSReadWriteTest.scala:115) 
> at org.apache.spark.examples.DFSReadWriteTest.main(DFSReadWriteTest.scala) 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 
> org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) 
> at 
> org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:928)
>  at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180) at 
> org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) at 
> org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) at 
> org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1007) 
> at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1016) at 
> org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
>  
> And the same program successfully ran with the Spark2 with HDP 3.1.5.0-152 
> and spark-examples_2.11-2.3.2.3.1.5.0-152.jar file.
> Any inputs appreciated. 
>  
> Thanks
> Raj



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-35428) Spark history Server to S3 doesn't show incomplete applications

2021-10-08 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-35428?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17426159#comment-17426159
 ] 

Steve Loughran commented on SPARK-35428:


# please stop using s3n; that connector is unsupported
# once you've moved to the s3a connector on a hadoop-3.1+ version of the hadoop 
binaries, does the problem remain?


> Spark history Server to S3 doesn't show incomplete applications
> ---
>
> Key: SPARK-35428
> URL: https://issues.apache.org/jira/browse/SPARK-35428
> Project: Spark
>  Issue Type: Bug
>  Components: Structured Streaming
>Affects Versions: 2.4.5
> Environment: Jupyter Notebook sparkmagic with Spark(2.4.5)  client 
> mode running on Kubernetes
>Reporter: Tianbin Jiang
>Priority: Major
>
> Jupyter Notebook sparkmagic with Spark(2.4.5)  client mode running on 
> Kubernetes.  I am redirecting the spark event logs to a S3 with the following 
> configuration:
>   
>  spark.eventLog.enabled = true
>  spark.history.ui.port = 18080
>  spark.eventLog.dir = s3://livy-spark-log/spark-history/
>  spark.history.fs.logDirectory = s3://livy-spark-log/spark-history/
>  spark.history.fs.update.interval = 5s
> spark.eventLog.buffer.kb = 1k
>  
> spark.streaming.driver.writeAheadLog.closeFileAfterWrite = true
>  spark.streaming.receiver.writeAheadLog.closeFileAfterWrite = true
>  
>   
>  Once my application is completed, I can see it shows up on the spark history 
> server. However, running applications doesn't show up on "incomplete 
> applications". I have also checked the log, whenever my application end, I 
> can see this message:
>   
>  {{21/05/17 06:14:18 INFO k8s.KubernetesClusterSchedulerBackend: Shutting 
> down all executors}}
>  {{21/05/17 06:14:18 INFO 
> k8s.KubernetesClusterSchedulerBackend$KubernetesDriverEndpoint: Asking each 
> executor to shut down}}
>  {{21/05/17 06:14:18 WARN k8s.ExecutorPodsWatchSnapshotSource: Kubernetes 
> client has been closed (this is expected if the application is shutting 
> down.)}}
>  *{{21/05/17 06:14:18 INFO s3n.MultipartUploadOutputStream: close 
> closed:false 
> s3://livy-spark-log/spark-history/spark-48c3141875fe4c67b5708400134ea3d6.inprogress}}*
>  *{{21/05/17 06:14:19 INFO s3n.S3NativeFileSystem: rename 
> s3://livy-spark-log/spark-history/spark-48c3141875fe4c67b5708400134ea3d6.inprogress
>  s3://livy-spark-log/spark-history/spark-48c3141875fe4c67b5708400134ea3d6}}*
>  {{21/05/17 06:14:19 INFO spark.MapOutputTrackerMasterEndpoint: 
> MapOutputTrackerMasterEndpoint stopped!}}
>  {{21/05/17 06:14:19 INFO memory.MemoryStore: MemoryStore cleared}}
>  {{21/05/17 06:14:19 INFO storage.BlockManager: BlockManager stopped}}
>   
>   
>  {{I am not able to see any xx.inprogress file on S3 though. Anyone had this 
> problem before? Otherwise, I would take it as a bug.}}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-36529) Decouple CPU with IO work in vectorized Parquet reader

2021-10-08 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-36529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17426158#comment-17426158
 ] 

Steve Loughran commented on SPARK-36529:


If you look at HADOOP-11867 / https://github.com/apache/hadoop/pull/3499 

We are adding a vectored read API to the FSDataInputStream with

* async fetch of different blocks
* order of return == "when the data comes back"
* read into bytebuffer
* caller provides their own bytebuffer factory

Will intially ship with
* base implementation to reorder/coalesce reads
* local FS to use native IO byte buffer reads

For the s3a and abfs object stores, our plan is to coalesce nearby ranges into 
aggregate ones, then issue multiple ranged GET requests in parallel. If/when 
the stores support multiple ranges in a GET, we could be even more efficient.

Please have a look @ the API and
1.  See if it will work with your code. Owen's clearly wrote knowing how ORC 
would make use of it.
1. try to make what you add now be able to support the API when spark is built 
against a version of hadoop with the API.

> Decouple CPU with IO work in vectorized Parquet reader
> --
>
> Key: SPARK-36529
> URL: https://issues.apache.org/jira/browse/SPARK-36529
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: Chao Sun
>Priority: Major
>
> Currently it seems the vectorized Parquet reader does almost everything in a 
> sequential manner:
> 1. read the row group using file system API (perhaps from remote storage like 
> S3)
> 2. allocate buffers and store those row group bytes into them
> 3. decompress the data pages
> 4. in Spark, decode all the read columns one by one
> 5. read the next row group and repeat from 1.
> A lot of improvements can be done to decouple the IO and CPU intensive work. 
> In addition, we could parallelize the row group loading and column decoding, 
> and utilizing all the cores available for a Spark task.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-36766) Spark SQL DDL does not recognize fs.s3.impl implied filesystem in LOCATION tag

2021-10-08 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-36766?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17426152#comment-17426152
 ] 

Steve Loughran commented on SPARK-36766:


I can see why you'd want to do this (consistent URLs on EMR and ASF spark 
builds), but be aware that there's a risk the s3a connector might assume its 
got an s3a URL, and sometimes it'll leak.

> Spark SQL DDL does not recognize fs.s3.impl implied filesystem in LOCATION tag
> --
>
> Key: SPARK-36766
> URL: https://issues.apache.org/jira/browse/SPARK-36766
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.0.2
>Reporter: Sungwon
>Priority: Major
>
> CREATE TABLE with the LOCATION tag via SPARK SQL DDL does not seem to 
> recognize the fileSystem implied by setting fs.s3.impl.
> The environment setting is set to:
> |spark.hadoop.fs.s3.impl|org.apache.hadoop.fs.s3a.S3AFileSystem|
> The following spark.sql ddl raises an exception:
>  spark.sql("""CREATE TABLE local.db.table (
>  col1 string,
>  col2 string,
>  )
>  LOCATION 's3://bucket/prefix'
>  """)
>  Exception: 
> Py4JJavaError: An error occurred while calling o86.sql. : 
> org.apache.iceberg.exceptions.CommitStateUnknownException: Got exception: 
> org.apache.hadoop.fs.UnsupportedFileSystemException No FileSystem for scheme 
> "s3"
> This is inconsistent with other spark functions such as 
> spark.read.parquet("s3://bucket/prefix") which succeeds with the same 
> configuration



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-36599) ExecutorClassLoader no longer works with Http based Class Servers

2021-08-27 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-36599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17405917#comment-17405917
 ] 

Steve Loughran commented on SPARK-36599:


I thought things had been fixed up so Hadoop's HTTP binding was not used 
through classloaders. If that isn't the case, that is something to fix.

> ExecutorClassLoader no longer works with Http based Class Servers
> -
>
> Key: SPARK-36599
> URL: https://issues.apache.org/jira/browse/SPARK-36599
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.2
>Reporter: VinothKumar Raman
>Priority: Major
>
> There are 2 primary issues,
>  * If the classServer uri is `{{http://host:port}}` instead of 
> `{{http://host:port/}}` the getPath on the URI object returns empty string 
> and subsequently causes Path creation to fail with Path cannot be empty++
>  * This is a regression issue and transitively a hadoop issue too, the Http 
> Filesystem fails with URI not absolute error unlike other filesystems when 
> the path doesn't have scheme or authority. This used to work when there was 
> http specific implementation and no longer work with Filesystem based 
> implementation
> https://issues.apache.org/jira/browse/HADOOP-17870



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-36024) Switch the datasource example due to the depreciation of the dataset

2021-07-28 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-36024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17388969#comment-17388969
 ] 

Steve Loughran commented on SPARK-36024:


yes, you can change the example. For hadoop we're trying to keep the landsat 
file around, because removing it breaks regression testing all old releases.

> Switch the datasource example due to the depreciation of the dataset
> 
>
> Key: SPARK-36024
> URL: https://issues.apache.org/jira/browse/SPARK-36024
> Project: Spark
>  Issue Type: Documentation
>  Components: Documentation
>Affects Versions: 3.1.2
>Reporter: Leona Yoda
>Priority: Trivial
>
> The S3 bucket that used for an example in "Integration with Cloud 
> Infrastructures" document will be deleted on Jul 1, 2021 
> [https://registry.opendata.aws/landsat-8/ 
> |https://registry.opendata.aws/landsat-8/]
> The dataset will move to another bucket but it requires `--request-payer 
> requester` option so users have to pay S3 cost. 
> [https://registry.opendata.aws/usgs-landsat/]
>  
> So I think it's better to change the datasource like this.
> [https://github.com/yoda-mon/spark/commit/cdb24acdbb57a429e5bf1729502653b91a600022]
>  
> I chose [NYC Taxi data| 
> [https://registry.opendata.aws/nyc-tlc-trip-records-pds/|https://registry.opendata.aws/nyc-tlc-trip-records-pds/),]]
>  here for an example. 
>  Unlike landat data it's not compressed, but it is just an example and there 
> are several tutorials using Spark  (e.g. 
> [https://github.com/aws-samples/amazon-eks-apache-spark-etl-sample)]
>  
> Reed test result
> {code:java}
> scala> sc.textFile("s3a://nyc-tlc/misc/taxi 
> _zone_lookup.csv").take(10).foreach(println) 
> "LocationID","Borough","Zone","service_zone" 1,"EWR","Newark Airport","EWR" 
> 2,"Queens","Jamaica Bay","Boro Zone" 3,"Bronx","Allerton/Pelham 
> Gardens","Boro Zone" 4,"Manhattan","Alphabet City","Yellow Zone" 5,"Staten 
> Island","Arden Heights","Boro Zone" 6,"Staten Island","Arrochar/Fort 
> Wadsworth","Boro Zone" 7,"Queens","Astoria","Boro Zone" 8,"Queens","Astoria 
> Park","Boro Zone" 9,"Queens","Auburndale","Boro Zone"
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-36024) Switch the datasource example due to the depreciation of the dataset

2021-07-06 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-36024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17375400#comment-17375400
 ] 

Steve Loughran commented on SPARK-36024:


similar to HADOOP-17784

I'm "in discussions" with them. Maybe I can persuade them to leave the index 
file up

And I'd like to move on to a dataset where (a) it's stable (b) got real 
ORC/Parquet data alongside the CSV

Finally: need to make sure that this time, not matter how "stable" the source 
is, whoever runs it knows we need it.

Where in the docs is this?

> Switch the datasource example due to the depreciation of the dataset
> 
>
> Key: SPARK-36024
> URL: https://issues.apache.org/jira/browse/SPARK-36024
> Project: Spark
>  Issue Type: Documentation
>  Components: Documentation
>Affects Versions: 3.1.2
>Reporter: Leona Yoda
>Priority: Trivial
>
> The S3 bucket that used for an example in "Integration with Cloud 
> Infrastructures" document will be deleted on Jul 1, 2021 
> [https://registry.opendata.aws/landsat-8/ 
> |https://registry.opendata.aws/landsat-8/]
> The dataset will move to another bucket but it requires `--request-payer 
> requester` option so users have to pay S3 cost. 
> [https://registry.opendata.aws/usgs-landsat/]
>  
> So I think it's better to change the datasource like this.
> [https://github.com/yoda-mon/spark/commit/cdb24acdbb57a429e5bf1729502653b91a600022]
>  
> I chose [NYC Taxi data| 
> [https://registry.opendata.aws/nyc-tlc-trip-records-pds/|https://registry.opendata.aws/nyc-tlc-trip-records-pds/),]]
>  here for an example. 
>  Unlike landat data it's not compressed, but it is just an example and there 
> are several tutorials using Spark  (e.g. 
> [https://github.com/aws-samples/amazon-eks-apache-spark-etl-sample)]
>  
> Reed test result
> {code:java}
> scala> sc.textFile("s3a://nyc-tlc/misc/taxi 
> _zone_lookup.csv").take(10).foreach(println) 
> "LocationID","Borough","Zone","service_zone" 1,"EWR","Newark Airport","EWR" 
> 2,"Queens","Jamaica Bay","Boro Zone" 3,"Bronx","Allerton/Pelham 
> Gardens","Boro Zone" 4,"Manhattan","Alphabet City","Yellow Zone" 5,"Staten 
> Island","Arden Heights","Boro Zone" 6,"Staten Island","Arrochar/Fort 
> Wadsworth","Boro Zone" 7,"Queens","Astoria","Boro Zone" 8,"Queens","Astoria 
> Park","Boro Zone" 9,"Queens","Auburndale","Boro Zone"
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Comment Edited] (SPARK-36024) Switch the datasource example due to the depreciation of the dataset

2021-07-06 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-36024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17375400#comment-17375400
 ] 

Steve Loughran edited comment on SPARK-36024 at 7/6/21, 9:32 AM:
-

similar to HADOOP-17784

I'm "in discussions" with them. Maybe I can persuade them to leave the index 
file up

And I'd like to move on to a dataset where (a) it's stable (b) got real 
ORC/Parquet data alongside the CSV

Finally: need to make sure that this time, not matter how "stable" the source 
is, whoever runs it knows we need it.

Where in the docs is this?

(oh, obviously it'll be something I wrote, won't it...)


was (Author: ste...@apache.org):
similar to HADOOP-17784

I'm "in discussions" with them. Maybe I can persuade them to leave the index 
file up

And I'd like to move on to a dataset where (a) it's stable (b) got real 
ORC/Parquet data alongside the CSV

Finally: need to make sure that this time, not matter how "stable" the source 
is, whoever runs it knows we need it.

Where in the docs is this?

> Switch the datasource example due to the depreciation of the dataset
> 
>
> Key: SPARK-36024
> URL: https://issues.apache.org/jira/browse/SPARK-36024
> Project: Spark
>  Issue Type: Documentation
>  Components: Documentation
>Affects Versions: 3.1.2
>Reporter: Leona Yoda
>Priority: Trivial
>
> The S3 bucket that used for an example in "Integration with Cloud 
> Infrastructures" document will be deleted on Jul 1, 2021 
> [https://registry.opendata.aws/landsat-8/ 
> |https://registry.opendata.aws/landsat-8/]
> The dataset will move to another bucket but it requires `--request-payer 
> requester` option so users have to pay S3 cost. 
> [https://registry.opendata.aws/usgs-landsat/]
>  
> So I think it's better to change the datasource like this.
> [https://github.com/yoda-mon/spark/commit/cdb24acdbb57a429e5bf1729502653b91a600022]
>  
> I chose [NYC Taxi data| 
> [https://registry.opendata.aws/nyc-tlc-trip-records-pds/|https://registry.opendata.aws/nyc-tlc-trip-records-pds/),]]
>  here for an example. 
>  Unlike landat data it's not compressed, but it is just an example and there 
> are several tutorials using Spark  (e.g. 
> [https://github.com/aws-samples/amazon-eks-apache-spark-etl-sample)]
>  
> Reed test result
> {code:java}
> scala> sc.textFile("s3a://nyc-tlc/misc/taxi 
> _zone_lookup.csv").take(10).foreach(println) 
> "LocationID","Borough","Zone","service_zone" 1,"EWR","Newark Airport","EWR" 
> 2,"Queens","Jamaica Bay","Boro Zone" 3,"Bronx","Allerton/Pelham 
> Gardens","Boro Zone" 4,"Manhattan","Alphabet City","Yellow Zone" 5,"Staten 
> Island","Arden Heights","Boro Zone" 6,"Staten Island","Arrochar/Fort 
> Wadsworth","Boro Zone" 7,"Queens","Astoria","Boro Zone" 8,"Queens","Astoria 
> Park","Boro Zone" 9,"Queens","Auburndale","Boro Zone"
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



  1   2   3   4   5   6   7   8   9   >