[jira] [Commented] (SPARK-27981) Remove `Illegal reflective access` warning for `java.nio.Bits.unaligned()`

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun commented on SPARK-27981:
---

For now, `java.nio.DirectByteBuffer` is an intentional one, [~kshukla].

[~srowen], do we have a JIRA to track `java.nio.DirectByteBuffer`?

> Remove `Illegal reflective access` warning for `java.nio.Bits.unaligned()`
> --
>
> Key: SPARK-27981
> URL: https://issues.apache.org/jira/browse/SPARK-27981
> Project: Spark
>  Issue Type: Sub-task
>  Components: Spark Core
>Affects Versions: 3.0.0
>Reporter: Dongjoon Hyun
>Assignee: Dongjoon Hyun
>Priority: Major
> Fix For: 3.0.0
>
>
> This PR aims to remove the following warnings for `java.nio.Bits.unaligned` 
> at JDK9/10/11/12. Please note that there are more warnings which is beyond of 
> this PR's scope.
> {code}
> bin/spark-shell --driver-java-options=--illegal-access=warn
> WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform 
> (file:/Users/dhyun/APACHE/spark-release/spark-3.0/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar)
>  to method java.nio.Bits.unaligned()
> ...
> {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-29451) Some queries with divisions in SQL windows are failling in Thrift

2019-10-14 Thread Yuming Wang (Jira)


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

Yuming Wang commented on SPARK-29451:
-

It works for me.
{code:sql}
CREATE OR REPLACE TEMPORARY VIEW tenk1
  (unique1 int, unique2 int, two int, four int, ten int, twenty int, hundred 
int,
thousand int, twothousand int, fivethous int, tenthous int, odd int, even 
int,
stringu1 string, stringu2 string, string4 string)
USING csv
  OPTIONS (path 
'/Users/yumwang/spark/SPARK-28216/sql/core/src/test/resources/test-data/postgresql/tenk.data',
  header 'false', delimiter '\t');

SELECT four, ten/4 as two,
sum(ten/4) over (partition by four order by ten/4 rows between unbounded 
preceding and current row),
last(ten/4) over (partition by four order by ten/4 rows between unbounded 
preceding and current row)
FROM (select distinct ten, four from tenk1) ss;

SELECT four, ten/4 as two,
sum(ten/4) over (partition by four order by ten/4 range between unbounded 
preceding and current row),
last(ten/4) over (partition by four order by ten/4 range between unbounded 
preceding and current row)
FROM (select distinct ten, four from tenk1) ss;
{code}
{noformat}
LM-SHC-16502798:SPARK-28216 yumwang$ bin/beeline -u jdbc:hive2://localhost:1
log4j:WARN No appenders could be found for logger 
(org.apache.hadoop.util.Shell).
log4j:WARN Please initialize the log4j system properly.
log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more 
info.
Connecting to jdbc:hive2://localhost:1
Connected to: Spark SQL (version 3.0.0-SNAPSHOT)
Driver: Hive JDBC (version 2.3.6)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Beeline version 2.3.6 by Apache Hive
0: jdbc:hive2://localhost:1> CREATE OR REPLACE TEMPORARY VIEW tenk1
. . . . . . . . . . . . . . . .>   (unique1 int, unique2 int, two int, four 
int, ten int, twenty int, hundred int,
. . . . . . . . . . . . . . . .> thousand int, twothousand int, fivethous 
int, tenthous int, odd int, even int,
. . . . . . . . . . . . . . . .> stringu1 string, stringu2 string, string4 
string)
. . . . . . . . . . . . . . . .> USING csv
. . . . . . . . . . . . . . . .>   OPTIONS (path 
'/Users/yumwang/spark/SPARK-28216/sql/core/src/test/resources/test-data/postgresql/tenk.data',
. . . . . . . . . . . . . . . .>   header 'false', delimiter '\t');
+-+
| Result  |
+-+
+-+
No rows selected (0.499 seconds)
0: jdbc:hive2://localhost:1> SELECT four, ten/4 as two,
. . . . . . . . . . . . . . . .> sum(ten/4) over (partition by four order by 
ten/4 rows between unbounded preceding and current row),
. . . . . . . . . . . . . . . .> last(ten/4) over (partition by four order by 
ten/4 rows between unbounded preceding and current row)
. . . . . . . . . . . . . . . .> FROM (select distinct ten, four from tenk1) ss;
+---+---+++
| four  |  two  | sum((CAST(ten AS DOUBLE) / CAST(4 AS DOUBLE))) OVER 
(PARTITION BY four ORDER BY (CAST(ten AS DOUBLE) / CAST(4 AS DOUBLE)) ASC NULLS 
FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) | last((CAST(ten AS 
DOUBLE) / CAST(4 AS DOUBLE)), false) OVER (PARTITION BY four ORDER BY (CAST(ten 
AS DOUBLE) / CAST(4 AS DOUBLE)) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED 
PRECEDING AND CURRENT ROW) |
+---+---+++
| 1 | 0.25  | 0.25   | 0.25 
  |
| 1 | 0.75  | 1.0| 0.75 
  |
| 1 | 1.25  | 2.25   | 1.25 
  |
| 1 | 1.75  | 4.0| 1.75 
  |
| 1 | 2.25  | 6.25   | 2.25 
  |
| 3 | 0.25  | 0.25   | 0.25 
  |
| 3 | 0.75  | 1.0| 0.75 
  |
| 3 | 1.25  | 2.25   | 1.25 
  |
| 3 | 1.75  | 4.0| 1.75 
  |
| 3 | 2.25  | 6.25   | 2.25 
  |
| 2 | 0.0   | 0.0| 0.0  
  |
| 2 | 0.5   | 0.5  

[jira] [Commented] (SPARK-27259) Processing Compressed HDFS files with spark failing with error: "java.lang.IllegalArgumentException: requirement failed: length (-1) cannot be negative" from spark 2.2

2019-10-14 Thread Praneet Sharma (Jira)


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

Praneet Sharma commented on SPARK-27259:


Submitted a PR for this issue: [https://github.com/apache/spark/pull/26123]

> Processing Compressed HDFS files with spark failing with error: 
> "java.lang.IllegalArgumentException: requirement failed: length (-1) cannot 
> be negative" from spark 2.2.X
> -
>
> Key: SPARK-27259
> URL: https://issues.apache.org/jira/browse/SPARK-27259
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 2.2.1, 2.2.2, 2.2.3, 2.3.0, 2.3.1, 2.3.2, 2.3.3, 2.4.0
>Reporter: Simon poortman
>Priority: Major
>
>  
> From spark 2.2.x versions, when spark job processing any compressed HDFS 
> files with custom input file format then spark jobs are failing with error 
> "java.lang.IllegalArgumentException: requirement failed: length (-1) cannot 
> be negative", the custom input file format will return the number of bytes 
> length value as -1 for compressed file formats due to the compressed HDFS 
> file are non splitable, so for compressed input file format the split will be 
> offset as 0 and number of bytes length as -1, spark should consider the bytes 
> length value -1 as valid split for the compressed file formats.
>  
> We observed that earlier versions of spark doesn’t have this validation, and 
> found that from spark 2.2.x new validation got introduced in the class 
> InputFileBlockHolder, so spark should accept the number of bytes length value 
> -1 as valid length for input splits from spark 2.2.x as well.
>  
> +Below is the stack trace.+
>  Caused by: java.lang.IllegalArgumentException: requirement failed: length 
> (-1) cannot be negative
>   at scala.Predef$.require(Predef.scala:224)
>   at 
> org.apache.spark.rdd.InputFileBlockHolder$.set(InputFileBlockHolder.scala:70)
>   at org.apache.spark.rdd.HadoopRDD$$anon$1.(HadoopRDD.scala:226)
>   at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:214)
>   at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:94)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
>   at org.apache.spark.scheduler.Task.run(Task.scala:109)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>   at java.lang.Thread.run(Thread.java:748)
>  
> +Below is the code snippet which caused this issue.+
>    **    {color:#ff}require(length >= 0, s"length ($length) cannot be 
> negative"){color} // This validation caused the issue. 
>  
> {code:java}
> // code placeholder
>  org.apache.spark.rdd.InputFileBlockHolder - spark-core
>  
> def set(filePath: String, startOffset: Long, length: Long): Unit = {
>     require(filePath != null, "filePath cannot be null")
>     require(startOffset >= 0, s"startOffset ($startOffset) cannot be 
> negative")
>     require(length >= 0, s"length ($length) cannot be negative")  
>     inputBlock.set(new FileBlock(UTF8String.fromString(filePath), 
> startOffset, length))
>   }
> {code}
>  
> +Steps to reproduce the issue.+
>  Please refer the below code to reproduce the issue.  
> {code:java}
> // code placeholder
> import org.apache.hadoop.mapred.JobConf
> val hadoopConf = new JobConf()
> import org.apache.hadoop.mapred.FileInputFormat
> import org.apache.hadoop.fs.Path
> FileInputFormat.setInputPaths(hadoopConf, new 
> Path("/output656/part-r-0.gz"))    
> val records = 
> sc.hadoopRDD(hadoopConf,classOf[com.platform.custom.storagehandler.INFAInputFormat],
>  classOf[org.apache.hadoop.io.LongWritable], 
> classOf[org.apache.hadoop.io.Writable]) 
> records.count()
> {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] [Resolved] (SPARK-29454) Reduce unsafeProjection call times when read parquet file

2019-10-14 Thread Wenchen Fan (Jira)


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

Wenchen Fan resolved SPARK-29454.
-
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26106
[https://github.com/apache/spark/pull/26106]

> Reduce unsafeProjection call times when read parquet file
> -
>
> Key: SPARK-29454
> URL: https://issues.apache.org/jira/browse/SPARK-29454
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 2.2.3, 2.3.4, 2.4.4
>Reporter: Yang Jie
>Assignee: Yang Jie
>Priority: Major
> Fix For: 3.0.0
>
>
> ParquetGroupConverter call unsafeProjection function to covert 
> SpecificInternalRow to UnsafeRow every times when read Parquet data file use 
> ParquetRecordReader, then ParquetFileFormat will call unsafeProjection 
> function to covert this UnsafeRow to another UnsafeRow again when 
> partitionSchema is not empty , and on the other hand 
> PartitionReaderWithPartitionValues  always do this convert process when use 
> DataSourceV2.
> I think the first time convert in ParquetGroupConverter is redundant and 
> ParquetRecordReader return a SpecificInternalRow is enough.



--
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] [Assigned] (SPARK-29454) Reduce unsafeProjection call times when read parquet file

2019-10-14 Thread Wenchen Fan (Jira)


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

Wenchen Fan reassigned SPARK-29454:
---

Assignee: Yang Jie

> Reduce unsafeProjection call times when read parquet file
> -
>
> Key: SPARK-29454
> URL: https://issues.apache.org/jira/browse/SPARK-29454
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 2.2.3, 2.3.4, 2.4.4
>Reporter: Yang Jie
>Assignee: Yang Jie
>Priority: Major
>
> ParquetGroupConverter call unsafeProjection function to covert 
> SpecificInternalRow to UnsafeRow every times when read Parquet data file use 
> ParquetRecordReader, then ParquetFileFormat will call unsafeProjection 
> function to covert this UnsafeRow to another UnsafeRow again when 
> partitionSchema is not empty , and on the other hand 
> PartitionReaderWithPartitionValues  always do this convert process when use 
> DataSourceV2.
> I think the first time convert in ParquetGroupConverter is redundant and 
> ParquetRecordReader return a SpecificInternalRow is enough.



--
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-29465) Unable to configure SPARK UI (spark.ui.port) in spark yarn cluster mode.

2019-10-14 Thread Sandeep Katta (Jira)


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

Sandeep Katta commented on SPARK-29465:
---

I see as per the comments in the [code 
|https://github.com/apache/spark/blob/master/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala#L212]
 behavior is correct. User can submit more spark applications in the same box.

ping [~hyukjin.kwon] [~dongjoon] what's your suggesstion

> Unable to configure SPARK UI (spark.ui.port) in spark yarn cluster mode. 
> -
>
> Key: SPARK-29465
> URL: https://issues.apache.org/jira/browse/SPARK-29465
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Submit, YARN
>Affects Versions: 1.6.2, 2.4.4
>Reporter: Vishwas Nalka
>Priority: Major
>
>  I'm trying to restrict the ports used by spark app which is launched in yarn 
> cluster mode. All ports (viz. driver, executor, blockmanager) could be 
> specified using the respective properties except the ui port. The spark app 
> is launched using JAVA code and setting the property spark.ui.port in 
> sparkConf doesn't seem to help. Even setting a JVM option 
> -Dspark.ui.port="some_port" does not spawn the UI is required port. 
> From the logs of the spark app, *_the property spark.ui.port is overridden 
> and the JVM property '-Dspark.ui.port=0' is set_* even though it is never set 
> to 0. 
> _(Run in Spark 1.6.2) From the logs ->_
> _command:LD_LIBRARY_PATH="/usr/hdp/2.6.4.0-91/hadoop/lib/native:$LD_LIBRARY_PATH"
>  {{JAVA_HOME}}/bin/java -server -XX:OnOutOfMemoryError='kill %p' -Xms4096m 
> -Xmx4096m -Djava.io.tmpdir={{PWD}}/tmp '-Dspark.blockManager.port=9900' 
> '-Dspark.driver.port=9902' '-Dspark.fileserver.port=9903' 
> '-Dspark.broadcast.port=9904' '-Dspark.port.maxRetries=20' 
> '-Dspark.ui.port=0' '-Dspark.executor.port=9905'_
> _19/10/14 16:39:59 INFO Utils: Successfully started service 'SparkUI' on port 
> 35167.19/10/14 16:39:59 INFO SparkUI: Started SparkUI at_ 
> [_http://10.65.170.98:35167_|http://10.65.170.98:35167/]
> Even tried using a *spark-submit command with --conf spark.ui.port* does 
> spawn UI in required port
> {color:#172b4d}_(Run in Spark 2.4.4)_{color}
>  {color:#172b4d}_./bin/spark-submit --class org.apache.spark.examples.SparkPi 
> --master yarn --deploy-mode cluster --driver-memory 4g --executor-memory 2g 
> --executor-cores 1 --conf spark.ui.port=12345 --conf spark.driver.port=12340 
> --queue default examples/jars/spark-examples_2.11-2.4.4.jar 10_{color}
> _From the logs::_
>  _19/10/15 00:04:05 INFO ui.SparkUI: Stopped Spark web UI at 
> [http://invrh74ace005.informatica.com:46622|http://invrh74ace005.informatica.com:46622/]_
> _command:{{JAVA_HOME}}/bin/java -server -Xmx2048m 
> -Djava.io.tmpdir={{PWD}}/tmp '-Dspark.ui.port=0'  'Dspark.driver.port=12340' 
> -Dspark.yarn.app.container.log.dir= -XX:OnOutOfMemoryError='kill %p' 
> org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url 
> spark://coarsegrainedschedu...@invrh74ace005.informatica.com:12340 
> --executor-id  --hostname  --cores 1 --app-id 
> application_1570992022035_0089 --user-class-path 
> [file:$PWD/__app__.jar1|file://%24pwd/__app__.jar1]>/stdout2>/stderr_
>  
> Looks like the application master override this and set a JVM property before 
> launch resulting in random UI port even though spark.ui.port is set by the 
> user.
> In these links
>  # 
> [https://github.com/apache/spark/blob/master/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala]
>  (line 214)
>  # 
> [https://github.com/cloudera/spark/blob/master/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala]
>  (line 75)
> I can see that the method _*run() in above files sets a system property 
> UI_PORT*_ and _*spark.ui.port respectively.*_



--
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-29437) CSV Writer should escape 'escapechar' when it exists in the data

2019-10-14 Thread Ankit Raj Boudh (Jira)


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

Ankit Raj Boudh commented on SPARK-29437:
-

Hi [~kretes],

This is the correct behaviour, as per my analysis. 

As per your suggestion i tried to escaped  escape character (default '\') but 
it will be a problem during storing same data 

inside view.

> CSV Writer should escape 'escapechar' when it exists in the data
> 
>
> Key: SPARK-29437
> URL: https://issues.apache.org/jira/browse/SPARK-29437
> Project: Spark
>  Issue Type: Bug
>  Components: Input/Output
>Affects Versions: 2.4.3
>Reporter: Tomasz Bartczak
>Priority: Trivial
>
> When the data contains escape character (default '\') it should either be 
> escaped or quoted.
> Steps to reproduce: 
> [https://gist.github.com/kretes/58f7f66a0780681a44c175a2ac3c0da2]
>  
> The effect can be either bad data read or sometimes even unable to properly 
> read the csv, e.g. when escape character is the last character in the column 
> - it break the column reading for that row and effectively break e.g. type 
> inference for a dataframe



--
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] [Assigned] (SPARK-10614) SystemClock uses non-monotonic time in its wait logic

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun reassigned SPARK-10614:
-

Assignee: Marcelo Masiero Vanzin

> SystemClock uses non-monotonic time in its wait logic
> -
>
> Key: SPARK-10614
> URL: https://issues.apache.org/jira/browse/SPARK-10614
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 1.3.0
>Reporter: Steve Loughran
>Assignee: Marcelo Masiero Vanzin
>Priority: Minor
>
> The consolidated (SPARK-4682) clock uses {{System.currentTimeMillis()}} for 
> measuring time, which means its {{waitTillTime()}} routine is brittle against 
> systems (VMs in particular) whose time can go backwards as well as forward.
> For the {{ExecutorAllocationManager}} this appears to be a regression.



--
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] [Resolved] (SPARK-10614) SystemClock uses non-monotonic time in its wait logic

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun resolved SPARK-10614.
---
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26058
[https://github.com/apache/spark/pull/26058]

> SystemClock uses non-monotonic time in its wait logic
> -
>
> Key: SPARK-10614
> URL: https://issues.apache.org/jira/browse/SPARK-10614
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 1.3.0
>Reporter: Steve Loughran
>Assignee: Marcelo Masiero Vanzin
>Priority: Minor
> Fix For: 3.0.0
>
>
> The consolidated (SPARK-4682) clock uses {{System.currentTimeMillis()}} for 
> measuring time, which means its {{waitTillTime()}} routine is brittle against 
> systems (VMs in particular) whose time can go backwards as well as forward.
> For the {{ExecutorAllocationManager}} this appears to be a regression.



--
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-29461) Spark dataframe writer does not expose metrics for JDBC writer

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun commented on SPARK-29461:
---

Thank you for filing a JIRA, [~rkalhans]. Thank you for making a PR, [~kabhwan].

> Spark dataframe writer does not expose metrics for JDBC writer 
> ---
>
> Key: SPARK-29461
> URL: https://issues.apache.org/jira/browse/SPARK-29461
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: ROHIT KALHANS
>Priority: Minor
>
> Spark does not expose the writer metrics when using the Dataframe JDBC 
> writer. Similar instances of such bugs have been fixed in previous versions. 
> However, it seems the fix was not exhaustive since it does not cover all the 
> writers. 
> Similar bugs: 
> https://issues.apache.org/jira/browse/SPARK-21882
> https://issues.apache.org/jira/browse/SPARK-22605
>  
>  
> Console reporter output 
>  app-name.1.executor.bytesWritten
>              count = 0 
>  
>  app-name.1.executor.recordsWritten
>              count = 0  
>  
>  



--
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] [Updated] (SPARK-29461) Spark dataframe writer does not expose metrics for JDBC writer

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun updated SPARK-29461:
--
Affects Version/s: (was: 2.4.4)
   (was: 2.4.0)
   3.0.0

> Spark dataframe writer does not expose metrics for JDBC writer 
> ---
>
> Key: SPARK-29461
> URL: https://issues.apache.org/jira/browse/SPARK-29461
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: ROHIT KALHANS
>Priority: Minor
>
> Spark does not expose the writer metrics when using the Dataframe JDBC 
> writer. Similar instances of such bugs have been fixed in previous versions. 
> However, it seems the fix was not exhaustive since it does not cover all the 
> writers. 
> Similar bugs: 
> https://issues.apache.org/jira/browse/SPARK-21882
> https://issues.apache.org/jira/browse/SPARK-22605
>  
>  
> Console reporter output 
>  app-name.1.executor.bytesWritten
>              count = 0 
>  
>  app-name.1.executor.recordsWritten
>              count = 0  
>  
>  



--
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] [Updated] (SPARK-29461) Spark dataframe writer does not expose metrics for JDBC writer

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun updated SPARK-29461:
--
Issue Type: Improvement  (was: Bug)

> Spark dataframe writer does not expose metrics for JDBC writer 
> ---
>
> Key: SPARK-29461
> URL: https://issues.apache.org/jira/browse/SPARK-29461
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 2.4.0, 2.4.4
>Reporter: ROHIT KALHANS
>Priority: Major
>
> Spark does not expose the writer metrics when using the Dataframe JDBC 
> writer. Similar instances of such bugs have been fixed in previous versions. 
> However, it seems the fix was not exhaustive since it does not cover all the 
> writers. 
> Similar bugs: 
> https://issues.apache.org/jira/browse/SPARK-21882
> https://issues.apache.org/jira/browse/SPARK-22605
>  
>  
> Console reporter output 
>  app-name.1.executor.bytesWritten
>              count = 0 
>  
>  app-name.1.executor.recordsWritten
>              count = 0  
>  
>  



--
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] [Updated] (SPARK-29461) Spark dataframe writer does not expose metrics for JDBC writer

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun updated SPARK-29461:
--
Component/s: (was: Spark Core)
 SQL

> Spark dataframe writer does not expose metrics for JDBC writer 
> ---
>
> Key: SPARK-29461
> URL: https://issues.apache.org/jira/browse/SPARK-29461
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.4.0, 2.4.4
>Reporter: ROHIT KALHANS
>Priority: Major
>
> Spark does not expose the writer metrics when using the Dataframe JDBC 
> writer. Similar instances of such bugs have been fixed in previous versions. 
> However, it seems the fix was not exhaustive since it does not cover all the 
> writers. 
> Similar bugs: 
> https://issues.apache.org/jira/browse/SPARK-21882
> https://issues.apache.org/jira/browse/SPARK-22605
>  
>  
> Console reporter output 
>  app-name.1.executor.bytesWritten
>              count = 0 
>  
>  app-name.1.executor.recordsWritten
>              count = 0  
>  
>  



--
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] [Updated] (SPARK-29461) Spark dataframe writer does not expose metrics for JDBC writer

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun updated SPARK-29461:
--
Priority: Minor  (was: Major)

> Spark dataframe writer does not expose metrics for JDBC writer 
> ---
>
> Key: SPARK-29461
> URL: https://issues.apache.org/jira/browse/SPARK-29461
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 2.4.0, 2.4.4
>Reporter: ROHIT KALHANS
>Priority: Minor
>
> Spark does not expose the writer metrics when using the Dataframe JDBC 
> writer. Similar instances of such bugs have been fixed in previous versions. 
> However, it seems the fix was not exhaustive since it does not cover all the 
> writers. 
> Similar bugs: 
> https://issues.apache.org/jira/browse/SPARK-21882
> https://issues.apache.org/jira/browse/SPARK-22605
>  
>  
> Console reporter output 
>  app-name.1.executor.bytesWritten
>              count = 0 
>  
>  app-name.1.executor.recordsWritten
>              count = 0  
>  
>  



--
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] [Resolved] (SPARK-29412) refine the document of v2 session catalog config

2019-10-14 Thread Wenchen Fan (Jira)


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

Wenchen Fan resolved SPARK-29412.
-
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26071
[https://github.com/apache/spark/pull/26071]

> refine the document of v2 session catalog config
> 
>
> Key: SPARK-29412
> URL: https://issues.apache.org/jira/browse/SPARK-29412
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Wenchen Fan
>Assignee: Wenchen Fan
>Priority: Major
> Fix For: 3.0.0
>
>




--
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] [Created] (SPARK-29472) Mechanism for Excluding Jars at Launch for YARN

2019-10-14 Thread Abhishek Modi (Jira)
Abhishek Modi created SPARK-29472:
-

 Summary: Mechanism for Excluding Jars at Launch for YARN
 Key: SPARK-29472
 URL: https://issues.apache.org/jira/browse/SPARK-29472
 Project: Spark
  Issue Type: New Feature
  Components: YARN
Affects Versions: 2.4.4
Reporter: Abhishek Modi


*Summary*

It would be convenient if there were an easy way to exclude jars from Spark’s 
classpath at launch time. This would complement the way in which jars can be 
added to the classpath using {{extraClassPath}}.

 

*Context*

The Spark build contains its dependency jars in the {{/jars}} directory. These 
jars become part of the executor’s classpath. By default on YARN, these jars 
are packaged and distributed to containers at launch ({{spark-submit}}) time.

 

While developing Spark applications, customers sometimes need to debug using 
different versions of dependencies. This can become difficult if the dependency 
(eg. Parquet 1.11.0) is one that Spark already has in {{/jars}} (eg. Parquet 
1.10.1 in Spark 2.4), as the dependency included with Spark is preferentially 
loaded. 

 

Configurations such as {{userClassPathFirst}} are available. However these have 
often come with other side effects. For example, if the customer’s build 
includes Avro they will likely see {{Caused by: java.lang.LinkageError: loader 
constraint violation: when resolving method 
"org.apache.spark.SparkConf.registerAvroSchemas(Lscala/collection/Seq;)Lorg/apache/spark/SparkConf;"
 the class loader (instance of org/apache/spark/util/ChildFirstURLClassLoader) 
of the current class, com/uber/marmaray/common/spark/SparkFactory, and the 
class loader (instance of sun/misc/Launcher$AppClassLoader) for the method's 
defining class, org/apache/spark/SparkConf, have different Class objects for 
the type scala/collection/Seq used in the signature}}. Resolving such issues 
often takes many hours.

 

To deal with these sorts of issues, customers often download the Spark build, 
remove the target jars and then do spark-submit. Other times, customers may not 
be able to do spark-submit as it is gated behind some Spark Job Server. In this 
case, customers may try downloading the build, removing the jars, and then 
using configurations such as {{spark.yarn.dist.jars}} or 
{{spark.yarn.dist.archives}}. Both of these options are undesirable as they are 
very operationally heavy, error prone and often result in the customer’s spark 
builds going out of sync with the authoritative build. 

 

*Solution*

I’d like to propose adding a {{spark.yarn.jars.exclusionRegex}} configuration. 
Customers could provide a regex such as {{.\*parquet.\*}} and jar files 
matching this regex would not be included in the driver and executor classpath.



--
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-26570) Out of memory when InMemoryFileIndex bulkListLeafFiles

2019-10-14 Thread fengchaoge (Jira)


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

fengchaoge commented on SPARK-26570:


It's just a snapshot of an instant, and it's actually getting bigger until the 
memory overflows and any information about the production environment can be 
exported。

> Out of memory when InMemoryFileIndex bulkListLeafFiles
> --
>
> Key: SPARK-26570
> URL: https://issues.apache.org/jira/browse/SPARK-26570
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.3.2
>Reporter: deshanxiao
>Priority: Major
> Attachments: image-2019-10-13-18-41-22-090.png, 
> image-2019-10-13-18-45-33-770.png, image-2019-10-14-10-00-27-361.png, 
> image-2019-10-14-10-32-17-949.png, image-2019-10-14-10-47-47-684.png, 
> image-2019-10-14-10-50-47-567.png, image-2019-10-14-10-51-28-374.png, 
> screenshot-1.png
>
>
> The *bulkListLeafFiles* will collect all filestatus in memory for every query 
> which may cause the oom of driver. I use the spark 2.3.2 meeting with the 
> problem. Maybe the latest one also exists the problem.



--
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-24915) Calling SparkSession.createDataFrame with schema can throw exception

2019-10-14 Thread Joachim Hereth (Jira)


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

Joachim Hereth commented on SPARK-24915:


this is fixed by [https://github.com/apache/spark/pull/26118.]

It's strange that Row is considered a tuple (it also causes the tests to look a 
bit strange).

However, changing the hierarchy seemed a bit too adventurous.

> Calling SparkSession.createDataFrame with schema can throw exception
> 
>
> Key: SPARK-24915
> URL: https://issues.apache.org/jira/browse/SPARK-24915
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark
>Affects Versions: 2.3.1
> Environment: Python 3.6.3
> PySpark 2.3.1 (installed via pip)
> OSX 10.12.6
>Reporter: Stephen Spencer
>Priority: Major
>
> There seems to be a bug in PySpark when using the PySparkSQL session to 
> create a dataframe with a pre-defined schema.
> Code to reproduce the error:
> {code:java}
> from pyspark import SparkConf, SparkContext
> from pyspark.sql import SparkSession
> from pyspark.sql.types import StructType, StructField, StringType, Row
> conf = SparkConf().setMaster("local").setAppName("repro") 
> context = SparkContext(conf=conf) 
> session = SparkSession(context)
> # Construct schema (the order of fields is important)
> schema = StructType([
> StructField('field2', StructType([StructField('sub_field', StringType(), 
> False)]), False),
> StructField('field1', StringType(), False),
> ])
> # Create data to populate data frame
> data = [
> Row(field1="Hello", field2=Row(sub_field='world'))
> ]
> # Attempt to create the data frame supplying the schema
> # this will throw a ValueError
> df = session.createDataFrame(data, schema=schema)
> df.show(){code}
> Running this throws a ValueError
> {noformat}
> Traceback (most recent call last):
> File "schema_bug.py", line 18, in 
> df = session.createDataFrame(data, schema=schema)
> File 
> "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/session.py",
>  line 691, in createDataFrame
> rdd, schema = self._createFromLocal(map(prepare, data), schema)
> File 
> "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/session.py",
>  line 423, in _createFromLocal
> data = [schema.toInternal(row) for row in data]
> File 
> "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/session.py",
>  line 423, in 
> data = [schema.toInternal(row) for row in data]
> File 
> "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/types.py",
>  line 601, in toInternal
> for f, v, c in zip(self.fields, obj, self._needConversion))
> File 
> "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/types.py",
>  line 601, in 
> for f, v, c in zip(self.fields, obj, self._needConversion))
> File 
> "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/types.py",
>  line 439, in toInternal
> return self.dataType.toInternal(obj)
> File 
> "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/types.py",
>  line 619, in toInternal
> raise ValueError("Unexpected tuple %r with StructType" % obj)
> ValueError: Unexpected tuple 'Hello' with StructType{noformat}
> The problem seems to be here:
> https://github.com/apache/spark/blob/3d5c61e5fd24f07302e39b5d61294da79aa0c2f9/python/pyspark/sql/types.py#L603
> specifically the bit
> {code:java}
> zip(self.fields, obj, self._needConversion)
> {code}
> This zip statement seems to assume that obj and self.fields are ordered in 
> the same way, so that the elements of obj will correspond to the right fields 
> in the schema. However this is not true, a Row orders its elements 
> alphabetically but the fields in the schema are in whatever order they are 
> specified. In this example field2 is being initialised with the field1 
> element 'Hello'. If you re-order the fields in the schema to go (field1, 
> field2), the given example works without error.
> The schema in the repro is specifically designed to elicit the problem, the 
> fields are out of alphabetical order and one field is a StructType, making 
> chema._needSerializeAnyField==True . However we encountered this in real use.



--
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] [Reopened] (SPARK-16483) Unifying struct fields and columns

2019-10-14 Thread Nicholas Chammas (Jira)


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

Nicholas Chammas reopened SPARK-16483:
--

Though this has been bulk closed, I still think it's a valuable potential 
improvement to the DataFrame API that makes working with nested datasets much 
more seamless and natural. (SPARK-18084 and SPARK-18277 are two hiccups I hit 
myself that would be addressed by this ticket.)

I hope this will get another look after Spark 3.0 is out.

> Unifying struct fields and columns
> --
>
> Key: SPARK-16483
> URL: https://issues.apache.org/jira/browse/SPARK-16483
> Project: Spark
>  Issue Type: New Feature
>  Components: SQL
>Affects Versions: 2.3.1
>Reporter: Simeon Simeonov
>Priority: Major
>  Labels: bulk-closed, sql
>
> This issue comes as a result of an exchange with Michael Armbrust outside of 
> the usual JIRA/dev list channels.
> DataFrame provides a full set of manipulation operations for top-level 
> columns. They have be added, removed, modified and renamed. The same is not 
> true about fields inside structs yet, from a logical standpoint, Spark users 
> may very well want to perform the same operations on struct fields, 
> especially since automatic schema discovery from JSON input tends to create 
> deeply nested structs.
> Common use-cases include:
>  - Remove and/or rename struct field(s) to adjust the schema
>  - Fix a data quality issue with a struct field (update/rewrite)
> To do this with the existing API by hand requires manually calling 
> {{named_struct}} and listing all fields, including ones we don't want to 
> manipulate. This leads to complex, fragile code that cannot survive schema 
> evolution.
> It would be far better if the various APIs that can now manipulate top-level 
> columns were extended to handle struct fields at arbitrary locations or, 
> alternatively, if we introduced new APIs for modifying any field in a 
> dataframe, whether it is a top-level one or one nested inside a struct.
> Purely for discussion purposes (overloaded methods are not shown):
> {code:java}
> class Column(val expr: Expression) extends Logging {
>   // ...
>   // matches Dataset.schema semantics
>   def schema: StructType
>   // matches Dataset.select() semantics
>   // '* support allows multiple new fields to be added easily, saving 
> cumbersome repeated withColumn() calls
>   def select(cols: Column*): Column
>   // matches Dataset.withColumn() semantics of add or replace
>   def withColumn(colName: String, col: Column): Column
>   // matches Dataset.drop() semantics
>   def drop(colName: String): Column
> }
> class Dataset[T] ... {
>   // ...
>   // Equivalent to sparkSession.createDataset(toDF.rdd, newSchema)
>   def cast(newShema: StructType): DataFrame
> }
> {code}
> The benefit of the above API is that it unifies manipulating top-level & 
> nested columns. The addition of {{schema}} and {{select()}} to {{Column}} 
> allows for nested field reordering, casting, etc., which is important in data 
> exchange scenarios where field position matters. That's also the reason to 
> add {{cast}} to {{Dataset}}: it improves consistency and readability (with 
> method chaining). Another way to think of {{Dataset.cast}} is as the Spark 
> schema equivalent of {{Dataset.as}}. {{as}} is to {{cast}} as a Scala 
> encodable type is to a {{StructType}} instance.



--
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] [Resolved] (SPARK-29347) External Row should be JSON serializable

2019-10-14 Thread Jira


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

Herman van Hövell resolved SPARK-29347.
---
Fix Version/s: 3.0.0
   Resolution: Fixed

> External Row should be JSON serializable
> 
>
> Key: SPARK-29347
> URL: https://issues.apache.org/jira/browse/SPARK-29347
> Project: Spark
>  Issue Type: New Feature
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Herman van Hövell
>Assignee: Herman van Hövell
>Priority: Major
> Fix For: 3.0.0
>
>
> External row should be exportable to json. This is needed for observable 
> metrics because we want to include these metrics in streaming query progress 
> (which is JSON serializable). 



--
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] [Created] (SPARK-29471) "TaskResultLost (result lost from block manager)" error message is misleading in case result fetch is caused by client-side issues

2019-10-14 Thread Josh Rosen (Jira)
Josh Rosen created SPARK-29471:
--

 Summary: "TaskResultLost (result lost from block manager)" error 
message is misleading in case result fetch is caused by client-side issues
 Key: SPARK-29471
 URL: https://issues.apache.org/jira/browse/SPARK-29471
 Project: Spark
  Issue Type: Bug
  Components: Block Manager
Affects Versions: 3.0.0
Reporter: Josh Rosen


I recently encountered a problem where jobs non-deterministically failed with
{code:java}
TaskResultLost (result lost from block manager) {code}
exceptions.

It turned out that this was due to some sort of networking issue where the 
Spark driver was unable to initiate outgoing connections to executors' block 
managers in order to fetch indirect task results.

In this situation, the error message was slightly misleading: the "result lost 
from block manager" makes it sound like we received an error / block-not-found 
response from the remote host, whereas in my case the problem was actually a 
network connectivity issue where we weren't even able to connect in the first 
place.

If it's easy to do so, it might be nice to refine the error-handling / logging 
code so that we distinguish between the receipt of an error response vs. a 
lower-level networking / connectivity issue. 



--
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] [Updated] (SPARK-29471) "TaskResultLost (result lost from block manager)" error message is misleading in case result fetch is caused by client-side network connectivity issues

2019-10-14 Thread Josh Rosen (Jira)


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

Josh Rosen updated SPARK-29471:
---
Summary: "TaskResultLost (result lost from block manager)" error message is 
misleading in case result fetch is caused by client-side network connectivity 
issues  (was: "TaskResultLost (result lost from block manager)" error message 
is misleading in case result fetch is caused by client-side issues)

> "TaskResultLost (result lost from block manager)" error message is misleading 
> in case result fetch is caused by client-side network connectivity issues
> ---
>
> Key: SPARK-29471
> URL: https://issues.apache.org/jira/browse/SPARK-29471
> Project: Spark
>  Issue Type: Bug
>  Components: Block Manager
>Affects Versions: 3.0.0
>Reporter: Josh Rosen
>Priority: Minor
>
> I recently encountered a problem where jobs non-deterministically failed with
> {code:java}
> TaskResultLost (result lost from block manager) {code}
> exceptions.
> It turned out that this was due to some sort of networking issue where the 
> Spark driver was unable to initiate outgoing connections to executors' block 
> managers in order to fetch indirect task results.
> In this situation, the error message was slightly misleading: the "result 
> lost from block manager" makes it sound like we received an error / 
> block-not-found response from the remote host, whereas in my case the problem 
> was actually a network connectivity issue where we weren't even able to 
> connect in the first place.
> If it's easy to do so, it might be nice to refine the error-handling / 
> logging code so that we distinguish between the receipt of an error response 
> vs. a lower-level networking / connectivity issue. 



--
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] [Created] (SPARK-29470) Update plugins to latest versions

2019-10-14 Thread Dongjoon Hyun (Jira)
Dongjoon Hyun created SPARK-29470:
-

 Summary: Update plugins to latest versions
 Key: SPARK-29470
 URL: https://issues.apache.org/jira/browse/SPARK-29470
 Project: Spark
  Issue Type: Improvement
  Components: Build
Affects Versions: 3.0.0
Reporter: Dongjoon Hyun






--
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-27623) Provider org.apache.spark.sql.avro.AvroFileFormat could not be instantiated

2019-10-14 Thread Tom Tang (Jira)


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

Tom Tang edited comment on SPARK-27623 at 10/14/19 9:42 PM:


I found the same issue with spark 2.4.3, and when I fall back to spark-avro 
2.11, that seems solve the issue.



{{spark-sql --packages org.apache.spark:spark-avro_2.11:2.4.3}}
{{pyspark --packages org.apache.spark:spark-avro_2.11:2.4.3}}



was (Author: tomtang):
I found the same issue with spark 2.4.3, and when I fall back to spark-avro 
2.11, that seems solve the issue.



{{ spark-sql --packages org.apache.spark:spark-avro_2.11:2.4.3 }}
{{ pyspark --packages org.apache.spark:spark-avro_2.11:2.4.3 }}


> Provider org.apache.spark.sql.avro.AvroFileFormat could not be instantiated
> ---
>
> Key: SPARK-27623
> URL: https://issues.apache.org/jira/browse/SPARK-27623
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark
>Affects Versions: 2.4.2
>Reporter: Alexandru Barbulescu
>Priority: Major
>
> After updating to spark 2.4.2 when using the 
> {code:java}
> spark.read.format().options().load()
> {code}
>  
> chain of methods, regardless of what parameter is passed to "format" we get 
> the following error related to avro:
>  
> {code:java}
> - .options(**load_options)
> - File "/opt/spark/python/lib/pyspark.zip/pyspark/sql/readwriter.py", line 
> 172, in load
> - File "/opt/spark/python/lib/py4j-0.10.7-src.zip/py4j/java_gateway.py", line 
> 1257, in __call__
> - File "/opt/spark/python/lib/pyspark.zip/pyspark/sql/utils.py", line 63, in 
> deco
> - File "/opt/spark/python/lib/py4j-0.10.7-src.zip/py4j/protocol.py", line 
> 328, in get_return_value
> - py4j.protocol.Py4JJavaError: An error occurred while calling o69.load.
> - : java.util.ServiceConfigurationError: 
> org.apache.spark.sql.sources.DataSourceRegister: Provider 
> org.apache.spark.sql.avro.AvroFileFormat could not be instantiated
> - at java.util.ServiceLoader.fail(ServiceLoader.java:232)
> - at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
> - at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
> - at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
> - at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
> - at 
> scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:44)
> - at scala.collection.Iterator.foreach(Iterator.scala:941)
> - at scala.collection.Iterator.foreach$(Iterator.scala:941)
> - at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
> - at scala.collection.IterableLike.foreach(IterableLike.scala:74)
> - at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
> - at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
> - at scala.collection.TraversableLike.filterImpl(TraversableLike.scala:250)
> - at scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:248)
> - at scala.collection.AbstractTraversable.filterImpl(Traversable.scala:108)
> - at scala.collection.TraversableLike.filter(TraversableLike.scala:262)
> - at scala.collection.TraversableLike.filter$(TraversableLike.scala:262)
> - at scala.collection.AbstractTraversable.filter(Traversable.scala:108)
> - at 
> org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:630)
> - at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:194)
> - at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:167)
> - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> - at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> - at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> - at java.lang.reflect.Method.invoke(Method.java:498)
> - at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
> - at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
> - at py4j.Gateway.invoke(Gateway.java:282)
> - at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
> - at py4j.commands.CallCommand.execute(CallCommand.java:79)
> - at py4j.GatewayConnection.run(GatewayConnection.java:238)
> - at java.lang.Thread.run(Thread.java:748)
> - Caused by: java.lang.NoClassDefFoundError: 
> org/apache/spark/sql/execution/datasources/FileFormat$class
> - at org.apache.spark.sql.avro.AvroFileFormat.(AvroFileFormat.scala:44)
> - at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
> - at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
> - at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
> - at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
> - at 

[jira] [Comment Edited] (SPARK-27623) Provider org.apache.spark.sql.avro.AvroFileFormat could not be instantiated

2019-10-14 Thread Tom Tang (Jira)


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

Tom Tang edited comment on SPARK-27623 at 10/14/19 9:42 PM:


I found the same issue with spark 2.4.3, and when I fall back to spark-avro 
2.11, that seems solve the issue.



{{ spark-sql --packages org.apache.spark:spark-avro_2.11:2.4.3 }}
{{ pyspark --packages org.apache.spark:spark-avro_2.11:2.4.3 }}



was (Author: tomtang):
I found the same issue with spark 2.4.3, and when I fall back to spark-avro 
2.11, that seems solve the issue.



{{
spark-sql --packages org.apache.spark:spark-avro_2.11:2.4.3

pyspark --packages org.apache.spark:spark-avro_2.11:2.4.3
}}


> Provider org.apache.spark.sql.avro.AvroFileFormat could not be instantiated
> ---
>
> Key: SPARK-27623
> URL: https://issues.apache.org/jira/browse/SPARK-27623
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark
>Affects Versions: 2.4.2
>Reporter: Alexandru Barbulescu
>Priority: Major
>
> After updating to spark 2.4.2 when using the 
> {code:java}
> spark.read.format().options().load()
> {code}
>  
> chain of methods, regardless of what parameter is passed to "format" we get 
> the following error related to avro:
>  
> {code:java}
> - .options(**load_options)
> - File "/opt/spark/python/lib/pyspark.zip/pyspark/sql/readwriter.py", line 
> 172, in load
> - File "/opt/spark/python/lib/py4j-0.10.7-src.zip/py4j/java_gateway.py", line 
> 1257, in __call__
> - File "/opt/spark/python/lib/pyspark.zip/pyspark/sql/utils.py", line 63, in 
> deco
> - File "/opt/spark/python/lib/py4j-0.10.7-src.zip/py4j/protocol.py", line 
> 328, in get_return_value
> - py4j.protocol.Py4JJavaError: An error occurred while calling o69.load.
> - : java.util.ServiceConfigurationError: 
> org.apache.spark.sql.sources.DataSourceRegister: Provider 
> org.apache.spark.sql.avro.AvroFileFormat could not be instantiated
> - at java.util.ServiceLoader.fail(ServiceLoader.java:232)
> - at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
> - at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
> - at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
> - at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
> - at 
> scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:44)
> - at scala.collection.Iterator.foreach(Iterator.scala:941)
> - at scala.collection.Iterator.foreach$(Iterator.scala:941)
> - at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
> - at scala.collection.IterableLike.foreach(IterableLike.scala:74)
> - at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
> - at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
> - at scala.collection.TraversableLike.filterImpl(TraversableLike.scala:250)
> - at scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:248)
> - at scala.collection.AbstractTraversable.filterImpl(Traversable.scala:108)
> - at scala.collection.TraversableLike.filter(TraversableLike.scala:262)
> - at scala.collection.TraversableLike.filter$(TraversableLike.scala:262)
> - at scala.collection.AbstractTraversable.filter(Traversable.scala:108)
> - at 
> org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:630)
> - at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:194)
> - at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:167)
> - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> - at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> - at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> - at java.lang.reflect.Method.invoke(Method.java:498)
> - at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
> - at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
> - at py4j.Gateway.invoke(Gateway.java:282)
> - at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
> - at py4j.commands.CallCommand.execute(CallCommand.java:79)
> - at py4j.GatewayConnection.run(GatewayConnection.java:238)
> - at java.lang.Thread.run(Thread.java:748)
> - Caused by: java.lang.NoClassDefFoundError: 
> org/apache/spark/sql/execution/datasources/FileFormat$class
> - at org.apache.spark.sql.avro.AvroFileFormat.(AvroFileFormat.scala:44)
> - at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
> - at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
> - at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
> - at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
> - at 

[jira] [Comment Edited] (SPARK-27623) Provider org.apache.spark.sql.avro.AvroFileFormat could not be instantiated

2019-10-14 Thread Tom Tang (Jira)


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

Tom Tang edited comment on SPARK-27623 at 10/14/19 9:41 PM:


I found the same issue with spark 2.4.3, and when I fall back to spark-avro 
2.11, that seems solve the issue.



{{
spark-sql --packages org.apache.spark:spark-avro_2.11:2.4.3

pyspark --packages org.apache.spark:spark-avro_2.11:2.4.3
}}



was (Author: tomtang):
I found the same issue with spark 2.4.3, and when I fall back to spark-avro 
2.11, that seems solve the issue.

```

spark-sql --packages org.apache.spark:spark-avro_2.11:2.4.3

pyspark --packages org.apache.spark:spark-avro_2.11:2.4.3

```

> Provider org.apache.spark.sql.avro.AvroFileFormat could not be instantiated
> ---
>
> Key: SPARK-27623
> URL: https://issues.apache.org/jira/browse/SPARK-27623
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark
>Affects Versions: 2.4.2
>Reporter: Alexandru Barbulescu
>Priority: Major
>
> After updating to spark 2.4.2 when using the 
> {code:java}
> spark.read.format().options().load()
> {code}
>  
> chain of methods, regardless of what parameter is passed to "format" we get 
> the following error related to avro:
>  
> {code:java}
> - .options(**load_options)
> - File "/opt/spark/python/lib/pyspark.zip/pyspark/sql/readwriter.py", line 
> 172, in load
> - File "/opt/spark/python/lib/py4j-0.10.7-src.zip/py4j/java_gateway.py", line 
> 1257, in __call__
> - File "/opt/spark/python/lib/pyspark.zip/pyspark/sql/utils.py", line 63, in 
> deco
> - File "/opt/spark/python/lib/py4j-0.10.7-src.zip/py4j/protocol.py", line 
> 328, in get_return_value
> - py4j.protocol.Py4JJavaError: An error occurred while calling o69.load.
> - : java.util.ServiceConfigurationError: 
> org.apache.spark.sql.sources.DataSourceRegister: Provider 
> org.apache.spark.sql.avro.AvroFileFormat could not be instantiated
> - at java.util.ServiceLoader.fail(ServiceLoader.java:232)
> - at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
> - at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
> - at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
> - at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
> - at 
> scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:44)
> - at scala.collection.Iterator.foreach(Iterator.scala:941)
> - at scala.collection.Iterator.foreach$(Iterator.scala:941)
> - at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
> - at scala.collection.IterableLike.foreach(IterableLike.scala:74)
> - at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
> - at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
> - at scala.collection.TraversableLike.filterImpl(TraversableLike.scala:250)
> - at scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:248)
> - at scala.collection.AbstractTraversable.filterImpl(Traversable.scala:108)
> - at scala.collection.TraversableLike.filter(TraversableLike.scala:262)
> - at scala.collection.TraversableLike.filter$(TraversableLike.scala:262)
> - at scala.collection.AbstractTraversable.filter(Traversable.scala:108)
> - at 
> org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:630)
> - at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:194)
> - at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:167)
> - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> - at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> - at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> - at java.lang.reflect.Method.invoke(Method.java:498)
> - at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
> - at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
> - at py4j.Gateway.invoke(Gateway.java:282)
> - at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
> - at py4j.commands.CallCommand.execute(CallCommand.java:79)
> - at py4j.GatewayConnection.run(GatewayConnection.java:238)
> - at java.lang.Thread.run(Thread.java:748)
> - Caused by: java.lang.NoClassDefFoundError: 
> org/apache/spark/sql/execution/datasources/FileFormat$class
> - at org.apache.spark.sql.avro.AvroFileFormat.(AvroFileFormat.scala:44)
> - at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
> - at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
> - at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
> - at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
> - at 

[jira] [Commented] (SPARK-27623) Provider org.apache.spark.sql.avro.AvroFileFormat could not be instantiated

2019-10-14 Thread Tom Tang (Jira)


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

Tom Tang commented on SPARK-27623:
--

I found the same issue with spark 2.4.3, and when I fall back to spark-avro 
2.11, that seems solve the issue.

```

spark-sql --packages org.apache.spark:spark-avro_2.11:2.4.3

pyspark --packages org.apache.spark:spark-avro_2.11:2.4.3

```

> Provider org.apache.spark.sql.avro.AvroFileFormat could not be instantiated
> ---
>
> Key: SPARK-27623
> URL: https://issues.apache.org/jira/browse/SPARK-27623
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark
>Affects Versions: 2.4.2
>Reporter: Alexandru Barbulescu
>Priority: Major
>
> After updating to spark 2.4.2 when using the 
> {code:java}
> spark.read.format().options().load()
> {code}
>  
> chain of methods, regardless of what parameter is passed to "format" we get 
> the following error related to avro:
>  
> {code:java}
> - .options(**load_options)
> - File "/opt/spark/python/lib/pyspark.zip/pyspark/sql/readwriter.py", line 
> 172, in load
> - File "/opt/spark/python/lib/py4j-0.10.7-src.zip/py4j/java_gateway.py", line 
> 1257, in __call__
> - File "/opt/spark/python/lib/pyspark.zip/pyspark/sql/utils.py", line 63, in 
> deco
> - File "/opt/spark/python/lib/py4j-0.10.7-src.zip/py4j/protocol.py", line 
> 328, in get_return_value
> - py4j.protocol.Py4JJavaError: An error occurred while calling o69.load.
> - : java.util.ServiceConfigurationError: 
> org.apache.spark.sql.sources.DataSourceRegister: Provider 
> org.apache.spark.sql.avro.AvroFileFormat could not be instantiated
> - at java.util.ServiceLoader.fail(ServiceLoader.java:232)
> - at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
> - at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
> - at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
> - at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
> - at 
> scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:44)
> - at scala.collection.Iterator.foreach(Iterator.scala:941)
> - at scala.collection.Iterator.foreach$(Iterator.scala:941)
> - at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
> - at scala.collection.IterableLike.foreach(IterableLike.scala:74)
> - at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
> - at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
> - at scala.collection.TraversableLike.filterImpl(TraversableLike.scala:250)
> - at scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:248)
> - at scala.collection.AbstractTraversable.filterImpl(Traversable.scala:108)
> - at scala.collection.TraversableLike.filter(TraversableLike.scala:262)
> - at scala.collection.TraversableLike.filter$(TraversableLike.scala:262)
> - at scala.collection.AbstractTraversable.filter(Traversable.scala:108)
> - at 
> org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:630)
> - at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:194)
> - at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:167)
> - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> - at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> - at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> - at java.lang.reflect.Method.invoke(Method.java:498)
> - at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
> - at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
> - at py4j.Gateway.invoke(Gateway.java:282)
> - at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
> - at py4j.commands.CallCommand.execute(CallCommand.java:79)
> - at py4j.GatewayConnection.run(GatewayConnection.java:238)
> - at java.lang.Thread.run(Thread.java:748)
> - Caused by: java.lang.NoClassDefFoundError: 
> org/apache/spark/sql/execution/datasources/FileFormat$class
> - at org.apache.spark.sql.avro.AvroFileFormat.(AvroFileFormat.scala:44)
> - at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
> - at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
> - at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
> - at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
> - at java.lang.Class.newInstance(Class.java:442)
> - at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:380)
> - ... 29 more
> - Caused by: java.lang.ClassNotFoundException: 
> org.apache.spark.sql.execution.datasources.FileFormat$class
> - at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
> - at 

[jira] [Assigned] (SPARK-29469) Avoid retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed

2019-10-14 Thread L. C. Hsieh (Jira)


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

L. C. Hsieh reassigned SPARK-29469:
---

Assignee: L. C. Hsieh

> Avoid retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed
> -
>
> Key: SPARK-29469
> URL: https://issues.apache.org/jira/browse/SPARK-29469
> Project: Spark
>  Issue Type: Improvement
>  Components: Shuffle
>Affects Versions: 3.0.0
>Reporter: L. C. Hsieh
>Assignee: L. C. Hsieh
>Priority: Minor
>
> Found that some NPE was thrown in job log:
> 2019-10-14 20:06:16 ERROR RetryingBlockFetcher:143 - Exception while 
> beginning fetch of 2 outstanding blocks (after 3 retries)
> java.lang.NullPointerException
>   at 
> org.apache.spark.network.shuffle.ExternalShuffleClient.lambda$fetchBlocks$0(ExternalShuffleClient.java:100)
>   at 
> org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:141)
>   at 
> org.apache.spark.network.shuffle.RetryingBlockFetcher.lambda$initiateRetry$0(RetryingBlockFetcher.java:169)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>   at 
> io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
> It was happened after BlockManager and ExternalBlockStoreClient was closed 
> due to previous errors. In this cases, RetryingBlockFetcher does not need to 
> retry. This NPE is harmless for job execution, but is a source of misleading 
> when looking at log. Especially for end-users.



--
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] [Resolved] (SPARK-29463) move v2 commands to a new file

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun resolved SPARK-29463.
---
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26111
[https://github.com/apache/spark/pull/26111]

> move v2 commands to a new file
> --
>
> Key: SPARK-29463
> URL: https://issues.apache.org/jira/browse/SPARK-29463
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Wenchen Fan
>Assignee: Wenchen Fan
>Priority: Major
> Fix For: 3.0.0
>
>




--
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] [Created] (SPARK-29469) Avoid retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed

2019-10-14 Thread L. C. Hsieh (Jira)
L. C. Hsieh created SPARK-29469:
---

 Summary: Avoid retries by RetryingBlockFetcher when 
ExternalBlockStoreClient is closed
 Key: SPARK-29469
 URL: https://issues.apache.org/jira/browse/SPARK-29469
 Project: Spark
  Issue Type: Improvement
  Components: Shuffle
Affects Versions: 3.0.0
Reporter: L. C. Hsieh


Found that some NPE was thrown in job log:

2019-10-14 20:06:16 ERROR RetryingBlockFetcher:143 - Exception while beginning 
fetch of 2 outstanding blocks (after 3 retries)
java.lang.NullPointerException
at 
org.apache.spark.network.shuffle.ExternalShuffleClient.lambda$fetchBlocks$0(ExternalShuffleClient.java:100)
at 
org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:141)
at 
org.apache.spark.network.shuffle.RetryingBlockFetcher.lambda$initiateRetry$0(RetryingBlockFetcher.java:169)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)


It was happened after BlockManager and ExternalBlockStoreClient was closed due 
to previous errors. In this cases, RetryingBlockFetcher does not need to retry. 
This NPE is harmless for job execution, but is a source of misleading when 
looking at log. Especially for end-users.



--
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] [Created] (SPARK-29468) Floating point literals produce incorrect SQL

2019-10-14 Thread Jose Torres (Jira)
Jose Torres created SPARK-29468:
---

 Summary: Floating point literals produce incorrect SQL
 Key: SPARK-29468
 URL: https://issues.apache.org/jira/browse/SPARK-29468
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 2.4.4
Reporter: Jose Torres


A FLOAT literal 1.2345 returns SQL `CAST(1.2345 AS FLOAT)`. For very small 
values this doesn't work; `CAST(1e-44 AS FLOAT)` for example doesn't parse, 
because the parser tries to squeeze the numeric literal 1e-44 into a 
DECIMAL(38).



--
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] [Updated] (SPARK-29467) dev/merge_spark_pr.py fails on CAPTCHA

2019-10-14 Thread Shane Knapp (Jira)


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

Shane Knapp updated SPARK-29467:

Description: 
so, i was merging a PR and when i tried to update the associated jira, it 
failed and dumped out the error response to my terminal.  the important bit is 
here:

jira.exceptions.JIRAError: JiraError HTTP 403 url: 
https://issues.apache.org/jira/rest/api/2/serverInfo
text: CAPTCHA_CHALLENGE; login-url=https://issues.apache.org/jira/login.jsp

when i went to log in to the site and close the issue manually, i had to enter 
the captcha.

three thoughts:
1) perhaps people will need to make sure they're logged in to the jira BEFORE 
running the merge script...
2) or we can remove the jira update section (which isn't ideal)
3) or we somehow bypass it for the script?

open to suggestions.  

¯\_(ツ)_/¯


  was:
so, i was merging a PR and when i tried to update the associated jira, it 
failed and dumped out the error response to my terminal.  the important bit is 
here:

{{
jira.exceptions.JIRAError: JiraError HTTP 403 url: 
https://issues.apache.org/jira/rest/api/2/serverInfo
text: CAPTCHA_CHALLENGE; login-url=https://issues.apache.org/jira/login.jsp
}}

when i went to log in to the site and close the issue manually, i had to enter 
the captcha.

three thoughts:
1) perhaps people will need to make sure they're logged in to the jira BEFORE 
running the merge script...
2) or we can remove the jira update section (which isn't ideal)
3) or we somehow bypass it for the script?

open to suggestions.  
{{
¯\_(ツ)_/¯
}}


> dev/merge_spark_pr.py fails on CAPTCHA
> --
>
> Key: SPARK-29467
> URL: https://issues.apache.org/jira/browse/SPARK-29467
> Project: Spark
>  Issue Type: Bug
>  Components: Project Infra
>Affects Versions: 3.0.0
>Reporter: Shane Knapp
>Priority: Minor
>
> so, i was merging a PR and when i tried to update the associated jira, it 
> failed and dumped out the error response to my terminal.  the important bit 
> is here:
> jira.exceptions.JIRAError: JiraError HTTP 403 url: 
> https://issues.apache.org/jira/rest/api/2/serverInfo
> text: CAPTCHA_CHALLENGE; login-url=https://issues.apache.org/jira/login.jsp
> when i went to log in to the site and close the issue manually, i had to 
> enter the captcha.
> three thoughts:
> 1) perhaps people will need to make sure they're logged in to the jira BEFORE 
> running the merge script...
> 2) or we can remove the jira update section (which isn't ideal)
> 3) or we somehow bypass it for the script?
> open to suggestions.  
> ¯\_(ツ)_/¯



--
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] [Updated] (SPARK-29467) dev/merge_spark_pr.py fails on CAPTCHA

2019-10-14 Thread Shane Knapp (Jira)


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

Shane Knapp updated SPARK-29467:

Description: 
so, i was merging a PR and when i tried to update the associated jira, it 
failed and dumped out the error response to my terminal.  the important bit is 
here:

{{
jira.exceptions.JIRAError: JiraError HTTP 403 url: 
https://issues.apache.org/jira/rest/api/2/serverInfo
text: CAPTCHA_CHALLENGE; login-url=https://issues.apache.org/jira/login.jsp
}}

when i went to log in to the site and close the issue manually, i had to enter 
the captcha.

three thoughts:
1) perhaps people will need to make sure they're logged in to the jira BEFORE 
running the merge script...
2) or we can remove the jira update section (which isn't ideal)
3) or we somehow bypass it for the script?

open to suggestions.  
{{
¯\_(ツ)_/¯
}}

  was:
so, i was merging a PR and when i tried to update the associated jira, it 
failed and dumped out the error response to my terminal.  the important bit is 
here:

{{jira.exceptions.JIRAError: JiraError HTTP 403 url: 
https://issues.apache.org/jira/rest/api/2/serverInfo
text: CAPTCHA_CHALLENGE; login-url=https://issues.apache.org/jira/login.jsp
}}

when i went to log in to the site and close the issue manually, i had to enter 
the captcha.

three thoughts:
1) perhaps people will need to make sure they're logged in to the jira BEFORE 
running the merge script...
2) or we can remove the jira update section (which isn't ideal)
3) or we somehow bypass it for the script?

open to suggestions.  {{ ¯\_(ツ)_/¯}}


> dev/merge_spark_pr.py fails on CAPTCHA
> --
>
> Key: SPARK-29467
> URL: https://issues.apache.org/jira/browse/SPARK-29467
> Project: Spark
>  Issue Type: Bug
>  Components: Project Infra
>Affects Versions: 3.0.0
>Reporter: Shane Knapp
>Priority: Minor
>
> so, i was merging a PR and when i tried to update the associated jira, it 
> failed and dumped out the error response to my terminal.  the important bit 
> is here:
> {{
> jira.exceptions.JIRAError: JiraError HTTP 403 url: 
> https://issues.apache.org/jira/rest/api/2/serverInfo
> text: CAPTCHA_CHALLENGE; login-url=https://issues.apache.org/jira/login.jsp
> }}
> when i went to log in to the site and close the issue manually, i had to 
> enter the captcha.
> three thoughts:
> 1) perhaps people will need to make sure they're logged in to the jira BEFORE 
> running the merge script...
> 2) or we can remove the jira update section (which isn't ideal)
> 3) or we somehow bypass it for the script?
> open to suggestions.  
> {{
> ¯\_(ツ)_/¯
> }}



--
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] [Created] (SPARK-29467) dev/merge_spark_pr.py fails on CAPTCHA

2019-10-14 Thread Shane Knapp (Jira)
Shane Knapp created SPARK-29467:
---

 Summary: dev/merge_spark_pr.py fails on CAPTCHA
 Key: SPARK-29467
 URL: https://issues.apache.org/jira/browse/SPARK-29467
 Project: Spark
  Issue Type: Bug
  Components: Project Infra
Affects Versions: 3.0.0
Reporter: Shane Knapp


so, i was merging a PR and when i tried to update the associated jira, it 
failed and dumped out the error response to my terminal.  the important bit is 
here:

{{jira.exceptions.JIRAError: JiraError HTTP 403 url: 
https://issues.apache.org/jira/rest/api/2/serverInfo
text: CAPTCHA_CHALLENGE; login-url=https://issues.apache.org/jira/login.jsp
}}

when i went to log in to the site and close the issue manually, i had to enter 
the captcha.

three thoughts:
1) perhaps people will need to make sure they're logged in to the jira BEFORE 
running the merge script...
2) or we can remove the jira update section (which isn't ideal)
3) or we somehow bypass it for the script?

open to suggestions.  {{ ¯\_(ツ)_/¯}}



--
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] [Resolved] (SPARK-29417) Resource Scheduling - add TaskContext.resource java api

2019-10-14 Thread Xiangrui Meng (Jira)


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

Xiangrui Meng resolved SPARK-29417.
---
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26083
[https://github.com/apache/spark/pull/26083]

> Resource Scheduling - add TaskContext.resource java api
> ---
>
> Key: SPARK-29417
> URL: https://issues.apache.org/jira/browse/SPARK-29417
> Project: Spark
>  Issue Type: Story
>  Components: Spark Core
>Affects Versions: 3.0.0
>Reporter: Thomas Graves
>Assignee: Thomas Graves
>Priority: Major
> Fix For: 3.0.0
>
>
> I noticed the TaskContext.resource() api we added returns a scala Map. This 
> isn't very nice for the java api usage, so we should add an api that returns 
> a java Map.



--
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] [Resolved] (SPARK-25152) Enable Spark on Kubernetes R Integration Tests

2019-10-14 Thread Shane Knapp (Jira)


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

Shane Knapp resolved SPARK-25152.
-
  Assignee: Ilan Filonenko
Resolution: Fixed

merged in to master

> Enable Spark on Kubernetes R Integration Tests
> --
>
> Key: SPARK-25152
> URL: https://issues.apache.org/jira/browse/SPARK-25152
> Project: Spark
>  Issue Type: Test
>  Components: Kubernetes, SparkR
>Affects Versions: 2.4.0
>Reporter: Matt Cheah
>Assignee: Ilan Filonenko
>Priority: Major
>
> We merged [https://github.com/apache/spark/pull/21584] for SPARK-24433 but we 
> had to turn off the integration tests due to issues with the Jenkins 
> environment. Re-enable the tests after the environment is fixed.



--
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] [Resolved] (SPARK-29442) Set `default` mode should override the existing mode

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun resolved SPARK-29442.
---
Fix Version/s: 3.0.0
 Assignee: Dongjoon Hyun
   Resolution: Fixed

This is resolved via https://github.com/apache/spark/pull/26094

> Set `default` mode should override the existing mode
> 
>
> Key: SPARK-29442
> URL: https://issues.apache.org/jira/browse/SPARK-29442
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Dongjoon Hyun
>Assignee: Dongjoon Hyun
>Priority: Minor
> Fix For: 3.0.0
>
>




--
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] [Updated] (SPARK-29442) Set `default` mode should override the existing mode

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun updated SPARK-29442:
--
Affects Version/s: (was: 2.4.4)
   (was: 2.3.4)

> Set `default` mode should override the existing mode
> 
>
> Key: SPARK-29442
> URL: https://issues.apache.org/jira/browse/SPARK-29442
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Dongjoon Hyun
>Priority: Minor
>




--
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] [Created] (SPARK-29466) Show `Duration` for running drivers in Standalone master web UI

2019-10-14 Thread Dongjoon Hyun (Jira)
Dongjoon Hyun created SPARK-29466:
-

 Summary: Show `Duration` for running drivers in Standalone master 
web UI
 Key: SPARK-29466
 URL: https://issues.apache.org/jira/browse/SPARK-29466
 Project: Spark
  Issue Type: Improvement
  Components: Web UI
Affects Versions: 3.0.0
Reporter: Dongjoon Hyun


This issue aims to add a new column for `Duration` for running drivers table in 
`Standalone` master web UI.



--
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] [Updated] (SPARK-29465) Unable to configure SPARK UI (spark.ui.port) in spark yarn cluster mode.

2019-10-14 Thread Vishwas Nalka (Jira)


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

Vishwas Nalka updated SPARK-29465:
--
Description: 
 I'm trying to restrict the ports used by spark app which is launched in yarn 
cluster mode. All ports (viz. driver, executor, blockmanager) could be 
specified using the respective properties except the ui port. The spark app is 
launched using JAVA code and setting the property spark.ui.port in sparkConf 
doesn't seem to help. Even setting a JVM option -Dspark.ui.port="some_port" 
does not spawn the UI is required port. 

>From the logs of the spark app, *_the property spark.ui.port is overridden and 
>the JVM property '-Dspark.ui.port=0' is set_* even though it is never set to 
>0. 

_(Run in Spark 1.6.2) From the logs ->_

_command:LD_LIBRARY_PATH="/usr/hdp/2.6.4.0-91/hadoop/lib/native:$LD_LIBRARY_PATH"
 {{JAVA_HOME}}/bin/java -server -XX:OnOutOfMemoryError='kill %p' -Xms4096m 
-Xmx4096m -Djava.io.tmpdir={{PWD}}/tmp '-Dspark.blockManager.port=9900' 
'-Dspark.driver.port=9902' '-Dspark.fileserver.port=9903' 
'-Dspark.broadcast.port=9904' '-Dspark.port.maxRetries=20' '-Dspark.ui.port=0' 
'-Dspark.executor.port=9905'_

_19/10/14 16:39:59 INFO Utils: Successfully started service 'SparkUI' on port 
35167.19/10/14 16:39:59 INFO SparkUI: Started SparkUI at_ 
[_http://10.65.170.98:35167_|http://10.65.170.98:35167/]

Even tried using a *spark-submit command with --conf spark.ui.port* does spawn 
UI in required port

{color:#172b4d}_(Run in Spark 2.4.4)_{color}
 {color:#172b4d}_./bin/spark-submit --class org.apache.spark.examples.SparkPi 
--master yarn --deploy-mode cluster --driver-memory 4g --executor-memory 2g 
--executor-cores 1 --conf spark.ui.port=12345 --conf spark.driver.port=12340 
--queue default examples/jars/spark-examples_2.11-2.4.4.jar 10_{color}

_From the logs::_
 _19/10/15 00:04:05 INFO ui.SparkUI: Stopped Spark web UI at 
[http://invrh74ace005.informatica.com:46622|http://invrh74ace005.informatica.com:46622/]_

_command:{{JAVA_HOME}}/bin/java -server -Xmx2048m -Djava.io.tmpdir={{PWD}}/tmp 
'-Dspark.ui.port=0'  'Dspark.driver.port=12340' 
-Dspark.yarn.app.container.log.dir= -XX:OnOutOfMemoryError='kill %p' 
org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url 
spark://coarsegrainedschedu...@invrh74ace005.informatica.com:12340 
--executor-id  --hostname  --cores 1 --app-id 
application_1570992022035_0089 --user-class-path 
[file:$PWD/__app__.jar1|file://%24pwd/__app__.jar1]>/stdout2>/stderr_

 

Looks like the application master override this and set a JVM property before 
launch resulting in random UI port even though spark.ui.port is set by the user.

In these links
 # 
[https://github.com/apache/spark/blob/master/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala]
 (line 214)
 # 
[https://github.com/cloudera/spark/blob/master/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala]
 (line 75)

I can see that the method _*run() in above files sets a system property 
UI_PORT*_ and _*spark.ui.port respectively.*_

  was:
 I'm trying to restrict the ports used by spark app which is launched in yarn 
cluster mode. All ports (viz. driver, executor, blockmanager) could be 
specified using the respective properties except the ui port. The spark app is 
launched using JAVA code and setting the property spark.ui.port in sparkConf 
doesn't seem to help. Even setting a JVM option -Dspark.ui.port="some_port" 
does not spawn the UI is required port. 

>From the logs of the spark app, *_the property spark.ui.port is overridden and 
>the JVM property '-Dspark.ui.port=0' is set_* even though it is never set to 
>0. 

_(Run in Spark 1.6.2) From the logs ->_

_command:LD_LIBRARY_PATH="/usr/hdp/2.6.4.0-91/hadoop/lib/native:$LD_LIBRARY_PATH"
 {{JAVA_HOME}}/bin/java -server -XX:OnOutOfMemoryError='kill %p' -Xms4096m 
-Xmx4096m -Djava.io.tmpdir={{PWD}}/tmp '-Dspark.blockManager.port=9900' 
'-Dspark.driver.port=9902' '-Dspark.fileserver.port=9903' 
'-Dspark.broadcast.port=9904' '-Dspark.port.maxRetries=20' '-Dspark.ui.port=0' 
'-Dspark.executor.port=9905'_

_19/10/14 16:39:59 INFO Utils: Successfully started service 'SparkUI' on port 
35167.19/10/14 16:39:59 INFO SparkUI: Started SparkUI at_ 
[_http://10.65.170.98:35167_|http://10.65.170.98:35167/]

Even tried using a *spark-submit command with --conf spark.ui.port* does spawn 
UI in required port

{color:#172b4d}_(Run in Spark 2.4.4)_{color}
 {color:#172b4d}_./bin/spark-submit --class org.apache.spark.examples.SparkPi 
--master yarn --deploy-mode cluster --driver-memory 4g --executor-memory 2g 
--executor-cores 1 --conf spark.ui.port=12345 --conf spark.driver.port=12340 
--queue default examples/jars/spark-examples_2.11-2.4.4.jar 10_{color}

_From the logs::_
 _19/10/15 00:04:05 INFO ui.SparkUI: Stopped Spark web UI at 

[jira] [Created] (SPARK-29465) Unable to configure SPARK UI (spark.ui.port) in spark yarn cluster mode.

2019-10-14 Thread Vishwas Nalka (Jira)
Vishwas Nalka created SPARK-29465:
-

 Summary: Unable to configure SPARK UI (spark.ui.port) in spark 
yarn cluster mode. 
 Key: SPARK-29465
 URL: https://issues.apache.org/jira/browse/SPARK-29465
 Project: Spark
  Issue Type: Bug
  Components: Spark Submit, YARN
Affects Versions: 2.4.4, 1.6.2
Reporter: Vishwas Nalka


 I'm trying to restrict the ports used by spark app which is launched in yarn 
cluster mode. All ports (viz. driver, executor, blockmanager) could be 
specified using the respective properties except the ui port. The spark app is 
launched using JAVA code and setting the property spark.ui.port in sparkConf 
doesn't seem to help. Even setting a JVM option -Dspark.ui.port="some_port" 
does not spawn the UI is required port. 


>From the logs of the spark app, *_the property spark.ui.port is overridden and 
>the JVM property '-Dspark.ui.port=0' is set_* even though it is never set to 
>0. 


_(Run in Spark 1.6.2) From the logs ->_

_command:LD_LIBRARY_PATH="/usr/hdp/2.6.4.0-91/hadoop/lib/native:$LD_LIBRARY_PATH"
 \{{JAVA_HOME}}/bin/java -server -XX:OnOutOfMemoryError='kill %p' -Xms4096m 
-Xmx4096m -Djava.io.tmpdir=\{{PWD}}/tmp '-Dspark.blockManager.port=9900' 
'-Dspark.driver.port=9902' '-Dspark.fileserver.port=9903' 
'-Dspark.broadcast.port=9904' '-Dspark.port.maxRetries=20' '-Dspark.ui.port=0' 
'-Dspark.executor.port=9905'_


_19/10/14 16:39:59 INFO Utils: Successfully started service 'SparkUI' on port 
35167.19/10/14 16:39:59 INFO SparkUI: Started SparkUI at_ 
[_http://10.65.170.98:35167_|http://10.65.170.98:35167/]


Even tried using a *spark-submit command with --conf spark.ui.port* does spawn 
UI in required port

{color:#172b4d}_(Run in Spark 2.4.4)_{color}
{color:#172b4d}_./bin/spark-submit --class org.apache.spark.examples.SparkPi 
--master yarn --deploy-mode cluster --driver-memory 4g --executor-memory 2g 
--executor-cores 1 --conf spark.ui.port=12345 --conf spark.driver.port=12340 
--queue default examples/jars/spark-examples_2.11-2.4.4.jar 10_{color}


_From the logs::_
_19/10/15 00:04:05 INFO ui.SparkUI: Stopped Spark web UI at 
[http://invrh74ace005.informatica.com:46622|http://invrh74ace005.informatica.com:46622/]_


_command:\{{JAVA_HOME}}/bin/java -server -Xmx2048m 
-Djava.io.tmpdir=\{{PWD}}/tmp '-Dspark.ui.port=0'  'Dspark.driver.port=12340' 
-Dspark.yarn.app.container.log.dir= -XX:OnOutOfMemoryError='kill %p' 
org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url 
spark://coarsegrainedschedu...@invrh74ace005.informatica.com:12340 
--executor-id  --hostname  --cores 1 --app-id 
application_1570992022035_0089 --user-class-path 
file:$PWD/__app__.jar1>/stdout2>/stderr_

 

Looks like the application master override this and set a JVM property before 
launch resulting in random UI port even though spark.ui.port is set.

 

In these links
 # 
[https://github.com/apache/spark/blob/master/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala]
 (line 214)
 # 
[https://github.com/cloudera/spark/blob/master/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala]
 (line 75)

I can see that the method _*run() in above files sets a system property 
UI_PORT*_ and _*spark.ui.port respectively.*_



--
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] [Updated] (SPARK-29465) Unable to configure SPARK UI (spark.ui.port) in spark yarn cluster mode.

2019-10-14 Thread Vishwas Nalka (Jira)


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

Vishwas Nalka updated SPARK-29465:
--
Description: 
 I'm trying to restrict the ports used by spark app which is launched in yarn 
cluster mode. All ports (viz. driver, executor, blockmanager) could be 
specified using the respective properties except the ui port. The spark app is 
launched using JAVA code and setting the property spark.ui.port in sparkConf 
doesn't seem to help. Even setting a JVM option -Dspark.ui.port="some_port" 
does not spawn the UI is required port. 

>From the logs of the spark app, *_the property spark.ui.port is overridden and 
>the JVM property '-Dspark.ui.port=0' is set_* even though it is never set to 
>0. 

_(Run in Spark 1.6.2) From the logs ->_

_command:LD_LIBRARY_PATH="/usr/hdp/2.6.4.0-91/hadoop/lib/native:$LD_LIBRARY_PATH"
 {{JAVA_HOME}}/bin/java -server -XX:OnOutOfMemoryError='kill %p' -Xms4096m 
-Xmx4096m -Djava.io.tmpdir={{PWD}}/tmp '-Dspark.blockManager.port=9900' 
'-Dspark.driver.port=9902' '-Dspark.fileserver.port=9903' 
'-Dspark.broadcast.port=9904' '-Dspark.port.maxRetries=20' '-Dspark.ui.port=0' 
'-Dspark.executor.port=9905'_

_19/10/14 16:39:59 INFO Utils: Successfully started service 'SparkUI' on port 
35167.19/10/14 16:39:59 INFO SparkUI: Started SparkUI at_ 
[_http://10.65.170.98:35167_|http://10.65.170.98:35167/]

Even tried using a *spark-submit command with --conf spark.ui.port* does spawn 
UI in required port

{color:#172b4d}_(Run in Spark 2.4.4)_{color}
 {color:#172b4d}_./bin/spark-submit --class org.apache.spark.examples.SparkPi 
--master yarn --deploy-mode cluster --driver-memory 4g --executor-memory 2g 
--executor-cores 1 --conf spark.ui.port=12345 --conf spark.driver.port=12340 
--queue default examples/jars/spark-examples_2.11-2.4.4.jar 10_{color}

_From the logs::_
 _19/10/15 00:04:05 INFO ui.SparkUI: Stopped Spark web UI at 
[http://invrh74ace005.informatica.com:46622|http://invrh74ace005.informatica.com:46622/]_

_command:{{JAVA_HOME}}/bin/java -server -Xmx2048m -Djava.io.tmpdir={{PWD}}/tmp 
'-Dspark.ui.port=0'  'Dspark.driver.port=12340' 
-Dspark.yarn.app.container.log.dir= -XX:OnOutOfMemoryError='kill %p' 
org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url 
spark://coarsegrainedschedu...@invrh74ace005.informatica.com:12340 
--executor-id  --hostname  --cores 1 --app-id 
application_1570992022035_0089 --user-class-path 
[file:$PWD/__app__.jar1|file://%24pwd/__app__.jar1]>/stdout2>/stderr_

 

Looks like the application master override this and set a JVM property before 
launch resulting in random UI port even though spark.ui.port is set.

In these links
 # 
[https://github.com/apache/spark/blob/master/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala]
 (line 214)
 # 
[https://github.com/cloudera/spark/blob/master/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala]
 (line 75)

I can see that the method _*run() in above files sets a system property 
UI_PORT*_ and _*spark.ui.port respectively.*_

  was:
 I'm trying to restrict the ports used by spark app which is launched in yarn 
cluster mode. All ports (viz. driver, executor, blockmanager) could be 
specified using the respective properties except the ui port. The spark app is 
launched using JAVA code and setting the property spark.ui.port in sparkConf 
doesn't seem to help. Even setting a JVM option -Dspark.ui.port="some_port" 
does not spawn the UI is required port. 


>From the logs of the spark app, *_the property spark.ui.port is overridden and 
>the JVM property '-Dspark.ui.port=0' is set_* even though it is never set to 
>0. 


_(Run in Spark 1.6.2) From the logs ->_

_command:LD_LIBRARY_PATH="/usr/hdp/2.6.4.0-91/hadoop/lib/native:$LD_LIBRARY_PATH"
 \{{JAVA_HOME}}/bin/java -server -XX:OnOutOfMemoryError='kill %p' -Xms4096m 
-Xmx4096m -Djava.io.tmpdir=\{{PWD}}/tmp '-Dspark.blockManager.port=9900' 
'-Dspark.driver.port=9902' '-Dspark.fileserver.port=9903' 
'-Dspark.broadcast.port=9904' '-Dspark.port.maxRetries=20' '-Dspark.ui.port=0' 
'-Dspark.executor.port=9905'_


_19/10/14 16:39:59 INFO Utils: Successfully started service 'SparkUI' on port 
35167.19/10/14 16:39:59 INFO SparkUI: Started SparkUI at_ 
[_http://10.65.170.98:35167_|http://10.65.170.98:35167/]


Even tried using a *spark-submit command with --conf spark.ui.port* does spawn 
UI in required port

{color:#172b4d}_(Run in Spark 2.4.4)_{color}
{color:#172b4d}_./bin/spark-submit --class org.apache.spark.examples.SparkPi 
--master yarn --deploy-mode cluster --driver-memory 4g --executor-memory 2g 
--executor-cores 1 --conf spark.ui.port=12345 --conf spark.driver.port=12340 
--queue default examples/jars/spark-examples_2.11-2.4.4.jar 10_{color}


_From the logs::_
_19/10/15 00:04:05 INFO ui.SparkUI: Stopped Spark web UI at 

[jira] [Commented] (SPARK-29464) PySpark ML should expose Params.clear() to unset a user supplied Param

2019-10-14 Thread Huaxin Gao (Jira)


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

Huaxin Gao commented on SPARK-29464:


I will work on this. Thanks!

> PySpark ML should expose Params.clear() to unset a user supplied Param
> --
>
> Key: SPARK-29464
> URL: https://issues.apache.org/jira/browse/SPARK-29464
> Project: Spark
>  Issue Type: Improvement
>  Components: ML, PySpark
>Affects Versions: 3.0.0
>Reporter: Bryan Cutler
>Priority: Minor
>
> PySpark ML currently has a private {{_clear()}} method that will unset a 
> param. This should be made public to match the Scala API and give users a way 
> to unset a user supplied param. Care should be take to ensure param values 
> are synced with the JVM and also work with save/load persistence.



--
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-10848) Applied JSON Schema Works for json RDD but not when loading json file

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun commented on SPARK-10848:
---

Hi, [~purijatin]. If you are sure the schema, there is a workaround. Please try 
the following in the above example.
{code}
scala> spark.createDataFrame(dfDFfromFile.rdd, mySchema)
res9: org.apache.spark.sql.DataFrame = [OrderID: bigint, CustomerID: int ... 5 
more fields]
{code}

> Applied JSON Schema Works for json RDD but not when loading json file
> -
>
> Key: SPARK-10848
> URL: https://issues.apache.org/jira/browse/SPARK-10848
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.5.0
>Reporter: Miklos Christine
>Priority: Minor
>
> Using a defined schema to load a json rdd works as expected. Loading the json 
> records from a file does not apply the supplied schema. Mainly the nullable 
> field isn't applied correctly. Loading from a file uses nullable=true on all 
> fields regardless of applied schema. 
> Code to reproduce:
> {code}
> import  org.apache.spark.sql.types._
> val jsonRdd = sc.parallelize(List(
>   """{"OrderID": 1, "CustomerID":452 , "OrderDate": "2015-05-16", 
> "ProductCode": "WQT648", "Qty": 5}""",
>   """{"OrderID": 2, "CustomerID":16  , "OrderDate": "2015-07-11", 
> "ProductCode": "LG4-Z5", "Qty": 10, "Discount":0.25, 
> "expressDelivery":true}"""))
> val mySchema = StructType(Array(
>   StructField(name="OrderID"   , dataType=LongType, nullable=false),
>   StructField("CustomerID", IntegerType, false),
>   StructField("OrderDate", DateType, false),
>   StructField("ProductCode", StringType, false),
>   StructField("Qty", IntegerType, false),
>   StructField("Discount", FloatType, true),
>   StructField("expressDelivery", BooleanType, true)
> ))
> val myDF = sqlContext.read.schema(mySchema).json(jsonRdd)
> val schema1 = myDF.printSchema
> val dfDFfromFile = sqlContext.read.schema(mySchema).json("Orders.json")
> val schema2 = dfDFfromFile.printSchema
> {code}
> Orders.json
> {code}
> {"OrderID": 1, "CustomerID":452 , "OrderDate": "2015-05-16", "ProductCode": 
> "WQT648", "Qty": 5}
> {"OrderID": 2, "CustomerID":16  , "OrderDate": "2015-07-11", "ProductCode": 
> "LG4-Z5", "Qty": 10, "Discount":0.25, "expressDelivery":true}
> {code}
> The behavior should be consistent. 



--
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-10848) Applied JSON Schema Works for json RDD but not when loading json file

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun edited comment on SPARK-10848 at 10/14/19 6:23 PM:
-

Hi, [~purijatin]. If you are sure that the non-nullable schema is consistent 
with your data, there is a workaround. Please try the following in the above 
example.
{code}
scala> spark.createDataFrame(dfDFfromFile.rdd, mySchema)
res9: org.apache.spark.sql.DataFrame = [OrderID: bigint, CustomerID: int ... 5 
more fields]
{code}


was (Author: dongjoon):
Hi, [~purijatin]. If you are sure the schema, there is a workaround. Please try 
the following in the above example.
{code}
scala> spark.createDataFrame(dfDFfromFile.rdd, mySchema)
res9: org.apache.spark.sql.DataFrame = [OrderID: bigint, CustomerID: int ... 5 
more fields]
{code}

> Applied JSON Schema Works for json RDD but not when loading json file
> -
>
> Key: SPARK-10848
> URL: https://issues.apache.org/jira/browse/SPARK-10848
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.5.0
>Reporter: Miklos Christine
>Priority: Minor
>
> Using a defined schema to load a json rdd works as expected. Loading the json 
> records from a file does not apply the supplied schema. Mainly the nullable 
> field isn't applied correctly. Loading from a file uses nullable=true on all 
> fields regardless of applied schema. 
> Code to reproduce:
> {code}
> import  org.apache.spark.sql.types._
> val jsonRdd = sc.parallelize(List(
>   """{"OrderID": 1, "CustomerID":452 , "OrderDate": "2015-05-16", 
> "ProductCode": "WQT648", "Qty": 5}""",
>   """{"OrderID": 2, "CustomerID":16  , "OrderDate": "2015-07-11", 
> "ProductCode": "LG4-Z5", "Qty": 10, "Discount":0.25, 
> "expressDelivery":true}"""))
> val mySchema = StructType(Array(
>   StructField(name="OrderID"   , dataType=LongType, nullable=false),
>   StructField("CustomerID", IntegerType, false),
>   StructField("OrderDate", DateType, false),
>   StructField("ProductCode", StringType, false),
>   StructField("Qty", IntegerType, false),
>   StructField("Discount", FloatType, true),
>   StructField("expressDelivery", BooleanType, true)
> ))
> val myDF = sqlContext.read.schema(mySchema).json(jsonRdd)
> val schema1 = myDF.printSchema
> val dfDFfromFile = sqlContext.read.schema(mySchema).json("Orders.json")
> val schema2 = dfDFfromFile.printSchema
> {code}
> Orders.json
> {code}
> {"OrderID": 1, "CustomerID":452 , "OrderDate": "2015-05-16", "ProductCode": 
> "WQT648", "Qty": 5}
> {"OrderID": 2, "CustomerID":16  , "OrderDate": "2015-07-11", "ProductCode": 
> "LG4-Z5", "Qty": 10, "Discount":0.25, "expressDelivery":true}
> {code}
> The behavior should be consistent. 



--
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-26570) Out of memory when InMemoryFileIndex bulkListLeafFiles

2019-10-14 Thread L. C. Hsieh (Jira)


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

L. C. Hsieh commented on SPARK-26570:
-

First from the stacktrace, I was thinking it is possible that your 
SerializableFileStatus is too many. Then it is possible to cause OOM when 
transforming them back to Status. The PR was created for that.

But from your latest posts, from jmap logs, looks like SerializableFileStatus 
do not hold too much memory. Are you sure SerializableFileStatus is the cause 
of the OOM?

> Out of memory when InMemoryFileIndex bulkListLeafFiles
> --
>
> Key: SPARK-26570
> URL: https://issues.apache.org/jira/browse/SPARK-26570
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.3.2
>Reporter: deshanxiao
>Priority: Major
> Attachments: image-2019-10-13-18-41-22-090.png, 
> image-2019-10-13-18-45-33-770.png, image-2019-10-14-10-00-27-361.png, 
> image-2019-10-14-10-32-17-949.png, image-2019-10-14-10-47-47-684.png, 
> image-2019-10-14-10-50-47-567.png, image-2019-10-14-10-51-28-374.png, 
> screenshot-1.png
>
>
> The *bulkListLeafFiles* will collect all filestatus in memory for every query 
> which may cause the oom of driver. I use the spark 2.3.2 meeting with the 
> problem. Maybe the latest one also exists the problem.



--
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] [Updated] (SPARK-29464) PySpark ML should expose Params.clear() to unset a user supplied Param

2019-10-14 Thread Bryan Cutler (Jira)


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

Bryan Cutler updated SPARK-29464:
-
Description: PySpark ML currently has a private {{_clear()}} method that 
will unset a param. This should be made public to match the Scala API and give 
users a way to unset a user supplied param. Care should be take to ensure param 
values are synced with the JVM and also work with save/load persistence.  (was: 
PySpark ML currently has a private {{_clear()}} method that will unset a param. 
This should be made public to match the Scala API and give users a way to unset 
a user supplied param.)

> PySpark ML should expose Params.clear() to unset a user supplied Param
> --
>
> Key: SPARK-29464
> URL: https://issues.apache.org/jira/browse/SPARK-29464
> Project: Spark
>  Issue Type: Improvement
>  Components: ML, PySpark
>Affects Versions: 3.0.0
>Reporter: Bryan Cutler
>Priority: Minor
>
> PySpark ML currently has a private {{_clear()}} method that will unset a 
> param. This should be made public to match the Scala API and give users a way 
> to unset a user supplied param. Care should be take to ensure param values 
> are synced with the JVM and also work with save/load persistence.



--
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] [Created] (SPARK-29464) PySpark ML should expose Params.clear() to unset a user supplied Param

2019-10-14 Thread Bryan Cutler (Jira)
Bryan Cutler created SPARK-29464:


 Summary: PySpark ML should expose Params.clear() to unset a user 
supplied Param
 Key: SPARK-29464
 URL: https://issues.apache.org/jira/browse/SPARK-29464
 Project: Spark
  Issue Type: Improvement
  Components: ML, PySpark
Affects Versions: 3.0.0
Reporter: Bryan Cutler


PySpark ML currently has a private {{_clear()}} method that will unset a param. 
This should be made public to match the Scala API and give users a way to unset 
a user supplied param.



--
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] [Resolved] (SPARK-29428) Can't persist/set None-valued param

2019-10-14 Thread Bryan Cutler (Jira)


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

Bryan Cutler resolved SPARK-29428.
--
Resolution: Not A Problem

> Can't persist/set None-valued param 
> 
>
> Key: SPARK-29428
> URL: https://issues.apache.org/jira/browse/SPARK-29428
> Project: Spark
>  Issue Type: Bug
>  Components: ML, PySpark
>Affects Versions: 2.3.2
>Reporter: Borys Biletskyy
>Priority: Major
>
> {code:java}
> import pytest
> from pyspark import keyword_only
> from pyspark.ml import Model
> from pyspark.sql import DataFrame
> from pyspark.ml.util import DefaultParamsReadable, DefaultParamsWritable
> from pyspark.ml.param.shared import HasInputCol
> from pyspark.sql.functions import *
> class NoneParamTester(Model,
>   HasInputCol,
>   DefaultParamsReadable,
>   DefaultParamsWritable
>   ):
> @keyword_only
> def __init__(self, inputCol: str = None):
> super(NoneParamTester, self).__init__()
> kwargs = self._input_kwargs
> self.setParams(**kwargs)
> @keyword_only
> def setParams(self, inputCol: str = None):
> kwargs = self._input_kwargs
> self._set(**kwargs)
> return self
> def _transform(self, data: DataFrame) -> DataFrame:
> return data
> class TestNoneParam(object):
> def test_persist_none(self, spark, temp_dir):
> path = temp_dir + '/test_model'
> model = NoneParamTester(inputCol=None)
> assert model.isDefined(model.inputCol)
> assert model.isSet(model.inputCol)
> assert model.getInputCol() is None
> model.write().overwrite().save(path)
> NoneParamTester.load(path)  # TypeError: Could not convert  'NoneType'> to string type
> def test_set_none(self, spark):
> model = NoneParamTester(inputCol=None)
> assert model.isDefined(model.inputCol)
> assert model.isSet(model.inputCol)
> assert model.getInputCol() is None
> model.set(model.inputCol, None)  # TypeError: Could not convert 
>  to string type
> {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-29428) Can't persist/set None-valued param

2019-10-14 Thread Bryan Cutler (Jira)


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

Bryan Cutler commented on SPARK-29428:
--

The usage of {{None}} in pyspark ml is a bit confusing in general, but at the 
very least it should be documented somewhere. [~borys.biletskyy] what was your 
intent in doing this, was it to unset the param?  It seems {{_clear()}} should 
be used for this, but is not public like it is in Scala. I'll close this and 
open another JIRA to expose {{clear()}}.

> Can't persist/set None-valued param 
> 
>
> Key: SPARK-29428
> URL: https://issues.apache.org/jira/browse/SPARK-29428
> Project: Spark
>  Issue Type: Bug
>  Components: ML, PySpark
>Affects Versions: 2.3.2
>Reporter: Borys Biletskyy
>Priority: Major
>
> {code:java}
> import pytest
> from pyspark import keyword_only
> from pyspark.ml import Model
> from pyspark.sql import DataFrame
> from pyspark.ml.util import DefaultParamsReadable, DefaultParamsWritable
> from pyspark.ml.param.shared import HasInputCol
> from pyspark.sql.functions import *
> class NoneParamTester(Model,
>   HasInputCol,
>   DefaultParamsReadable,
>   DefaultParamsWritable
>   ):
> @keyword_only
> def __init__(self, inputCol: str = None):
> super(NoneParamTester, self).__init__()
> kwargs = self._input_kwargs
> self.setParams(**kwargs)
> @keyword_only
> def setParams(self, inputCol: str = None):
> kwargs = self._input_kwargs
> self._set(**kwargs)
> return self
> def _transform(self, data: DataFrame) -> DataFrame:
> return data
> class TestNoneParam(object):
> def test_persist_none(self, spark, temp_dir):
> path = temp_dir + '/test_model'
> model = NoneParamTester(inputCol=None)
> assert model.isDefined(model.inputCol)
> assert model.isSet(model.inputCol)
> assert model.getInputCol() is None
> model.write().overwrite().save(path)
> NoneParamTester.load(path)  # TypeError: Could not convert  'NoneType'> to string type
> def test_set_none(self, spark):
> model = NoneParamTester(inputCol=None)
> assert model.isDefined(model.inputCol)
> assert model.isSet(model.inputCol)
> assert model.getInputCol() is None
> model.set(model.inputCol, None)  # TypeError: Could not convert 
>  to string type
> {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] [Assigned] (SPARK-29107) Add window.sql - Part 1

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun reassigned SPARK-29107:
-

Assignee: Dylan Guedes

> Add window.sql - Part 1
> ---
>
> Key: SPARK-29107
> URL: https://issues.apache.org/jira/browse/SPARK-29107
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL, Tests
>Affects Versions: 3.0.0
>Reporter: Dylan Guedes
>Assignee: Dylan Guedes
>Priority: Major
>
> In this ticket, we plan to add the regression test cases of 
> https://github.com/postgres/postgres/blob/REL_12_BETA3/src/test/regress/sql/window.sql#L1-L319



--
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] [Resolved] (SPARK-29107) Add window.sql - Part 1

2019-10-14 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun resolved SPARK-29107.
---
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 25816
[https://github.com/apache/spark/pull/25816]

> Add window.sql - Part 1
> ---
>
> Key: SPARK-29107
> URL: https://issues.apache.org/jira/browse/SPARK-29107
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL, Tests
>Affects Versions: 3.0.0
>Reporter: Dylan Guedes
>Assignee: Dylan Guedes
>Priority: Major
> Fix For: 3.0.0
>
>
> In this ticket, we plan to add the regression test cases of 
> https://github.com/postgres/postgres/blob/REL_12_BETA3/src/test/regress/sql/window.sql#L1-L319



--
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-21492) Memory leak in SortMergeJoin

2019-10-14 Thread Min Shen (Jira)


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

Min Shen edited comment on SPARK-21492 at 10/14/19 5:13 PM:


Want to further clarify the scope of the fix in PR 
[https://github.com/apache/spark/pull/25888].

Based on previous work by [~taoluo], this PR further fixes the issue for SMJ 
codegen.

[~hvanhovell] raised 2 concerns in [~taoluo]'s PR in 
[https://github.com/apache/spark/pull/23762]:
 # This only works for a SMJ with Sorts as its direct input.
 # Not sure if it safe to assume that you can close an underlying child like 
this.

The fix in PR [https://github.com/apache/spark/pull/25888] should have 
addressed concern #2, i.e. it guarantees safeness on closing the iterator for a 
Sort operator early.

This fix does not yet propagate the requests to close iterators of both child 
operators of a SMJ throughout the plan tree to reach the Sort operators.

However, with our experiences in operating all Spark workloads at LinkedIn, it 
is mostly common for SMJ not having Sort as its direct input when there are 
multiple SMJs stacked together.

In this case, even if we are not yet propagating the requests, each SMJ can 
still properly handle its local child operators which would still help to 
release the resources early.


was (Author: mshen):
Want to further clarify the scope of the fix in PR 
[https://github.com/apache/spark/pull/25888].

Based on previous work by [~taoluo], this PR further fixes the issue for SMJ 
codegen.

[~hvanhovell] raised 2 concerns in [~taoluo]'s PR in 
[https://github.com/apache/spark/pull/23762]:
 # This only works for a SMJ with Sorts as its direct input.
 # Not sure if it safe to assume that you can close an underlying child like 
this.

The fix in PR [https://github.com/apache/spark/pull/25888] should have 
addressed concern #2, i.e. it guarantees safeness on closing the iterator for a 
Sort operator early.

This fix does not yet propagate the requests to close iterators of both child 
operators of a SMJ throughout the plan tree to reach the Sort operators.

However, with our experiences in operating all Spark workloads at LI, it is 
mostly common for SMJ not having Sort as its direct input when there are 
multiple SMJs stacked together.

In this case, even if we are not yet propagating the requests, each SMJ can 
still properly handle its local child operators which would still help to 
release the resources early.

> Memory leak in SortMergeJoin
> 
>
> Key: SPARK-21492
> URL: https://issues.apache.org/jira/browse/SPARK-21492
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0, 2.3.0, 2.3.1, 3.0.0
>Reporter: Zhan Zhang
>Priority: Major
>
> In SortMergeJoin, if the iterator is not exhausted, there will be memory leak 
> caused by the Sort. The memory is not released until the task end, and cannot 
> be used by other operators causing performance drop or OOM.



--
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-21492) Memory leak in SortMergeJoin

2019-10-14 Thread Min Shen (Jira)


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

Min Shen commented on SPARK-21492:
--

Want to further clarify the scope of the fix in PR 
[https://github.com/apache/spark/pull/25888].

Based on previous work by [~taoluo], this PR further fixes the issue for SMJ 
codegen.

[~hvanhovell] raised 2 concerns in [~taoluo]'s PR in 
[https://github.com/apache/spark/pull/23762]:
 # This only works for a SMJ with Sorts as its direct input.
 # Not sure if it safe to assume that you can close an underlying child like 
this.

The fix in PR [https://github.com/apache/spark/pull/25888] should have 
addressed concern #2, i.e. it guarantees safeness on closing the iterator for a 
Sort operator early.

This fix does not yet propagate the requests to close iterators of both child 
operators of a SMJ throughout the plan tree to reach the Sort operators.

However, with our experiences in operating all Spark workloads at LI, it is 
mostly common for SMJ not having Sort as its direct input when there are 
multiple SMJs stacked together.

In this case, even if we are not yet propagating the requests, each SMJ can 
still properly handle its local child operators which would still help to 
release the resources early.

> Memory leak in SortMergeJoin
> 
>
> Key: SPARK-21492
> URL: https://issues.apache.org/jira/browse/SPARK-21492
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0, 2.3.0, 2.3.1, 3.0.0
>Reporter: Zhan Zhang
>Priority: Major
>
> In SortMergeJoin, if the iterator is not exhausted, there will be memory leak 
> caused by the Sort. The memory is not released until the task end, and cannot 
> be used by other operators causing performance drop or OOM.



--
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-21492) Memory leak in SortMergeJoin

2019-10-14 Thread Min Shen (Jira)


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

Min Shen commented on SPARK-21492:
--

We have deployed the latest version of the PR in 
[https://github.com/apache/spark/pull/25888] in LinkedIn's production clusters 
for a week now.

With the most recent changes, all corner cases seem to have been handled.

We are seeing jobs previously failing due to this issue now able to complete.

We have also observed a general reduction of spills during join in our cluster.

Want to see if the community is also working on a fix of this issue, and if so 
whether there's a timeline for the fix.

[~cloud_fan] [~jiangxb1987] [~taoluo]

> Memory leak in SortMergeJoin
> 
>
> Key: SPARK-21492
> URL: https://issues.apache.org/jira/browse/SPARK-21492
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0, 2.3.0, 2.3.1, 3.0.0
>Reporter: Zhan Zhang
>Priority: Major
>
> In SortMergeJoin, if the iterator is not exhausted, there will be memory leak 
> caused by the Sort. The memory is not released until the task end, and cannot 
> be used by other operators causing performance drop or OOM.



--
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-29206) Number of shuffle Netty server threads should be a multiple of number of chunk fetch handler threads

2019-10-14 Thread Min Shen (Jira)


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

Min Shen commented on SPARK-29206:
--

It appears that simply by making sure the number of Netty server threads is a 
multiple of the number of chunk fetch handler threads is still not enough to 
guarantee the isolation of control/data plane requests.

We are still investigating what could be leading to the SASL timeout issue.

Will update the ticket once we have more findings.

> Number of shuffle Netty server threads should be a multiple of number of 
> chunk fetch handler threads
> 
>
> Key: SPARK-29206
> URL: https://issues.apache.org/jira/browse/SPARK-29206
> Project: Spark
>  Issue Type: Improvement
>  Components: Shuffle
>Affects Versions: 3.0.0
>Reporter: Min Shen
>Priority: Major
>
> In SPARK-24355, we proposed to use a separate chunk fetch handler thread pool 
> to handle the slow-to-process chunk fetch requests in order to improve the 
> responsiveness of shuffle service for RPC requests.
> Initially, we thought by making the number of Netty server threads larger 
> than the number of chunk fetch handler threads, it would reserve some threads 
> for RPC requests thus resolving the various RPC request timeout issues we 
> experienced previously. The solution worked in our cluster initially. 
> However, as the number of Spark applications in our cluster continues to 
> increase, we saw the RPC request (SASL authentication specifically) timeout 
> issue again:
> {noformat}
> java.lang.RuntimeException: java.util.concurrent.TimeoutException: Timeout 
> waiting for task.
>   at 
> org.spark-project.guava.base.Throwables.propagate(Throwables.java:160)
>   at 
> org.apache.spark.network.client.TransportClient.sendRpcSync(TransportClient.java:278)
>   at 
> org.apache.spark.network.sasl.SaslClientBootstrap.doBootstrap(SaslClientBootstrap.java:80)
>   at 
> org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:228)
>   at 
> org.apache.spark.network.client.TransportClientFactory.createUnmanagedClient(TransportClientFactory.java:181)
>   at 
> org.apache.spark.network.shuffle.ExternalShuffleClient.registerWithShuffleServer(ExternalShuffleClient.java:141)
>   at 
> org.apache.spark.storage.BlockManager$$anonfun$registerWithExternalShuffleServer$1.apply$mcVI$sp(BlockManager.scala:218)
>  {noformat}
> After further investigation, we realized that as the number of concurrent 
> clients connecting to a shuffle service increases, it becomes _VERY_ 
> important to configure the number of Netty server threads and number of chunk 
> fetch handler threads correctly. Specifically, the number of Netty server 
> threads needs to be a multiple of the number of chunk fetch handler threads. 
> The reason is explained in details below:
> When a channel is established on the Netty server, it is registered with both 
> the Netty server default EventLoopGroup and the chunk fetch handler 
> EventLoopGroup. Once registered, this channel sticks with a given thread in 
> both EventLoopGroups, i.e. all requests from this channel is going to be 
> handled by the same thread. Right now, Spark shuffle Netty server uses the 
> default Netty strategy to select a thread from a EventLoopGroup to be 
> associated with a new channel, which is simply round-robin (Netty's 
> DefaultEventExecutorChooserFactory).
> In SPARK-24355, with the introduced chunk fetch handler thread pool, all 
> chunk fetch requests from a given channel will be first added to the task 
> queue of the chunk fetch handler thread associated with that channel. When 
> the requests get processed, the chunk fetch request handler thread will 
> submit a task to the task queue of the Netty server thread that's also 
> associated with this channel. If the number of Netty server threads is not a 
> multiple of the number of chunk fetch handler threads, it would become a 
> problem when the server has a large number of concurrent connections.
> Assume we configure the number of Netty server threads as 40 and the 
> percentage of chunk fetch handler threads as 87, which leads to 35 chunk 
> fetch handler threads. Then according to the round-robin policy, channel 0, 
> 40, 80, 120, 160, 200, 240, and 280 will all be associated with the 1st Netty 
> server thread in the default EventLoopGroup. However, since the chunk fetch 
> handler thread pool only has 35 threads, out of these 8 channels, only 
> channel 0 and 280 will be associated with the same chunk fetch handler 
> thread. Thus, channel 0, 40, 80, 120, 160, 200, 240 will all be associated 
> with different chunk fetch handler threads but associated with the same Netty 
> server 

[jira] [Resolved] (SPARK-29435) Spark 3 doesnt work with older shuffle service

2019-10-14 Thread Wenchen Fan (Jira)


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

Wenchen Fan resolved SPARK-29435.
-
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26095
[https://github.com/apache/spark/pull/26095]

> Spark 3 doesnt work with older shuffle service
> --
>
> Key: SPARK-29435
> URL: https://issues.apache.org/jira/browse/SPARK-29435
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 3.0.0
> Environment: Spark 3 from Sept 26, commit 
> 8beb736a00b004f97de7fcdf9ff09388d80fc548
> Spark 2.4.1 shuffle service in yarn 
>Reporter: koert kuipers
>Assignee: Sandeep Katta
>Priority: Major
> Fix For: 3.0.0
>
>
> SPARK-27665 introduced a change to the shuffle protocol. It also introduced a 
> setting spark.shuffle.useOldFetchProtocol which would allow spark 3 to run 
> with old shuffle service.
> However i have not gotten that to work. I have been testing with Spark 3 
> master (from Sept 26) and shuffle service from Spark 2.4.1 in yarn.
> The errors i see are for example on EMR:
> {code}
> Error occurred while fetching local blocks
> java.nio.file.NoSuchFileException: 
> /mnt1/yarn/usercache/hadoop/appcache/application_1570697024032_0058/blockmgr-d1d009b1-1c95-4e2a-9a71-0ff20078b9a8/38/shuffle_0_0_0.index
> {code}
> And on CDH5:
> {code}
> org.apache.spark.shuffle.FetchFailedException: 
> /data/9/hadoop/nm/usercache/koert/appcache/application_1568061697664_8250/blockmgr-57f28014-cdf2-431e-8e11-447ba5c2b2f2/0b/shuffle_0_0_0.index
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:596)
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:511)
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:67)
>   at 
> org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:29)
>   at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
>   at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
>   at 
> org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:31)
>   at 
> org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
>   at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
>   at scala.collection.Iterator$SliceIterator.hasNext(Iterator.scala:266)
>   at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:337)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:850)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:850)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:127)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:455)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:458)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.file.NoSuchFileException: 
> /data/9/hadoop/nm/usercache/koert/appcache/application_1568061697664_8250/blockmgr-57f28014-cdf2-431e-8e11-447ba5c2b2f2/0b/shuffle_0_0_0.index
>   at 
> sun.nio.fs.UnixException.translateToIOException(UnixException.java:86)
>   at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102)
>   at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107)
>   at 
> sun.nio.fs.UnixFileSystemProvider.newByteChannel(UnixFileSystemProvider.java:214)
>   at java.nio.file.Files.newByteChannel(Files.java:361)
>   at java.nio.file.Files.newByteChannel(Files.java:407)
>   at 
> org.apache.spark.shuffle.IndexShuffleBlockResolver.getBlockData(IndexShuffleBlockResolver.scala:204)
>   at 
> org.apache.spark.storage.BlockManager.getBlockData(BlockManager.scala:551)
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.fetchLocalBlocks(ShuffleBlockFetcherIterator.scala:349)
>   at 
> 

[jira] [Assigned] (SPARK-29435) Spark 3 doesnt work with older shuffle service

2019-10-14 Thread Wenchen Fan (Jira)


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

Wenchen Fan reassigned SPARK-29435:
---

Assignee: Sandeep Katta

> Spark 3 doesnt work with older shuffle service
> --
>
> Key: SPARK-29435
> URL: https://issues.apache.org/jira/browse/SPARK-29435
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 3.0.0
> Environment: Spark 3 from Sept 26, commit 
> 8beb736a00b004f97de7fcdf9ff09388d80fc548
> Spark 2.4.1 shuffle service in yarn 
>Reporter: koert kuipers
>Assignee: Sandeep Katta
>Priority: Major
>
> SPARK-27665 introduced a change to the shuffle protocol. It also introduced a 
> setting spark.shuffle.useOldFetchProtocol which would allow spark 3 to run 
> with old shuffle service.
> However i have not gotten that to work. I have been testing with Spark 3 
> master (from Sept 26) and shuffle service from Spark 2.4.1 in yarn.
> The errors i see are for example on EMR:
> {code}
> Error occurred while fetching local blocks
> java.nio.file.NoSuchFileException: 
> /mnt1/yarn/usercache/hadoop/appcache/application_1570697024032_0058/blockmgr-d1d009b1-1c95-4e2a-9a71-0ff20078b9a8/38/shuffle_0_0_0.index
> {code}
> And on CDH5:
> {code}
> org.apache.spark.shuffle.FetchFailedException: 
> /data/9/hadoop/nm/usercache/koert/appcache/application_1568061697664_8250/blockmgr-57f28014-cdf2-431e-8e11-447ba5c2b2f2/0b/shuffle_0_0_0.index
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:596)
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:511)
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:67)
>   at 
> org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:29)
>   at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
>   at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
>   at 
> org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:31)
>   at 
> org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
>   at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
>   at scala.collection.Iterator$SliceIterator.hasNext(Iterator.scala:266)
>   at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:337)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:850)
>   at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:850)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:127)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:455)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:458)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.file.NoSuchFileException: 
> /data/9/hadoop/nm/usercache/koert/appcache/application_1568061697664_8250/blockmgr-57f28014-cdf2-431e-8e11-447ba5c2b2f2/0b/shuffle_0_0_0.index
>   at 
> sun.nio.fs.UnixException.translateToIOException(UnixException.java:86)
>   at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102)
>   at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107)
>   at 
> sun.nio.fs.UnixFileSystemProvider.newByteChannel(UnixFileSystemProvider.java:214)
>   at java.nio.file.Files.newByteChannel(Files.java:361)
>   at java.nio.file.Files.newByteChannel(Files.java:407)
>   at 
> org.apache.spark.shuffle.IndexShuffleBlockResolver.getBlockData(IndexShuffleBlockResolver.scala:204)
>   at 
> org.apache.spark.storage.BlockManager.getBlockData(BlockManager.scala:551)
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.fetchLocalBlocks(ShuffleBlockFetcherIterator.scala:349)
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.initialize(ShuffleBlockFetcherIterator.scala:391)
>   at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.(ShuffleBlockFetcherIterator.scala:161)
>   at 
> 

[jira] [Resolved] (SPARK-29381) Add 'private' _XXXParams classes for classification & regression

2019-10-14 Thread Sean R. Owen (Jira)


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

Sean R. Owen resolved SPARK-29381.
--
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26103
[https://github.com/apache/spark/pull/26103]

> Add 'private' _XXXParams classes for classification & regression
> 
>
> Key: SPARK-29381
> URL: https://issues.apache.org/jira/browse/SPARK-29381
> Project: Spark
>  Issue Type: Sub-task
>  Components: ML, PySpark
>Affects Versions: 3.0.0
>Reporter: zhengruifeng
>Assignee: Huaxin Gao
>Priority: Major
> Fix For: 3.0.0
>
>
> ping [~huaxingao]  would you like to work on this?



--
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] [Assigned] (SPARK-29381) Add 'private' _XXXParams classes for classification & regression

2019-10-14 Thread Sean R. Owen (Jira)


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

Sean R. Owen reassigned SPARK-29381:


Assignee: Huaxin Gao

> Add 'private' _XXXParams classes for classification & regression
> 
>
> Key: SPARK-29381
> URL: https://issues.apache.org/jira/browse/SPARK-29381
> Project: Spark
>  Issue Type: Sub-task
>  Components: ML, PySpark
>Affects Versions: 3.0.0
>Reporter: zhengruifeng
>Assignee: Huaxin Gao
>Priority: Major
>
> ping [~huaxingao]  would you like to work on this?



--
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] [Updated] (SPARK-29381) Add 'private' _XXXParams classes for classification & regression

2019-10-14 Thread Sean R. Owen (Jira)


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

Sean R. Owen updated SPARK-29381:
-
Priority: Minor  (was: Major)

> Add 'private' _XXXParams classes for classification & regression
> 
>
> Key: SPARK-29381
> URL: https://issues.apache.org/jira/browse/SPARK-29381
> Project: Spark
>  Issue Type: Sub-task
>  Components: ML, PySpark
>Affects Versions: 3.0.0
>Reporter: zhengruifeng
>Assignee: Huaxin Gao
>Priority: Minor
> Fix For: 3.0.0
>
>
> ping [~huaxingao]  would you like to work on this?



--
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] [Resolved] (SPARK-29369) Accept strings without `interval` prefix in casting to intervals

2019-10-14 Thread Wenchen Fan (Jira)


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

Wenchen Fan resolved SPARK-29369.
-
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26079
[https://github.com/apache/spark/pull/26079]

> Accept strings without `interval` prefix in casting to intervals
> 
>
> Key: SPARK-29369
> URL: https://issues.apache.org/jira/browse/SPARK-29369
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Maxim Gekk
>Assignee: Maxim Gekk
>Priority: Minor
> Fix For: 3.0.0
>
>
> PostgreSQL allows strings without the `interval` prefix while casting a 
> string to an interval:
> {code}
> maxim=# select cast('1 day' as interval);
>  interval 
> --
>  1 day
> (1 row)
> maxim=# select interval '1 microsecond';
> interval 
> -
>  00:00:00.01
> (1 row)
> {code}
> See 
> https://www.postgresql.org/docs/current/datatype-datetime.html#DATATYPE-INTERVAL-INPUT



--
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] [Assigned] (SPARK-29369) Accept strings without `interval` prefix in casting to intervals

2019-10-14 Thread Wenchen Fan (Jira)


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

Wenchen Fan reassigned SPARK-29369:
---

Assignee: Maxim Gekk

> Accept strings without `interval` prefix in casting to intervals
> 
>
> Key: SPARK-29369
> URL: https://issues.apache.org/jira/browse/SPARK-29369
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Maxim Gekk
>Assignee: Maxim Gekk
>Priority: Minor
>
> PostgreSQL allows strings without the `interval` prefix while casting a 
> string to an interval:
> {code}
> maxim=# select cast('1 day' as interval);
>  interval 
> --
>  1 day
> (1 row)
> maxim=# select interval '1 microsecond';
> interval 
> -
>  00:00:00.01
> (1 row)
> {code}
> See 
> https://www.postgresql.org/docs/current/datatype-datetime.html#DATATYPE-INTERVAL-INPUT



--
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] [Created] (SPARK-29463) move v2 commands to a new file

2019-10-14 Thread Wenchen Fan (Jira)
Wenchen Fan created SPARK-29463:
---

 Summary: move v2 commands to a new file
 Key: SPARK-29463
 URL: https://issues.apache.org/jira/browse/SPARK-29463
 Project: Spark
  Issue Type: Improvement
  Components: SQL
Affects Versions: 3.0.0
Reporter: Wenchen Fan
Assignee: Wenchen Fan






--
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] [Resolved] (SPARK-29279) DataSourceV2: merge SHOW NAMESPACES and SHOW DATABASES code path

2019-10-14 Thread Wenchen Fan (Jira)


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

Wenchen Fan resolved SPARK-29279.
-
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26006
[https://github.com/apache/spark/pull/26006]

> DataSourceV2: merge SHOW NAMESPACES and SHOW DATABASES code path
> 
>
> Key: SPARK-29279
> URL: https://issues.apache.org/jira/browse/SPARK-29279
> Project: Spark
>  Issue Type: New Feature
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Terry Kim
>Assignee: Terry Kim
>Priority: Major
> Fix For: 3.0.0
>
>
> Currently,  SHOW NAMESPACES and SHOW DATABASES are separate code paths. These 
> should be merged.



--
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] [Assigned] (SPARK-29279) DataSourceV2: merge SHOW NAMESPACES and SHOW DATABASES code path

2019-10-14 Thread Wenchen Fan (Jira)


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

Wenchen Fan reassigned SPARK-29279:
---

Assignee: Terry Kim

> DataSourceV2: merge SHOW NAMESPACES and SHOW DATABASES code path
> 
>
> Key: SPARK-29279
> URL: https://issues.apache.org/jira/browse/SPARK-29279
> Project: Spark
>  Issue Type: New Feature
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Terry Kim
>Assignee: Terry Kim
>Priority: Major
>
> Currently,  SHOW NAMESPACES and SHOW DATABASES are separate code paths. These 
> should be merged.



--
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] [Created] (SPARK-29462) The data type of "array()" should be array

2019-10-14 Thread Gengliang Wang (Jira)
Gengliang Wang created SPARK-29462:
--

 Summary: The data type of "array()" should be array
 Key: SPARK-29462
 URL: https://issues.apache.org/jira/browse/SPARK-29462
 Project: Spark
  Issue Type: Task
  Components: SQL
Affects Versions: 3.0.0
Reporter: Gengliang Wang


In the current implmentation:
> spark.sql("select array()")
res0: org.apache.spark.sql.DataFrame = [array(): array]

The output type should be array



--
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-17914) Spark SQL casting to TimestampType with nanosecond results in incorrect timestamp

2019-10-14 Thread Alexandre Gattiker (Jira)


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

Alexandre Gattiker edited comment on SPARK-17914 at 10/14/19 11:37 AM:
---

As reported by other commenters, the issue is still outstanding with from_json 
in Spark 2.4.3 (Azure Databricks 5.5 LTS):

{{sc.parallelize(List("2019-10-14T{color:#00875a}09:39{color}:07.322Z")).toDF}}
{{.select('value.cast("timestamp"))}}
{{// 2019-10-14T{color:#00875a}09:39{color}:07.322+}}
{{// correct time parsing outside of from_json}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T}}{color:#00875a}{{09:39}}{color}{{:07.322Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#de350b}10:32{color}:47.000+"}}}
{{// wrong time, corresponds to 09:39+3220 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#de350b}09:44{color}:29.000+"}}}
{{// wrong time, corresponds to 09:39+322 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#00875a}09:39{color}:07.322+"}}}
{{// correct time}}


was (Author: agattiker):
As reported by other commenters, the issue is still outstanding with from_json 
in Spark 2.4.3 (Azure Databricks 5.5 LTS):

{{sc.parallelize(List("2019-10-14T{color:#00875a}09:39{color}:07.322Z")).toDF}}
{{.select('value.cast("timestamp"))}}
{{// 2019-10-14T{color:#00875a}09:39{color}:07.322+}}
{{// correct time parsing outside of from_json}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T}}{color:#00875a}{{09:39}}{color}{{:07.322Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#de350b}10:32{color}:47.000+"}}}
{{// wrong time, corresponds to 09:39+3220 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#de350b}09:44{color}:29.000+"}}}
{{// wrong time, corresponds to 09:39+322 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#00875a}09:39{color}:07.322+"}}}
{{// correct time}}

> Spark SQL casting to TimestampType with nanosecond results in incorrect 
> timestamp
> -
>
> Key: SPARK-17914
> URL: https://issues.apache.org/jira/browse/SPARK-17914
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.1
>Reporter: Oksana Romankova
>Assignee: Anton Okolnychyi
>Priority: Major
> Fix For: 2.2.0, 2.3.0
>
>
> In some cases when timestamps contain nanoseconds they will be parsed 
> incorrectly. 
> Examples: 
> "2016-05-14T15:12:14.0034567Z" -> "2016-05-14 15:12:14.034567"
> "2016-05-14T15:12:14.000345678Z" -> "2016-05-14 15:12:14.345678"
> The issue seems to be happening in DateTimeUtils.stringToTimestamp(). It 
> assumes that only 6 digit fraction of a second will be passed.
> With this being the case I would suggest either discarding nanoseconds 
> automatically, or throw an exception prompting to pre-format timestamps to 
> microsecond precision first before casting to the Timestamp.



--
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-17914) Spark SQL casting to TimestampType with nanosecond results in incorrect timestamp

2019-10-14 Thread Alexandre Gattiker (Jira)


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

Alexandre Gattiker edited comment on SPARK-17914 at 10/14/19 11:36 AM:
---

As reported by other commenters, the issue is still outstanding with from_json 
in Spark 2.4.3 (Azure Databricks 5.5 LTS):

{{sc.parallelize(List("2019-10-14T{color:#00875a}09:39{color}:07.322Z")).toDF}}
{{.select('value.cast("timestamp"))}}
{{// 2019-10-14T{color:#00875a}09:39{color}:07.322+}}
{{// correct time parsing outside of from_json}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T}}{color:#00875a}{{09:39}}{color}{{:07.322Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#de350b}10:32{color}:47.000+"}}}
{{// wrong time, corresponds to 09:39+3220 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}09:44{color}:29.000+"}}}
{{ // wrong time, corresponds to 09:39+322 seconds}}

{{ val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#00875a}09:39{color}:07.322+"}}}
{{ // correct time}}


was (Author: agattiker):
As reported by other commenters, the issue is still outstanding with from_json 
in Spark 2.4.3 (Azure Databricks 5.5 LTS):

{{sc.parallelize(List("2019-10-14T{color:#00875a}09:39{color}:07.322Z")).toDF}}
{{ .select('value.cast("timestamp"))}}
{{ // 2019-10-14T{color:#00875a}09:39{color}:07.322+}}
{{// correct time parsing outside of from_json}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T}}{color:#00875a}{{09:39}}{color}{{:07.322Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}10:32{color}:47.000+"}}}
{{ // wrong time, corresponds to 09:39+3220 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}09:44{color}:29.000+"}}}
{{ // wrong time, corresponds to 09:39+322 seconds}}

{{ val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#00875a}09:39{color}:07.322+"}}}
{{ // correct time}}

> Spark SQL casting to TimestampType with nanosecond results in incorrect 
> timestamp
> -
>
> Key: SPARK-17914
> URL: https://issues.apache.org/jira/browse/SPARK-17914
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.1
>Reporter: Oksana Romankova
>Assignee: Anton Okolnychyi
>Priority: Major
> Fix For: 2.2.0, 2.3.0
>
>
> In some cases when timestamps contain nanoseconds they will be parsed 
> incorrectly. 
> Examples: 
> "2016-05-14T15:12:14.0034567Z" -> "2016-05-14 15:12:14.034567"
> "2016-05-14T15:12:14.000345678Z" -> "2016-05-14 15:12:14.345678"
> The issue seems to be happening in DateTimeUtils.stringToTimestamp(). It 
> assumes that only 6 digit fraction of a second will be passed.
> With this being the case I would suggest either discarding nanoseconds 
> automatically, or throw an exception prompting to pre-format timestamps to 
> microsecond precision first before casting to the Timestamp.



--
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-17914) Spark SQL casting to TimestampType with nanosecond results in incorrect timestamp

2019-10-14 Thread Alexandre Gattiker (Jira)


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

Alexandre Gattiker edited comment on SPARK-17914 at 10/14/19 11:36 AM:
---

As reported by other commenters, the issue is still outstanding with from_json 
in Spark 2.4.3 (Azure Databricks 5.5 LTS):

{{sc.parallelize(List("2019-10-14T{color:#00875a}09:39{color}:07.322Z")).toDF}}
{{ .select('value.cast("timestamp"))}}
{{ // 2019-10-14T{color:#00875a}09:39{color}:07.322+}}
{{// correct time parsing outside of from_json}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T}}{color:#00875a}{{09:39}}{color}{{:07.322Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}10:32{color}:47.000+"}}}
{{ // wrong time, corresponds to 09:39+3220 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}09:44{color}:29.000+"}}}
{{ // wrong time, corresponds to 09:39+322 seconds}}

{{ val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#00875a}09:39{color}:07.322+"}}}
{{ // correct time}}


was (Author: agattiker):
As reported by other commenters, the issue is still outstanding with from_json 
in Spark 2.4.3 (Azure Databricks 5.5 LTS):

{{sc.parallelize(List("2019-10-14T{color:#00875a}09:39{color}:07.322Z")).toDF}}
{{ .select('value.cast("timestamp"))}}
{{ // 2019-10-14T{color:#00875a}09:39{color}:07.322+}}
{{// correct time parsing outside of}}{{}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T}}{color:#00875a}{{09:39}}{color}{{:07.322Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}10:32{color}:47.000+"}}}
{{ // wrong time, corresponds to 09:39+3220 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}09:44{color}:29.000+"}}}
{{ // wrong time, corresponds to 09:39+322 seconds}}

{{ val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#00875a}09:39{color}:07.322+"}}}
{{ // correct time}}

> Spark SQL casting to TimestampType with nanosecond results in incorrect 
> timestamp
> -
>
> Key: SPARK-17914
> URL: https://issues.apache.org/jira/browse/SPARK-17914
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.1
>Reporter: Oksana Romankova
>Assignee: Anton Okolnychyi
>Priority: Major
> Fix For: 2.2.0, 2.3.0
>
>
> In some cases when timestamps contain nanoseconds they will be parsed 
> incorrectly. 
> Examples: 
> "2016-05-14T15:12:14.0034567Z" -> "2016-05-14 15:12:14.034567"
> "2016-05-14T15:12:14.000345678Z" -> "2016-05-14 15:12:14.345678"
> The issue seems to be happening in DateTimeUtils.stringToTimestamp(). It 
> assumes that only 6 digit fraction of a second will be passed.
> With this being the case I would suggest either discarding nanoseconds 
> automatically, or throw an exception prompting to pre-format timestamps to 
> microsecond precision first before casting to the Timestamp.



--
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-17914) Spark SQL casting to TimestampType with nanosecond results in incorrect timestamp

2019-10-14 Thread Alexandre Gattiker (Jira)


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

Alexandre Gattiker edited comment on SPARK-17914 at 10/14/19 11:36 AM:
---

As reported by other commenters, the issue is still outstanding with from_json 
in Spark 2.4.3 (Azure Databricks 5.5 LTS):

{{sc.parallelize(List("2019-10-14T{color:#00875a}09:39{color}:07.322Z")).toDF}}
{{.select('value.cast("timestamp"))}}
{{// 2019-10-14T{color:#00875a}09:39{color}:07.322+}}
{{// correct time parsing outside of from_json}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T}}{color:#00875a}{{09:39}}{color}{{:07.322Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#de350b}10:32{color}:47.000+"}}}
{{// wrong time, corresponds to 09:39+3220 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#de350b}09:44{color}:29.000+"}}}
{{// wrong time, corresponds to 09:39+322 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#00875a}09:39{color}:07.322+"}}}
{{// correct time}}


was (Author: agattiker):
As reported by other commenters, the issue is still outstanding with from_json 
in Spark 2.4.3 (Azure Databricks 5.5 LTS):

{{sc.parallelize(List("2019-10-14T{color:#00875a}09:39{color}:07.322Z")).toDF}}
{{.select('value.cast("timestamp"))}}
{{// 2019-10-14T{color:#00875a}09:39{color}:07.322+}}
{{// correct time parsing outside of from_json}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{sc.parallelize(List("""{"a":"2019-10-14T}}{color:#00875a}{{09:39}}{color}{{:07.322Z"}""")).toDF}}
{{.select(from_json('value, schema))}}
{{// {"a":"2019-10-14T{color:#de350b}10:32{color}:47.000+"}}}
{{// wrong time, corresponds to 09:39+3220 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}09:44{color}:29.000+"}}}
{{ // wrong time, corresponds to 09:39+322 seconds}}

{{ val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#00875a}09:39{color}:07.322+"}}}
{{ // correct time}}

> Spark SQL casting to TimestampType with nanosecond results in incorrect 
> timestamp
> -
>
> Key: SPARK-17914
> URL: https://issues.apache.org/jira/browse/SPARK-17914
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.1
>Reporter: Oksana Romankova
>Assignee: Anton Okolnychyi
>Priority: Major
> Fix For: 2.2.0, 2.3.0
>
>
> In some cases when timestamps contain nanoseconds they will be parsed 
> incorrectly. 
> Examples: 
> "2016-05-14T15:12:14.0034567Z" -> "2016-05-14 15:12:14.034567"
> "2016-05-14T15:12:14.000345678Z" -> "2016-05-14 15:12:14.345678"
> The issue seems to be happening in DateTimeUtils.stringToTimestamp(). It 
> assumes that only 6 digit fraction of a second will be passed.
> With this being the case I would suggest either discarding nanoseconds 
> automatically, or throw an exception prompting to pre-format timestamps to 
> microsecond precision first before casting to the Timestamp.



--
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-17914) Spark SQL casting to TimestampType with nanosecond results in incorrect timestamp

2019-10-14 Thread Alexandre Gattiker (Jira)


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

Alexandre Gattiker commented on SPARK-17914:


As reported by other commenters, the issue is still outstanding with from_json 
in Spark 2.4.3 (Azure Databricks 5.5 LTS):

{{sc.parallelize(List("2019-10-14T{color:#00875a}09:39{color}:07.322Z")).toDF}}
{{ .select('value.cast("timestamp"))}}
{{ // 2019-10-14T{color:#00875a}09:39{color}:07.322+}}
{{// correct time parsing outside of}}{{}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T}}{color:#00875a}{{09:39}}{color}{{:07.322Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}10:32{color}:47.000+"}}}
{{ // wrong time, corresponds to 09:39+3220 seconds}}

{{val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#de350b}09:44{color}:29.000+"}}}
{{ // wrong time, corresponds to 09:39+322 seconds}}

{{ val schema = StructType(StructField("a", TimestampType, false) :: Nil)}}
{{ 
sc.parallelize(List("""{"a":"2019-10-14T{color:#00875a}09:39{color}:322000Z"}""")).toDF}}
{{ .select(from_json('value, schema))}}
{{ // {"a":"2019-10-14T{color:#00875a}09:39{color}:07.322+"}}}
{{ // correct time}}

> Spark SQL casting to TimestampType with nanosecond results in incorrect 
> timestamp
> -
>
> Key: SPARK-17914
> URL: https://issues.apache.org/jira/browse/SPARK-17914
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.1
>Reporter: Oksana Romankova
>Assignee: Anton Okolnychyi
>Priority: Major
> Fix For: 2.2.0, 2.3.0
>
>
> In some cases when timestamps contain nanoseconds they will be parsed 
> incorrectly. 
> Examples: 
> "2016-05-14T15:12:14.0034567Z" -> "2016-05-14 15:12:14.034567"
> "2016-05-14T15:12:14.000345678Z" -> "2016-05-14 15:12:14.345678"
> The issue seems to be happening in DateTimeUtils.stringToTimestamp(). It 
> assumes that only 6 digit fraction of a second will be passed.
> With this being the case I would suggest either discarding nanoseconds 
> automatically, or throw an exception prompting to pre-format timestamps to 
> microsecond precision first before casting to the Timestamp.



--
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] [Updated] (SPARK-29274) Should not coerce decimal type to double type when it's join column

2019-10-14 Thread Yuming Wang (Jira)


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

Yuming Wang updated SPARK-29274:

Summary: Should not coerce decimal type to double type when it's join 
column  (was: Can not coerce decimal type to double type when it's join key)

> Should not coerce decimal type to double type when it's join column
> ---
>
> Key: SPARK-29274
> URL: https://issues.apache.org/jira/browse/SPARK-29274
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.3.4, 2.4.4, 3.0.0
>Reporter: Yuming Wang
>Assignee: Pengfei Chang
>Priority: Major
> Attachments: image-2019-09-27-20-20-24-238.png
>
>
> How to reproduce this issue:
> {code:sql}
> create table t1 (incdata_id decimal(21,0), v string) using parquet;
> create table t2 (incdata_id string, v string) using parquet;
> explain select * from t1 join t2 on (t1.incdata_id = t2.incdata_id);
> == Physical Plan ==
> *(5) SortMergeJoin 
> [knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#31 as 
> double)))], 
> [knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#33 as 
> double)))], Inner
> :- *(2) Sort 
> [knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#31 as 
> double))) ASC NULLS FIRST], false, 0
> :  +- Exchange 
> hashpartitioning(knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#31
>  as double))), 200), true, [id=#104]
> : +- *(1) Filter isnotnull(incdata_id#31)
> :+- Scan hive default.t1 [incdata_id#31, v#32], HiveTableRelation 
> `default`.`t1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, 
> [incdata_id#31, v#32], Statistics(sizeInBytes=8.0 EiB)
> +- *(4) Sort 
> [knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#33 as 
> double))) ASC NULLS FIRST], false, 0
>+- Exchange 
> hashpartitioning(knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#33
>  as double))), 200), true, [id=#112]
>   +- *(3) Filter isnotnull(incdata_id#33)
>  +- Scan hive default.t2 [incdata_id#33, v#34], HiveTableRelation 
> `default`.`t2`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, 
> [incdata_id#33, v#34], Statistics(sizeInBytes=8.0 EiB)
> {code}
> {code:sql}
> select cast(v1 as double) as v3, cast(v2 as double) as v4,
>   cast(v1 as double) = cast(v2 as double), v1 = v2 
> from (select cast('1001636981212' as decimal(21, 0)) as v1,
>   cast('1001636981213' as decimal(21, 0)) as v2) t;
> 1.00163697E20 1.00163697E20   truefalse
> {code}
>  
> It's a realy case in our production:
> !image-2019-09-27-20-20-24-238.png|width=100%!



--
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-29461) Spark dataframe writer does not expose metrics for JDBC writer

2019-10-14 Thread Jungtaek Lim (Jira)


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

Jungtaek Lim commented on SPARK-29461:
--

I'm taking a look at this.

> Spark dataframe writer does not expose metrics for JDBC writer 
> ---
>
> Key: SPARK-29461
> URL: https://issues.apache.org/jira/browse/SPARK-29461
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 2.4.0, 2.4.4
>Reporter: ROHIT KALHANS
>Priority: Major
>
> Spark does not expose the writer metrics when using the Dataframe JDBC 
> writer. Similar instances of such bugs have been fixed in previous versions. 
> However, it seems the fix was not exhaustive since it does not cover all the 
> writers. 
> Similar bugs: 
> https://issues.apache.org/jira/browse/SPARK-21882
> https://issues.apache.org/jira/browse/SPARK-22605
>  
>  
> Console reporter output 
>  app-name.1.executor.bytesWritten
>              count = 0 
>  
>  app-name.1.executor.recordsWritten
>              count = 0  
>  
>  



--
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] [Updated] (SPARK-29461) Spark dataframe writer does not expose metrics for JDBC writer

2019-10-14 Thread Yuming Wang (Jira)


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

Yuming Wang updated SPARK-29461:

Target Version/s:   (was: 2.4.0)

> Spark dataframe writer does not expose metrics for JDBC writer 
> ---
>
> Key: SPARK-29461
> URL: https://issues.apache.org/jira/browse/SPARK-29461
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 2.4.0, 2.4.4
>Reporter: ROHIT KALHANS
>Priority: Major
>
> Spark does not expose the writer metrics when using the Dataframe JDBC 
> writer. Similar instances of such bugs have been fixed in previous versions. 
> However, it seems the fix was not exhaustive since it does not cover all the 
> writers. 
> Similar bugs: 
> https://issues.apache.org/jira/browse/SPARK-21882
> https://issues.apache.org/jira/browse/SPARK-22605
>  
>  
> Console reporter output 
>  app-name.1.executor.bytesWritten
>              count = 0 
>  
>  app-name.1.executor.recordsWritten
>              count = 0  
>  
>  



--
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] [Created] (SPARK-29461) Spark dataframe writer does not expose metrics for JDBC writer

2019-10-14 Thread ROHIT KALHANS (Jira)
ROHIT KALHANS created SPARK-29461:
-

 Summary: Spark dataframe writer does not expose metrics for JDBC 
writer 
 Key: SPARK-29461
 URL: https://issues.apache.org/jira/browse/SPARK-29461
 Project: Spark
  Issue Type: Bug
  Components: Spark Core
Affects Versions: 2.4.4, 2.4.0
Reporter: ROHIT KALHANS


Spark does not expose the writer metrics when using the Dataframe JDBC writer. 
Similar instances of such bugs have been fixed in previous versions. However, 
it seems the fix was not exhaustive since it does not cover all the writers. 

Similar bugs: 
https://issues.apache.org/jira/browse/SPARK-21882
https://issues.apache.org/jira/browse/SPARK-22605
 
 
Console reporter output 
 app-name.1.executor.bytesWritten
             count = 0 
 
 app-name.1.executor.recordsWritten
             count = 0  
 
 



--
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-29294) Update Kafka to a verison that supports Scala 2.13

2019-10-14 Thread Gabor Somogyi (Jira)


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

Gabor Somogyi commented on SPARK-29294:
---

[~srowen] I'm intended to upgrade kafka version to 2.4 (where java 11 + scala 
2.13 available) but not yet filed a jira. Since you've done this I'm happy to 
track this and do the necessary changes when needed.

> Update Kafka to a verison that supports Scala 2.13
> --
>
> Key: SPARK-29294
> URL: https://issues.apache.org/jira/browse/SPARK-29294
> Project: Spark
>  Issue Type: Sub-task
>  Components: Structured Streaming
>Affects Versions: 3.0.0
>Reporter: Sean R. Owen
>Priority: Major
>
> Kafka 2.3.0 doesn't seem to support 2.13 yet. We'll need to update when one 
> is published, to support 2.13.



--
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-29323) Add tooltip for The Executors Tab's column names in the Spark history server Page

2019-10-14 Thread ABHISHEK KUMAR GUPTA (Jira)


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

ABHISHEK KUMAR GUPTA commented on SPARK-29323:
--

[~liucht-inspur] Let me know if new sub Jira to be raised to handle this in 
Spark LIVE UI as I see this sub Jira already in RESOLVED state.

> Add tooltip for The Executors Tab's column names in the Spark history server 
> Page
> -
>
> Key: SPARK-29323
> URL: https://issues.apache.org/jira/browse/SPARK-29323
> Project: Spark
>  Issue Type: Sub-task
>  Components: Web UI
>Affects Versions: 3.0.0
>Reporter: liucht-inspur
>Assignee: liucht-inspur
>Priority: Trivial
> Fix For: 3.0.0
>
> Attachments: image-2019-10-04-09-42-14-174.png, image.png
>
>
> the spark Executors of history Tab page, the Summary part shows the line in 
> the list of title, but format is irregular.
> Some column names have tooltip, such as Storage Memory, Task Time(GC Time), 
> Input, Shuffle Read,Shuffle Write and Blacklisted, but there are still some 
> list names that have not tooltip. They are RDD Blocks, Disk Used,Cores, 
> Activity Tasks, Failed Tasks , Complete Tasks and Total Tasks. oddly, 
> Executors section below,All the column names Contains the column names above 
> have tooltip .



--
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-29460) Improve tooltip for Job Tab

2019-10-14 Thread pavithra ramachandran (Jira)


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

pavithra ramachandran commented on SPARK-29460:
---

i shall work on this

> Improve tooltip for Job Tab
> ---
>
> Key: SPARK-29460
> URL: https://issues.apache.org/jira/browse/SPARK-29460
> Project: Spark
>  Issue Type: Sub-task
>  Components: Web UI
>Affects Versions: 3.0.0
>Reporter: ABHISHEK KUMAR GUPTA
>Priority: Major
>
> [~LI,Xiao] I see there is inconsistency in the tool tip added for the column 
> across the tab. like Duration column in Job Tab does not have tooltip but 
> Duration Column in JDBC/ODBC Server tab has tooltip. 
> I submitted this Jira to handle this inconsistency in Job Tab Table Column.



--
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] [Created] (SPARK-29460) Improve tooltip for Job Tab

2019-10-14 Thread ABHISHEK KUMAR GUPTA (Jira)
ABHISHEK KUMAR GUPTA created SPARK-29460:


 Summary: Improve tooltip for Job Tab
 Key: SPARK-29460
 URL: https://issues.apache.org/jira/browse/SPARK-29460
 Project: Spark
  Issue Type: Sub-task
  Components: Web UI
Affects Versions: 3.0.0
Reporter: ABHISHEK KUMAR GUPTA


[~LI,Xiao] I see there is inconsistency in the tool tip added for the column 
across the tab. like Duration column in Job Tab does not have tooltip but 
Duration Column in JDBC/ODBC Server tab has tooltip. 

I submitted this Jira to handle this inconsistency in Job Tab Table Column.



--
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] [Assigned] (SPARK-29377) parity between scala ml tuning and python ml tuning

2019-10-14 Thread zhengruifeng (Jira)


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

zhengruifeng reassigned SPARK-29377:


Assignee: Huaxin Gao

> parity between scala ml tuning and python ml tuning
> ---
>
> Key: SPARK-29377
> URL: https://issues.apache.org/jira/browse/SPARK-29377
> Project: Spark
>  Issue Type: Sub-task
>  Components: ML, PySpark
>Affects Versions: 3.0.0
>Reporter: Huaxin Gao
>Assignee: Huaxin Gao
>Priority: Minor
>




--
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] [Resolved] (SPARK-29377) parity between scala ml tuning and python ml tuning

2019-10-14 Thread zhengruifeng (Jira)


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

zhengruifeng resolved SPARK-29377.
--
Fix Version/s: 3.0.0
   Resolution: Fixed

Issue resolved by pull request 26057
[https://github.com/apache/spark/pull/26057]

> parity between scala ml tuning and python ml tuning
> ---
>
> Key: SPARK-29377
> URL: https://issues.apache.org/jira/browse/SPARK-29377
> Project: Spark
>  Issue Type: Sub-task
>  Components: ML, PySpark
>Affects Versions: 3.0.0
>Reporter: Huaxin Gao
>Assignee: Huaxin Gao
>Priority: Minor
> Fix For: 3.0.0
>
>




--
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] [Updated] (SPARK-29323) Add tooltip for The Executors Tab's column names in the Spark history server Page

2019-10-14 Thread ABHISHEK KUMAR GUPTA (Jira)


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

ABHISHEK KUMAR GUPTA updated SPARK-29323:
-
Attachment: image.png

> Add tooltip for The Executors Tab's column names in the Spark history server 
> Page
> -
>
> Key: SPARK-29323
> URL: https://issues.apache.org/jira/browse/SPARK-29323
> Project: Spark
>  Issue Type: Sub-task
>  Components: Web UI
>Affects Versions: 3.0.0
>Reporter: liucht-inspur
>Assignee: liucht-inspur
>Priority: Trivial
> Fix For: 3.0.0
>
> Attachments: image-2019-10-04-09-42-14-174.png, image.png
>
>
> the spark Executors of history Tab page, the Summary part shows the line in 
> the list of title, but format is irregular.
> Some column names have tooltip, such as Storage Memory, Task Time(GC Time), 
> Input, Shuffle Read,Shuffle Write and Blacklisted, but there are still some 
> list names that have not tooltip. They are RDD Blocks, Disk Used,Cores, 
> Activity Tasks, Failed Tasks , Complete Tasks and Total Tasks. oddly, 
> Executors section below,All the column names Contains the column names above 
> have tooltip .



--
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] [Created] (SPARK-29459) Spark takes lot of time to initialize when the network is not connected to internet.

2019-10-14 Thread Raman (Jira)
Raman created SPARK-29459:
-

 Summary: Spark takes lot of time to initialize when the network is 
not connected to internet.
 Key: SPARK-29459
 URL: https://issues.apache.org/jira/browse/SPARK-29459
 Project: Spark
  Issue Type: Question
  Components: ML
Affects Versions: 1.5.2
Reporter: Raman


Hi, I have faced two problems in using Spark ML when my network is not 
connected to the internet (airplane mode). All these issues occurred in windows 
10 and java 8 environment.
 # Spark by default searches for the internal IP for various operations. (like 
hosting spark web ui). There was some problem initializing spark and I have 
fixed this issue by setting "spark.driver.host" to "localhost".
 # It takes around 120 secs to initialize the spark. May I know that is there 
any way to reduce this time. Like, If there is any param/config which related 
to timeout can resolve this?



--
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-29323) Add tooltip for The Executors Tab's column names in the Spark history server Page

2019-10-14 Thread ABHISHEK KUMAR GUPTA (Jira)


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

ABHISHEK KUMAR GUPTA commented on SPARK-29323:
--

This require to handle for Spark Live UI Page also. Attached is the screen shot

> Add tooltip for The Executors Tab's column names in the Spark history server 
> Page
> -
>
> Key: SPARK-29323
> URL: https://issues.apache.org/jira/browse/SPARK-29323
> Project: Spark
>  Issue Type: Sub-task
>  Components: Web UI
>Affects Versions: 3.0.0
>Reporter: liucht-inspur
>Assignee: liucht-inspur
>Priority: Trivial
> Fix For: 3.0.0
>
> Attachments: image-2019-10-04-09-42-14-174.png
>
>
> the spark Executors of history Tab page, the Summary part shows the line in 
> the list of title, but format is irregular.
> Some column names have tooltip, such as Storage Memory, Task Time(GC Time), 
> Input, Shuffle Read,Shuffle Write and Blacklisted, but there are still some 
> list names that have not tooltip. They are RDD Blocks, Disk Used,Cores, 
> Activity Tasks, Failed Tasks , Complete Tasks and Total Tasks. oddly, 
> Executors section below,All the column names Contains the column names above 
> have tooltip .



--
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