[jira] [Commented] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17784115#comment-17784115 ] koert kuipers commented on SPARK-45282: --- it does look like same issue and partitioning being the cause makes sense too > Join loses records for cached datasets > -- > > Key: SPARK-45282 > URL: https://issues.apache.org/jira/browse/SPARK-45282 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1, 3.5.0 > Environment: spark 3.4.1 on apache hadoop 3.3.6 or kubernetes 1.26 or > databricks 13.3 >Reporter: koert kuipers >Priority: Blocker > Labels: CorrectnessBug, correctness > > we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is > not present on spark 3.3.1. > it only shows up in distributed environment. i cannot replicate in unit test. > however i did get it to show up on hadoop cluster, kubernetes, and on > databricks 13.3 > the issue is that records are dropped when two cached dataframes are joined. > it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an > optimization while in spark 3.3.1 these Exhanges are still present. it seems > to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. > to reproduce on distributed cluster these settings needed are: > {code:java} > spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 > spark.sql.adaptive.coalescePartitions.parallelismFirst false > spark.sql.adaptive.enabled true > spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} > code using scala to reproduce is: > {code:java} > import java.util.UUID > import org.apache.spark.sql.functions.col > import spark.implicits._ > val data = (1 to 100).toDS().map(i => > UUID.randomUUID().toString).persist() > val left = data.map(k => (k, 1)) > val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! > println("number of left " + left.count()) > println("number of right " + right.count()) > println("number of (left join right) " + > left.toDF("key", "value1").join(right.toDF("key", "value2"), "key").count() > ) > val left1 = left > .toDF("key", "value1") > .repartition(col("key")) // comment out this line to make it work > .persist() > println("number of left1 " + left1.count()) > val right1 = right > .toDF("key", "value2") > .repartition(col("key")) // comment out this line to make it work > .persist() > println("number of right1 " + right1.count()) > println("number of (left1 join right1) " + left1.join(right1, > "key").count()) // this gives incorrect result{code} > this produces the following output: > {code:java} > number of left 100 > number of right 100 > number of (left join right) 100 > number of left1 100 > number of right1 100 > number of (left1 join right1) 859531 {code} > note that the last number (the incorrect one) actually varies depending on > settings and cluster size etc. > -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17781171#comment-17781171 ] koert kuipers commented on SPARK-45282: --- last time i checked this issue was still present in 3.4/3.5/master > Join loses records for cached datasets > -- > > Key: SPARK-45282 > URL: https://issues.apache.org/jira/browse/SPARK-45282 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1, 3.5.0 > Environment: spark 3.4.1 on apache hadoop 3.3.6 or kubernetes 1.26 or > databricks 13.3 >Reporter: koert kuipers >Priority: Blocker > Labels: CorrectnessBug, correctness > > we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is > not present on spark 3.3.1. > it only shows up in distributed environment. i cannot replicate in unit test. > however i did get it to show up on hadoop cluster, kubernetes, and on > databricks 13.3 > the issue is that records are dropped when two cached dataframes are joined. > it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an > optimization while in spark 3.3.1 these Exhanges are still present. it seems > to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. > to reproduce on distributed cluster these settings needed are: > {code:java} > spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 > spark.sql.adaptive.coalescePartitions.parallelismFirst false > spark.sql.adaptive.enabled true > spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} > code using scala to reproduce is: > {code:java} > import java.util.UUID > import org.apache.spark.sql.functions.col > import spark.implicits._ > val data = (1 to 100).toDS().map(i => > UUID.randomUUID().toString).persist() > val left = data.map(k => (k, 1)) > val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! > println("number of left " + left.count()) > println("number of right " + right.count()) > println("number of (left join right) " + > left.toDF("key", "value1").join(right.toDF("key", "value2"), "key").count() > ) > val left1 = left > .toDF("key", "value1") > .repartition(col("key")) // comment out this line to make it work > .persist() > println("number of left1 " + left1.count()) > val right1 = right > .toDF("key", "value2") > .repartition(col("key")) // comment out this line to make it work > .persist() > println("number of right1 " + right1.count()) > println("number of (left1 join right1) " + left1.join(right1, > "key").count()) // this gives incorrect result{code} > this produces the following output: > {code:java} > number of left 100 > number of right 100 > number of (left join right) 100 > number of left1 100 > number of right1 100 > number of (left1 join right1) 859531 {code} > note that the last number (the incorrect one) actually varies depending on > settings and cluster size etc. > -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17769893#comment-17769893 ] koert kuipers edited comment on SPARK-45282 at 9/28/23 4:07 AM: yes i can reproduce it. master branch on commit: {code:java} commit 7e8aafd2c0f1f6fcd03a69afe2b85fd3fda95d20 (HEAD -> master, upstream/master) Author: lanmengran1 Date: Tue Sep 26 21:01:02 2023 -0500 [SPARK-45334][SQL] Remove misleading comment in parquetSchemaConverter {code} i build spark for k8s using: {code:java} $ dev/make-distribution.sh --name kubernetes --tgz -Pkubernetes -Phadoop-cloud {code} created docker container using Dockerfile provided in resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile launch pod and shell inside: {code:java} 185@proxy:~/work-dir$ export SPARK_LOCAL_HOSTNAME=$(hostname -i 185@proxy:~/work-dir$ export SPARK_PUBLIC_DNS=$(hostname -i) 185@proxy:~/work-dir$ /opt/spark/bin/spark-shell --master k8s://https://kubernetes.default:443 --deploy-mode client --num-executors 4 --executor-memory 2G --conf spark.serializer=org.apache.spark.serializer.KryoSerializer --conf spark.kubernetes.namespace=default --conf spark.sql.adaptive.coalescePartitions.parallelismFirst=false --conf spark.sql.adaptive.enabled=true --conf spark.sql.adaptive.advisoryPartitionSizeInBytes=33554432 --conf spark.sql.optimizer.canChangeCachedPlanOutputPartitioning=true --conf spark.kubernetes.container.image=.dkr.ecr.us-east-1.amazonaws.com/spark:4.0.0-SNAPSHOT 23/09/28 03:44:57 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 4.0.0-SNAPSHOT /_/ Using Scala version 2.13.11 (OpenJDK 64-Bit Server VM, Java 21) Type in expressions to have them evaluated. Type :help for more information. Spark context Web UI available at http://10.177.71.94:4040 Spark context available as 'sc' (master = k8s://https://kubernetes.default:443, app id = spark-5ab0957571944828866a2f23068ff180). Spark session available as 'spark'.scala> :paste // Entering paste mode (ctrl-D to finish)import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result // Exiting paste mode, now interpreting. 23/09/28 03:45:30 WARN TaskSetManager: Stage 0 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. 23/09/28 03:45:34 WARN TaskSetManager: Stage 1 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of left 100 23/09/28 03:45:36 WARN TaskSetManager: Stage 4 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of right 100 23/09/28 03:45:39 WARN TaskSetManager: Stage 7 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. 23/09/28 03:45:40 WARN TaskSetManager: Stage 8 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of (left join right) 100 23/09/28 03:45:45 WARN TaskSetManager: Stage 16 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of left1 100 23/09/28 03:45:48 WARN TaskSetManager: Stage 24 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of right1 100 number of (left1 join right1) 850735 import
[jira] [Comment Edited] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17769893#comment-17769893 ] koert kuipers edited comment on SPARK-45282 at 9/28/23 4:06 AM: yes i can reproduce it. master branch on commit: {code:java} commit 7e8aafd2c0f1f6fcd03a69afe2b85fd3fda95d20 (HEAD -> master, upstream/master) Author: lanmengran1 Date: Tue Sep 26 21:01:02 2023 -0500 [SPARK-45334][SQL] Remove misleading comment in parquetSchemaConverter {code} i build spark for k8s using: {code:java} $ dev/make-distribution.sh --name kubernetes --tgz -Pkubernetes -Phadoop-cloud {code} created docker container using Dockerfile provided in resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile launch pod and shell inside: {code:java} 185@proxy:~/work-dir$ export SPARK_LOCAL_HOSTNAME=$(hostname -i 185@proxy:~/work-dir$ export SPARK_PUBLIC_DNS=$(hostname -i) 185@proxy:~/work-dir$ /opt/spark/bin/spark-shell --master k8s://https://kubernetes.default:443 --deploy-mode client --num-executors 4 --executor-memory 2G --conf spark.serializer=org.apache.spark.serializer.KryoSerializer --conf spark.kubernetes.namespace=default --conf spark.sql.adaptive.coalescePartitions.parallelismFirst=false --conf spark.sql.adaptive.enabled=true --conf spark.sql.adaptive.advisoryPartitionSizeInBytes=33554432 --conf spark.sql.optimizer.canChangeCachedPlanOutputPartitioning=true --conf spark.kubernetes.container.image=.dkr.ecr.us-east-1.amazonaws.com/spark:4.0.0-SNAPSHOT 23/09/28 03:44:57 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 4.0.0-SNAPSHOT /_/ Using Scala version 2.13.11 (OpenJDK 64-Bit Server VM, Java 21) Type in expressions to have them evaluated. Type :help for more information. Spark context Web UI available at http://10.177.71.94:4040 Spark context available as 'sc' (master = k8s://https://kubernetes.default:443, app id = spark-5ab0957571944828866a2f23068ff180). Spark session available as 'spark'.scala> :paste // Entering paste mode (ctrl-D to finish)import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result // Exiting paste mode, now interpreting. 23/09/28 03:45:30 WARN TaskSetManager: Stage 0 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. 23/09/28 03:45:34 WARN TaskSetManager: Stage 1 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of left 100 23/09/28 03:45:36 WARN TaskSetManager: Stage 4 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of right 100 23/09/28 03:45:39 WARN TaskSetManager: Stage 7 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. 23/09/28 03:45:40 WARN TaskSetManager: Stage 8 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of (left join right) 100 23/09/28 03:45:45 WARN TaskSetManager: Stage 16 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of left1 100 23/09/28 03:45:48 WARN TaskSetManager: Stage 24 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of right1 100 number of (left1 join right1) 850735 import
[jira] [Commented] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17769893#comment-17769893 ] koert kuipers commented on SPARK-45282: --- yes i can reproduce it. master branch on commit: {code:java} commit 7e8aafd2c0f1f6fcd03a69afe2b85fd3fda95d20 (HEAD -> master, upstream/master) Author: lanmengran1 Date: Tue Sep 26 21:01:02 2023 -0500 [SPARK-45334][SQL] Remove misleading comment in parquetSchemaConverter {code} i build spark for k8s using: {code:java} $ dev/make-distribution.sh --name kubernetes --tgz -Pkubernetes -Phadoop-cloud {code} created docker container using Dockerfile provided in resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile launch pod and shell inside: {code:java} 185@proxy:~/work-dir$ export SPARK_LOCAL_HOSTNAME=$(hostname -i 185@proxy:~/work-dir$ export SPARK_PUBLIC_DNS=$(hostname -i) 185@proxy:~/work-dir$ /opt/spark/bin/spark-shell --master k8s://https://kubernetes.default:443 --deploy-mode client --num-executors 4 --executor-memory 2G --conf spark.serializer=org.apache.spark.serializer.KryoSerializer --conf spark.kubernetes.namespace=default --conf spark.sql.adaptive.coalescePartitions.parallelismFirst=false --conf spark.sql.adaptive.enabled=true --conf spark.sql.adaptive.advisoryPartitionSizeInBytes=33554432 --conf spark.sql.optimizer.canChangeCachedPlanOutputPartitioning=true --conf spark.kubernetes.container.image=.dkr.ecr.us-east-1.amazonaws.com/spark:4.0.0-SNAPSHOT 23/09/28 03:44:57 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 4.0.0-SNAPSHOT /_/ Using Scala version 2.13.11 (OpenJDK 64-Bit Server VM, Java 21) Type in expressions to have them evaluated. Type :help for more information. Spark context Web UI available at http://10.177.71.94:4040 Spark context available as 'sc' (master = k8s://https://kubernetes.default:443, app id = spark-5ab0957571944828866a2f23068ff180). Spark session available as 'spark'.scala> :paste // Entering paste mode (ctrl-D to finish)import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result // Exiting paste mode, now interpreting. 23/09/28 03:45:30 WARN TaskSetManager: Stage 0 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. 23/09/28 03:45:34 WARN TaskSetManager: Stage 1 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of left 100 23/09/28 03:45:36 WARN TaskSetManager: Stage 4 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of right 100 23/09/28 03:45:39 WARN TaskSetManager: Stage 7 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. 23/09/28 03:45:40 WARN TaskSetManager: Stage 8 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of (left join right) 100 23/09/28 03:45:45 WARN TaskSetManager: Stage 16 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of left1 100 23/09/28 03:45:48 WARN TaskSetManager: Stage 24 contains a task of very large size (6631 KiB). The maximum recommended task size is 1000 KiB. number of right1 100 number of (left1 join right1) 850735 import java.util.UUID import
[jira] [Comment Edited] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17768123#comment-17768123 ] koert kuipers edited comment on SPARK-45282 at 9/22/23 7:04 PM: after reverting SPARK-41048 the issue went away. was (Author: koert): after reverting SPARK-41048 the issue went away. so i think this is the cause. > Join loses records for cached datasets > -- > > Key: SPARK-45282 > URL: https://issues.apache.org/jira/browse/SPARK-45282 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1, 3.5.0 > Environment: spark 3.4.1 on apache hadoop 3.3.6 or kubernetes 1.26 or > databricks 13.3 >Reporter: koert kuipers >Priority: Major > Labels: CorrectnessBug, correctness > > we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is > not present on spark 3.3.1. > it only shows up in distributed environment. i cannot replicate in unit test. > however i did get it to show up on hadoop cluster, kubernetes, and on > databricks 13.3 > the issue is that records are dropped when two cached dataframes are joined. > it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an > optimization while in spark 3.3.1 these Exhanges are still present. it seems > to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. > to reproduce on distributed cluster these settings needed are: > {code:java} > spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 > spark.sql.adaptive.coalescePartitions.parallelismFirst false > spark.sql.adaptive.enabled true > spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} > code using scala to reproduce is: > {code:java} > import java.util.UUID > import org.apache.spark.sql.functions.col > import spark.implicits._ > val data = (1 to 100).toDS().map(i => > UUID.randomUUID().toString).persist() > val left = data.map(k => (k, 1)) > val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! > println("number of left " + left.count()) > println("number of right " + right.count()) > println("number of (left join right) " + > left.toDF("key", "value1").join(right.toDF("key", "value2"), "key").count() > ) > val left1 = left > .toDF("key", "value1") > .repartition(col("key")) // comment out this line to make it work > .persist() > println("number of left1 " + left1.count()) > val right1 = right > .toDF("key", "value2") > .repartition(col("key")) // comment out this line to make it work > .persist() > println("number of right1 " + right1.count()) > println("number of (left1 join right1) " + left1.join(right1, > "key").count()) // this gives incorrect result{code} > this produces the following output: > {code:java} > number of left 100 > number of right 100 > number of (left join right) 100 > number of left1 100 > number of right1 100 > number of (left1 join right1) 859531 {code} > note that the last number (the incorrect one) actually varies depending on > settings and cluster size etc. > -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-45282: -- Description: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code using scala to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "value1").join(right.toDF("key", "value2"), "key").count() ) val left1 = left .toDF("key", "value1") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "value2") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} this produces the following output: {code:java} number of left 100 number of right 100 number of (left join right) 100 number of left1 100 number of right1 100 number of (left1 join right1) 859531 {code} note that the last number (the incorrect one) actually varies depending on settings and cluster size etc. was: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code using scala to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} this produces the following output: {code:java} number of left 100 number of right 100 number of (left join right) 100 number of left1 100 number of right1 100 number of (left1 join right1) 859531 {code} note that the last number (the incorrect one) actually varies depending on settings and cluster size etc. > Join loses records for cached datasets > -- > > Key: SPARK-45282 > URL: https://issues.apache.org/jira/browse/SPARK-45282 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1, 3.5.0 > Environment: spark 3.4.1 on apache hadoop 3.3.6 or
[jira] [Updated] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-45282: -- Description: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code using scala to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} this produces the following output: {code:java} number of left 100 number of right 100 number of (left join right) 100 number of left1 100 number of right1 100 number of (left1 join right1) 859531 {code} note that the last number (the incorrect one) actually varies depending on settings and cluster size etc. was: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code using scala 2.13 to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} this produces the following output: {code:java} number of left 100 number of right 100 number of (left join right) 100 number of left1 100 number of right1 100 number of (left1 join right1) 859531 {code} note that the last number (the incorrect one) actually varies depending on settings and cluster size etc. > Join loses records for cached datasets > -- > > Key: SPARK-45282 > URL: https://issues.apache.org/jira/browse/SPARK-45282 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1, 3.5.0 > Environment: spark 3.4.1 on apache hadoop 3.3.6 or
[jira] [Commented] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17768123#comment-17768123 ] koert kuipers commented on SPARK-45282: --- after reverting SPARK-41048 the issue went away. so i think this is the cause. > Join loses records for cached datasets > -- > > Key: SPARK-45282 > URL: https://issues.apache.org/jira/browse/SPARK-45282 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1, 3.5.0 > Environment: spark 3.4.1 on apache hadoop 3.3.6 or kubernetes 1.26 or > databricks 13.3 >Reporter: koert kuipers >Priority: Major > Labels: CorrectnessBug, correctness > > we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is > not present on spark 3.3.1. > it only shows up in distributed environment. i cannot replicate in unit test. > however i did get it to show up on hadoop cluster, kubernetes, and on > databricks 13.3 > the issue is that records are dropped when two cached dataframes are joined. > it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an > optimization while in spark 3.3.1 these Exhanges are still present. it seems > to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. > to reproduce on distributed cluster these settings needed are: > {code:java} > spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 > spark.sql.adaptive.coalescePartitions.parallelismFirst false > spark.sql.adaptive.enabled true > spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} > code using scala 2.13 to reproduce is: > {code:java} > import java.util.UUID > import org.apache.spark.sql.functions.col > import spark.implicits._ > val data = (1 to 100).toDS().map(i => > UUID.randomUUID().toString).persist() > val left = data.map(k => (k, 1)) > val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! > println("number of left " + left.count()) > println("number of right " + right.count()) > println("number of (left join right) " + > left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() > ) > val left1 = left > .toDF("key", "vertex") > .repartition(col("key")) // comment out this line to make it work > .persist() > println("number of left1 " + left1.count()) > val right1 = right > .toDF("key", "state") > .repartition(col("key")) // comment out this line to make it work > .persist() > println("number of right1 " + right1.count()) > println("number of (left1 join right1) " + left1.join(right1, > "key").count()) // this gives incorrect result{code} > this produces the following output: > {code:java} > number of left 100 > number of right 100 > number of (left join right) 100 > number of left1 100 > number of right1 100 > number of (left1 join right1) 859531 {code} > note that the last number (the incorrect one) actually varies depending on > settings and cluster size etc. > -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-45282: -- Description: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code using scala 2.13 to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} this produces the following output: {code:java} number of left 100 number of right 100 number of (left join right) 100 number of left1 100 number of right1 100 number of (left1 join right1) 859531 {code} note that the last number (the incorrect one) actually varies depending on settings and cluster size etc. was: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} this produces the following output: {code:java} number of left 100 number of right 100 number of (left join right) 100 number of left1 100 number of right1 100 number of (left1 join right1) 859531 {code} note that the last number (the incorrect one) actually varies depending on settings and cluster size etc. > Join loses records for cached datasets > -- > > Key: SPARK-45282 > URL: https://issues.apache.org/jira/browse/SPARK-45282 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1, 3.5.0 > Environment: spark 3.4.1 on apache hadoop 3.3.6 or kubernetes
[jira] [Updated] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-45282: -- Description: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} this produces the following output: {code:java} number of left 100 number of right 100 number of (left join right) 100 number of left1 100 number of right1 100 number of (left1 join right1) 859531 {code} note that the last number (the incorrect one) actually varies depending on settings and cluster size etc. was: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} > Join loses records for cached datasets > -- > > Key: SPARK-45282 > URL: https://issues.apache.org/jira/browse/SPARK-45282 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1, 3.5.0 > Environment: spark 3.4.1 on apache hadoop 3.3.6 or kubernetes 1.26 or > databricks 13.3 >Reporter: koert kuipers >Priority: Major > Labels: CorrectnessBug, correctness > > we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is > not present on spark 3.3.1. > it only shows up in distributed environment. i cannot replicate in unit test. > however
[jira] [Updated] (SPARK-45282) Join loses records for cached datasets
[ https://issues.apache.org/jira/browse/SPARK-45282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-45282: -- Description: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} was: we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} > Join loses records for cached datasets > -- > > Key: SPARK-45282 > URL: https://issues.apache.org/jira/browse/SPARK-45282 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1, 3.5.0 > Environment: spark 3.4.1 on apache hadoop 3.3.6 or kubernetes 1.26 or > databricks 13.3 >Reporter: koert kuipers >Priority: Major > Labels: CorrectnessBug, correctness > > we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is > not present on spark 3.3.1. > it only shows up in distributed environment. i cannot replicate in unit test. > however i did get it to show up on hadoop cluster, kubernetes, and on > databricks 13.3 > the issue is that records are dropped when two cached dataframes are joined. > it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an > optimization while in spark 3.3.1 these Exhanges are still present. it seems > to
[jira] [Created] (SPARK-45282) Join loses records for cached datasets
koert kuipers created SPARK-45282: - Summary: Join loses records for cached datasets Key: SPARK-45282 URL: https://issues.apache.org/jira/browse/SPARK-45282 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.5.0, 3.4.1 Environment: spark 3.4.1 on apache hadoop 3.3.6 or kubernetes 1.26 or databricks 13.3 Reporter: koert kuipers we observed this issue on spark 3.4.1 but it is also present on 3.5.0. it is not present on spark 3.3.1. it only shows up in distributed environment. i cannot replicate in unit test. however i did get it to show up on hadoop cluster, kubernetes, and on databricks 13.3 the issue is that records are dropped when two cached dataframes are joined. it seems in spark 3.4.1 in queryplan some Exchanges are dropped as an optimization while in spark 3.3.1 these Exhanges are still present. it seems to be an issue with AQE with canChangeCachedPlanOutputPartitioning=true. to reproduce on distributed cluster these settings needed are: {code:java} spark.sql.adaptive.advisoryPartitionSizeInBytes 33554432 spark.sql.adaptive.coalescePartitions.parallelismFirst false spark.sql.adaptive.enabled true spark.sql.optimizer.canChangeCachedPlanOutputPartitioning true {code} code to reproduce is: {code:java} import java.util.UUID import org.apache.spark.sql.functions.col import spark.implicits._ val data = (1 to 100).toDS().map(i => UUID.randomUUID().toString).persist() val left = data.map(k => (k, 1)) val right = data.map(k => (k, k)) // if i change this to k => (k, 1) it works! println("number of left " + left.count()) println("number of right " + right.count()) println("number of (left join right) " + left.toDF("key", "vertex").join(right.toDF("key", "state"), "key").count() ) val left1 = left .toDF("key", "vertex") .repartition(col("key")) // comment out this line to make it work .persist() println("number of left1 " + left1.count()) val right1 = right .toDF("key", "state") .repartition(col("key")) // comment out this line to make it work .persist() println("number of right1 " + right1.count()) println("number of (left1 join right1) " + left1.join(right1, "key").count()) // this gives incorrect result{code} -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-44323) Scala None shows up as null for Aggregator BUF or OUT
[ https://issues.apache.org/jira/browse/SPARK-44323?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17741400#comment-17741400 ] koert kuipers commented on SPARK-44323: --- not sure why pullreq isnt getting linked automatically but its here: https://github.com/apache/spark/pull/41903 > Scala None shows up as null for Aggregator BUF or OUT > --- > > Key: SPARK-44323 > URL: https://issues.apache.org/jira/browse/SPARK-44323 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1 >Reporter: koert kuipers >Priority: Major > > when doing an upgrade from spark 3.3.1 to spark 3.4.1 we suddenly started > getting null pointer exceptions in Aggregators (classes extending > org.apache.spark.sql.expressions.Aggregator) that use scala Option for BUF > and/or OUT. basically None is now showing up as null. > after adding a simple test case and doing a binary search on commits we > landed on SPARK-37829 being the cause. > we observed the issue at first with NPE inside Aggregator.merge because None > was null. i am having a hard time replicating that in a spark unit test, but > i did manage to get a None become null in the output. simple test that now > fails: > > {code:java} > diff --git > a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > index e9daa825dd4..a1959d7065d 100644 > --- > a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > +++ > b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > @@ -228,6 +228,16 @@ case class FooAgg(s: Int) extends Aggregator[Row, Int, > Int] { > def outputEncoder: Encoder[Int] = Encoders.scalaInt > } > > +object OptionStringAgg extends Aggregator[Option[String], Option[String], > Option[String]] { > + override def zero: Option[String] = None > + override def reduce(b: Option[String], a: Option[String]): Option[String] > = merge(b, a) > + override def finish(reduction: Option[String]): Option[String] = reduction > + override def merge(b1: Option[String], b2: Option[String]): Option[String] > = > + b1.map{ b1v => b2.map{ b2v => b1v ++ b2v }.getOrElse(b1v) }.orElse(b2) > + override def bufferEncoder: Encoder[Option[String]] = ExpressionEncoder() > + override def outputEncoder: Encoder[Option[String]] = ExpressionEncoder() > +} > + > class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { > import testImplicits._ > > @@ -432,4 +442,15 @@ class DatasetAggregatorSuite extends QueryTest with > SharedSparkSession { > val agg = df.select(mode(col("a"))).as[String] > checkDataset(agg, "3") > } > + > + test("typed aggregation: option string") { > + val ds = Seq((1, Some("a")), (1, None), (1, Some("c")), (2, None)).toDS() > + > + checkDataset( > + ds.groupByKey(_._1).mapValues(_._2).agg( > + OptionStringAgg.toColumn > + ), > + (1, Some("ac")), (2, None) > + ) > + } > } > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-37829) An outer-join using joinWith on DataFrames returns Rows with null fields instead of null values
[ https://issues.apache.org/jira/browse/SPARK-37829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17740726#comment-17740726 ] koert kuipers edited comment on SPARK-37829 at 7/8/23 4:19 PM: --- since this (admittedly somewhat weird) behavior of returning a Row with null values has been present since spark 3.0.x (a major breaking release, and 3 years ago) i would argue this is the default behavior and this jira introduces a breaking change. basically i am saying if one argues this was a breaking change in going from spark 2.x to 3.x then i agree but a major version can make a breaking change. introducing a fix in 3.4.1 that reverts that breaking change is basically introducing a breaking change going from 3.4.0 to 3.4.1 which is worse in my opinion. also expressionencoders are used for other purposes than dataset joins and now we find nulls popping up in places they should not. this is how i ran into this issue. was (Author: koert): since this (admittedly somewhat weird) behavior of returning a Row with null values has been present since spark 3.0.x (a major breaking release, and 3 years ago) i would argue this is the default behavior and this jira introduces a breaking change. also expressionencoders are used for other purposes than dataset joins and now we find nulls popping up in places they should not. > An outer-join using joinWith on DataFrames returns Rows with null fields > instead of null values > --- > > Key: SPARK-37829 > URL: https://issues.apache.org/jira/browse/SPARK-37829 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0, 3.0.1, 3.0.2, 3.0.3, 3.1.0, 3.1.1, 3.1.2, 3.2.0 >Reporter: Clément de Groc >Assignee: Jason Xu >Priority: Major > Fix For: 3.3.3, 3.4.1, 3.5.0 > > > Doing an outer-join using {{joinWith}} on {{{}DataFrame{}}}s used to return > missing values as {{null}} in Spark 2.4.8, but returns them as {{Rows}} with > {{null}} values in Spark 3+. > The issue can be reproduced with [the following > test|https://github.com/cdegroc/spark/commit/79f4d6a1ec6c69b10b72dbc8f92ab6490d5ef5e5] > that succeeds on Spark 2.4.8 but fails starting from Spark 3.0.0. > The problem only arises when working with DataFrames: Datasets of case > classes work as expected as demonstrated by [this other > test|https://github.com/apache/spark/blob/v3.0.0/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala#L1200-L1223]. > I couldn't find an explanation for this change in the Migration guide so I'm > assuming this is a bug. > A {{git bisect}} pointed me to [that > commit|https://github.com/apache/spark/commit/cd92f25be5a221e0d4618925f7bc9dfd3bb8cb59]. > Reverting the commit solves the problem. > A similar solution, but without reverting, is shown > [here|https://github.com/cdegroc/spark/commit/684c675bf070876a475a9b225f6c2f92edce4c8a]. > Happy to help if you think of another approach / can provide some guidance. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-44323) Scala None shows up as null for Aggregator BUF or OUT
[ https://issues.apache.org/jira/browse/SPARK-44323?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17740744#comment-17740744 ] koert kuipers commented on SPARK-44323: --- i think the issue is that Nones inside Tuples now become null. so its the usage of nullSafe inside the childrenDeserializers for tuples introduced in https://github.com/apache/spark/pull/40755 > Scala None shows up as null for Aggregator BUF or OUT > --- > > Key: SPARK-44323 > URL: https://issues.apache.org/jira/browse/SPARK-44323 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1 >Reporter: koert kuipers >Priority: Major > > when doing an upgrade from spark 3.3.1 to spark 3.4.1 we suddenly started > getting null pointer exceptions in Aggregators (classes extending > org.apache.spark.sql.expressions.Aggregator) that use scala Option for BUF > and/or OUT. basically None is now showing up as null. > after adding a simple test case and doing a binary search on commits we > landed on SPARK-37829 being the cause. > we observed the issue at first with NPE inside Aggregator.merge because None > was null. i am having a hard time replicating that in a spark unit test, but > i did manage to get a None become null in the output. simple test that now > fails: > > {code:java} > diff --git > a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > index e9daa825dd4..a1959d7065d 100644 > --- > a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > +++ > b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > @@ -228,6 +228,16 @@ case class FooAgg(s: Int) extends Aggregator[Row, Int, > Int] { > def outputEncoder: Encoder[Int] = Encoders.scalaInt > } > > +object OptionStringAgg extends Aggregator[Option[String], Option[String], > Option[String]] { > + override def zero: Option[String] = None > + override def reduce(b: Option[String], a: Option[String]): Option[String] > = merge(b, a) > + override def finish(reduction: Option[String]): Option[String] = reduction > + override def merge(b1: Option[String], b2: Option[String]): Option[String] > = > + b1.map{ b1v => b2.map{ b2v => b1v ++ b2v }.getOrElse(b1v) }.orElse(b2) > + override def bufferEncoder: Encoder[Option[String]] = ExpressionEncoder() > + override def outputEncoder: Encoder[Option[String]] = ExpressionEncoder() > +} > + > class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { > import testImplicits._ > > @@ -432,4 +442,15 @@ class DatasetAggregatorSuite extends QueryTest with > SharedSparkSession { > val agg = df.select(mode(col("a"))).as[String] > checkDataset(agg, "3") > } > + > + test("typed aggregation: option string") { > + val ds = Seq((1, Some("a")), (1, None), (1, Some("c")), (2, None)).toDS() > + > + checkDataset( > + ds.groupByKey(_._1).mapValues(_._2).agg( > + OptionStringAgg.toColumn > + ), > + (1, Some("ac")), (2, None) > + ) > + } > } > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-44323) Scala None shows up as null for Aggregator BUF or OUT
[ https://issues.apache.org/jira/browse/SPARK-44323?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17740744#comment-17740744 ] koert kuipers edited comment on SPARK-44323 at 7/6/23 6:28 PM: --- i think the issue is that Nones inside Tuples now become nulls. so its the usage of nullSafe inside the childrenDeserializers for tuples introduced in [https://github.com/apache/spark/pull/40755] was (Author: koert): i think the issue is that Nones inside Tuples now become null. so its the usage of nullSafe inside the childrenDeserializers for tuples introduced in https://github.com/apache/spark/pull/40755 > Scala None shows up as null for Aggregator BUF or OUT > --- > > Key: SPARK-44323 > URL: https://issues.apache.org/jira/browse/SPARK-44323 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1 >Reporter: koert kuipers >Priority: Major > > when doing an upgrade from spark 3.3.1 to spark 3.4.1 we suddenly started > getting null pointer exceptions in Aggregators (classes extending > org.apache.spark.sql.expressions.Aggregator) that use scala Option for BUF > and/or OUT. basically None is now showing up as null. > after adding a simple test case and doing a binary search on commits we > landed on SPARK-37829 being the cause. > we observed the issue at first with NPE inside Aggregator.merge because None > was null. i am having a hard time replicating that in a spark unit test, but > i did manage to get a None become null in the output. simple test that now > fails: > > {code:java} > diff --git > a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > index e9daa825dd4..a1959d7065d 100644 > --- > a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > +++ > b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > @@ -228,6 +228,16 @@ case class FooAgg(s: Int) extends Aggregator[Row, Int, > Int] { > def outputEncoder: Encoder[Int] = Encoders.scalaInt > } > > +object OptionStringAgg extends Aggregator[Option[String], Option[String], > Option[String]] { > + override def zero: Option[String] = None > + override def reduce(b: Option[String], a: Option[String]): Option[String] > = merge(b, a) > + override def finish(reduction: Option[String]): Option[String] = reduction > + override def merge(b1: Option[String], b2: Option[String]): Option[String] > = > + b1.map{ b1v => b2.map{ b2v => b1v ++ b2v }.getOrElse(b1v) }.orElse(b2) > + override def bufferEncoder: Encoder[Option[String]] = ExpressionEncoder() > + override def outputEncoder: Encoder[Option[String]] = ExpressionEncoder() > +} > + > class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { > import testImplicits._ > > @@ -432,4 +442,15 @@ class DatasetAggregatorSuite extends QueryTest with > SharedSparkSession { > val agg = df.select(mode(col("a"))).as[String] > checkDataset(agg, "3") > } > + > + test("typed aggregation: option string") { > + val ds = Seq((1, Some("a")), (1, None), (1, Some("c")), (2, None)).toDS() > + > + checkDataset( > + ds.groupByKey(_._1).mapValues(_._2).agg( > + OptionStringAgg.toColumn > + ), > + (1, Some("ac")), (2, None) > + ) > + } > } > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-37829) An outer-join using joinWith on DataFrames returns Rows with null fields instead of null values
[ https://issues.apache.org/jira/browse/SPARK-37829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17740726#comment-17740726 ] koert kuipers edited comment on SPARK-37829 at 7/6/23 5:32 PM: --- since this (admittedly somewhat weird) behavior of returning a Row with null values has been present since spark 3.0.x (a major breaking release, and 3 years ago) i would argue this is the default behavior and this jira introduces a breaking change. also expressionencoders are used for other purposes than dataset joins and now we find nulls popping up in places they should not. was (Author: koert): since this behavior of returning a Row with null values has been present since spark 3.0.x (a major breaking release, and 3 years ago) i would argue this is the default behavior and this jira introduces a breaking change. also expressionencoders are used for other purposes than dataset joins and now we find nulls popping up in places they should not. > An outer-join using joinWith on DataFrames returns Rows with null fields > instead of null values > --- > > Key: SPARK-37829 > URL: https://issues.apache.org/jira/browse/SPARK-37829 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0, 3.0.1, 3.0.2, 3.0.3, 3.1.0, 3.1.1, 3.1.2, 3.2.0 >Reporter: Clément de Groc >Assignee: Jason Xu >Priority: Major > Fix For: 3.3.3, 3.4.1, 3.5.0 > > > Doing an outer-join using {{joinWith}} on {{{}DataFrame{}}}s used to return > missing values as {{null}} in Spark 2.4.8, but returns them as {{Rows}} with > {{null}} values in Spark 3+. > The issue can be reproduced with [the following > test|https://github.com/cdegroc/spark/commit/79f4d6a1ec6c69b10b72dbc8f92ab6490d5ef5e5] > that succeeds on Spark 2.4.8 but fails starting from Spark 3.0.0. > The problem only arises when working with DataFrames: Datasets of case > classes work as expected as demonstrated by [this other > test|https://github.com/apache/spark/blob/v3.0.0/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala#L1200-L1223]. > I couldn't find an explanation for this change in the Migration guide so I'm > assuming this is a bug. > A {{git bisect}} pointed me to [that > commit|https://github.com/apache/spark/commit/cd92f25be5a221e0d4618925f7bc9dfd3bb8cb59]. > Reverting the commit solves the problem. > A similar solution, but without reverting, is shown > [here|https://github.com/cdegroc/spark/commit/684c675bf070876a475a9b225f6c2f92edce4c8a]. > Happy to help if you think of another approach / can provide some guidance. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-37829) An outer-join using joinWith on DataFrames returns Rows with null fields instead of null values
[ https://issues.apache.org/jira/browse/SPARK-37829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17740726#comment-17740726 ] koert kuipers edited comment on SPARK-37829 at 7/6/23 5:23 PM: --- since this behavior of returning a Row with null values has been present since spark 3.0.x (a major breaking release, and 3 years ago) i would argue this is the default behavior and this jira introduces a breaking change. also expressionencoders are used for other purposes than dataset joins and now we find nulls popping up in places they should not. was (Author: koert): since this behavior of returning a Row with null values has been present since spark 3.0.x (a major breaking release, and 3 years ago) i would argue this is the default behavior and this jira introduces a breaking change. > An outer-join using joinWith on DataFrames returns Rows with null fields > instead of null values > --- > > Key: SPARK-37829 > URL: https://issues.apache.org/jira/browse/SPARK-37829 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0, 3.0.1, 3.0.2, 3.0.3, 3.1.0, 3.1.1, 3.1.2, 3.2.0 >Reporter: Clément de Groc >Assignee: Jason Xu >Priority: Major > Fix For: 3.3.3, 3.4.1, 3.5.0 > > > Doing an outer-join using {{joinWith}} on {{{}DataFrame{}}}s used to return > missing values as {{null}} in Spark 2.4.8, but returns them as {{Rows}} with > {{null}} values in Spark 3+. > The issue can be reproduced with [the following > test|https://github.com/cdegroc/spark/commit/79f4d6a1ec6c69b10b72dbc8f92ab6490d5ef5e5] > that succeeds on Spark 2.4.8 but fails starting from Spark 3.0.0. > The problem only arises when working with DataFrames: Datasets of case > classes work as expected as demonstrated by [this other > test|https://github.com/apache/spark/blob/v3.0.0/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala#L1200-L1223]. > I couldn't find an explanation for this change in the Migration guide so I'm > assuming this is a bug. > A {{git bisect}} pointed me to [that > commit|https://github.com/apache/spark/commit/cd92f25be5a221e0d4618925f7bc9dfd3bb8cb59]. > Reverting the commit solves the problem. > A similar solution, but without reverting, is shown > [here|https://github.com/cdegroc/spark/commit/684c675bf070876a475a9b225f6c2f92edce4c8a]. > Happy to help if you think of another approach / can provide some guidance. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-37829) An outer-join using joinWith on DataFrames returns Rows with null fields instead of null values
[ https://issues.apache.org/jira/browse/SPARK-37829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17740726#comment-17740726 ] koert kuipers commented on SPARK-37829: --- since this behavior of returning a Row with null values has been present since spark 3.0.x (a major breaking release, and 3 years ago) i would argue this is the default behavior and this jira introduces a breaking change. > An outer-join using joinWith on DataFrames returns Rows with null fields > instead of null values > --- > > Key: SPARK-37829 > URL: https://issues.apache.org/jira/browse/SPARK-37829 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0, 3.0.1, 3.0.2, 3.0.3, 3.1.0, 3.1.1, 3.1.2, 3.2.0 >Reporter: Clément de Groc >Assignee: Jason Xu >Priority: Major > Fix For: 3.3.3, 3.4.1, 3.5.0 > > > Doing an outer-join using {{joinWith}} on {{{}DataFrame{}}}s used to return > missing values as {{null}} in Spark 2.4.8, but returns them as {{Rows}} with > {{null}} values in Spark 3+. > The issue can be reproduced with [the following > test|https://github.com/cdegroc/spark/commit/79f4d6a1ec6c69b10b72dbc8f92ab6490d5ef5e5] > that succeeds on Spark 2.4.8 but fails starting from Spark 3.0.0. > The problem only arises when working with DataFrames: Datasets of case > classes work as expected as demonstrated by [this other > test|https://github.com/apache/spark/blob/v3.0.0/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala#L1200-L1223]. > I couldn't find an explanation for this change in the Migration guide so I'm > assuming this is a bug. > A {{git bisect}} pointed me to [that > commit|https://github.com/apache/spark/commit/cd92f25be5a221e0d4618925f7bc9dfd3bb8cb59]. > Reverting the commit solves the problem. > A similar solution, but without reverting, is shown > [here|https://github.com/cdegroc/spark/commit/684c675bf070876a475a9b225f6c2f92edce4c8a]. > Happy to help if you think of another approach / can provide some guidance. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-44323) Scala None shows up as null for Aggregator BUF or OUT
[ https://issues.apache.org/jira/browse/SPARK-44323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-44323: -- Description: when doing an upgrade from spark 3.3.1 to spark 3.4.1 we suddenly started getting null pointer exceptions in Aggregators (classes extending org.apache.spark.sql.expressions.Aggregator) that use scala Option for BUF and/or OUT. basically None is now showing up as null. after adding a simple test case and doing a binary search on commits we landed on SPARK-37829 being the cause. we observed the issue at first with NPE inside Aggregator.merge because None was null. i am having a hard time replicating that in a spark unit test, but i did manage to get a None become null in the output. simple test that now fails: {code:java} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index e9daa825dd4..a1959d7065d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -228,6 +228,16 @@ case class FooAgg(s: Int) extends Aggregator[Row, Int, Int] { def outputEncoder: Encoder[Int] = Encoders.scalaInt } +object OptionStringAgg extends Aggregator[Option[String], Option[String], Option[String]] { + override def zero: Option[String] = None + override def reduce(b: Option[String], a: Option[String]): Option[String] = merge(b, a) + override def finish(reduction: Option[String]): Option[String] = reduction + override def merge(b1: Option[String], b2: Option[String]): Option[String] = + b1.map{ b1v => b2.map{ b2v => b1v ++ b2v }.getOrElse(b1v) }.orElse(b2) + override def bufferEncoder: Encoder[Option[String]] = ExpressionEncoder() + override def outputEncoder: Encoder[Option[String]] = ExpressionEncoder() +} + class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -432,4 +442,15 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { val agg = df.select(mode(col("a"))).as[String] checkDataset(agg, "3") } + + test("typed aggregation: option string") { + val ds = Seq((1, Some("a")), (1, None), (1, Some("c")), (2, None)).toDS() + + checkDataset( + ds.groupByKey(_._1).mapValues(_._2).agg( + OptionStringAgg.toColumn + ), + (1, Some("ac")), (2, None) + ) + } } {code} was: when doing an upgrade from spark 3.3.1 to spark 3.4.1 we suddenly started getting null pointer exceptions in Aggregators (classes extending org.apache.spark.sql.expressions.Aggregator) that use scala Option for BUF and/or OUT. basically None is now showing up as null. after adding a simple test case and doing a binary search on commits we landed on SPARK-37829 being the cause. > Scala None shows up as null for Aggregator BUF or OUT > --- > > Key: SPARK-44323 > URL: https://issues.apache.org/jira/browse/SPARK-44323 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.4.1 >Reporter: koert kuipers >Priority: Major > > when doing an upgrade from spark 3.3.1 to spark 3.4.1 we suddenly started > getting null pointer exceptions in Aggregators (classes extending > org.apache.spark.sql.expressions.Aggregator) that use scala Option for BUF > and/or OUT. basically None is now showing up as null. > after adding a simple test case and doing a binary search on commits we > landed on SPARK-37829 being the cause. > we observed the issue at first with NPE inside Aggregator.merge because None > was null. i am having a hard time replicating that in a spark unit test, but > i did manage to get a None become null in the output. simple test that now > fails: > > {code:java} > diff --git > a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > index e9daa825dd4..a1959d7065d 100644 > --- > a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > +++ > b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala > @@ -228,6 +228,16 @@ case class FooAgg(s: Int) extends Aggregator[Row, Int, > Int] { > def outputEncoder: Encoder[Int] = Encoders.scalaInt > } > > +object OptionStringAgg extends Aggregator[Option[String], Option[String], > Option[String]] { > + override def zero: Option[String] = None > + override def reduce(b: Option[String], a: Option[String]): Option[String] > = merge(b, a) > + override def finish(reduction: Option[String]): Option[String] = reduction > + override def merge(b1: Option[String], b2: Option[String]): Option[String] > = > + b1.map{ b1v => b2.map{
[jira] [Created] (SPARK-44323) Scala None shows up as null for Aggregator BUF or OUT
koert kuipers created SPARK-44323: - Summary: Scala None shows up as null for Aggregator BUF or OUT Key: SPARK-44323 URL: https://issues.apache.org/jira/browse/SPARK-44323 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.4.1 Reporter: koert kuipers when doing an upgrade from spark 3.3.1 to spark 3.4.1 we suddenly started getting null pointer exceptions in Aggregators (classes extending org.apache.spark.sql.expressions.Aggregator) that use scala Option for BUF and/or OUT. basically None is now showing up as null. after adding a simple test case and doing a binary search on commits we landed on SPARK-37829 being the cause. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-39770) Support Avro schema evolution
koert kuipers created SPARK-39770: - Summary: Support Avro schema evolution Key: SPARK-39770 URL: https://issues.apache.org/jira/browse/SPARK-39770 Project: Spark Issue Type: Improvement Components: SQL Affects Versions: 3.3.0 Reporter: koert kuipers currently the avro source in connector/avro does not yet support schema evolution. from source code of AvroUtils: {code:java} // Schema evolution is not supported yet. Here we only pick first random readable sample file to // figure out the schema of the whole dataset. {code} i added schema evolution for our inhouse spark version. if there is interest in this i could contribute it. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-31726) Make spark.files available in driver with cluster deploy mode on kubernetes
[ https://issues.apache.org/jira/browse/SPARK-31726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17480756#comment-17480756 ] koert kuipers edited comment on SPARK-31726 at 1/23/22, 11:22 PM: -- [~beregon87] about --jars, are you seeing that the jars are also not available on driver, or not added to classpath, or both? i ran a simple test where i added a jar from s3, e.g. --jars s3a://some/jar.jar, and was surprised to find the driver could not find a class in that jar (on kubernetes with cluster deploy mode). this would be a more serious bug given the description of --jars clearly says it should: --jars JARS Comma-separated list of jars to include on the driver and executor classpaths. now with --files its to bad the drivers dont get it but at least it does what it says on the tin (which does not include a promise to get the files to the driver): --files FILES Comma-separated list of files to be placed in the working directory of each executor. was (Author: koert): [~beregon87] about --jars, are you seeing that the jars are also not available on driver, or not added to classpath, or both? i ran a simple test where i added a jar from s3, e.g. --jars s3a://some/jar.jar, and was surprised to find the driver could not find a class in that jar (on kubernetes with cluster deploy mode). this would be a more serious bug given the description of --jars clearly says it should: Comma-separated list of jars to include on the driver and executor classpaths. now with --files its to bad the drivers dont get it but at least it does what it says on the tin (which does not include a promise to get the files to the driver): Comma-separated list of files to be placed in the working directory of each executor. > Make spark.files available in driver with cluster deploy mode on kubernetes > --- > > Key: SPARK-31726 > URL: https://issues.apache.org/jira/browse/SPARK-31726 > Project: Spark > Issue Type: Improvement > Components: Kubernetes, Spark Core >Affects Versions: 3.0.0 >Reporter: koert kuipers >Priority: Minor > > currently on yarn with cluster deploy mode --files makes the files available > for driver and executors and also put them on classpath for driver and > executors. > on k8s with cluster deploy mode --files makes the files available on > executors but they are not on classpath. it does not make the files available > on driver and they are not on driver classpath. > it would be nice if the k8s behavior was consistent with yarn, or at least > makes the files available on driver. once the files are available there is a > simple workaround to get them on classpath using > spark.driver.extraClassPath="./" > background: > we recently started testing kubernetes for spark. our main platform is yarn > on which we use client deploy mode. our first experience was that client > deploy mode was difficult to use on k8s (we dont launch from inside a pod). > so we switched to cluster deploy mode, which seems to behave well on k8s. but > then we realized that our program rely on reading files on classpath > (application.conf, log4j.properties etc.) that are on the client but now are > no longer on the driver (since driver is no longer on client). an easy fix > for this seems to be to ship the files using --files to make them available > on driver, but we could not get this to work. > -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-31726) Make spark.files available in driver with cluster deploy mode on kubernetes
[ https://issues.apache.org/jira/browse/SPARK-31726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17480756#comment-17480756 ] koert kuipers commented on SPARK-31726: --- [~beregon87] about --jars, are you seeing that the jars are also not available on driver, or not added to classpath, or both? i ran a simple test where i added a jar from s3, e.g. --jars s3a://some/jar.jar, and was surprised to find the driver could not find a class in that jar (on kubernetes with cluster deploy mode). this would be a more serious bug given the description of --jars clearly says it should: Comma-separated list of jars to include on the driver and executor classpaths. now with --files its to bad the drivers dont get it but at least it does what it says on the tin (which does not include a promise to get the files to the driver): Comma-separated list of files to be placed in the working directory of each executor. > Make spark.files available in driver with cluster deploy mode on kubernetes > --- > > Key: SPARK-31726 > URL: https://issues.apache.org/jira/browse/SPARK-31726 > Project: Spark > Issue Type: Improvement > Components: Kubernetes, Spark Core >Affects Versions: 3.0.0 >Reporter: koert kuipers >Priority: Minor > > currently on yarn with cluster deploy mode --files makes the files available > for driver and executors and also put them on classpath for driver and > executors. > on k8s with cluster deploy mode --files makes the files available on > executors but they are not on classpath. it does not make the files available > on driver and they are not on driver classpath. > it would be nice if the k8s behavior was consistent with yarn, or at least > makes the files available on driver. once the files are available there is a > simple workaround to get them on classpath using > spark.driver.extraClassPath="./" > background: > we recently started testing kubernetes for spark. our main platform is yarn > on which we use client deploy mode. our first experience was that client > deploy mode was difficult to use on k8s (we dont launch from inside a pod). > so we switched to cluster deploy mode, which seems to behave well on k8s. but > then we realized that our program rely on reading files on classpath > (application.conf, log4j.properties etc.) that are on the client but now are > no longer on the driver (since driver is no longer on client). an easy fix > for this seems to be to ship the files using --files to make them available > on driver, but we could not get this to work. > -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-31726) Make spark.files available in driver with cluster deploy mode on kubernetes
[ https://issues.apache.org/jira/browse/SPARK-31726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17476940#comment-17476940 ] koert kuipers commented on SPARK-31726: --- https://stackoverflow.com/questions/65353164/how-to-read-files-uploaded-by-spark-submit-on-kubernetes > Make spark.files available in driver with cluster deploy mode on kubernetes > --- > > Key: SPARK-31726 > URL: https://issues.apache.org/jira/browse/SPARK-31726 > Project: Spark > Issue Type: Improvement > Components: Kubernetes, Spark Core >Affects Versions: 3.0.0 >Reporter: koert kuipers >Priority: Minor > > currently on yarn with cluster deploy mode --files makes the files available > for driver and executors and also put them on classpath for driver and > executors. > on k8s with cluster deploy mode --files makes the files available on > executors but they are not on classpath. it does not make the files available > on driver and they are not on driver classpath. > it would be nice if the k8s behavior was consistent with yarn, or at least > makes the files available on driver. once the files are available there is a > simple workaround to get them on classpath using > spark.driver.extraClassPath="./" > background: > we recently started testing kubernetes for spark. our main platform is yarn > on which we use client deploy mode. our first experience was that client > deploy mode was difficult to use on k8s (we dont launch from inside a pod). > so we switched to cluster deploy mode, which seems to behave well on k8s. but > then we realized that our program rely on reading files on classpath > (application.conf, log4j.properties etc.) that are on the client but now are > no longer on the driver (since driver is no longer on client). an easy fix > for this seems to be to ship the files using --files to make them available > on driver, but we could not get this to work. > -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17452118#comment-17452118 ] koert kuipers commented on SPARK-37476: --- note that changing my case class to: {code:java} case class SumAndProduct(sum: java.lang.Double, product: java.lang.Double) {code} did not fix anything. same errors. > udaf doesnt work with nullable (or option of) case class result > > > Key: SPARK-37476 > URL: https://issues.apache.org/jira/browse/SPARK-37476 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.2.0 > Environment: spark master branch on nov 27 >Reporter: koert kuipers >Priority: Minor > > i have a need to have a dataframe aggregation return a nullable case class. > there seems to be no way to get this to work. the suggestion to wrap the > result in an option doesnt work either. > first attempt using nulls: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { > def zero: SumAndProduct = null > def reduce(b: SumAndProduct, a: Double): SumAndProduct = > if (b == null) { > SumAndProduct(a, a) > } else { > SumAndProduct(b.sum + a, b.product * a) > } > def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = > if (b1 == null) { > b2 > } else if (b2 == null) { > b1 > } else { > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > } > def finish(r: SumAndProduct): SumAndProduct = r > def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$3(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1491.0 (TID 1929) > java.lang.RuntimeException: Error while encoding: > java.lang.NullPointerException: Null value appeared in non-nullable field: > top level Product or row object > If the schema is inferred from a Scala tuple/case class, or a Java bean, > please try to use scala.Option[_] or other nullable types (e.g. > java.lang.Integer instead of int/scala.Int). > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).sum AS sum#20070 > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).product AS product#20071 > at > org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) > {code} > i dont really understand the error, this result is not a top-level row object. > anyhow taking the advice to heart and using option we get to the second > attempt using options: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], > Option[SumAndProduct]] { > def zero: Option[SumAndProduct] = None > def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = > b > .map{ b => SumAndProduct(b.sum + a, b.product * a) } > .orElse{ Option(SumAndProduct(a, a)) } > def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): > Option[SumAndProduct] = > b1.map{ b1 => > b2.map{ b2 => > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > }.getOrElse(b1) > }.orElse(b2) > def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r > def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$4(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1493.0 (TID 1930) > java.lang.AssertionError: index (1) should < 1 > at > org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) > at > org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown > Source) > at >
[jira] [Comment Edited] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17452101#comment-17452101 ] koert kuipers edited comment on SPARK-37476 at 12/2/21, 12:58 AM: -- i get that scala Double cannot be null. however i dont understand how this is relevant? my case class can be null, yet it fails when i try to return null for the case class. in so far i know a nullable struct is perfectly valid in spark? was (Author: koert): i get that scala Double cannot be null. however i dont understand how this is relevant? my case class can be null, yet it fails when i try to return null for the case class. > udaf doesnt work with nullable (or option of) case class result > > > Key: SPARK-37476 > URL: https://issues.apache.org/jira/browse/SPARK-37476 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.2.0 > Environment: spark master branch on nov 27 >Reporter: koert kuipers >Priority: Minor > > i have a need to have a dataframe aggregation return a nullable case class. > there seems to be no way to get this to work. the suggestion to wrap the > result in an option doesnt work either. > first attempt using nulls: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { > def zero: SumAndProduct = null > def reduce(b: SumAndProduct, a: Double): SumAndProduct = > if (b == null) { > SumAndProduct(a, a) > } else { > SumAndProduct(b.sum + a, b.product * a) > } > def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = > if (b1 == null) { > b2 > } else if (b2 == null) { > b1 > } else { > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > } > def finish(r: SumAndProduct): SumAndProduct = r > def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$3(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1491.0 (TID 1929) > java.lang.RuntimeException: Error while encoding: > java.lang.NullPointerException: Null value appeared in non-nullable field: > top level Product or row object > If the schema is inferred from a Scala tuple/case class, or a Java bean, > please try to use scala.Option[_] or other nullable types (e.g. > java.lang.Integer instead of int/scala.Int). > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).sum AS sum#20070 > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).product AS product#20071 > at > org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) > {code} > i dont really understand the error, this result is not a top-level row object. > anyhow taking the advice to heart and using option we get to the second > attempt using options: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], > Option[SumAndProduct]] { > def zero: Option[SumAndProduct] = None > def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = > b > .map{ b => SumAndProduct(b.sum + a, b.product * a) } > .orElse{ Option(SumAndProduct(a, a)) } > def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): > Option[SumAndProduct] = > b1.map{ b1 => > b2.map{ b2 => > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > }.getOrElse(b1) > }.orElse(b2) > def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r > def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$4(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1493.0 (TID 1930) > java.lang.AssertionError: index (1) should < 1 > at > org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) > at >
[jira] [Commented] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17452101#comment-17452101 ] koert kuipers commented on SPARK-37476: --- i get that scala Double cannot be null. however i dont understand how this is relevant? my case class can be null, yet it fails when i try to return null for the case class. > udaf doesnt work with nullable (or option of) case class result > > > Key: SPARK-37476 > URL: https://issues.apache.org/jira/browse/SPARK-37476 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.2.0 > Environment: spark master branch on nov 27 >Reporter: koert kuipers >Priority: Minor > > i have a need to have a dataframe aggregation return a nullable case class. > there seems to be no way to get this to work. the suggestion to wrap the > result in an option doesnt work either. > first attempt using nulls: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { > def zero: SumAndProduct = null > def reduce(b: SumAndProduct, a: Double): SumAndProduct = > if (b == null) { > SumAndProduct(a, a) > } else { > SumAndProduct(b.sum + a, b.product * a) > } > def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = > if (b1 == null) { > b2 > } else if (b2 == null) { > b1 > } else { > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > } > def finish(r: SumAndProduct): SumAndProduct = r > def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$3(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1491.0 (TID 1929) > java.lang.RuntimeException: Error while encoding: > java.lang.NullPointerException: Null value appeared in non-nullable field: > top level Product or row object > If the schema is inferred from a Scala tuple/case class, or a Java bean, > please try to use scala.Option[_] or other nullable types (e.g. > java.lang.Integer instead of int/scala.Int). > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).sum AS sum#20070 > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).product AS product#20071 > at > org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) > {code} > i dont really understand the error, this result is not a top-level row object. > anyhow taking the advice to heart and using option we get to the second > attempt using options: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], > Option[SumAndProduct]] { > def zero: Option[SumAndProduct] = None > def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = > b > .map{ b => SumAndProduct(b.sum + a, b.product * a) } > .orElse{ Option(SumAndProduct(a, a)) } > def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): > Option[SumAndProduct] = > b1.map{ b1 => > b2.map{ b2 => > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > }.getOrElse(b1) > }.orElse(b2) > def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r > def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$4(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1493.0 (TID 1930) > java.lang.AssertionError: index (1) should < 1 > at > org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) > at > org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown > Source) > at >
[jira] [Commented] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17451921#comment-17451921 ] koert kuipers commented on SPARK-37476: --- yes it works well {code:java} import java.lang.{Double => JDouble} val sumAgg = new Aggregator[Double, JDouble, JDouble] { def zero: JDouble = null def reduce(b: JDouble, a: Double): JDouble = if (b == null) { a } else { b + a } def merge(b1: JDouble, b2: JDouble): JDouble = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { b1 + b2 } def finish(r: JDouble): JDouble = r def bufferEncoder: Encoder[JDouble] = ExpressionEncoder() def outputEncoder: Encoder[JDouble] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAgg).apply(col("value"))) df.printSchema() root |-- $anon$3(value): double (nullable = true) df.show() +--+ |$anon$3(value)| +--+ | null| +--+ {code} it works with Option without issues too > udaf doesnt work with nullable (or option of) case class result > > > Key: SPARK-37476 > URL: https://issues.apache.org/jira/browse/SPARK-37476 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.2.0 > Environment: spark master branch on nov 27 >Reporter: koert kuipers >Priority: Minor > > i have a need to have a dataframe aggregation return a nullable case class. > there seems to be no way to get this to work. the suggestion to wrap the > result in an option doesnt work either. > first attempt using nulls: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { > def zero: SumAndProduct = null > def reduce(b: SumAndProduct, a: Double): SumAndProduct = > if (b == null) { > SumAndProduct(a, a) > } else { > SumAndProduct(b.sum + a, b.product * a) > } > def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = > if (b1 == null) { > b2 > } else if (b2 == null) { > b1 > } else { > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > } > def finish(r: SumAndProduct): SumAndProduct = r > def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$3(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1491.0 (TID 1929) > java.lang.RuntimeException: Error while encoding: > java.lang.NullPointerException: Null value appeared in non-nullable field: > top level Product or row object > If the schema is inferred from a Scala tuple/case class, or a Java bean, > please try to use scala.Option[_] or other nullable types (e.g. > java.lang.Integer instead of int/scala.Int). > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).sum AS sum#20070 > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).product AS product#20071 > at > org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) > {code} > i dont really understand the error, this result is not a top-level row object. > anyhow taking the advice to heart and using option we get to the second > attempt using options: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], > Option[SumAndProduct]] { > def zero: Option[SumAndProduct] = None > def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = > b > .map{ b => SumAndProduct(b.sum + a, b.product * a) } > .orElse{ Option(SumAndProduct(a, a)) } > def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): > Option[SumAndProduct] = > b1.map{ b1 => > b2.map{ b2 => > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > }.getOrElse(b1) > }.orElse(b2) > def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r > def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$4(value): struct
[jira] [Comment Edited] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17451921#comment-17451921 ] koert kuipers edited comment on SPARK-37476 at 12/1/21, 4:57 PM: - yes it works well {code:java} import java.lang.{Double => JDouble} val sumAgg = new Aggregator[Double, JDouble, JDouble] { def zero: JDouble = null def reduce(b: JDouble, a: Double): JDouble = if (b == null) { a } else { b + a } def merge(b1: JDouble, b2: JDouble): JDouble = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { b1 + b2 } def finish(r: JDouble): JDouble = r def bufferEncoder: Encoder[JDouble] = ExpressionEncoder() def outputEncoder: Encoder[JDouble] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAgg).apply(col("value"))) df.printSchema() root |-- $anon$3(value): double (nullable = true) df.show() +--+ |$anon$3(value)| +--+ | null| +--+ {code} it works with Option without issues too was (Author: koert): yes it works well {code:java} import java.lang.{Double => JDouble} val sumAgg = new Aggregator[Double, JDouble, JDouble] { def zero: JDouble = null def reduce(b: JDouble, a: Double): JDouble = if (b == null) { a } else { b + a } def merge(b1: JDouble, b2: JDouble): JDouble = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { b1 + b2 } def finish(r: JDouble): JDouble = r def bufferEncoder: Encoder[JDouble] = ExpressionEncoder() def outputEncoder: Encoder[JDouble] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAgg).apply(col("value"))) df.printSchema() root |-- $anon$3(value): double (nullable = true) df.show() +--+ |$anon$3(value)| +--+ | null| +--+ {code} it works with Option without issues too > udaf doesnt work with nullable (or option of) case class result > > > Key: SPARK-37476 > URL: https://issues.apache.org/jira/browse/SPARK-37476 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.2.0 > Environment: spark master branch on nov 27 >Reporter: koert kuipers >Priority: Minor > > i have a need to have a dataframe aggregation return a nullable case class. > there seems to be no way to get this to work. the suggestion to wrap the > result in an option doesnt work either. > first attempt using nulls: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { > def zero: SumAndProduct = null > def reduce(b: SumAndProduct, a: Double): SumAndProduct = > if (b == null) { > SumAndProduct(a, a) > } else { > SumAndProduct(b.sum + a, b.product * a) > } > def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = > if (b1 == null) { > b2 > } else if (b2 == null) { > b1 > } else { > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > } > def finish(r: SumAndProduct): SumAndProduct = r > def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$3(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1491.0 (TID 1929) > java.lang.RuntimeException: Error while encoding: > java.lang.NullPointerException: Null value appeared in non-nullable field: > top level Product or row object > If the schema is inferred from a Scala tuple/case class, or a Java bean, > please try to use scala.Option[_] or other nullable types (e.g. > java.lang.Integer instead of int/scala.Int). > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).sum AS sum#20070 > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).product AS product#20071 > at > org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) > {code} > i dont really understand the error, this result is not a top-level row object. > anyhow taking the advice to heart and using option we get to the second > attempt using options: > {code:java} > case class SumAndProduct(sum: Double, product: Double) > val sumAndProductAgg = new Aggregator[Double,
[jira] [Updated] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-37476: -- Description: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$3(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929) java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field: top level Product or row object If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int). knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070 knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071 at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) {code} i dont really understand the error, this result is not a top-level row object. anyhow taking the advice to heart and using option we get to the second attempt using options: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] { def zero: Option[SumAndProduct] = None def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = b .map{ b => SumAndProduct(b.sum + a, b.product * a) } .orElse{ Option(SumAndProduct(a, a)) } def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] = b1.map{ b1 => b2.map{ b2 => SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) }.getOrElse(b1) }.orElse(b2) def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$4(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930) java.lang.AssertionError: index (1) should < 1 at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) {code} was: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 ==
[jira] [Updated] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-37476: -- Description: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$3(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929) java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field: top level Product or row object If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int). knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070 knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071 at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) {code} i dont really understand the error, this result is not a top-level row object. anyhow taking the advice to heart and using option we get to the second attempt using options: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] { def zero: Option[SumAndProduct] = None def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = b .map{ b => SumAndProduct(b.sum + a, b.product * a) } .orElse{ Option(SumAndProduct(a, a)) } def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] = b1.map{ b1 => b2.map{ b2 => SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) }.getOrElse(b1) }.orElse(b2) def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$4(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930) java.lang.AssertionError: index (1) should < 1 at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) {code} was: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2
[jira] [Updated] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-37476: -- Description: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$3(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929) java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field: top level Product or row object If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int). knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070 knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071 at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) {code} i dont really understand the error, this result is not a top-level row object. anyhow taking the advice to heart and using option we get to the second attempt using options: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] { def zero: Option[SumAndProduct] = None def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = b .map{ b => SumAndProduct(b.sum + a, b.product * a) } .orElse{ Option(SumAndProduct(a, a)) } def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] = b1.map{ b1 => b2.map{ b2 => SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) }.getOrElse(b1) }.orElse(b2) def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$4(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930) java.lang.AssertionError: index (1) should < 1 at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) {code} was: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2
[jira] [Updated] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-37476: -- Description: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$3(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929) java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field: top level Product or row object If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int). knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070 knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071 at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) {code} i dont really understand the error, this result is not a top-level row object. anyhow taking the advice to heart and using option we get to the second attempt using options: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] { def zero: Option[SumAndProduct] = None def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = b .map{ b => SumAndProduct(b.sum + a, b.product * a) } .orElse{ Option(SumAndProduct(a, a)) } def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] = b1.map{ b1 => b2.map{ b2 => SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) }.getOrElse(b1) }.orElse(b2) def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$4(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930) java.lang.AssertionError: index (1) should < 1 at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) {code} was: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 ==
[jira] [Updated] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-37476: -- Description: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$3(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929) java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field: top level Product or row object If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int). knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070 knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071 at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) {code} taking the advice to heart and using option we get to the second attempt using options: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] { def zero: Option[SumAndProduct] = None def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = b .map{ b => SumAndProduct(b.sum + a, b.product * a) } .orElse{ Option(SumAndProduct(a, a)) } def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] = b1.map{ b1 => b2.map{ b2 => SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) }.getOrElse(b1) }.orElse(b2) def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$4(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930) java.lang.AssertionError: index (1) should < 1 at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) {code} was: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product *
[jira] [Updated] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-37476: -- Summary: udaf doesnt work with nullable (or option of) case class result (was: udaf doesnt work with nullable case class result ) > udaf doesnt work with nullable (or option of) case class result > > > Key: SPARK-37476 > URL: https://issues.apache.org/jira/browse/SPARK-37476 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.2.0 > Environment: spark master branch on nov 27 >Reporter: koert kuipers >Priority: Minor > > i have a need to have a dataframe aggregation return a nullable case class. > there seems to be no way to get this to work. the suggestion to wrap the > result in an option doesnt work either. > first attempt using nulls: > {code:java} > val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { > def zero: SumAndProduct = null > def reduce(b: SumAndProduct, a: Double): SumAndProduct = > if (b == null) { > SumAndProduct(a, a) > } else { > SumAndProduct(b.sum + a, b.product * a) > } > def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = > if (b1 == null) { > b2 > } else if (b2 == null) { > b1 > } else { > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > } > def finish(r: SumAndProduct): SumAndProduct = r > def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$3(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1491.0 (TID 1929) > java.lang.RuntimeException: Error while encoding: > java.lang.NullPointerException: Null value appeared in non-nullable field: > top level Product or row object > If the schema is inferred from a Scala tuple/case class, or a Java bean, > please try to use scala.Option[_] or other nullable types (e.g. > java.lang.Integer instead of int/scala.Int). > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).sum AS sum#20070 > knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, > true])).product AS product#20071 > at > org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) > {code} > taking the advice to heart and using option we get to the second attempt > using options: > {code:java} > val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], > Option[SumAndProduct]] { > def zero: Option[SumAndProduct] = None > def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = > b > .map{ b => SumAndProduct(b.sum + a, b.product * a) } > .orElse{ Option(SumAndProduct(a, a)) } > def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): > Option[SumAndProduct] = > b1.map{ b1 => > b2.map{ b2 => > SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) > }.getOrElse(b1) > }.orElse(b2) > def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r > def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() > } > val df = Seq.empty[Double] > .toDF() > .select(udaf(sumAndProductAgg).apply(col("value"))) > df.printSchema() > df.show() > {code} > this gives: > {code:java} > root > |-- $anon$4(value): struct (nullable = true) > | |-- sum: double (nullable = false) > | |-- product: double (nullable = false) > 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 > in stage 1493.0 (TID 1930) > java.lang.AssertionError: index (1) should < 1 > at > org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) > at > org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown > Source) > at > org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) > at > org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) > {code} > > > -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Updated] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-37476: -- Description: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} case class SumAndProduct(sum: Double, product: Double) val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$3(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929) java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field: top level Product or row object If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int). knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070 knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071 at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) {code} taking the advice to heart and using option we get to the second attempt using options: {code:java} val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] { def zero: Option[SumAndProduct] = None def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = b .map{ b => SumAndProduct(b.sum + a, b.product * a) } .orElse{ Option(SumAndProduct(a, a)) } def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] = b1.map{ b1 => b2.map{ b2 => SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) }.getOrElse(b1) }.orElse(b2) def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$4(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930) java.lang.AssertionError: index (1) should < 1 at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) {code} was: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] =
[jira] [Updated] (SPARK-37476) udaf doesnt work with nullable case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-37476: -- Description: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$3(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929) java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field: top level Product or row object If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int). knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070 knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071 at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) {code} taking the advice to heart and using option we get to the second attempt using options: {code:java} val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] { def zero: Option[SumAndProduct] = None def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = b .map{ b => SumAndProduct(b.sum + a, b.product * a) } .orElse{ Option(SumAndProduct(a, a)) } def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] = b1.map{ b1 => b2.map{ b2 => SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) }.getOrElse(b1) }.orElse(b2) def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$4(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930) java.lang.AssertionError: index (1) should < 1 at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) {code} was: i have a need to have an aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder:
[jira] [Updated] (SPARK-37476) udaf doesnt work with nullable case class result
[ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-37476: -- Description: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$3(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929) java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field: top level Product or row object If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int). knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070 knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071 at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) {code} taking the advice to heart and using option we get to the second attempt using options: {code:java} val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] { def zero: Option[SumAndProduct] = None def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = b .map{ b => SumAndProduct(b.sum + a, b.product * a) } .orElse{ Option(SumAndProduct(a, a)) } def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] = b1.map{ b1 => b2.map{ b2 => SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) }.getOrElse(b1) }.orElse(b2) def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$4(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930) java.lang.AssertionError: index (1) should < 1 at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) {code} was: i have a need to have a dataframe aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder:
[jira] [Created] (SPARK-37476) udaf doesnt work with nullable case class result
koert kuipers created SPARK-37476: - Summary: udaf doesnt work with nullable case class result Key: SPARK-37476 URL: https://issues.apache.org/jira/browse/SPARK-37476 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.2.0 Environment: spark master branch on nov 27 Reporter: koert kuipers i have a need to have an aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either. first attempt using nulls: {code:java} val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] { def zero: SumAndProduct = null def reduce(b: SumAndProduct, a: Double): SumAndProduct = if (b == null) { SumAndProduct(a, a) } else { SumAndProduct(b.sum + a, b.product * a) } def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct = if (b1 == null) { b2 } else if (b2 == null) { b1 } else { SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) } def finish(r: SumAndProduct): SumAndProduct = r def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder() def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$3(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929) java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field: top level Product or row object If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int). knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070 knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071 at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125) {code} taking the advice to heart and using option we get to the second attempt using options: {code:java} val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] { def zero: Option[SumAndProduct] = None def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] = b .map{ b => SumAndProduct(b.sum + a, b.product * a) } .orElse{ Option(SumAndProduct(a, a)) } def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] = b1.map{ b1 => b2.map{ b2 => SumAndProduct(b1.sum + b2.sum, b1.product * b2.product) }.getOrElse(b1) }.orElse(b2) def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder() } val df = Seq.empty[Double] .toDF() .select(udaf(sumAndProductAgg).apply(col("value"))) df.printSchema() df.show() {code} this gives: {code:java} root |-- $anon$4(value): struct (nullable = true) | |-- sum: double (nullable = false) | |-- product: double (nullable = false) 16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930) java.lang.AssertionError: index (1) should < 1 at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142) at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107) {code} -- This message was sent by Atlassian Jira (v8.20.1#820001) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-36681) Fail to load Snappy codec
[ https://issues.apache.org/jira/browse/SPARK-36681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17430303#comment-17430303 ] koert kuipers commented on SPARK-36681: --- hadoop Jira issue: https://issues.apache.org/jira/browse/HADOOP-17891 i have my doubt this only impacts sequence files. i am seeing this issue with snappy compressed csv files, snappy compress json files, etc. > Fail to load Snappy codec > - > > Key: SPARK-36681 > URL: https://issues.apache.org/jira/browse/SPARK-36681 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 3.2.0 >Reporter: L. C. Hsieh >Priority: Major > > snappy-java as a native library should not be relocated in Hadoop shaded > client libraries. Currently we use Hadoop shaded client libraries in Spark. > If trying to use SnappyCodec to write sequence file, we will encounter the > following error: > {code} > [info] Cause: java.lang.UnsatisfiedLinkError: > org.apache.hadoop.shaded.org.xerial.snappy.SnappyNative.rawCompress(Ljava/nio/ByteBuffer;IILjava/nio/ByteBuffer;I)I > [info] at > org.apache.hadoop.shaded.org.xerial.snappy.SnappyNative.rawCompress(Native > Method) > > [info] at > org.apache.hadoop.shaded.org.xerial.snappy.Snappy.compress(Snappy.java:151) > > > [info] at > org.apache.hadoop.io.compress.snappy.SnappyCompressor.compressDirectBuf(SnappyCompressor.java:282) > [info] at > org.apache.hadoop.io.compress.snappy.SnappyCompressor.compress(SnappyCompressor.java:210) > [info] at > org.apache.hadoop.io.compress.BlockCompressorStream.compress(BlockCompressorStream.java:149) > [info] at > org.apache.hadoop.io.compress.BlockCompressorStream.finish(BlockCompressorStream.java:142) > [info] at > org.apache.hadoop.io.SequenceFile$BlockCompressWriter.writeBuffer(SequenceFile.java:1589) > > [info] at > org.apache.hadoop.io.SequenceFile$BlockCompressWriter.sync(SequenceFile.java:1605) > [info] at > org.apache.hadoop.io.SequenceFile$BlockCompressWriter.close(SequenceFile.java:1629) > > {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] [Created] (SPARK-36210) Preserve column insertion order in Dataset.withColumns
koert kuipers created SPARK-36210: - Summary: Preserve column insertion order in Dataset.withColumns Key: SPARK-36210 URL: https://issues.apache.org/jira/browse/SPARK-36210 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.1.2 Reporter: koert kuipers Dataset.withColumns uses a Map (columnMap) to store the mapping of column name to column. however this loses the order of the columns. also none of the operations used on the Map (find and filter) benefits from the map's lookup features. so it seems simpler to use a Seq instead, which also preserves the insertion order. -- 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-35079) Transform with udf gives incorrect result
[ https://issues.apache.org/jira/browse/SPARK-35079?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17346970#comment-17346970 ] koert kuipers commented on SPARK-35079: --- looks to me like this is a duplicate of SPARK-34829 > Transform with udf gives incorrect result > - > > Key: SPARK-35079 > URL: https://issues.apache.org/jira/browse/SPARK-35079 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.1.1 >Reporter: koert kuipers >Priority: Minor > Fix For: 3.1.2, 3.2.0 > > > i think this is a correctness bug in spark 3.1.1 > the behavior is correct in spark 3.0.1 > in spark 3.0.1: > {code:java} > scala> import spark.implicits._ > scala> import org.apache.spark.sql.functions._ > scala> val x = Seq(Seq("aa", "bb", "cc")).toDF > x: org.apache.spark.sql.DataFrame = [value: array] > scala> x.select(transform(col("value"), col => udf((_: > String).drop(1)).apply(col))).show > +---+ > |transform(value, lambdafunction(UDF(lambda 'x), x))| > +---+ > | [a, b, c]| > +---+ > {code} > in spark 3.1.1: > {code:java} > scala> import spark.implicits._ > scala> import org.apache.spark.sql.functions._ > scala> val x = Seq(Seq("aa", "bb", "cc")).toDF > x: org.apache.spark.sql.DataFrame = [value: array] > scala> x.select(transform(col("value"), col => udf((_: > String).drop(1)).apply(col))).show > +---+ > |transform(value, lambdafunction(UDF(lambda 'x), x))| > +---+ > | [c, c, c]| > +---+ > {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] [Updated] (SPARK-35079) Transform with udf gives incorrect result
[ https://issues.apache.org/jira/browse/SPARK-35079?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-35079: -- Description: i think this is a correctness bug in spark 3.1.1 the behavior is correct in spark 3.0.1 in spark 3.0.1: {code:java} scala> import spark.implicits._ scala> import org.apache.spark.sql.functions._ scala> val x = Seq(Seq("aa", "bb", "cc")).toDF x: org.apache.spark.sql.DataFrame = [value: array] scala> x.select(transform(col("value"), col => udf((_: String).drop(1)).apply(col))).show +---+ |transform(value, lambdafunction(UDF(lambda 'x), x))| +---+ | [a, b, c]| +---+ {code} in spark 3.1.1: {code:java} scala> import spark.implicits._ scala> import org.apache.spark.sql.functions._ scala> val x = Seq(Seq("aa", "bb", "cc")).toDF x: org.apache.spark.sql.DataFrame = [value: array] scala> x.select(transform(col("value"), col => udf((_: String).drop(1)).apply(col))).show +---+ |transform(value, lambdafunction(UDF(lambda 'x), x))| +---+ | [c, c, c]| +---+ {code} was: i think this is a correctness bug in spark 3.1.1 the behavior is correct in spark 3.0.1 in spark 3.0.1: {code:java} scala> import spark.implicits._ scala> import org.apache.spark.sql.functions._ scala> val x = Seq(Seq("11", "22", "33")).toDF x: org.apache.spark.sql.DataFrame = [value: array] scala> x.select(transform(col("value"), col => udf((_: String).drop(1)).apply(col))).show +---+ |transform(value, lambdafunction(UDF(lambda 'x), x))| +---+ | [1, 2, 3]| +---+ {code} in spark 3.1.1: {code:java} scala> import spark.implicits._ scala> import org.apache.spark.sql.functions._ scala> val x = Seq(Seq("11", "22", "33")).toDF x: org.apache.spark.sql.DataFrame = [value: array] scala> x.select(transform(col("value"), col => udf((_: String).drop(1)).apply(col))).show +---+ |transform(value, lambdafunction(UDF(lambda 'x), x))| +---+ | [3, 3, 3]| +---+ {code} > Transform with udf gives incorrect result > - > > Key: SPARK-35079 > URL: https://issues.apache.org/jira/browse/SPARK-35079 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.1.1 >Reporter: koert kuipers >Priority: Minor > > i think this is a correctness bug in spark 3.1.1 > the behavior is correct in spark 3.0.1 > in spark 3.0.1: > {code:java} > scala> import spark.implicits._ > scala> import org.apache.spark.sql.functions._ > scala> val x = Seq(Seq("aa", "bb", "cc")).toDF > x: org.apache.spark.sql.DataFrame = [value: array] > scala> x.select(transform(col("value"), col => udf((_: > String).drop(1)).apply(col))).show > +---+ > |transform(value, lambdafunction(UDF(lambda 'x), x))| > +---+ > | [a, b, c]| > +---+ > {code} > in spark 3.1.1: > {code:java} > scala> import spark.implicits._ > scala> import org.apache.spark.sql.functions._ > scala> val x = Seq(Seq("aa", "bb", "cc")).toDF > x: org.apache.spark.sql.DataFrame = [value: array] > scala> x.select(transform(col("value"), col => udf((_: > String).drop(1)).apply(col))).show > +---+ > |transform(value, lambdafunction(UDF(lambda 'x), x))| > +---+ > | [c, c, c]| > +---+ > {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] [Created] (SPARK-35079) Transform with udf gives incorrect result
koert kuipers created SPARK-35079: - Summary: Transform with udf gives incorrect result Key: SPARK-35079 URL: https://issues.apache.org/jira/browse/SPARK-35079 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.1.1 Reporter: koert kuipers i think this is a correctness bug in spark 3.1.1 the behavior is correct in spark 3.0.1 in spark 3.0.1: {code:java} scala> import spark.implicits._ scala> import org.apache.spark.sql.functions._ scala> val x = Seq(Seq("11", "22", "33")).toDF x: org.apache.spark.sql.DataFrame = [value: array] scala> x.select(transform(col("value"), col => udf((_: String).drop(1)).apply(col))).show +---+ |transform(value, lambdafunction(UDF(lambda 'x), x))| +---+ | [1, 2, 3]| +---+ {code} in spark 3.1.1: {code:java} scala> import spark.implicits._ scala> import org.apache.spark.sql.functions._ scala> val x = Seq(Seq("11", "22", "33")).toDF x: org.apache.spark.sql.DataFrame = [value: array] scala> x.select(transform(col("value"), col => udf((_: String).drop(1)).apply(col))).show +---+ |transform(value, lambdafunction(UDF(lambda 'x), x))| +---+ | [3, 3, 3]| +---+ {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-13860) TPCDS query 39 returns wrong results compared to TPC official result set
[ https://issues.apache.org/jira/browse/SPARK-13860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17320450#comment-17320450 ] koert kuipers edited comment on SPARK-13860 at 4/13/21, 6:38 PM: - this is what i now see: {code:java} scala> Seq(1.0, 2.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ |0.7071067811865476| +--+ scala> Seq(1.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ | null| +--+ {code} so basically now when we have division by zero we return null instead of NaN. i think for division by zero NaN is a far better answer than null? it captures exactly what the issue is. us changing correct answers to match what SQL does (which has many warts and mistakes we should not try to emulate) saddens me. spark is more than SQL... was (Author: koert): this is what i now see: {code:java} scala> Seq(1.0, 2.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ |0.7071067811865476| +--+ scala> Seq(1.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ | null| +--+ {code} so basically now when we have division by zero we return null instead of NaN. i think for division by zero NaN is a far better answer than null? it captures exactly what the issue is. us changing correct answers to match what some other SQL system does (which is a system with many warts and mistakes we should not try to emulate) saddens me. > TPCDS query 39 returns wrong results compared to TPC official result set > - > > Key: SPARK-13860 > URL: https://issues.apache.org/jira/browse/SPARK-13860 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 1.6.0, 2.1.1, 2.2.0, 2.3.4, 2.4.7, 3.0.1 >Reporter: JESSE CHEN >Assignee: Leanken.Lin >Priority: Major > Labels: correctness > Fix For: 3.1.0 > > > Testing Spark SQL using TPC queries. Query 39 returns wrong results compared > to official result set. This is at 1GB SF (validation run). > q39a - 3 extra rows in SparkSQL output (eg. > [1,1155,1,184.0,NaN,1,1155,2,343.3,1.1700233592269733]) ; q39b > - 3 extra rows in SparkSQL output (eg. > [1,1155,1,184.0,NaN,1,1155,2,343.3,1.1700233592269733]) > Actual results 39a: > {noformat} > [1,265,1,324.75,1.2438391781531353,1,265,2,329.0,1.0151581328149208] > [1,363,1,499.5,1.031941572270649,1,363,2,321.0,1.1411766752007977] > [1,679,1,373.75,1.0955498064867504,1,679,2,417.5,1.042970994259454] > [1,695,1,450.75,1.0835888283564505,1,695,2,368.75,1.1356494125569416] > [1,789,1,357.25,1.03450938027956,1,789,2,410.0,1.0284221852702604] > [1,815,1,216.5,1.1702270938111008,1,815,2,150.5,1.3057281471249382] > [1,827,1,271.75,1.1046890134130438,1,827,2,424.75,1.1653198631238286] > [1,1041,1,382.5,1.284808399803008,1,1041,2,424.75,1.000577271456812] > [1,1155,1,184.0,NaN,1,1155,2,343.3,1.1700233592269733] > [1,1569,1,212.0,1.630213519639535,1,1569,2,239.25,1.2641513267800557] > [1,1623,1,338.25,1.1285483279713715,1,1623,2,261.3,1.2717809002195564] > [1,2581,1,448.5,1.060429041250449,1,2581,2,476.25,1.0362984739390064] > [1,2705,1,246.25,1.0120308357959693,1,2705,2,294.7,1.0742134101583702] > [1,3131,1,393.75,1.0037613982687346,1,3131,2,480.5,1.0669144981482768] > [1,3291,1,374.5,1.195189833087008,1,3291,2,265.25,1.572972106948466] > [1,3687,1,279.75,1.4260909081999698,1,3687,2,157.25,1.4534340882531784] > [1,4955,1,495.25,1.0318296151625301,1,4955,2,322.5,1.1693842343776149] > [1,5627,1,282.75,1.5657032366359889,1,5627,2,297.5,1.2084286841430678] > [1,7017,1,175.5,1.0427454215644427,1,7017,2,321.3,1.0183356932936254] > [1,7317,1,366.3,1.025466403613547,1,7317,2,378.0,1.2172513189920555] > [1,7569,1,430.5,1.0874396852180854,1,7569,2,360.25,1.047005559314515] > [1,7999,1,166.25,1.7924231710846223,1,7999,2,375.3,1.008092263550718] > [1,8319,1,306.75,1.1615378040478215,1,8319,2,276.0,1.1420996385609428] > [1,8443,1,327.75,1.256718374192724,1,8443,2,332.5,1.0044167259988928] > [1,8583,1,319.5,1.024108893111539,1,8583,2,310.25,1.2358813775861328] > [1,8591,1,398.0,1.1478168692042447,1,8591,2,355.75,1.0024472149348966] > [1,8611,1,300.5,1.5191545184147954,1,8611,2,243.75,1.2342122780960432] > [1,9081,1,367.0,1.0878932141280895,1,9081,2,435.0,1.0330530776324107] >
[jira] [Comment Edited] (SPARK-13860) TPCDS query 39 returns wrong results compared to TPC official result set
[ https://issues.apache.org/jira/browse/SPARK-13860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17320450#comment-17320450 ] koert kuipers edited comment on SPARK-13860 at 4/13/21, 6:33 PM: - this is what i now see: {code:java} scala> Seq(1.0, 2.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ |0.7071067811865476| +--+ scala> Seq(1.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ | null| +--+ {code} so basically now when we have division by zero we return null instead of NaN. i think for division by zero NaN is a far better answer than null? it captures exactly what the issue is. us changing correct answers to match what some other SQL system does (which is a system with many warts and mistakes we should not try to emulate) saddens me. was (Author: koert): this is what i now see: {code:java} scala> Seq(1.0, 2.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ |0.7071067811865476| +--+ scala> Seq(1.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ | null| +--+ {code} so basically now when we have division by zero we return null instead of NaN. i think for division by zero NaN is a far better answer than null? it captures exactly what the issue is. us changing correct answers to match what some other SQL does (which is a system with many warts and mistakes we should not try to emulate) saddens me. > TPCDS query 39 returns wrong results compared to TPC official result set > - > > Key: SPARK-13860 > URL: https://issues.apache.org/jira/browse/SPARK-13860 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 1.6.0, 2.1.1, 2.2.0, 2.3.4, 2.4.7, 3.0.1 >Reporter: JESSE CHEN >Assignee: Leanken.Lin >Priority: Major > Labels: correctness > Fix For: 3.1.0 > > > Testing Spark SQL using TPC queries. Query 39 returns wrong results compared > to official result set. This is at 1GB SF (validation run). > q39a - 3 extra rows in SparkSQL output (eg. > [1,1155,1,184.0,NaN,1,1155,2,343.3,1.1700233592269733]) ; q39b > - 3 extra rows in SparkSQL output (eg. > [1,1155,1,184.0,NaN,1,1155,2,343.3,1.1700233592269733]) > Actual results 39a: > {noformat} > [1,265,1,324.75,1.2438391781531353,1,265,2,329.0,1.0151581328149208] > [1,363,1,499.5,1.031941572270649,1,363,2,321.0,1.1411766752007977] > [1,679,1,373.75,1.0955498064867504,1,679,2,417.5,1.042970994259454] > [1,695,1,450.75,1.0835888283564505,1,695,2,368.75,1.1356494125569416] > [1,789,1,357.25,1.03450938027956,1,789,2,410.0,1.0284221852702604] > [1,815,1,216.5,1.1702270938111008,1,815,2,150.5,1.3057281471249382] > [1,827,1,271.75,1.1046890134130438,1,827,2,424.75,1.1653198631238286] > [1,1041,1,382.5,1.284808399803008,1,1041,2,424.75,1.000577271456812] > [1,1155,1,184.0,NaN,1,1155,2,343.3,1.1700233592269733] > [1,1569,1,212.0,1.630213519639535,1,1569,2,239.25,1.2641513267800557] > [1,1623,1,338.25,1.1285483279713715,1,1623,2,261.3,1.2717809002195564] > [1,2581,1,448.5,1.060429041250449,1,2581,2,476.25,1.0362984739390064] > [1,2705,1,246.25,1.0120308357959693,1,2705,2,294.7,1.0742134101583702] > [1,3131,1,393.75,1.0037613982687346,1,3131,2,480.5,1.0669144981482768] > [1,3291,1,374.5,1.195189833087008,1,3291,2,265.25,1.572972106948466] > [1,3687,1,279.75,1.4260909081999698,1,3687,2,157.25,1.4534340882531784] > [1,4955,1,495.25,1.0318296151625301,1,4955,2,322.5,1.1693842343776149] > [1,5627,1,282.75,1.5657032366359889,1,5627,2,297.5,1.2084286841430678] > [1,7017,1,175.5,1.0427454215644427,1,7017,2,321.3,1.0183356932936254] > [1,7317,1,366.3,1.025466403613547,1,7317,2,378.0,1.2172513189920555] > [1,7569,1,430.5,1.0874396852180854,1,7569,2,360.25,1.047005559314515] > [1,7999,1,166.25,1.7924231710846223,1,7999,2,375.3,1.008092263550718] > [1,8319,1,306.75,1.1615378040478215,1,8319,2,276.0,1.1420996385609428] > [1,8443,1,327.75,1.256718374192724,1,8443,2,332.5,1.0044167259988928] > [1,8583,1,319.5,1.024108893111539,1,8583,2,310.25,1.2358813775861328] > [1,8591,1,398.0,1.1478168692042447,1,8591,2,355.75,1.0024472149348966] > [1,8611,1,300.5,1.5191545184147954,1,8611,2,243.75,1.2342122780960432] > [1,9081,1,367.0,1.0878932141280895,1,9081,2,435.0,1.0330530776324107] >
[jira] [Commented] (SPARK-13860) TPCDS query 39 returns wrong results compared to TPC official result set
[ https://issues.apache.org/jira/browse/SPARK-13860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17320450#comment-17320450 ] koert kuipers commented on SPARK-13860: --- this is what i now see: {code:java} scala> Seq(1.0, 2.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ |0.7071067811865476| +--+ scala> Seq(1.0).toDF.select(stddev(col("value"))).show +--+ |stddev_samp(value)| +--+ | null| +--+ {code} so basically now when we have division by zero we return null instead of NaN. i think for division by zero NaN is a far better answer than null? it captures exactly what the issue is. us changing correct answers to match what some other SQL does (which is a system with many warts and mistakes we should not try to emulate) saddens me. > TPCDS query 39 returns wrong results compared to TPC official result set > - > > Key: SPARK-13860 > URL: https://issues.apache.org/jira/browse/SPARK-13860 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 1.6.0, 2.1.1, 2.2.0, 2.3.4, 2.4.7, 3.0.1 >Reporter: JESSE CHEN >Assignee: Leanken.Lin >Priority: Major > Labels: correctness > Fix For: 3.1.0 > > > Testing Spark SQL using TPC queries. Query 39 returns wrong results compared > to official result set. This is at 1GB SF (validation run). > q39a - 3 extra rows in SparkSQL output (eg. > [1,1155,1,184.0,NaN,1,1155,2,343.3,1.1700233592269733]) ; q39b > - 3 extra rows in SparkSQL output (eg. > [1,1155,1,184.0,NaN,1,1155,2,343.3,1.1700233592269733]) > Actual results 39a: > {noformat} > [1,265,1,324.75,1.2438391781531353,1,265,2,329.0,1.0151581328149208] > [1,363,1,499.5,1.031941572270649,1,363,2,321.0,1.1411766752007977] > [1,679,1,373.75,1.0955498064867504,1,679,2,417.5,1.042970994259454] > [1,695,1,450.75,1.0835888283564505,1,695,2,368.75,1.1356494125569416] > [1,789,1,357.25,1.03450938027956,1,789,2,410.0,1.0284221852702604] > [1,815,1,216.5,1.1702270938111008,1,815,2,150.5,1.3057281471249382] > [1,827,1,271.75,1.1046890134130438,1,827,2,424.75,1.1653198631238286] > [1,1041,1,382.5,1.284808399803008,1,1041,2,424.75,1.000577271456812] > [1,1155,1,184.0,NaN,1,1155,2,343.3,1.1700233592269733] > [1,1569,1,212.0,1.630213519639535,1,1569,2,239.25,1.2641513267800557] > [1,1623,1,338.25,1.1285483279713715,1,1623,2,261.3,1.2717809002195564] > [1,2581,1,448.5,1.060429041250449,1,2581,2,476.25,1.0362984739390064] > [1,2705,1,246.25,1.0120308357959693,1,2705,2,294.7,1.0742134101583702] > [1,3131,1,393.75,1.0037613982687346,1,3131,2,480.5,1.0669144981482768] > [1,3291,1,374.5,1.195189833087008,1,3291,2,265.25,1.572972106948466] > [1,3687,1,279.75,1.4260909081999698,1,3687,2,157.25,1.4534340882531784] > [1,4955,1,495.25,1.0318296151625301,1,4955,2,322.5,1.1693842343776149] > [1,5627,1,282.75,1.5657032366359889,1,5627,2,297.5,1.2084286841430678] > [1,7017,1,175.5,1.0427454215644427,1,7017,2,321.3,1.0183356932936254] > [1,7317,1,366.3,1.025466403613547,1,7317,2,378.0,1.2172513189920555] > [1,7569,1,430.5,1.0874396852180854,1,7569,2,360.25,1.047005559314515] > [1,7999,1,166.25,1.7924231710846223,1,7999,2,375.3,1.008092263550718] > [1,8319,1,306.75,1.1615378040478215,1,8319,2,276.0,1.1420996385609428] > [1,8443,1,327.75,1.256718374192724,1,8443,2,332.5,1.0044167259988928] > [1,8583,1,319.5,1.024108893111539,1,8583,2,310.25,1.2358813775861328] > [1,8591,1,398.0,1.1478168692042447,1,8591,2,355.75,1.0024472149348966] > [1,8611,1,300.5,1.5191545184147954,1,8611,2,243.75,1.2342122780960432] > [1,9081,1,367.0,1.0878932141280895,1,9081,2,435.0,1.0330530776324107] > [1,9357,1,351.7,1.1902922622025887,1,9357,2,427.0,1.0438583026358363] > [1,9449,1,406.25,1.0183183104803557,1,9449,2,175.0,1.0544779796296408] > [1,9713,1,242.5,1.1035044355064203,1,9713,2,393.0,1.208474608738988] > [1,9809,1,479.0,1.0189602512117633,1,9809,2,317.5,1.0614142074924882] > [1,9993,1,417.75,1.0099832672435247,1,9993,2,204.5,1.552870745350107] > [1,10127,1,239.75,1.0561770587198123,1,10127,2,359.25,1.1857980403742183] > [1,11159,1,407.25,1.0785507154337637,1,11159,2,250.0,1.334757905639321] > [1,11277,1,211.25,1.2615858275316627,1,11277,2,330.75,1.0808767951625093] > [1,11937,1,344.5,1.085804026843784,1,11937,2,200.34,1.0638527063883725] > [1,12373,1,387.75,1.1014904822941258,1,12373,2,306.0,1.0761744390394028] > [1,12471,1,365.25,1.0607570183728479,1,12471,2,327.25,1.0547560580567852] > [1,12625,1,279.0,1.3016560542373208,1,12625,2,443.25,1.0604958838068959] >
[jira] [Commented] (SPARK-32109) SQL hash function handling of nulls makes collision too likely
[ https://issues.apache.org/jira/browse/SPARK-32109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17147358#comment-17147358 ] koert kuipers commented on SPARK-32109: --- the issue is that Row here isnt really a sequence. it represent an object. if you have say an object Person(name: String, nickname: String) you would not want Person("john", null) and Person(null, "john") to have same hashCode. see for example the suggested hashcode implementations in effective java by joshua bloch. they do something similar to what you suggest to solve this problem. so unfortunately i think our current implementation is flawed :( PS even for pure sequences i do not think this implementation as it is right now is acceptable. but that is less of a worry than the object represenation of row. > SQL hash function handling of nulls makes collision too likely > -- > > Key: SPARK-32109 > URL: https://issues.apache.org/jira/browse/SPARK-32109 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 >Reporter: koert kuipers >Priority: Minor > > this ticket is about org.apache.spark.sql.functions.hash and sparks handling > of nulls when hashing sequences. > {code:java} > scala> spark.sql("SELECT hash('bar', null)").show() > +---+ > |hash(bar, NULL)| > +---+ > |-1808790533| > +---+ > scala> spark.sql("SELECT hash(null, 'bar')").show() > +---+ > |hash(NULL, bar)| > +---+ > |-1808790533| > +---+ > {code} > these are differences sequences. e.g. these could be positions 0 and 1 in a > dataframe which are diffferent columns with entirely different meanings. the > hashes should not be the same. > another example: > {code:java} > scala> Seq(("john", null), (null, "john")).toDF("name", > "alias").withColumn("hash", hash(col("name"), col("alias"))).show > ++-+-+ > |name|alias| hash| > ++-+-+ > |john| null|487839701| > |null| john|487839701| > ++-+-+ {code} > instead of ignoring nulls each null show do a transform to the hash so that > the order of elements including the nulls matters for the outcome. > -- 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-32109) SQL hash function handling of nulls makes collision too likely
[ https://issues.apache.org/jira/browse/SPARK-32109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17147358#comment-17147358 ] koert kuipers edited comment on SPARK-32109 at 6/28/20, 2:58 PM: - the issue is that row here isnt really a sequence. it represent an object. if you have say an object Person(name: String, nickname: String) you would not want Person("john", null) and Person(null, "john") to have same hashCode. see for example the suggested hashcode implementations in effective java by joshua bloch. they do something similar to what you suggest to solve this problem. so unfortunately i think our current implementation is flawed :( p.s. even for pure sequences i do not think this implementation as it is right now is acceptable. but that is less of a worry than the object represenation of row. was (Author: koert): the issue is that Row here isnt really a sequence. it represent an object. if you have say an object Person(name: String, nickname: String) you would not want Person("john", null) and Person(null, "john") to have same hashCode. see for example the suggested hashcode implementations in effective java by joshua bloch. they do something similar to what you suggest to solve this problem. so unfortunately i think our current implementation is flawed :( PS even for pure sequences i do not think this implementation as it is right now is acceptable. but that is less of a worry than the object represenation of row. > SQL hash function handling of nulls makes collision too likely > -- > > Key: SPARK-32109 > URL: https://issues.apache.org/jira/browse/SPARK-32109 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 >Reporter: koert kuipers >Priority: Minor > > this ticket is about org.apache.spark.sql.functions.hash and sparks handling > of nulls when hashing sequences. > {code:java} > scala> spark.sql("SELECT hash('bar', null)").show() > +---+ > |hash(bar, NULL)| > +---+ > |-1808790533| > +---+ > scala> spark.sql("SELECT hash(null, 'bar')").show() > +---+ > |hash(NULL, bar)| > +---+ > |-1808790533| > +---+ > {code} > these are differences sequences. e.g. these could be positions 0 and 1 in a > dataframe which are diffferent columns with entirely different meanings. the > hashes should not be the same. > another example: > {code:java} > scala> Seq(("john", null), (null, "john")).toDF("name", > "alias").withColumn("hash", hash(col("name"), col("alias"))).show > ++-+-+ > |name|alias| hash| > ++-+-+ > |john| null|487839701| > |null| john|487839701| > ++-+-+ {code} > instead of ignoring nulls each null show do a transform to the hash so that > the order of elements including the nulls matters for the outcome. > -- 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-32109) SQL hash function handling of nulls makes collision too likely
[ https://issues.apache.org/jira/browse/SPARK-32109?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-32109: -- Description: this ticket is about org.apache.spark.sql.functions.hash and sparks handling of nulls when hashing sequences. {code:java} scala> spark.sql("SELECT hash('bar', null)").show() +---+ |hash(bar, NULL)| +---+ |-1808790533| +---+ scala> spark.sql("SELECT hash(null, 'bar')").show() +---+ |hash(NULL, bar)| +---+ |-1808790533| +---+ {code} these are differences sequences. e.g. these could be positions 0 and 1 in a dataframe which are diffferent columns with entirely different meanings. the hashes should not be the same. another example: {code:java} scala> Seq(("john", null), (null, "john")).toDF("name", "alias").withColumn("hash", hash(col("name"), col("alias"))).show ++-+-+ |name|alias| hash| ++-+-+ |john| null|487839701| |null| john|487839701| ++-+-+ {code} instead of ignoring nulls each null show do a transform to the hash so that the order of elements including the nulls matters for the outcome. was: this ticket is about org.apache.spark.sql.functions.hash and sparks handling of nulls when hashing sequences. {code:java} scala> spark.sql("SELECT hash('bar', null)").show() +---+ |hash(bar, NULL)| +---+ |-1808790533| +---+ scala> spark.sql("SELECT hash(null, 'bar')").show() +---+ |hash(NULL, bar)| +---+ |-1808790533| +---+ {code} these are differences sequences. e.g. these could be positions 0 and 1 in a dataframe which are diffferent columns with entirely different meanings. the hashes should bot be the same. another example: {code:java} scala> Seq(("john", null), (null, "john")).toDF("name", "alias").withColumn("hash", hash(col("name"), col("alias"))).show ++-+-+ |name|alias| hash| ++-+-+ |john| null|487839701| |null| john|487839701| ++-+-+ {code} > SQL hash function handling of nulls makes collision too likely > -- > > Key: SPARK-32109 > URL: https://issues.apache.org/jira/browse/SPARK-32109 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 >Reporter: koert kuipers >Priority: Minor > > this ticket is about org.apache.spark.sql.functions.hash and sparks handling > of nulls when hashing sequences. > {code:java} > scala> spark.sql("SELECT hash('bar', null)").show() > +---+ > |hash(bar, NULL)| > +---+ > |-1808790533| > +---+ > scala> spark.sql("SELECT hash(null, 'bar')").show() > +---+ > |hash(NULL, bar)| > +---+ > |-1808790533| > +---+ > {code} > these are differences sequences. e.g. these could be positions 0 and 1 in a > dataframe which are diffferent columns with entirely different meanings. the > hashes should not be the same. > another example: > {code:java} > scala> Seq(("john", null), (null, "john")).toDF("name", > "alias").withColumn("hash", hash(col("name"), col("alias"))).show > ++-+-+ > |name|alias| hash| > ++-+-+ > |john| null|487839701| > |null| john|487839701| > ++-+-+ {code} > instead of ignoring nulls each null show do a transform to the hash so that > the order of elements including the nulls matters for the outcome. > -- 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-32109) SQL hash function handling of nulls makes collision too likely
koert kuipers created SPARK-32109: - Summary: SQL hash function handling of nulls makes collision too likely Key: SPARK-32109 URL: https://issues.apache.org/jira/browse/SPARK-32109 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.0.0 Reporter: koert kuipers this ticket is about org.apache.spark.sql.functions.hash and sparks handling of nulls when hashing sequences. {code:java} scala> spark.sql("SELECT hash('bar', null)").show() +---+ |hash(bar, NULL)| +---+ |-1808790533| +---+ scala> spark.sql("SELECT hash(null, 'bar')").show() +---+ |hash(NULL, bar)| +---+ |-1808790533| +---+ {code} these are differences sequences. e.g. these could be positions 0 and 1 in a dataframe which are diffferent columns with entirely different meanings. the hashes should bot be the same. another example: {code:java} scala> Seq(("john", null), (null, "john")).toDF("name", "alias").withColumn("hash", hash(col("name"), col("alias"))).show ++-+-+ |name|alias| hash| ++-+-+ |john| null|487839701| |null| john|487839701| ++-+-+ {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] [Updated] (SPARK-32056) Repartition by key should support partition coalesce for AQE
[ https://issues.apache.org/jira/browse/SPARK-32056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-32056: -- Priority: Minor (was: Major) > Repartition by key should support partition coalesce for AQE > > > Key: SPARK-32056 > URL: https://issues.apache.org/jira/browse/SPARK-32056 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 > Environment: spark release 3.0.0 >Reporter: koert kuipers >Priority: Minor > > when adaptive query execution is enabled the following expression should > support coalescing of partitions: > {code:java} > dataframe.repartition(col("somecolumn")) {code} > currently it does not because it simply calls the repartition implementation > where number of partitions is specified: > {code:java} > def repartition(partitionExprs: Column*): Dataset[T] = { > repartition(sparkSession.sessionState.conf.numShufflePartitions, > partitionExprs: _*) > }{code} > and repartition with the number of partitions specified does now allow for > coalescing of partitions (since this breaks the user's expectation that it > will have the number of partitions specified). > for more context see the discussion here: > [https://github.com/apache/spark/pull/27986] > a simple test to confirm that repartition by key does not support coalescing > of partitions can be added in AdaptiveQueryExecSuite like this (it currently > fails): > {code:java} > test("SPARK-32056 repartition has less partitions for small data when > adaptiveExecutionEnabled") { > Seq(true, false).foreach { enableAQE => > withSQLConf( > SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, > SQLConf.SHUFFLE_PARTITIONS.key -> "50", > SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "50", > SQLConf.SHUFFLE_PARTITIONS.key -> "50") { > val partitionsNum = (1 to 10).toDF.repartition($"value") > .rdd.collectPartitions().length > if (enableAQE) { > assert(partitionsNum < 50) > } else { > assert(partitionsNum === 50) > } > } > } > } > {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] [Updated] (SPARK-32056) Repartition by key should support partition coalesce for AQE
[ https://issues.apache.org/jira/browse/SPARK-32056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-32056: -- Description: when adaptive query execution is enabled the following expression should support coalescing of partitions: {code:java} dataframe.repartition(col("somecolumn")) {code} currently it does not because it simply calls the repartition implementation where number of partitions is specified: {code:java} def repartition(partitionExprs: Column*): Dataset[T] = { repartition(sparkSession.sessionState.conf.numShufflePartitions, partitionExprs: _*) }{code} and repartition with the number of partitions specified does now allow for coalescing of partitions (since this breaks the user's expectation that it will have the number of partitions specified). for more context see the discussion here: [https://github.com/apache/spark/pull/27986] a simple test to confirm that repartition by key does not support coalescing of partitions can be added in AdaptiveQueryExecSuite like this (it currently fails): {code:java} test("SPARK-32056 repartition has less partitions for small data when adaptiveExecutionEnabled") { Seq(true, false).foreach { enableAQE => withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, SQLConf.SHUFFLE_PARTITIONS.key -> "50", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "50", SQLConf.SHUFFLE_PARTITIONS.key -> "50") { val partitionsNum = (1 to 10).toDF.repartition($"value") .rdd.collectPartitions().length if (enableAQE) { assert(partitionsNum < 50) } else { assert(partitionsNum === 50) } } } } {code} was: when adaptive query execution is enabled the following expression should support coalescing of partitions: {code:java} dataframe.repartition(col("somecolumn")) {code} currently it does not because it simply calls the repartition implementation where number of partitions is specified: {code:java} def repartition(partitionExprs: Column*): Dataset[T] = { repartition(sparkSession.sessionState.conf.numShufflePartitions, partitionExprs: _*) }{code} and repartition with the number of partitions specified does now allow for coalescing of partitions (since this breaks the user's expectation that it will have the number of partitions specified). for more context see the discussion here: [https://github.com/apache/spark/pull/27986] a simple test to confirm that repartition by key does not support coalescing of partitions can be added in AdaptiveQueryExecSuite like this (it currently fails): {code:java} test("SPARK-? repartition has less partitions for small data when adaptiveExecutionEnabled") { Seq(true, false).foreach { enableAQE => withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, SQLConf.SHUFFLE_PARTITIONS.key -> "50", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "50", SQLConf.SHUFFLE_PARTITIONS.key -> "50") { val partitionsNum = (1 to 10).toDF.repartition($"value") .rdd.collectPartitions().length if (enableAQE) { assert(partitionsNum < 50) } else { assert(partitionsNum === 50) } } } } {code} > Repartition by key should support partition coalesce for AQE > > > Key: SPARK-32056 > URL: https://issues.apache.org/jira/browse/SPARK-32056 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 > Environment: spark release 3.0.0 >Reporter: koert kuipers >Priority: Major > > when adaptive query execution is enabled the following expression should > support coalescing of partitions: > {code:java} > dataframe.repartition(col("somecolumn")) {code} > currently it does not because it simply calls the repartition implementation > where number of partitions is specified: > {code:java} > def repartition(partitionExprs: Column*): Dataset[T] = { > repartition(sparkSession.sessionState.conf.numShufflePartitions, > partitionExprs: _*) > }{code} > and repartition with the number of partitions specified does now allow for > coalescing of partitions (since this breaks the user's expectation that it > will have the number of partitions specified). > for more context see the discussion here: > [https://github.com/apache/spark/pull/27986] > a simple test to confirm that repartition by key does not support coalescing > of partitions can be added in AdaptiveQueryExecSuite like this (it currently > fails): > {code:java} > test("SPARK-32056 repartition has less partitions for small data when > adaptiveExecutionEnabled") { > Seq(true, false).foreach { enableAQE => > withSQLConf(
[jira] [Created] (SPARK-32056) Repartition by key should support partition coalesce for AQE
koert kuipers created SPARK-32056: - Summary: Repartition by key should support partition coalesce for AQE Key: SPARK-32056 URL: https://issues.apache.org/jira/browse/SPARK-32056 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.0.0 Environment: spark release 3.0.0 Reporter: koert kuipers when adaptive query execution is enabled the following expression should support coalescing of partitions: {code:java} dataframe.repartition(col("somecolumn")) {code} currently it does not because it simply calls the repartition implementation where number of partitions is specified: {code:java} def repartition(partitionExprs: Column*): Dataset[T] = { repartition(sparkSession.sessionState.conf.numShufflePartitions, partitionExprs: _*) }{code} and repartition with the number of partitions specified does now allow for coalescing of partitions (since this breaks the user's expectation that it will have the number of partitions specified). for more context see the discussion here: [https://github.com/apache/spark/pull/27986] a simple test to confirm that repartition by key does not support coalescing of partitions can be added in AdaptiveQueryExecSuite like this (it currently fails): {code:java} test("SPARK-? repartition has less partitions for small data when adaptiveExecutionEnabled") { Seq(true, false).foreach { enableAQE => withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, SQLConf.SHUFFLE_PARTITIONS.key -> "50", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "50", SQLConf.SHUFFLE_PARTITIONS.key -> "50") { val partitionsNum = (1 to 10).toDF.repartition($"value") .rdd.collectPartitions().length if (enableAQE) { assert(partitionsNum < 50) } else { assert(partitionsNum === 50) } } } } {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-31841) Dataset.repartition leverage adaptive execution
[ https://issues.apache.org/jira/browse/SPARK-31841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17118120#comment-17118120 ] koert kuipers commented on SPARK-31841: --- that is right. its a feature request i believe (unless i misunderstood whats happening). should i delete it here? > Dataset.repartition leverage adaptive execution > --- > > Key: SPARK-31841 > URL: https://issues.apache.org/jira/browse/SPARK-31841 > Project: Spark > Issue Type: Improvement > Components: SQL >Affects Versions: 3.0.0 > Environment: spark branch-3.0 from may 1 this year >Reporter: koert kuipers >Priority: Minor > > hello, > we are very happy users of adaptive query execution. its a great feature to > now have to think about and tune the number of partitions anymore in a > shuffle. > i noticed that Dataset.groupBy consistently uses adaptive execution when its > enabled (e.g. i don't see the default 200 partitions) but when i do > Dataset.repartition it seems i am back to a hardcoded number of partitions. > Should adaptive execution also be used for repartition? It would be nice to > be able to repartition without having to think about optimal number of > partitions. > An example: > {code:java} > $ spark-shell --conf spark.sql.adaptive.enabled=true --conf > spark.sql.adaptive.advisoryPartitionSizeInBytes=10 > Welcome to > __ > / __/__ ___ _/ /__ > _\ \/ _ \/ _ `/ __/ '_/ >/___/ .__/\_,_/_/ /_/\_\ version 3.0.0-SNAPSHOT > /_/ > > Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_252) > Type in expressions to have them evaluated. > Type :help for more information. > scala> val x = (1 to 100).toDF > x: org.apache.spark.sql.DataFrame = [value: int] > scala> x.rdd.getNumPartitions > res0: Int = 2scala> x.repartition($"value").rdd.getNumPartitions > res1: Int = 200 > scala> x.groupBy("value").count.rdd.getNumPartitions > res2: Int = 67 > {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] [Created] (SPARK-31841) Dataset.repartition leverage adaptive execution
koert kuipers created SPARK-31841: - Summary: Dataset.repartition leverage adaptive execution Key: SPARK-31841 URL: https://issues.apache.org/jira/browse/SPARK-31841 Project: Spark Issue Type: Improvement Components: SQL Affects Versions: 3.0.0 Environment: spark branch-3.0 from may 1 this year Reporter: koert kuipers hello, we are very happy users of adaptive query execution. its a great feature to now have to think about and tune the number of partitions anymore in a shuffle. i noticed that Dataset.groupBy consistently uses adaptive execution when its enabled (e.g. i don't see the default 200 partitions) but when i do Dataset.repartition it seems i am back to a hardcoded number of partitions. Should adaptive execution also be used for repartition? It would be nice to be able to repartition without having to think about optimal number of partitions. An example: {code:java} $ spark-shell --conf spark.sql.adaptive.enabled=true --conf spark.sql.adaptive.advisoryPartitionSizeInBytes=10 Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 3.0.0-SNAPSHOT /_/ Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_252) Type in expressions to have them evaluated. Type :help for more information. scala> val x = (1 to 100).toDF x: org.apache.spark.sql.DataFrame = [value: int] scala> x.rdd.getNumPartitions res0: Int = 2scala> x.repartition($"value").rdd.getNumPartitions res1: Int = 200 scala> x.groupBy("value").count.rdd.getNumPartitions res2: Int = 67 {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] [Created] (SPARK-31726) Make spark.files available in driver with cluster deploy mode on kubernetes
koert kuipers created SPARK-31726: - Summary: Make spark.files available in driver with cluster deploy mode on kubernetes Key: SPARK-31726 URL: https://issues.apache.org/jira/browse/SPARK-31726 Project: Spark Issue Type: Improvement Components: Kubernetes Affects Versions: 3.0.0 Reporter: koert kuipers currently on yarn with cluster deploy mode --files makes the files available for driver and executors and also put them on classpath for driver and executors. on k8s with cluster deploy mode --files makes the files available on executors but they are not on classpath. it does not make the files available on driver and they are not on driver classpath. it would be nice if the k8s behavior was consistent with yarn, or at least makes the files available on driver. once the files are available there is a simple workaround to get them on classpath using spark.driver.extraClassPath="./" background: we recently started testing kubernetes for spark. our main platform is yarn on which we use client deploy mode. our first experience was that client deploy mode was difficult to use on k8s (we dont launch from inside a pod). so we switched to cluster deploy mode, which seems to behave well on k8s. but then we realized that our program rely on reading files on classpath (application.conf, log4j.properties etc.) that are on the client but now are no longer on the driver (since driver is no longer on client). an easy fix for this seems to be to ship the files using --files to make them available on driver, but we could not get this to work. -- 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-29302) dynamic partition overwrite with speculation enabled
[ https://issues.apache.org/jira/browse/SPARK-29302?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-29302: -- Affects Version/s: (was: 3.0.0) > dynamic partition overwrite with speculation enabled > > > Key: SPARK-29302 > URL: https://issues.apache.org/jira/browse/SPARK-29302 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.4.4 >Reporter: feiwang >Priority: Major > Attachments: screenshot-1.png, screenshot-2.png > > > Now, for a dynamic partition overwrite operation, the filename of a task > output is determinable. > So, if speculation is enabled, would a task conflict with its relative > speculation task? > Would the two tasks concurrent write a same file? -- 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-29302) dynamic partition overwrite with speculation enabled
[ https://issues.apache.org/jira/browse/SPARK-29302?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-29302: -- Affects Version/s: 3.0.0 > dynamic partition overwrite with speculation enabled > > > Key: SPARK-29302 > URL: https://issues.apache.org/jira/browse/SPARK-29302 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.4.4, 3.0.0 >Reporter: feiwang >Priority: Major > Attachments: screenshot-1.png, screenshot-2.png > > > Now, for a dynamic partition overwrite operation, the filename of a task > output is determinable. > So, if speculation is enabled, would a task conflict with its relative > speculation task? > Would the two tasks concurrent write a same file? -- 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-30320) Insert overwrite to DataSource table with dynamic partition error when running multiple task attempts
[ https://issues.apache.org/jira/browse/SPARK-30320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17068005#comment-17068005 ] koert kuipers commented on SPARK-30320: --- i believe we are seeing this issue. it shows up in particular when pre-emption is turned on and we are using dynamic partition overwrite. pre-emption kills tasks, they get restarted, and then they fail again because the output directory already exists (so task throws FileAlreadyExistsException). as a result entire job fails. so i dont think this is just a speculative execution issue. this is a general issue with dynamic partition overwrite not being able to recover from task failure. > Insert overwrite to DataSource table with dynamic partition error when > running multiple task attempts > - > > Key: SPARK-30320 > URL: https://issues.apache.org/jira/browse/SPARK-30320 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.4.4, 3.0.0 >Reporter: Du Ripeng >Priority: Major > > Inserting overwrite to a DataSource table with dynamic partition might fail > when running multiple task attempts. Suppose there are a task attempt and a > speculative task attempt, the speculative attempt would raise > FileAlreadyExistException -- 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-30320) Insert overwrite to DataSource table with dynamic partition error when running multiple task attempts
[ https://issues.apache.org/jira/browse/SPARK-30320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17068005#comment-17068005 ] koert kuipers edited comment on SPARK-30320 at 3/26/20, 8:11 PM: - i believe we are seeing this issue. it shows up in particular when pre-emption is turned on and we are using dynamic partition overwrite. pre-emption kills tasks, they get restarted, and then they fail again because the output directory already exists (so task throws FileAlreadyExistsException). as a result entire job fails. so i dont think this is just a speculative execution issue. this is a general issue with dynamic partition overwrite not being able to recover from task failure. see also SPARK-29302 which is same issue i believe. was (Author: koert): i believe we are seeing this issue. it shows up in particular when pre-emption is turned on and we are using dynamic partition overwrite. pre-emption kills tasks, they get restarted, and then they fail again because the output directory already exists (so task throws FileAlreadyExistsException). as a result entire job fails. so i dont think this is just a speculative execution issue. this is a general issue with dynamic partition overwrite not being able to recover from task failure. > Insert overwrite to DataSource table with dynamic partition error when > running multiple task attempts > - > > Key: SPARK-30320 > URL: https://issues.apache.org/jira/browse/SPARK-30320 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.4.4, 3.0.0 >Reporter: Du Ripeng >Priority: Major > > Inserting overwrite to a DataSource table with dynamic partition might fail > when running multiple task attempts. Suppose there are a task attempt and a > speculative task attempt, the speculative attempt would raise > FileAlreadyExistException -- 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-29302) dynamic partition overwrite with speculation enabled
[ https://issues.apache.org/jira/browse/SPARK-29302?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17067972#comment-17067972 ] koert kuipers edited comment on SPARK-29302 at 3/26/20, 7:23 PM: - i believe we are seeing this issue. it shows up in particular when pre-emption is turned on and we are using dynamic partition overwrite. pre-emption kills tasks, they get restarted, and then they fail again because the output directory already exists (so task throws FileAlreadyExistsException). as a result entire job fails. so i dont think this is just a speculative execution issue. this is a general issue with dynamic partition overwrite not being able to recover from task failure. was (Author: koert): i believe we are seeing this issue. it shows up in particular when pre-emption is turned on and we are using dynamic partition overwrite. pre-emption kills tasks, they get restarted, and then they fail again because the output directory already exists (so task throws FileAlreadyExistsException). as a result entire job fails. > dynamic partition overwrite with speculation enabled > > > Key: SPARK-29302 > URL: https://issues.apache.org/jira/browse/SPARK-29302 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.4.4 >Reporter: feiwang >Priority: Major > Attachments: screenshot-1.png, screenshot-2.png > > > Now, for a dynamic partition overwrite operation, the filename of a task > output is determinable. > So, if speculation is enabled, would a task conflict with its relative > speculation task? > Would the two tasks concurrent write a same file? -- 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-29302) dynamic partition overwrite with speculation enabled
[ https://issues.apache.org/jira/browse/SPARK-29302?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17067972#comment-17067972 ] koert kuipers commented on SPARK-29302: --- i believe we are seeing this issue. it shows up in particular when pre-emption is turned on and we are using dynamic partition overwrite. pre-emption kills tasks, they get restarted, and then they fail again because the output directory alreay exsists (so task throws FileAlreadyExistsException). as a result entire job fails. > dynamic partition overwrite with speculation enabled > > > Key: SPARK-29302 > URL: https://issues.apache.org/jira/browse/SPARK-29302 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.4.4 >Reporter: feiwang >Priority: Major > Attachments: screenshot-1.png, screenshot-2.png > > > Now, for a dynamic partition overwrite operation, the filename of a task > output is determinable. > So, if speculation is enabled, would a task conflict with its relative > speculation task? > Would the two tasks concurrent write a same file? -- 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-29302) dynamic partition overwrite with speculation enabled
[ https://issues.apache.org/jira/browse/SPARK-29302?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17067972#comment-17067972 ] koert kuipers edited comment on SPARK-29302 at 3/26/20, 7:16 PM: - i believe we are seeing this issue. it shows up in particular when pre-emption is turned on and we are using dynamic partition overwrite. pre-emption kills tasks, they get restarted, and then they fail again because the output directory already exists (so task throws FileAlreadyExistsException). as a result entire job fails. was (Author: koert): i believe we are seeing this issue. it shows up in particular when pre-emption is turned on and we are using dynamic partition overwrite. pre-emption kills tasks, they get restarted, and then they fail again because the output directory alreay exsists (so task throws FileAlreadyExistsException). as a result entire job fails. > dynamic partition overwrite with speculation enabled > > > Key: SPARK-29302 > URL: https://issues.apache.org/jira/browse/SPARK-29302 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.4.4 >Reporter: feiwang >Priority: Major > Attachments: screenshot-1.png, screenshot-2.png > > > Now, for a dynamic partition overwrite operation, the filename of a task > output is determinable. > So, if speculation is enabled, would a task conflict with its relative > speculation task? > Would the two tasks concurrent write a same file? -- 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-31183) Incompatible Avro dates/timestamps with Spark 2.4
[ https://issues.apache.org/jira/browse/SPARK-31183?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17063725#comment-17063725 ] koert kuipers edited comment on SPARK-31183 at 3/22/20, 2:03 PM: - hi! all spark tests pass for me except hive and avro. hive always seems to fail and i never figured out why so i ignore it. but avro failure is new and i think its related to this ticket. i see errors like this: {code:java} - SPARK-31183: rebasing milliseconds timestamps in write *** FAILED *** [3529/9624] org.apache.spark.SparkException: Job aborted. at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:226) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:178) at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:108) at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:106) at org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:131) at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:175) at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:213) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:210) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:171) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 186.0 failed 1 times, most recent failure: Lost task 0.0 in $ tage 186.0 (TID 316, server04.tresata.com, executor driver): org.apache.spark.SparkException: Task failed while writing rows. at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:291) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205) 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:444) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:447) 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.lang.ArithmeticException: long overflow at java.lang.Math.multiplyExact(Math.java:892) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.fromMillis(DateTimeUtils.scala:185) at org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$17(AvroSerializer.scala:155) at org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$17$adapted(AvroSerializer.scala:152) at org.apache.spark.sql.avro.AvroSerializer.$anonfun$newStructConverter$2(AvroSerializer.scala:245) at org.apache.spark.sql.avro.AvroSerializer.serialize(AvroSerializer.scala:51) at org.apache.spark.sql.avro.AvroOutputWriter.write(AvroOutputWriter.scala:64) at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.write(FileFormatDataWriter.scala:140) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:273) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1411) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:281) {code} this is on branch-3.0 on master it seems all tests pass i think. i will check again. was (Author: koert): hi! all spark tests pass for me except hive and avro. hive always seems to fail and i never figured out why so i ignore it. but avro failure is new and i think its related to this ticket. i see errors like this: {code:java} - SPARK-31183: rebasing milliseconds timestamps in write *** FAILED *** [3529/9624] org.apache.spark.SparkException: Job aborted. at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:226)
[jira] [Commented] (SPARK-31183) Incompatible Avro dates/timestamps with Spark 2.4
[ https://issues.apache.org/jira/browse/SPARK-31183?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17063725#comment-17063725 ] koert kuipers commented on SPARK-31183: --- hi! all spark tests pass for me except hive and avro. hive always seems to fail and i never figured out why so i ignore it. but avro failure is new and i think its related to this ticket. i see errors like this: {code:java} - SPARK-31183: rebasing milliseconds timestamps in write *** FAILED *** [3529/9624] org.apache.spark.SparkException: Job aborted. at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:226) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:178) at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:108) at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:106) at org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:131) at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:175) at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:213) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:210) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:171) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 186.0 failed 1 times, most recent failure: Lost task 0.0 in $ tage 186.0 (TID 316, server04.tresata.com, executor driver): org.apache.spark.SparkException: Task failed while writing rows. at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:291) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205) 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:444) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:447) 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.lang.ArithmeticException: long overflow at java.lang.Math.multiplyExact(Math.java:892) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.fromMillis(DateTimeUtils.scala:185) at org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$17(AvroSerializer.scala:155) at org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$17$adapted(AvroSerializer.scala:152) at org.apache.spark.sql.avro.AvroSerializer.$anonfun$newStructConverter$2(AvroSerializer.scala:245) at org.apache.spark.sql.avro.AvroSerializer.serialize(AvroSerializer.scala:51) at org.apache.spark.sql.avro.AvroOutputWriter.write(AvroOutputWriter.scala:64) at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.write(FileFormatDataWriter.scala:140) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:273) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1411) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:281) {code} > Incompatible Avro dates/timestamps with Spark 2.4 > - > > Key: SPARK-31183 > URL: https://issues.apache.org/jira/browse/SPARK-31183 > Project: Spark > Issue Type: Sub-task > Components: SQL >Affects Versions: 3.0.0 >Reporter: Maxim Gekk >Assignee: Maxim Gekk >Priority: Major > Fix For: 3.0.0 > > > Write dates/timestamps to Avro file in Spark 2.4.5: > {code} > $ export TZ="America/Los_Angeles" > $ bin/spark-shell --packages org.apache.spark:spark-avro_2.11:2.4.5 > {code} > {code:scala} > scala> > df.write.format("avro").save("/Users/maxim/tmp/before_1582/2_4_5_ts_avro") > scala> >
[jira] [Updated] (SPARK-29906) Reading of csv file fails with adaptive execution turned on
[ https://issues.apache.org/jira/browse/SPARK-29906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-29906: -- Priority: Minor (was: Major) > Reading of csv file fails with adaptive execution turned on > --- > > Key: SPARK-29906 > URL: https://issues.apache.org/jira/browse/SPARK-29906 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 > Environment: build from master today nov 14 > commit fca0a6c394990b86304a8f9a64bf4c7ec58abbd6 (HEAD -> master, > upstream/master, upstream/HEAD) > Author: Kevin Yu > Date: Thu Nov 14 14:58:32 2019 -0600 > build using: > $ dev/make-distribution.sh --tgz -Phadoop-2.7 -Dhadoop.version=2.7.4 -Pyarn > deployed on AWS EMR 5.28 with 10 m5.xlarge slaves > in spark-env.sh: > HADOOP_CONF_DIR=/etc/hadoop/conf > in spark-defaults.conf: > spark.master yarn > spark.submit.deployMode client > spark.serializer org.apache.spark.serializer.KryoSerializer > spark.hadoop.yarn.timeline-service.enabled false > spark.driver.extraClassPath /usr/lib/hadoop-lzo/lib/hadoop-lzo.jar > spark.driver.extraLibraryPath > /usr/lib/hadoop/lib/native:/usr/lib/hadoop-lzo/lib/native > spark.executor.extraClassPath /usr/lib/hadoop-lzo/lib/hadoop-lzo.jar > spark.executor.extraLibraryPath > /usr/lib/hadoop/lib/native:/usr/lib/hadoop-lzo/lib/native >Reporter: koert kuipers >Priority: Minor > Labels: correctness > > we observed an issue where spark seems to confuse a data line (not the first > line of the csv file) for the csv header when it creates the schema. > {code} > $ wget http://download.cms.gov/openpayments/PGYR13_P062819.ZIP > $ unzip PGYR13_P062819.ZIP > $ hadoop fs -put OP_DTL_GNRL_PGYR2013_P06282019.csv > $ spark-3.0.0-SNAPSHOT-bin-2.7.4/bin/spark-shell --conf > spark.sql.adaptive.enabled=true --num-executors 10 > Setting default log level to "WARN". > To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use > setLogLevel(newLevel). > 19/11/15 00:26:47 WARN yarn.Client: Neither spark.yarn.jars nor > spark.yarn.archive is set, falling back to uploading libraries under > SPARK_HOME. > Spark context Web UI available at http://ip-xx-xxx-x-xxx.ec2.internal:4040 > Spark context available as 'sc' (master = yarn, app id = > application_1573772077642_0006). > Spark session available as 'spark'. > Welcome to > __ > / __/__ ___ _/ /__ > _\ \/ _ \/ _ `/ __/ '_/ >/___/ .__/\_,_/_/ /_/\_\ version 3.0.0-SNAPSHOT > /_/ > > Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_222) > Type in expressions to have them evaluated. > Type :help for more information. > scala> spark.read.format("csv").option("header", > true).option("enforceSchema", > false).load("OP_DTL_GNRL_PGYR2013_P06282019.csv").show(1) > 19/11/15 00:27:10 WARN util.package: Truncated the string representation of a > plan since it was too large. This behavior can be adjusted by setting > 'spark.sql.debug.maxToStringFields'. > [Stage 2:>(0 + 10) / > 17]19/11/15 00:27:11 WARN scheduler.TaskSetManager: Lost task 0.0 in stage > 2.0 (TID 35, ip-xx-xxx-x-xxx.ec2.internal, executor 1): > java.lang.IllegalArgumentException: CSV header does not conform to the schema. > Header: Change_Type, Covered_Recipient_Type, Teaching_Hospital_CCN, > Teaching_Hospital_ID, Teaching_Hospital_Name, Physician_Profile_ID, > Physician_First_Name, Physician_Middle_Name, Physician_Last_Name, > Physician_Name_Suffix, Recipient_Primary_Business_Street_Address_Line1, > Recipient_Primary_Business_Street_Address_Line2, Recipient_City, > Recipient_State, Recipient_Zip_Code, Recipient_Country, Recipient_Province, > Recipient_Postal_Code, Physician_Primary_Type, Physician_Specialty, > Physician_License_State_code1, Physician_License_State_code2, > Physician_License_State_code3, Physician_License_State_code4, > Physician_License_State_code5, > Submitting_Applicable_Manufacturer_or_Applicable_GPO_Name, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_ID, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Name, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_State, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Country, > Total_Amount_of_Payment_USDollars, Date_of_Payment, > Number_of_Payments_Included_in_Total_Amount, > Form_of_Payment_or_Transfer_of_Value, Nature_of_Payment_or_Transfer_of_Value, > City_of_Travel, State_of_Travel, Country_of_Travel, > Physician_Ownership_Indicator, Third_Party_Payment_Recipient_Indicator, > Name_of_Third_Party_Entity_Receiving_Payment_or_Transfer_of_Value, > Charity_Indicator, Third_Party_Equals_Covered_Recipient_Indicator, > Contextual_Information,
[jira] [Updated] (SPARK-29906) Reading of csv file fails with adaptive execution turned on
[ https://issues.apache.org/jira/browse/SPARK-29906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-29906: -- Labels: correctness (was: ) > Reading of csv file fails with adaptive execution turned on > --- > > Key: SPARK-29906 > URL: https://issues.apache.org/jira/browse/SPARK-29906 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 > Environment: build from master today nov 14 > commit fca0a6c394990b86304a8f9a64bf4c7ec58abbd6 (HEAD -> master, > upstream/master, upstream/HEAD) > Author: Kevin Yu > Date: Thu Nov 14 14:58:32 2019 -0600 > build using: > $ dev/make-distribution.sh --tgz -Phadoop-2.7 -Dhadoop.version=2.7.4 -Pyarn > deployed on AWS EMR 5.28 with 10 m5.xlarge slaves > in spark-env.sh: > HADOOP_CONF_DIR=/etc/hadoop/conf > in spark-defaults.conf: > spark.master yarn > spark.submit.deployMode client > spark.serializer org.apache.spark.serializer.KryoSerializer > spark.hadoop.yarn.timeline-service.enabled false > spark.driver.extraClassPath /usr/lib/hadoop-lzo/lib/hadoop-lzo.jar > spark.driver.extraLibraryPath > /usr/lib/hadoop/lib/native:/usr/lib/hadoop-lzo/lib/native > spark.executor.extraClassPath /usr/lib/hadoop-lzo/lib/hadoop-lzo.jar > spark.executor.extraLibraryPath > /usr/lib/hadoop/lib/native:/usr/lib/hadoop-lzo/lib/native >Reporter: koert kuipers >Priority: Major > Labels: correctness > > we observed an issue where spark seems to confuse a data line (not the first > line of the csv file) for the csv header when it creates the schema. > {code} > $ wget http://download.cms.gov/openpayments/PGYR13_P062819.ZIP > $ unzip PGYR13_P062819.ZIP > $ hadoop fs -put OP_DTL_GNRL_PGYR2013_P06282019.csv > $ spark-3.0.0-SNAPSHOT-bin-2.7.4/bin/spark-shell --conf > spark.sql.adaptive.enabled=true --num-executors 10 > Setting default log level to "WARN". > To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use > setLogLevel(newLevel). > 19/11/15 00:26:47 WARN yarn.Client: Neither spark.yarn.jars nor > spark.yarn.archive is set, falling back to uploading libraries under > SPARK_HOME. > Spark context Web UI available at http://ip-xx-xxx-x-xxx.ec2.internal:4040 > Spark context available as 'sc' (master = yarn, app id = > application_1573772077642_0006). > Spark session available as 'spark'. > Welcome to > __ > / __/__ ___ _/ /__ > _\ \/ _ \/ _ `/ __/ '_/ >/___/ .__/\_,_/_/ /_/\_\ version 3.0.0-SNAPSHOT > /_/ > > Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_222) > Type in expressions to have them evaluated. > Type :help for more information. > scala> spark.read.format("csv").option("header", > true).option("enforceSchema", > false).load("OP_DTL_GNRL_PGYR2013_P06282019.csv").show(1) > 19/11/15 00:27:10 WARN util.package: Truncated the string representation of a > plan since it was too large. This behavior can be adjusted by setting > 'spark.sql.debug.maxToStringFields'. > [Stage 2:>(0 + 10) / > 17]19/11/15 00:27:11 WARN scheduler.TaskSetManager: Lost task 0.0 in stage > 2.0 (TID 35, ip-xx-xxx-x-xxx.ec2.internal, executor 1): > java.lang.IllegalArgumentException: CSV header does not conform to the schema. > Header: Change_Type, Covered_Recipient_Type, Teaching_Hospital_CCN, > Teaching_Hospital_ID, Teaching_Hospital_Name, Physician_Profile_ID, > Physician_First_Name, Physician_Middle_Name, Physician_Last_Name, > Physician_Name_Suffix, Recipient_Primary_Business_Street_Address_Line1, > Recipient_Primary_Business_Street_Address_Line2, Recipient_City, > Recipient_State, Recipient_Zip_Code, Recipient_Country, Recipient_Province, > Recipient_Postal_Code, Physician_Primary_Type, Physician_Specialty, > Physician_License_State_code1, Physician_License_State_code2, > Physician_License_State_code3, Physician_License_State_code4, > Physician_License_State_code5, > Submitting_Applicable_Manufacturer_or_Applicable_GPO_Name, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_ID, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Name, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_State, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Country, > Total_Amount_of_Payment_USDollars, Date_of_Payment, > Number_of_Payments_Included_in_Total_Amount, > Form_of_Payment_or_Transfer_of_Value, Nature_of_Payment_or_Transfer_of_Value, > City_of_Travel, State_of_Travel, Country_of_Travel, > Physician_Ownership_Indicator, Third_Party_Payment_Recipient_Indicator, > Name_of_Third_Party_Entity_Receiving_Payment_or_Transfer_of_Value, > Charity_Indicator, Third_Party_Equals_Covered_Recipient_Indicator, > Contextual_Information,
[jira] [Commented] (SPARK-29906) Reading of csv file fails with adaptive execution turned on
[ https://issues.apache.org/jira/browse/SPARK-29906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16975286#comment-16975286 ] koert kuipers commented on SPARK-29906: --- note that with the default option for csv being enforceSchema=false this will not fail but produce incorrect results. therefore it is correctness issue. > Reading of csv file fails with adaptive execution turned on > --- > > Key: SPARK-29906 > URL: https://issues.apache.org/jira/browse/SPARK-29906 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 > Environment: build from master today nov 14 > commit fca0a6c394990b86304a8f9a64bf4c7ec58abbd6 (HEAD -> master, > upstream/master, upstream/HEAD) > Author: Kevin Yu > Date: Thu Nov 14 14:58:32 2019 -0600 > build using: > $ dev/make-distribution.sh --tgz -Phadoop-2.7 -Dhadoop.version=2.7.4 -Pyarn > deployed on AWS EMR 5.28 with 10 m5.xlarge slaves > in spark-env.sh: > HADOOP_CONF_DIR=/etc/hadoop/conf > in spark-defaults.conf: > spark.master yarn > spark.submit.deployMode client > spark.serializer org.apache.spark.serializer.KryoSerializer > spark.hadoop.yarn.timeline-service.enabled false > spark.driver.extraClassPath /usr/lib/hadoop-lzo/lib/hadoop-lzo.jar > spark.driver.extraLibraryPath > /usr/lib/hadoop/lib/native:/usr/lib/hadoop-lzo/lib/native > spark.executor.extraClassPath /usr/lib/hadoop-lzo/lib/hadoop-lzo.jar > spark.executor.extraLibraryPath > /usr/lib/hadoop/lib/native:/usr/lib/hadoop-lzo/lib/native >Reporter: koert kuipers >Priority: Major > Labels: correctness > > we observed an issue where spark seems to confuse a data line (not the first > line of the csv file) for the csv header when it creates the schema. > {code} > $ wget http://download.cms.gov/openpayments/PGYR13_P062819.ZIP > $ unzip PGYR13_P062819.ZIP > $ hadoop fs -put OP_DTL_GNRL_PGYR2013_P06282019.csv > $ spark-3.0.0-SNAPSHOT-bin-2.7.4/bin/spark-shell --conf > spark.sql.adaptive.enabled=true --num-executors 10 > Setting default log level to "WARN". > To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use > setLogLevel(newLevel). > 19/11/15 00:26:47 WARN yarn.Client: Neither spark.yarn.jars nor > spark.yarn.archive is set, falling back to uploading libraries under > SPARK_HOME. > Spark context Web UI available at http://ip-xx-xxx-x-xxx.ec2.internal:4040 > Spark context available as 'sc' (master = yarn, app id = > application_1573772077642_0006). > Spark session available as 'spark'. > Welcome to > __ > / __/__ ___ _/ /__ > _\ \/ _ \/ _ `/ __/ '_/ >/___/ .__/\_,_/_/ /_/\_\ version 3.0.0-SNAPSHOT > /_/ > > Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_222) > Type in expressions to have them evaluated. > Type :help for more information. > scala> spark.read.format("csv").option("header", > true).option("enforceSchema", > false).load("OP_DTL_GNRL_PGYR2013_P06282019.csv").show(1) > 19/11/15 00:27:10 WARN util.package: Truncated the string representation of a > plan since it was too large. This behavior can be adjusted by setting > 'spark.sql.debug.maxToStringFields'. > [Stage 2:>(0 + 10) / > 17]19/11/15 00:27:11 WARN scheduler.TaskSetManager: Lost task 0.0 in stage > 2.0 (TID 35, ip-xx-xxx-x-xxx.ec2.internal, executor 1): > java.lang.IllegalArgumentException: CSV header does not conform to the schema. > Header: Change_Type, Covered_Recipient_Type, Teaching_Hospital_CCN, > Teaching_Hospital_ID, Teaching_Hospital_Name, Physician_Profile_ID, > Physician_First_Name, Physician_Middle_Name, Physician_Last_Name, > Physician_Name_Suffix, Recipient_Primary_Business_Street_Address_Line1, > Recipient_Primary_Business_Street_Address_Line2, Recipient_City, > Recipient_State, Recipient_Zip_Code, Recipient_Country, Recipient_Province, > Recipient_Postal_Code, Physician_Primary_Type, Physician_Specialty, > Physician_License_State_code1, Physician_License_State_code2, > Physician_License_State_code3, Physician_License_State_code4, > Physician_License_State_code5, > Submitting_Applicable_Manufacturer_or_Applicable_GPO_Name, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_ID, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Name, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_State, > Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Country, > Total_Amount_of_Payment_USDollars, Date_of_Payment, > Number_of_Payments_Included_in_Total_Amount, > Form_of_Payment_or_Transfer_of_Value, Nature_of_Payment_or_Transfer_of_Value, > City_of_Travel, State_of_Travel, Country_of_Travel, > Physician_Ownership_Indicator, Third_Party_Payment_Recipient_Indicator, >
[jira] [Commented] (SPARK-29906) Reading of csv file fails with adaptive execution turned on
[ https://issues.apache.org/jira/browse/SPARK-29906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16974836#comment-16974836 ] koert kuipers commented on SPARK-29906: --- i added a bit of debug logging: {code:java} $ git diff diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala index 375cec5971..7e5b7fb235 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala @@ -86,7 +86,7 @@ object CSVDataSource extends Logging { } } -object TextInputCSVDataSource extends CSVDataSource { +object TextInputCSVDataSource extends CSVDataSource with Logging { override val isSplitable: Boolean = true override def readFile( @@ -110,9 +110,13 @@ object TextInputCSVDataSource extends CSVDataSource { sparkSession: SparkSession, inputPaths: Seq[FileStatus], parsedOptions: CSVOptions): StructType = { +logInfo(s"!! inputPaths ${inputPaths}") val csv = createBaseDataset(sparkSession, inputPaths, parsedOptions) val maybeFirstLine = CSVUtils.filterCommentAndEmpty(csv, parsedOptions).take(1).headOption -inferFromDataset(sparkSession, csv, maybeFirstLine, parsedOptions) +logInfo(s"!! maybeFirstLine ${maybeFirstLine}") +val schema = inferFromDataset(sparkSession, csv, maybeFirstLine, parsedOptions) +logInfo(s"!! schema ${schema}") +schema } {code} and this shows when spark.sql.adaptive.enabled=true: {code:java} 19/11/15 05:52:06 INFO csv.TextInputCSVDataSource: !! inputPaths List(LocatedFileStatus{path=hdfs://ip-xx-xxx-x-xxx.ec2.internal:8020/user/hadoop/OP_DTL_GNRL_PGYR2013_P06282019.csv; isDirectory=false; length=2242114396; replication=3; blocksize=134217728; modification_time=1573794115499; access_time=1573794109887; owner=hadoop; group=hadoop; permission=rw-r--r--; isSymlink=false}) 19/11/15 05:52:10 INFO csv.TextInputCSVDataSource: !! maybeFirstLine Some("UNCHANGED","Covered Recipient Physician""195068","SCOTT","KEVIN","FORMAN",,"360 SAN MIGUEL DR","SUITE 701","NEWPORT BEACH","CA","92660-7853","United States",,,"Medical Doctor","Allopathic & Osteopathic Physicians|Orthopaedic Surgery","CA","Wright Medical Technology, Inc.","10011065","Wright Medical Technology, Inc.","TN","United States",12.50,"08/20/2013","1","In-kind items and services","Food and Beverage""No","No Third Party Payment",,"No",,,"No","105165962","No","Covered","Foot and Ankle",,,"2013","06/28/2019") 19/11/15 05:52:10 INFO csv.TextInputCSVDataSource: !! schema StructType(StructField(UNCHANGED,StringType,true), StructField(Covered Recipient Physician,StringType,true), StructField(_c2,StringType,true), StructField(_c3,StringType,true), StructField(_c4,StringType,true), StructField(195068,StringType,true), StructField(SCOTT,StringType,true), StructField(KEVIN,StringType,true), StructField(FORMAN,StringType,true), StructField(_c9,StringType,true), StructField(360 SAN MIGUEL DR,StringType,true), StructField(SUITE 701,StringType,true), StructField(NEWPORT BEACH,StringType,true), StructField(CA13,StringType,true), StructField(92660-7853,StringType,true), StructField(United States15,StringType,true), StructField(_c16,StringType,true), StructField(_c17,StringType,true), StructField(Medical Doctor,StringType,true), StructField(Allopathic & Osteopathic Physicians|Orthopaedic Surgery,StringType,true), StructField(CA20,StringType,true), StructField(_c21,StringType,true), StructField(_c22,StringType,true), StructField(_c23,StringType,true), StructField(_c24,StringType,true), StructField(Wright Medical Technology, Inc.25,StringType,true), StructField(10011065,StringType,true), StructField(Wright Medical Technology, Inc.27,StringType,true), StructField(TN,StringType,true), StructField(United States29,StringType,true), StructField(12.50,StringType,true), StructField(08/20/2013,StringType,true), StructField(1,StringType,true), StructField(In-kind items and services,StringType,true), StructField(Food and Beverage,StringType,true), StructField(_c35,StringType,true), StructField(_c36,StringType,true), StructField(_c37,StringType,true), StructField(No38,StringType,true), StructField(No Third Party Payment,StringType,true), StructField(_c40,StringType,true), StructField(No41,StringType,true), StructField(_c42,StringType,true), StructField(_c43,StringType,true), StructField(No44,StringType,true), StructField(105165962,StringType,true), StructField(No46,StringType,true), StructField(Covered,StringType,true), StructField(Foot and Ankle,StringType,true), StructField(_c49,StringType,true), StructField(_c50,StringType,true),
[jira] [Updated] (SPARK-29906) Reading of csv file fails with adaptive execution turned on
[ https://issues.apache.org/jira/browse/SPARK-29906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-29906: -- Description: we observed an issue where spark seems to confuse a data line (not the first line of the csv file) for the csv header when it creates the schema. {code} $ wget http://download.cms.gov/openpayments/PGYR13_P062819.ZIP $ unzip PGYR13_P062819.ZIP $ hadoop fs -put OP_DTL_GNRL_PGYR2013_P06282019.csv $ spark-3.0.0-SNAPSHOT-bin-2.7.4/bin/spark-shell --conf spark.sql.adaptive.enabled=true --num-executors 10 Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). 19/11/15 00:26:47 WARN yarn.Client: Neither spark.yarn.jars nor spark.yarn.archive is set, falling back to uploading libraries under SPARK_HOME. Spark context Web UI available at http://ip-xx-xxx-x-xxx.ec2.internal:4040 Spark context available as 'sc' (master = yarn, app id = application_1573772077642_0006). Spark session available as 'spark'. Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 3.0.0-SNAPSHOT /_/ Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_222) Type in expressions to have them evaluated. Type :help for more information. scala> spark.read.format("csv").option("header", true).option("enforceSchema", false).load("OP_DTL_GNRL_PGYR2013_P06282019.csv").show(1) 19/11/15 00:27:10 WARN util.package: Truncated the string representation of a plan since it was too large. This behavior can be adjusted by setting 'spark.sql.debug.maxToStringFields'. [Stage 2:>(0 + 10) / 17]19/11/15 00:27:11 WARN scheduler.TaskSetManager: Lost task 0.0 in stage 2.0 (TID 35, ip-xx-xxx-x-xxx.ec2.internal, executor 1): java.lang.IllegalArgumentException: CSV header does not conform to the schema. Header: Change_Type, Covered_Recipient_Type, Teaching_Hospital_CCN, Teaching_Hospital_ID, Teaching_Hospital_Name, Physician_Profile_ID, Physician_First_Name, Physician_Middle_Name, Physician_Last_Name, Physician_Name_Suffix, Recipient_Primary_Business_Street_Address_Line1, Recipient_Primary_Business_Street_Address_Line2, Recipient_City, Recipient_State, Recipient_Zip_Code, Recipient_Country, Recipient_Province, Recipient_Postal_Code, Physician_Primary_Type, Physician_Specialty, Physician_License_State_code1, Physician_License_State_code2, Physician_License_State_code3, Physician_License_State_code4, Physician_License_State_code5, Submitting_Applicable_Manufacturer_or_Applicable_GPO_Name, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_ID, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Name, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_State, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Country, Total_Amount_of_Payment_USDollars, Date_of_Payment, Number_of_Payments_Included_in_Total_Amount, Form_of_Payment_or_Transfer_of_Value, Nature_of_Payment_or_Transfer_of_Value, City_of_Travel, State_of_Travel, Country_of_Travel, Physician_Ownership_Indicator, Third_Party_Payment_Recipient_Indicator, Name_of_Third_Party_Entity_Receiving_Payment_or_Transfer_of_Value, Charity_Indicator, Third_Party_Equals_Covered_Recipient_Indicator, Contextual_Information, Delay_in_Publication_Indicator, Record_ID, Dispute_Status_for_Publication, Product_Indicator, Name_of_Associated_Covered_Drug_or_Biological1, Name_of_Associated_Covered_Drug_or_Biological2, Name_of_Associated_Covered_Drug_or_Biological3, Name_of_Associated_Covered_Drug_or_Biological4, Name_of_Associated_Covered_Drug_or_Biological5, NDC_of_Associated_Covered_Drug_or_Biological1, NDC_of_Associated_Covered_Drug_or_Biological2, NDC_of_Associated_Covered_Drug_or_Biological3, NDC_of_Associated_Covered_Drug_or_Biological4, NDC_of_Associated_Covered_Drug_or_Biological5, Name_of_Associated_Covered_Device_or_Medical_Supply1, Name_of_Associated_Covered_Device_or_Medical_Supply2, Name_of_Associated_Covered_Device_or_Medical_Supply3, Name_of_Associated_Covered_Device_or_Medical_Supply4, Name_of_Associated_Covered_Device_or_Medical_Supply5, Program_Year, Payment_Publication_Date Schema: UNCHANGED, Covered Recipient Physician, _c2, _c3, _c4, 278352, JOHN, M, RAY, JR, 3625 CAPE CENTER DR, _c11, FAYETTEVILLE, NC13, 28304-4457, United States15, _c16, _c17, Medical Doctor, Allopathic & Osteopathic Physicians|Family Medicine, NC20, _c21, _c22, _c23, _c24, Par Pharmaceutical, Inc.25, 10010989, Par Pharmaceutical, Inc.27, NY, United States29, 17.29, 10/23/2013, 1, In-kind items and services, Food and Beverage, _c35, _c36, _c37, No38, No Third Party Payment, _c40, _c41, _c42, _c43, No44, 104522962, No46, Covered, MEGACE ES MEGESTROL ACETATE, _c49, _c50, _c51, _c52, 4988409496, _c54, _c55, _c56, _c57,
[jira] [Updated] (SPARK-29906) Reading of csv file fails with adaptive execution turned on
[ https://issues.apache.org/jira/browse/SPARK-29906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-29906: -- Description: we observed an issue where spark seems to confuse a data line (not the first line of the csv file) for the csv header. {code} $ wget http://download.cms.gov/openpayments/PGYR13_P062819.ZIP $ unzip PGYR13_P062819.ZIP $ hadoop fs -put OP_DTL_GNRL_PGYR2013_P06282019.csv $ spark-3.0.0-SNAPSHOT-bin-2.7.4/bin/spark-shell --conf spark.sql.adaptive.enabled=true --num-executors 10 Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). 19/11/15 00:26:47 WARN yarn.Client: Neither spark.yarn.jars nor spark.yarn.archive is set, falling back to uploading libraries under SPARK_HOME. Spark context Web UI available at http://ip-xx-xxx-x-xxx.ec2.internal:4040 Spark context available as 'sc' (master = yarn, app id = application_1573772077642_0006). Spark session available as 'spark'. Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 3.0.0-SNAPSHOT /_/ Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_222) Type in expressions to have them evaluated. Type :help for more information. scala> spark.read.format("csv").option("header", true).option("enforceSchema", false).load("OP_DTL_GNRL_PGYR2013_P06282019.csv").show(1) 19/11/15 00:27:10 WARN util.package: Truncated the string representation of a plan since it was too large. This behavior can be adjusted by setting 'spark.sql.debug.maxToStringFields'. [Stage 2:>(0 + 10) / 17]19/11/15 00:27:11 WARN scheduler.TaskSetManager: Lost task 0.0 in stage 2.0 (TID 35, ip-xx-xxx-x-xxx.ec2.internal, executor 1): java.lang.IllegalArgumentException: CSV header does not conform to the schema. Header: Change_Type, Covered_Recipient_Type, Teaching_Hospital_CCN, Teaching_Hospital_ID, Teaching_Hospital_Name, Physician_Profile_ID, Physician_First_Name, Physician_Middle_Name, Physician_Last_Name, Physician_Name_Suffix, Recipient_Primary_Business_Street_Address_Line1, Recipient_Primary_Business_Street_Address_Line2, Recipient_City, Recipient_State, Recipient_Zip_Code, Recipient_Country, Recipient_Province, Recipient_Postal_Code, Physician_Primary_Type, Physician_Specialty, Physician_License_State_code1, Physician_License_State_code2, Physician_License_State_code3, Physician_License_State_code4, Physician_License_State_code5, Submitting_Applicable_Manufacturer_or_Applicable_GPO_Name, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_ID, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Name, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_State, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Country, Total_Amount_of_Payment_USDollars, Date_of_Payment, Number_of_Payments_Included_in_Total_Amount, Form_of_Payment_or_Transfer_of_Value, Nature_of_Payment_or_Transfer_of_Value, City_of_Travel, State_of_Travel, Country_of_Travel, Physician_Ownership_Indicator, Third_Party_Payment_Recipient_Indicator, Name_of_Third_Party_Entity_Receiving_Payment_or_Transfer_of_Value, Charity_Indicator, Third_Party_Equals_Covered_Recipient_Indicator, Contextual_Information, Delay_in_Publication_Indicator, Record_ID, Dispute_Status_for_Publication, Product_Indicator, Name_of_Associated_Covered_Drug_or_Biological1, Name_of_Associated_Covered_Drug_or_Biological2, Name_of_Associated_Covered_Drug_or_Biological3, Name_of_Associated_Covered_Drug_or_Biological4, Name_of_Associated_Covered_Drug_or_Biological5, NDC_of_Associated_Covered_Drug_or_Biological1, NDC_of_Associated_Covered_Drug_or_Biological2, NDC_of_Associated_Covered_Drug_or_Biological3, NDC_of_Associated_Covered_Drug_or_Biological4, NDC_of_Associated_Covered_Drug_or_Biological5, Name_of_Associated_Covered_Device_or_Medical_Supply1, Name_of_Associated_Covered_Device_or_Medical_Supply2, Name_of_Associated_Covered_Device_or_Medical_Supply3, Name_of_Associated_Covered_Device_or_Medical_Supply4, Name_of_Associated_Covered_Device_or_Medical_Supply5, Program_Year, Payment_Publication_Date Schema: UNCHANGED, Covered Recipient Physician, _c2, _c3, _c4, 278352, JOHN, M, RAY, JR, 3625 CAPE CENTER DR, _c11, FAYETTEVILLE, NC13, 28304-4457, United States15, _c16, _c17, Medical Doctor, Allopathic & Osteopathic Physicians|Family Medicine, NC20, _c21, _c22, _c23, _c24, Par Pharmaceutical, Inc.25, 10010989, Par Pharmaceutical, Inc.27, NY, United States29, 17.29, 10/23/2013, 1, In-kind items and services, Food and Beverage, _c35, _c36, _c37, No38, No Third Party Payment, _c40, _c41, _c42, _c43, No44, 104522962, No46, Covered, MEGACE ES MEGESTROL ACETATE, _c49, _c50, _c51, _c52, 4988409496, _c54, _c55, _c56, _c57, _c58, _c59, _c60, _c61,
[jira] [Created] (SPARK-29906) Reading of csv file fails with adaptive execution turned on
koert kuipers created SPARK-29906: - Summary: Reading of csv file fails with adaptive execution turned on Key: SPARK-29906 URL: https://issues.apache.org/jira/browse/SPARK-29906 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.0.0 Environment: build from master today nov 14 commit fca0a6c394990b86304a8f9a64bf4c7ec58abbd6 (HEAD -> master, upstream/master, upstream/HEAD) Author: Kevin Yu Date: Thu Nov 14 14:58:32 2019 -0600 build using: $ dev/make-distribution.sh --tgz -Phadoop-2.7 -Dhadoop.version=2.7.4 -Pyarn deployed on AWS EMR 5.28 with 10 m5.xlarge slaves in spark-env.sh: HADOOP_CONF_DIR=/etc/hadoop/conf in spark-defaults.conf: spark.master yarn spark.submit.deployMode client spark.serializer org.apache.spark.serializer.KryoSerializer spark.hadoop.yarn.timeline-service.enabled false spark.driver.extraClassPath /usr/lib/hadoop-lzo/lib/hadoop-lzo.jar spark.driver.extraLibraryPath /usr/lib/hadoop/lib/native:/usr/lib/hadoop-lzo/lib/native spark.executor.extraClassPath /usr/lib/hadoop-lzo/lib/hadoop-lzo.jar spark.executor.extraLibraryPath /usr/lib/hadoop/lib/native:/usr/lib/hadoop-lzo/lib/native Reporter: koert kuipers we observed an issue where spark seems to confuse a data line (not the first line of the csv file) for the csv header. {code} $ wget http://download.cms.gov/openpayments/PGYR13_P062819.ZIP $ unzip PGYR13_P062819.ZIP $ hadoop fs -put OP_DTL_GNRL_PGYR2013_P06282019.csv $ spark-3.0.0-SNAPSHOT-bin-2.7.4/bin/spark-shell --conf spark.sql.adaptive.enabled=true --num-executors 10 Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). 19/11/15 00:26:47 WARN yarn.Client: Neither spark.yarn.jars nor spark.yarn.archive is set, falling back to uploading libraries under SPARK_HOME. Spark context Web UI available at http://ip-xx-xxx-x-xxx.ec2.internal:4040 Spark context available as 'sc' (master = yarn, app id = application_1573772077642_0006). Spark session available as 'spark'. Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 3.0.0-SNAPSHOT /_/ Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_222) Type in expressions to have them evaluated. Type :help for more information. scala> spark.read.format("csv").option("header", true).option("enforceSchema", false).load("OP_DTL_GNRL_PGYR2013_P06282019.csv").show(1) 19/11/15 00:27:10 WARN util.package: Truncated the string representation of a plan since it was too large. This behavior can be adjusted by setting 'spark.sql.debug.maxToStringFields'. [Stage 2:>(0 + 10) / 17]19/11/15 00:27:11 WARN scheduler.TaskSetManager: Lost task 0.0 in stage 2.0 (TID 35, ip-xx-xxx-x-xxx.ec2.internal, executor 1): java.lang.IllegalArgumentException: CSV header does not conform to the schema. Header: Change_Type, Covered_Recipient_Type, Teaching_Hospital_CCN, Teaching_Hospital_ID, Teaching_Hospital_Name, Physician_Profile_ID, Physician_First_Name, Physician_Middle_Name, Physician_Last_Name, Physician_Name_Suffix, Recipient_Primary_Business_Street_Address_Line1, Recipient_Primary_Business_Street_Address_Line2, Recipient_City, Recipient_State, Recipient_Zip_Code, Recipient_Country, Recipient_Province, Recipient_Postal_Code, Physician_Primary_Type, Physician_Specialty, Physician_License_State_code1, Physician_License_State_code2, Physician_License_State_code3, Physician_License_State_code4, Physician_License_State_code5, Submitting_Applicable_Manufacturer_or_Applicable_GPO_Name, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_ID, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Name, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_State, Applicable_Manufacturer_or_Applicable_GPO_Making_Payment_Country, Total_Amount_of_Payment_USDollars, Date_of_Payment, Number_of_Payments_Included_in_Total_Amount, Form_of_Payment_or_Transfer_of_Value, Nature_of_Payment_or_Transfer_of_Value, City_of_Travel, State_of_Travel, Country_of_Travel, Physician_Ownership_Indicator, Third_Party_Payment_Recipient_Indicator, Name_of_Third_Party_Entity_Receiving_Payment_or_Transfer_of_Value, Charity_Indicator, Third_Party_Equals_Covered_Recipient_Indicator, Contextual_Information, Delay_in_Publication_Indicator, Record_ID, Dispute_Status_for_Publication, Product_Indicator, Name_of_Associated_Covered_Drug_or_Biological1, Name_of_Associated_Covered_Drug_or_Biological2, Name_of_Associated_Covered_Drug_or_Biological3, Name_of_Associated_Covered_Drug_or_Biological4, Name_of_Associated_Covered_Drug_or_Biological5, NDC_of_Associated_Covered_Drug_or_Biological1, NDC_of_Associated_Covered_Drug_or_Biological2,
[jira] [Commented] (SPARK-28945) Allow concurrent writes to different partitions with dynamic partition overwrite
[ https://issues.apache.org/jira/browse/SPARK-28945?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16973715#comment-16973715 ] koert kuipers commented on SPARK-28945: --- i understand there is a great deal of complexity in the committer and this might require more work to get it right but its still unclear to me if the committer is doing anything at all in case of dynamic partition overwrite. what do i lose by disabling all committer activity (committer.setupJob, committer.commitJob, etc.) when dynamicPartitionOverwrite is true? and if i lose nothing, is that a good thing, or does that mean i should be worried about the current state? > Allow concurrent writes to different partitions with dynamic partition > overwrite > > > Key: SPARK-28945 > URL: https://issues.apache.org/jira/browse/SPARK-28945 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.4.3 >Reporter: koert kuipers >Priority: Minor > > It is desirable to run concurrent jobs that write to different partitions > within same baseDir using partitionBy and dynamic partitionOverwriteMode. > See for example here: > https://stackoverflow.com/questions/38964736/multiple-spark-jobs-appending-parquet-data-to-same-base-path-with-partitioning > Or the discussion here: > https://github.com/delta-io/delta/issues/9 > This doesnt seem that difficult. I suspect only changes needed are in > org.apache.spark.internal.io.HadoopMapReduceCommitProtocol, which already has > a flag for dynamicPartitionOverwrite. I got a quick test to work by disabling > all committer activity (committer.setupJob, committer.commitJob, etc.) when > dynamicPartitionOverwrite is true. -- 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-29435) Spark 3 doesnt work with older shuffle service
[ https://issues.apache.org/jira/browse/SPARK-29435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16950122#comment-16950122 ] koert kuipers edited comment on SPARK-29435 at 10/12/19 11:50 PM: -- i checked the patch and it works with dynamic execution using spark 3 shuffle service and using spark 2 shuffle service was (Author: koert): i checked and it works with dynamic execution using spark 3 shuffle service and using spark 2 shuffle service > 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 >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 >
[jira] [Commented] (SPARK-29435) Spark 3 doesnt work with older shuffle service
[ https://issues.apache.org/jira/browse/SPARK-29435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16950122#comment-16950122 ] koert kuipers commented on SPARK-29435: --- i checked and it works with dynamic execution using spark 3 shuffle service and using spark 2 shuffle service > 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 >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 >
[jira] [Comment Edited] (SPARK-29435) Spark 3 doesnt work with older shuffle service
[ https://issues.apache.org/jira/browse/SPARK-29435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16949095#comment-16949095 ] koert kuipers edited comment on SPARK-29435 at 10/11/19 3:20 AM: - actually, it doesnt matter if i use spark 2 or spark 3 shuffle service. it doesnt matter if i use dynamic allocation or not. as soon as i set spark.shuffle.useOldFetchProtocol=true i get these errors, always. was (Author: koert): actually, it doesnt matter if i use spark 2 or spark 3 shuffle service. it doesnt matter if i use dynamic allocation and shuffle service, or not. as soon as i set spark.shuffle.useOldFetchProtocol=true i get these errors, always. > 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 >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 >
[jira] [Commented] (SPARK-29435) Spark 3 doesnt work with older shuffle service
[ https://issues.apache.org/jira/browse/SPARK-29435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16949095#comment-16949095 ] koert kuipers commented on SPARK-29435: --- actually, it doesnt matter if i use spark 2 or spark 3 shuffle service. it doesnt matter if i use dynamic allocation and shuffle service, or not. as soon as i set spark.shuffle.useOldFetchProtocol=true i get these errors, always. > 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 >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 >
[jira] [Commented] (SPARK-29435) Spark 3 doesnt work with older shuffle service
[ https://issues.apache.org/jira/browse/SPARK-29435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16948989#comment-16948989 ] koert kuipers commented on SPARK-29435: --- [~vanzin] sorry i should have been more clear, i did set spark.shuffle.useOldFetchProtocol=true and i could not get it to work > 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 >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 >
[jira] [Created] (SPARK-29435) Spark 3 doesnt work with older shuffle service
koert kuipers created SPARK-29435: - Summary: 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 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 org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:60) at org.apache.spark.sql.execution.ShuffledRowRDD.compute(ShuffledRowRDD.scala:172) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327) at org.apache.spark.rdd.RDD.iterator(RDD.scala:291) at
[jira] [Commented] (SPARK-27665) Split fetch shuffle blocks protocol from OpenBlocks
[ https://issues.apache.org/jira/browse/SPARK-27665?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16948979#comment-16948979 ] koert kuipers commented on SPARK-27665: --- i tried using spark.shuffle.useOldFetchProtocol=true while using spark 3 (master) to launch job, with spark 2.4.1 shuffle service running in yarn. i cannot get it to work. for example on one cluster i saw: {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} on another: {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 org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:60) at org.apache.spark.sql.execution.ShuffledRowRDD.compute(ShuffledRowRDD.scala:172) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327) at org.apache.spark.rdd.RDD.iterator(RDD.scala:291) 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) ... 11 more {code} > Split fetch shuffle blocks protocol from OpenBlocks > --- > > Key: SPARK-27665 > URL: https://issues.apache.org/jira/browse/SPARK-27665 >
[jira] [Commented] (SPARK-27665) Split fetch shuffle blocks protocol from OpenBlocks
[ https://issues.apache.org/jira/browse/SPARK-27665?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16932833#comment-16932833 ] koert kuipers commented on SPARK-27665: --- oh wait i didnt realize there is a setting spark.shuffle.useOldFetchProtocol never mind! i will try that > Split fetch shuffle blocks protocol from OpenBlocks > --- > > Key: SPARK-27665 > URL: https://issues.apache.org/jira/browse/SPARK-27665 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Affects Versions: 2.4.0 >Reporter: Yuanjian Li >Assignee: Yuanjian Li >Priority: Major > Fix For: 3.0.0 > > > As the current approach in OneForOneBlockFetcher, we reuse the OpenBlocks > protocol to describe the fetch request for shuffle blocks, and it causes the > extension work for shuffle fetching like SPARK-9853 and SPARK-25341 very > awkward. We need a new protocol only for shuffle blocks fetcher. -- 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-27665) Split fetch shuffle blocks protocol from OpenBlocks
[ https://issues.apache.org/jira/browse/SPARK-27665?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16932823#comment-16932823 ] koert kuipers commented on SPARK-27665: --- i am a little nervous that this got merged into master without resolving the blocker SPARK-27780 currently this means spark 3.x will not be able to support dynamic allocation at all on yarn clusters that have spark 2 shuffle managers installed, which is all our client clusters pretty much. > Split fetch shuffle blocks protocol from OpenBlocks > --- > > Key: SPARK-27665 > URL: https://issues.apache.org/jira/browse/SPARK-27665 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Affects Versions: 2.4.0 >Reporter: Yuanjian Li >Assignee: Yuanjian Li >Priority: Major > Fix For: 3.0.0 > > > As the current approach in OneForOneBlockFetcher, we reuse the OpenBlocks > protocol to describe the fetch request for shuffle blocks, and it causes the > extension work for shuffle fetching like SPARK-9853 and SPARK-25341 very > awkward. We need a new protocol only for shuffle blocks fetcher. -- 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-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16927912#comment-16927912 ] koert kuipers commented on SPARK-29027: --- [~gsomogyi] if you email me at koert at tresata dot com i can send logs > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] > [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 > s] > [INFO] Spark Project Tags . SUCCESS [ 9.373 > s] > [INFO] Spark Project Sketch ... SUCCESS [ 24.586 > s] > [INFO] Spark Project Local DB . SUCCESS [ 5.456 > s] > [INFO] Spark Project Networking
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16927889#comment-16927889 ] koert kuipers commented on SPARK-29027: --- just for this one test debug logs is 62mb of kerberos and ldap stuff. its difficult to say whats sensitive and whats not. > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] > [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 > s] > [INFO] Spark Project Tags . SUCCESS [ 9.373 > s] > [INFO] Spark Project Sketch ... SUCCESS [ 24.586 > s] > [INFO] Spark Project Local DB . SUCCESS [ 5.456 >
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16927785#comment-16927785 ] koert kuipers commented on SPARK-29027: --- [~gsomogyi] i can email you debug log file directly if thats ok. rather not post it publicly. > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] > [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 > s] > [INFO] Spark Project Tags . SUCCESS [ 9.373 > s] > [INFO] Spark Project Sketch ... SUCCESS [ 24.586 > s] > [INFO] Spark Project Local DB . SUCCESS [ 5.456 > s] > [INFO] Spark Project
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16927783#comment-16927783 ] koert kuipers commented on SPARK-29027: --- i get same error in sbt i think, plus i find sbt a lot easier to handle :) {code} [info] KafkaDelegationTokenSuite: [info] org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** (10 seconds, 543 milliseconds) [info] org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure [info] at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) [info] at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) [info] at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) [info] at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) [info] at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) [info] at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) [info] at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) [info] at org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) [info] at org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) [info] at org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) [info] at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:212) [info] at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210) [info] at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208) [info] at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:56) [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:314) [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:507) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:296) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:286) [info] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [info] at java.lang.Thread.run(Thread.java:748) org.apache.directory.api.ldap.model.exception.LdapOperationErrorException: /home/koert/src/spark/target/tmp/spark-dc223dd0-e499-4ccf-9600-c70e4706a909/1568218986864/partitions/system/1.3.6.1.4.1.18060.0.4.1.2.50.lg (No such file or directory) at org.apache.directory.server.core.partition.impl.btree.AbstractBTreePartition.modify(AbstractBTreePartition.java:1183) at org.apache.directory.server.core.shared.partition.DefaultPartitionNexus.sync(DefaultPartitionNexus.java:335) at org.apache.directory.server.core.DefaultDirectoryService.shutdown(DefaultDirectoryService.java:1299) at org.apache.directory.server.core.DefaultDirectoryService$1.run(DefaultDirectoryService.java:1230) at java.lang.Thread.run(Thread.java:748) Caused by: java.io.FileNotFoundException: /home/koert/src/spark/target/tmp/spark-dc223dd0-e499-4ccf-9600-c70e4706a909/1568218986864/partitions/system/1.3.6.1.4.1.18060.0.4.1.2.50.lg (No such file or directory) at java.io.FileOutputStream.open0(Native Method) at java.io.FileOutputStream.open(FileOutputStream.java:270) at java.io.FileOutputStream.(FileOutputStream.java:213) at java.io.FileOutputStream.(FileOutputStream.java:101) at jdbm.recman.TransactionManager.open(TransactionManager.java:209) at jdbm.recman.TransactionManager.synchronizeLogFromMemory(TransactionManager.java:202) at jdbm.recman.TransactionManager.synchronizeLog(TransactionManager.java:135) at org.apache.directory.server.core.partition.impl.btree.jdbm.JdbmIndex.sync(JdbmIndex.java:698) at org.apache.directory.server.core.partition.impl.btree.jdbm.JdbmPartition.sync(JdbmPartition.java:312) at org.apache.directory.server.core.partition.impl.btree.AbstractBTreePartition.modify(AbstractBTreePartition.java:1228) at org.apache.directory.server.core.partition.impl.btree.AbstractBTreePartition.modify(AbstractBTreePartition.java:1173) ... 4 more java.io.FileNotFoundException: /home/koert/src/spark/target/tmp/spark-dc223dd0-e499-4ccf-9600-c70e4706a909/1568218986864/partitions/example/1.3.6.1.4.1.18060.0.4.1.2.5.lg (No such file or directory) at java.io.FileOutputStream.open0(Native Method) at java.io.FileOutputStream.open(FileOutputStream.java:270) at java.io.FileOutputStream.(FileOutputStream.java:213) at java.io.FileOutputStream.(FileOutputStream.java:101) at jdbm.recman.TransactionManager.open(TransactionManager.java:209) at jdbm.recman.TransactionManager.synchronizeLogFromMemory(TransactionManager.java:202) at jdbm.recman.TransactionManager.synchronizeLog(TransactionManager.java:135) at
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16927715#comment-16927715 ] koert kuipers commented on SPARK-29027: --- i renamed /etc/krb5.conf and it did not change anything. still same failure. {code} ~/spark/external/kafka-0-10-sql$ mvn dependency:tree -Dverbose | grep zookeeper [INFO] +- org.apache.zookeeper:zookeeper:jar:3.4.7:test {code} > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] > [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 > s] > [INFO] Spark Project Tags . SUCCESS [ 9.373 > s] > [INFO] Spark Project Sketch ...
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16927646#comment-16927646 ] koert kuipers commented on SPARK-29027: --- let me try to get debug logs > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] > [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 > s] > [INFO] Spark Project Tags . SUCCESS [ 9.373 > s] > [INFO] Spark Project Sketch ... SUCCESS [ 24.586 > s] > [INFO] Spark Project Local DB . SUCCESS [ 5.456 > s] > [INFO] Spark Project Networking ... SUCCESS [ 49.819 > s] >
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16926809#comment-16926809 ] koert kuipers commented on SPARK-29027: --- [~gsomogyi] do you use any services that require open ports perhaps? i am thinking it could be firewall issue, or host to ip mapping? > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] > [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 > s] > [INFO] Spark Project Tags . SUCCESS [ 9.373 > s] > [INFO] Spark Project Sketch ... SUCCESS [ 24.586 > s] > [INFO] Spark Project Local DB . SUCCESS
[jira] [Comment Edited] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16926707#comment-16926707 ] koert kuipers edited comment on SPARK-29027 at 9/10/19 2:53 PM: i tried doing tests in a virtual machine and they pass so its something in my environment (or really in all our corporate laptops and servers) but i have no idea what it could be right now was (Author: koert): i tried doing tests in a virtual machine and they pass so its something in my environment (or should u say in all our corporate laptops and servers) but i have no idea what it could be right now > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] >
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16926707#comment-16926707 ] koert kuipers commented on SPARK-29027: --- i tried doing tests in a virtual machine and they pass so its something in my environment (or should u say in all our corporate laptops and servers) but i have no idea what it could be right now > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] > [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 > s] > [INFO] Spark Project Tags . SUCCESS [ 9.373 > s] > [INFO] Spark Project Sketch ... SUCCESS [ 24.586 > s] > [INFO]
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16926620#comment-16926620 ] koert kuipers commented on SPARK-29027: --- i am going to try running tests on a virtual machine to try to isolate what the issue could be in environment > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] > [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 > s] > [INFO] Spark Project Tags . SUCCESS [ 9.373 > s] > [INFO] Spark Project Sketch ... SUCCESS [ 24.586 > s] > [INFO] Spark Project Local DB . SUCCESS [ 5.456 > s] > [INFO]
[jira] [Comment Edited] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16926577#comment-16926577 ] koert kuipers edited comment on SPARK-29027 at 9/10/19 1:02 PM: i am running test on my work laptop. it has kerberos client installed (e.g. i can kinit, klist, kdestroy on it). i get the same error on other laptop (ubuntu 18) and one of our build servers. they also have kerberos client installed. was (Author: koert): i am running test on my work laptop. it has kerberos client installed (e.g. i can kinit, klist, kdestroy on it). i get the same error on other laptop (ubuntu 18) and one of our build servers. they also have kerberos client installed. i tried temporarily renaming /etc/krb5.conf to something else and then the tests passed it seems. so now i suspect that a functioning kerberos client interferes with test. i will repeat the confirm this is not coincidence. > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16926577#comment-16926577 ] koert kuipers commented on SPARK-29027: --- i am running test on my work laptop. it has kerberos client installed (e.g. i can kinit, klist, kdestroy on it). i get the same error on other laptop (ubuntu 18) and one of our build servers. they also have kerberos client installed. i tried temporarily renaming /etc/krb5.conf to something else and then the tests passed it seems. so now i suspect that a functioning kerberos client interferes with test. i will repeat the confirm this is not coincidence. > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: >
[jira] [Commented] (SPARK-29027) KafkaDelegationTokenSuite fails
[ https://issues.apache.org/jira/browse/SPARK-29027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16926563#comment-16926563 ] koert kuipers commented on SPARK-29027: --- hey the command i run is: mvn clean test -fae i am not aware of downstream changes. where/how do you see that in reactor summary? in so far i know this is spark master. to be sure i will do new clone of repo. > KafkaDelegationTokenSuite fails > --- > > Key: SPARK-29027 > URL: https://issues.apache.org/jira/browse/SPARK-29027 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 3.0.0 > Environment: {code} > commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 > Author: Sean Owen > Date: Mon Sep 9 10:19:40 2019 -0500 > {code} > Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) >Reporter: koert kuipers >Priority: Minor > > i am seeing consistent failure of KafkaDelegationTokenSuite on master > {code} > JsonUtilsSuite: > - parsing partitions > - parsing partitionOffsets > KafkaDelegationTokenSuite: > javax.security.sasl.SaslException: Failure to initialize security context > [Caused by GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails)] > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) > at > com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) > at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) > at > org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) > at > org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) > at > org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) > at java.lang.Thread.run(Thread.java:748) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos credentails) > at > sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) > at > sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) > at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) > at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) > at > sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) > at > com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) > ... 12 more > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** > org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure > at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) > at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) > at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) > at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) > at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) > at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) > at > org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) > at > org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) > ... > KafkaSourceOffsetSuite: > - comparison {"t":{"0":1}} <=> {"t":{"0":2}} > - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} > - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} > - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} > - basic serialization - deserialization > - OffsetSeqLog serialization - deserialization > - read Spark 2.1.0 offset format > {code} > {code} > [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: > [INFO] > [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 > s] > [INFO] Spark Project Tags . SUCCESS [ 9.373 > s] > [INFO] Spark Project Sketch ... SUCCESS [ 24.586
[jira] [Created] (SPARK-29027) KafkaDelegationTokenSuite fails
koert kuipers created SPARK-29027: - Summary: KafkaDelegationTokenSuite fails Key: SPARK-29027 URL: https://issues.apache.org/jira/browse/SPARK-29027 Project: Spark Issue Type: Bug Components: Structured Streaming Affects Versions: 3.0.0 Environment: {code} commit 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 Author: Sean Owen Date: Mon Sep 9 10:19:40 2019 -0500 {code} Ubuntu 16.04 with OpenJDK 1.8 (1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) Reporter: koert kuipers i am seeing consistent failure of KafkaDelegationTokenSuite on master {code} JsonUtilsSuite: - parsing partitions - parsing partitionOffsets KafkaDelegationTokenSuite: javax.security.sasl.SaslException: Failure to initialize security context [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos credentails)] at com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) at com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) at org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:118) at org.apache.zookeeper.server.ZooKeeperSaslServer$1.run(ZooKeeperSaslServer.java:114) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:114) at org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:48) at org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) at org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:156) at org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:197) at java.lang.Thread.run(Thread.java:748) Caused by: GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos credentails) at sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) at sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) at sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) at sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) at com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) ... 12 more org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** org.I0Itec.zkclient.exception.ZkAuthFailedException: Authentication failure at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:947) at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:924) at org.I0Itec.zkclient.ZkClient.connect(ZkClient.java:1231) at org.I0Itec.zkclient.ZkClient.(ZkClient.java:157) at org.I0Itec.zkclient.ZkClient.(ZkClient.java:131) at kafka.utils.ZkUtils$.createZkClientAndConnection(ZkUtils.scala:93) at kafka.utils.ZkUtils$.apply(ZkUtils.scala:75) at org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedZookeeper(KafkaTestUtils.scala:202) at org.apache.spark.sql.kafka010.KafkaTestUtils.setup(KafkaTestUtils.scala:243) at org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite.beforeAll(KafkaDelegationTokenSuite.scala:49) ... KafkaSourceOffsetSuite: - comparison {"t":{"0":1}} <=> {"t":{"0":2}} - comparison {"t":{"1":0,"0":1}} <=> {"t":{"1":1,"0":2}} - comparison {"t":{"0":1},"T":{"0":0}} <=> {"t":{"0":2},"T":{"0":1}} - comparison {"t":{"0":1}} <=> {"t":{"1":1,"0":2}} - comparison {"t":{"0":1}} <=> {"t":{"1":3,"0":2}} - basic serialization - deserialization - OffsetSeqLog serialization - deserialization - read Spark 2.1.0 offset format {code} {code} [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-SNAPSHOT: [INFO] [INFO] Spark Project Parent POM ... SUCCESS [ 4.178 s] [INFO] Spark Project Tags . SUCCESS [ 9.373 s] [INFO] Spark Project Sketch ... SUCCESS [ 24.586 s] [INFO] Spark Project Local DB . SUCCESS [ 5.456 s] [INFO] Spark Project Networking ... SUCCESS [ 49.819 s] [INFO] Spark Project Shuffle Streaming Service SUCCESS [ 6.096 s] [INFO] Spark Project Unsafe ... SUCCESS [ 14.714 s] [INFO] Spark Project Launcher . SUCCESS [ 5.277 s] [INFO] Spark Project Core . SUCCESS [32:58 min] [INFO] Spark Project ML Local Library . SUCCESS [ 41.076
[jira] [Updated] (SPARK-28945) Allow concurrent writes to different partitions with dynamic partition overwrite
[ https://issues.apache.org/jira/browse/SPARK-28945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koert kuipers updated SPARK-28945: -- Summary: Allow concurrent writes to different partitions with dynamic partition overwrite (was: Allow concurrent writes to unrelated partitions with dynamic partition overwrite) > Allow concurrent writes to different partitions with dynamic partition > overwrite > > > Key: SPARK-28945 > URL: https://issues.apache.org/jira/browse/SPARK-28945 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.4.3 >Reporter: koert kuipers >Priority: Minor > > It is desirable to run concurrent jobs that write to different partitions > within same baseDir using partitionBy and dynamic partitionOverwriteMode. > See for example here: > https://stackoverflow.com/questions/38964736/multiple-spark-jobs-appending-parquet-data-to-same-base-path-with-partitioning > Or the discussion here: > https://github.com/delta-io/delta/issues/9 > This doesnt seem that difficult. I suspect only changes needed are in > org.apache.spark.internal.io.HadoopMapReduceCommitProtocol, which already has > a flag for dynamicPartitionOverwrite. I got a quick test to work by disabling > all committer activity (committer.setupJob, committer.commitJob, etc.) when > dynamicPartitionOverwrite is true. -- This message was sent by Atlassian Jira (v8.3.2#803003) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org